Review fixes

This commit is contained in:
Alexey Milovidov 2021-07-11 02:14:03 +03:00
parent 78cde85ea8
commit 9c0097cd9b
6 changed files with 10 additions and 14 deletions

View File

@ -243,7 +243,7 @@ ExplainSettings<Settings> checkAndGetSettings(const ASTPtr & ast_settings)
BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
{
auto & ast = query->as<ASTExplainQuery &>();
const auto & ast = query->as<const ASTExplainQuery &>();
Block sample_block = getSampleBlock(ast.getKind());
MutableColumns res_columns = sample_block.cloneEmptyColumns();
@ -336,22 +336,17 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
else if (ast.getKind() == ASTExplainQuery::QueryEstimates)
{
if (!dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
throw Exception("Only SELECT is supported for EXPLAIN ESTIMATES query", ErrorCodes::INCORRECT_QUERY);
throw Exception("Only SELECT is supported for EXPLAIN ESTIMATE query", ErrorCodes::INCORRECT_QUERY);
auto settings = checkAndGetSettings<QueryPlanSettings>(ast.getSettings());
QueryPlan plan;
// It should output the result even the format is Null, for example EXPLAIN ESTIMATES select * from x format Null;
if (ast.format && ast.format->getColumnName() == "Null")
{
ast.format = nullptr;
}
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions());
interpreter.buildQueryPlan(plan);
if (settings.optimize)
plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext()));
plan.explainEstimates(res_columns);
plan.explainEstimate(res_columns);
}
if (ast.getKind() != ASTExplainQuery::QueryEstimates)
{

View File

@ -17,7 +17,7 @@ public:
AnalyzedSyntax, /// 'EXPLAIN SYNTAX SELECT ...'
QueryPlan, /// 'EXPLAIN SELECT ...'
QueryPipeline, /// 'EXPLAIN PIPELINE ...'
QueryEstimates, /// 'EXPLAIN ESTIMATES ...'
QueryEstimates, /// 'EXPLAIN ESTIMATE ...'
};
explicit ASTExplainQuery(ExplainKind kind_) : kind(kind_) {}
@ -77,7 +77,7 @@ private:
case AnalyzedSyntax: return "EXPLAIN SYNTAX";
case QueryPlan: return "EXPLAIN";
case QueryPipeline: return "EXPLAIN PIPELINE";
case QueryEstimates: return "EXPLAIN ESTIMATES";
case QueryEstimates: return "EXPLAIN ESTIMATE";
}
__builtin_unreachable();

View File

@ -19,7 +19,7 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ParserKeyword s_syntax("SYNTAX");
ParserKeyword s_pipeline("PIPELINE");
ParserKeyword s_plan("PLAN");
ParserKeyword s_estimates("ESTIMATES");
ParserKeyword s_estimates("ESTIMATE");
if (s_explain.ignore(pos, expected))
{

View File

@ -435,7 +435,7 @@ void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_sett
QueryPlanOptimizations::optimizeTree(optimization_settings, *root, nodes);
}
void QueryPlan::explainEstimates(MutableColumns & columns)
void QueryPlan::explainEstimate(MutableColumns & columns)
{
checkInitialized();
@ -447,6 +447,7 @@ void QueryPlan::explainEstimates(MutableColumns & columns)
UInt64 rows = 0;
UInt64 marks = 0;
UInt64 bytes = 0;
EstimateCounters(const std::string & database, const std::string & table) : database_name(database), table_name(table)
{
}

View File

@ -86,7 +86,7 @@ public:
JSONBuilder::ItemPtr explainPlan(const ExplainPlanOptions & options);
void explainPlan(WriteBuffer & buffer, const ExplainPlanOptions & options);
void explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptions & options);
void explainEstimates(MutableColumns & columns);
void explainEstimate(MutableColumns & columns);
/// Set upper limit for the recommend number of threads. Will be applied to the newly-created pipelines.
/// TODO: make it in a better way.

View File

@ -20,5 +20,5 @@ def test_explain_estimates(start_cluster):
node1.query("INSERT INTO test SELECT number FROM numbers(1000)")
node1.query("OPTIMIZE TABLE test")
system_parts_result = node1.query("SELECT any(database), any(table), count() as parts, sum(rows) as rows, sum(marks) as marks, sum(bytes_on_disk) as bytes FROM system.parts WHERE database = 'default' AND table = 'test' and active = 1 GROUP BY (database, table)")
explain_estimates_result = node1.query("EXPLAIN ESTIMATES SELECT * FROM test")
explain_estimates_result = node1.query("EXPLAIN ESTIMATE SELECT * FROM test")
assert(system_parts_result == explain_estimates_result)