mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Analyzer TableFunctionExplain fix
This commit is contained in:
parent
2830877e22
commit
cd3e8482ae
@ -209,12 +209,14 @@ ASTPtr MatcherNode::toASTImpl() const
|
||||
ASTPtr result;
|
||||
ASTPtr transformers;
|
||||
|
||||
if (!children.empty())
|
||||
const auto & column_transformers = getColumnTransformers().getNodes();
|
||||
|
||||
if (!column_transformers.empty())
|
||||
{
|
||||
transformers = std::make_shared<ASTColumnsTransformerList>();
|
||||
|
||||
for (const auto & child : children)
|
||||
transformers->children.push_back(child->toAST());
|
||||
for (const auto & column_transformer : column_transformers)
|
||||
transformers->children.push_back(column_transformer->toAST());
|
||||
}
|
||||
|
||||
if (matcher_type == MatcherNodeType::ASTERISK)
|
||||
|
@ -108,6 +108,7 @@ namespace ErrorCodes
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int ALIAS_REQUIRED;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
/** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before.
|
||||
|
@ -26,7 +26,6 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TYPE_MISMATCH;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int UNION_ALL_RESULT_STRUCTURES_MISMATCH;
|
||||
extern const int INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH;
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <TableFunctions/TableFunctionExplain.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
#include <Analyzer/TableFunctionNode.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -19,6 +20,18 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
std::vector<size_t> TableFunctionExplain::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr /*context*/) const
|
||||
{
|
||||
const auto & table_function_node = query_node_table_function->as<TableFunctionNode &>();
|
||||
const auto & table_function_node_arguments = table_function_node.getArguments().getNodes();
|
||||
size_t table_function_node_arguments_size = table_function_node_arguments.size();
|
||||
|
||||
if (table_function_node_arguments_size == 3)
|
||||
return {2};
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
void TableFunctionExplain::parseArguments(const ASTPtr & ast_function, ContextPtr /*context*/)
|
||||
{
|
||||
const auto * function = ast_function->as<ASTFunction>();
|
||||
|
@ -13,13 +13,18 @@ class TableFunctionExplain : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "viewExplain";
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns) const override;
|
||||
|
||||
const char * getStorageTypeName() const override { return "Explain"; }
|
||||
|
||||
std::vector<size_t> skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override;
|
||||
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
|
||||
|
||||
InterpreterExplainQuery getInterpreter(ContextPtr context) const;
|
||||
@ -27,5 +32,4 @@ private:
|
||||
ASTPtr query = nullptr;
|
||||
};
|
||||
|
||||
|
||||
}
|
||||
|
@ -24,12 +24,13 @@ SELECT * FROM viewExplain('EXPLAIN AST', ''); -- { serverError BAD_ARGUMENTS }
|
||||
SELECT * FROM viewExplain('EXPLAIN AST', '', 1); -- { serverError BAD_ARGUMENTS }
|
||||
SELECT * FROM viewExplain('EXPLAIN AST', '', ''); -- { serverError BAD_ARGUMENTS }
|
||||
|
||||
DROP TABLE IF EXISTS t1;
|
||||
CREATE TABLE t1 ( a UInt64 ) Engine = MergeTree ORDER BY tuple() AS SELECT number AS a FROM system.numbers LIMIT 100000;
|
||||
|
||||
SELECT rows > 1000 FROM (EXPLAIN ESTIMATE SELECT sum(a) FROM t1);
|
||||
SELECT count() == 1 FROM (EXPLAIN ESTIMATE SELECT sum(a) FROM t1);
|
||||
|
||||
DROP TABLE IF EXISTS t1;
|
||||
DROP TABLE t1;
|
||||
|
||||
SET allow_experimental_analyzer = 1;
|
||||
|
||||
@ -38,32 +39,7 @@ SELECT count() > 0 FROM (EXPLAIN PLAN SELECT * FROM system.numbers ORDER BY numb
|
||||
SELECT count() > 0 FROM (EXPLAIN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%';
|
||||
SELECT count() > 0 FROM (EXPLAIN CURRENT TRANSACTION);
|
||||
SELECT count() == 1 FROM (EXPLAIN SYNTAX SELECT number FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE 'SELECT%';
|
||||
|
||||
-- We have `Identifier number` instead of `Asterisk` because query argument of `viewExplain` table function was analyzed.
|
||||
-- Compare:
|
||||
-- :) EXPLAIN AST SELECT *;
|
||||
-- ┌─explain───────────────────────────┐
|
||||
-- │ SelectWithUnionQuery (children 1) │
|
||||
-- │ ExpressionList (children 1) │
|
||||
-- │ SelectQuery (children 1) │
|
||||
-- │ ExpressionList (children 1) │
|
||||
-- │ Asterisk │
|
||||
-- └───────────────────────────────────┘
|
||||
-- :) SELECT * FROM (EXPLAIN AST SELECT *);
|
||||
-- ┌─explain─────────────────────────────────────┐
|
||||
-- │ SelectWithUnionQuery (children 1) │
|
||||
-- │ ExpressionList (children 1) │
|
||||
-- │ SelectQuery (children 2) │
|
||||
-- │ ExpressionList (children 1) │
|
||||
-- │ Identifier dummy │
|
||||
-- │ TablesInSelectQuery (children 1) │
|
||||
-- │ TablesInSelectQueryElement (children 1) │
|
||||
-- │ TableExpression (children 1) │
|
||||
-- │ TableIdentifier system.one │
|
||||
-- └─────────────────────────────────────────────┘
|
||||
-- TODO: argument of `viewExplain` (and subquery in `EXAPLAN ...`) should not be analyzed.
|
||||
-- See _Support query tree in table functions_ in https://github.com/ClickHouse/ClickHouse/issues/42648
|
||||
SELECT trim(explain) == 'Identifier number' FROM (EXPLAIN AST SELECT * FROM system.numbers LIMIT 10) WHERE explain LIKE '%Identifier number%';
|
||||
SELECT trim(explain) == 'Asterisk' FROM (EXPLAIN AST SELECT * FROM system.numbers LIMIT 10) WHERE explain LIKE '%Asterisk%';
|
||||
|
||||
SELECT * FROM (
|
||||
EXPLAIN AST SELECT * FROM (
|
||||
|
Loading…
Reference in New Issue
Block a user