Analyzer TableFunctionExplain fix

This commit is contained in:
Maksim Kita 2023-01-26 14:46:32 +01:00
parent 2830877e22
commit cd3e8482ae
6 changed files with 27 additions and 32 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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