2019-02-11 19:53:55 +00:00
|
|
|
#include <Interpreters/InterpreterExplainQuery.h>
|
2018-09-21 15:20:23 +00:00
|
|
|
|
2021-10-15 20:18:20 +00:00
|
|
|
#include <QueryPipeline/BlockIO.h>
|
2022-05-20 19:49:31 +00:00
|
|
|
#include <QueryPipeline/QueryPipelineBuilder.h>
|
2021-09-15 19:35:48 +00:00
|
|
|
#include <Processors/Sources/SourceFromSingleChunk.h>
|
2018-09-21 15:20:23 +00:00
|
|
|
#include <DataTypes/DataTypeString.h>
|
2020-02-05 08:22:25 +00:00
|
|
|
#include <Interpreters/InDepthNodeVisitor.h>
|
2019-02-11 19:53:55 +00:00
|
|
|
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
2020-06-15 12:36:10 +00:00
|
|
|
#include <Interpreters/InterpreterSelectQuery.h>
|
2022-08-15 16:34:10 +00:00
|
|
|
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
2022-01-10 19:01:41 +00:00
|
|
|
#include <Interpreters/InterpreterInsertQuery.h>
|
2020-05-20 20:16:32 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2021-12-16 08:26:37 +00:00
|
|
|
#include <Interpreters/TableOverrideUtils.h>
|
2022-02-14 19:47:17 +00:00
|
|
|
#include <Interpreters/MergeTreeTransaction.h>
|
2021-04-14 14:07:56 +00:00
|
|
|
#include <Formats/FormatFactory.h>
|
2018-10-09 14:32:11 +00:00
|
|
|
#include <Parsers/DumpASTNode.h>
|
2019-02-11 19:53:55 +00:00
|
|
|
#include <Parsers/queryToString.h>
|
2020-02-05 08:22:25 +00:00
|
|
|
#include <Parsers/ASTExplainQuery.h>
|
2021-12-16 08:26:37 +00:00
|
|
|
#include <Parsers/ASTFunction.h>
|
2020-06-15 12:36:10 +00:00
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
2021-11-26 18:27:16 +00:00
|
|
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
2022-01-10 19:01:41 +00:00
|
|
|
#include <Parsers/ASTSetQuery.h>
|
2020-02-06 00:30:14 +00:00
|
|
|
|
2020-02-05 08:22:25 +00:00
|
|
|
#include <Storages/StorageView.h>
|
2020-06-22 14:37:42 +00:00
|
|
|
#include <Processors/QueryPlan/QueryPlan.h>
|
2021-03-04 17:38:12 +00:00
|
|
|
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
|
|
|
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
2021-10-16 14:03:50 +00:00
|
|
|
#include <QueryPipeline/printPipeline.h>
|
2018-09-21 15:20:23 +00:00
|
|
|
|
2021-04-14 14:07:56 +00:00
|
|
|
#include <Common/JSONBuilder.h>
|
2021-04-09 14:44:58 +00:00
|
|
|
|
2022-07-14 11:20:16 +00:00
|
|
|
#include <Analyzer/QueryTreeBuilder.h>
|
|
|
|
#include <Analyzer/QueryTreePassManager.h>
|
|
|
|
|
2018-09-21 15:20:23 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2020-06-22 14:37:42 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int INCORRECT_QUERY;
|
2020-06-23 11:26:06 +00:00
|
|
|
extern const int INVALID_SETTING_VALUE;
|
|
|
|
extern const int UNKNOWN_SETTING;
|
2020-06-23 13:52:36 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2023-03-29 14:59:01 +00:00
|
|
|
extern const int NOT_IMPLEMENTED;
|
2020-06-22 14:37:42 +00:00
|
|
|
}
|
|
|
|
|
2020-02-05 08:22:25 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
struct ExplainAnalyzedSyntaxMatcher
|
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
struct Data : public WithContext
|
2020-02-05 08:22:25 +00:00
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
explicit Data(ContextPtr context_) : WithContext(context_) {}
|
2020-02-05 08:22:25 +00:00
|
|
|
};
|
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
static bool needChildVisit(ASTPtr & node, ASTPtr &)
|
2020-02-05 08:22:25 +00:00
|
|
|
{
|
2020-06-15 12:36:10 +00:00
|
|
|
return !node->as<ASTSelectQuery>();
|
2020-02-05 08:22:25 +00:00
|
|
|
}
|
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
static void visit(ASTPtr & ast, Data & data)
|
2020-02-05 08:22:25 +00:00
|
|
|
{
|
2020-06-15 12:36:10 +00:00
|
|
|
if (auto * select = ast->as<ASTSelectQuery>())
|
|
|
|
visit(*select, ast, data);
|
2020-02-05 08:22:25 +00:00
|
|
|
}
|
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
static void visit(ASTSelectQuery & select, ASTPtr & node, Data & data)
|
2020-02-05 08:22:25 +00:00
|
|
|
{
|
2020-06-15 12:36:10 +00:00
|
|
|
InterpreterSelectQuery interpreter(
|
2021-04-10 23:33:54 +00:00
|
|
|
node, data.getContext(), SelectQueryOptions(QueryProcessingStage::FetchColumns).analyze().modify());
|
2020-02-05 08:22:25 +00:00
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
const SelectQueryInfo & query_info = interpreter.getQueryInfo();
|
|
|
|
if (query_info.view_query)
|
2020-02-05 08:22:25 +00:00
|
|
|
{
|
2020-06-15 12:36:10 +00:00
|
|
|
ASTPtr tmp;
|
2022-10-19 16:30:03 +00:00
|
|
|
StorageView::replaceWithSubquery(select, query_info.view_query->clone(), tmp, query_info.is_parameterized_view);
|
2020-02-05 08:22:25 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
using ExplainAnalyzedSyntaxVisitor = InDepthNodeVisitor<ExplainAnalyzedSyntaxMatcher, true>;
|
|
|
|
|
|
|
|
}
|
|
|
|
|
2018-09-21 15:20:23 +00:00
|
|
|
BlockIO InterpreterExplainQuery::execute()
|
|
|
|
{
|
|
|
|
BlockIO res;
|
2021-09-15 19:35:48 +00:00
|
|
|
res.pipeline = executeImpl();
|
2018-09-21 15:20:23 +00:00
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-07-09 10:29:45 +00:00
|
|
|
Block InterpreterExplainQuery::getSampleBlock(const ASTExplainQuery::ExplainKind kind)
|
2018-09-21 15:20:23 +00:00
|
|
|
{
|
2021-07-09 10:29:45 +00:00
|
|
|
if (kind == ASTExplainQuery::ExplainKind::QueryEstimates)
|
|
|
|
{
|
|
|
|
auto cols = NamesAndTypes{
|
|
|
|
{"database", std::make_shared<DataTypeString>()},
|
|
|
|
{"table", std::make_shared<DataTypeString>()},
|
|
|
|
{"parts", std::make_shared<DataTypeUInt64>()},
|
|
|
|
{"rows", std::make_shared<DataTypeUInt64>()},
|
2021-07-09 11:17:03 +00:00
|
|
|
{"marks", std::make_shared<DataTypeUInt64>()},
|
2021-07-09 10:29:45 +00:00
|
|
|
};
|
|
|
|
return Block({
|
|
|
|
{cols[0].type->createColumn(), cols[0].type, cols[0].name},
|
|
|
|
{cols[1].type->createColumn(), cols[1].type, cols[1].name},
|
|
|
|
{cols[2].type->createColumn(), cols[2].type, cols[2].name},
|
|
|
|
{cols[3].type->createColumn(), cols[3].type, cols[3].name},
|
|
|
|
{cols[4].type->createColumn(), cols[4].type, cols[4].name},
|
|
|
|
});
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
Block res;
|
|
|
|
ColumnWithTypeAndName col;
|
|
|
|
col.name = "explain";
|
|
|
|
col.type = std::make_shared<DataTypeString>();
|
|
|
|
col.column = col.type->createColumn();
|
|
|
|
res.insert(col);
|
|
|
|
return res;
|
|
|
|
}
|
2018-09-21 15:20:23 +00:00
|
|
|
}
|
|
|
|
|
2020-06-27 09:14:38 +00:00
|
|
|
/// Split str by line feed and write as separate row to ColumnString.
|
2020-06-22 14:37:42 +00:00
|
|
|
static void fillColumn(IColumn & column, const std::string & str)
|
|
|
|
{
|
|
|
|
size_t start = 0;
|
|
|
|
size_t end = 0;
|
|
|
|
size_t size = str.size();
|
|
|
|
|
|
|
|
while (end < size)
|
|
|
|
{
|
|
|
|
if (str[end] == '\n')
|
|
|
|
{
|
|
|
|
column.insertData(str.data() + start, end - start);
|
|
|
|
start = end + 1;
|
|
|
|
}
|
|
|
|
|
|
|
|
++end;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (start < end)
|
|
|
|
column.insertData(str.data() + start, end - start);
|
|
|
|
}
|
2018-09-21 15:20:23 +00:00
|
|
|
|
2020-06-23 13:52:36 +00:00
|
|
|
namespace
|
|
|
|
{
|
2020-06-23 11:26:06 +00:00
|
|
|
|
2020-06-27 09:14:38 +00:00
|
|
|
/// Settings. Different for each explain type.
|
|
|
|
|
2022-03-10 10:04:35 +00:00
|
|
|
struct QueryASTSettings
|
|
|
|
{
|
|
|
|
bool graph = false;
|
2022-07-19 12:40:50 +00:00
|
|
|
bool optimize = false;
|
2022-03-10 10:04:35 +00:00
|
|
|
|
|
|
|
constexpr static char name[] = "AST";
|
|
|
|
|
|
|
|
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
|
|
|
|
{
|
|
|
|
{"graph", graph},
|
2022-07-19 12:40:50 +00:00
|
|
|
{"optimize", optimize}
|
2022-03-10 10:04:35 +00:00
|
|
|
};
|
2022-07-14 11:20:16 +00:00
|
|
|
|
|
|
|
std::unordered_map<std::string, std::reference_wrapper<Int64>> integer_settings;
|
|
|
|
};
|
|
|
|
|
|
|
|
struct QueryTreeSettings
|
|
|
|
{
|
2022-11-21 15:05:52 +00:00
|
|
|
bool run_passes = true;
|
2022-07-19 10:54:45 +00:00
|
|
|
bool dump_passes = false;
|
|
|
|
bool dump_ast = false;
|
2022-07-14 11:20:16 +00:00
|
|
|
Int64 passes = -1;
|
|
|
|
|
2022-07-19 10:54:45 +00:00
|
|
|
constexpr static char name[] = "QUERY TREE";
|
2022-07-14 11:20:16 +00:00
|
|
|
|
|
|
|
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
|
|
|
|
{
|
|
|
|
{"run_passes", run_passes},
|
2022-07-19 10:54:45 +00:00
|
|
|
{"dump_passes", dump_passes},
|
|
|
|
{"dump_ast", dump_ast}
|
2022-07-14 11:20:16 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
std::unordered_map<std::string, std::reference_wrapper<Int64>> integer_settings =
|
|
|
|
{
|
|
|
|
{"passes", passes}
|
|
|
|
};
|
2022-03-10 10:04:35 +00:00
|
|
|
};
|
|
|
|
|
2020-06-26 13:58:28 +00:00
|
|
|
struct QueryPlanSettings
|
2020-06-23 11:26:06 +00:00
|
|
|
{
|
2020-06-26 13:58:28 +00:00
|
|
|
QueryPlan::ExplainPlanOptions query_plan_options;
|
2020-06-26 15:07:49 +00:00
|
|
|
|
2021-02-26 16:29:56 +00:00
|
|
|
/// Apply query plan optimizations.
|
2020-09-23 17:42:46 +00:00
|
|
|
bool optimize = true;
|
2021-04-16 12:44:27 +00:00
|
|
|
bool json = false;
|
2020-09-23 17:42:46 +00:00
|
|
|
|
2020-06-26 13:58:28 +00:00
|
|
|
constexpr static char name[] = "PLAN";
|
2020-06-23 13:52:36 +00:00
|
|
|
|
|
|
|
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
|
|
|
|
{
|
|
|
|
{"header", query_plan_options.header},
|
2020-06-23 14:11:15 +00:00
|
|
|
{"description", query_plan_options.description},
|
2020-09-23 17:42:46 +00:00
|
|
|
{"actions", query_plan_options.actions},
|
2021-04-16 11:11:45 +00:00
|
|
|
{"indexes", query_plan_options.indexes},
|
2021-04-16 12:44:27 +00:00
|
|
|
{"optimize", optimize},
|
2022-08-04 22:02:53 +00:00
|
|
|
{"json", json},
|
2022-08-15 15:14:59 +00:00
|
|
|
{"sorting", query_plan_options.sorting},
|
2020-06-23 13:52:36 +00:00
|
|
|
};
|
2022-07-14 11:20:16 +00:00
|
|
|
|
|
|
|
std::unordered_map<std::string, std::reference_wrapper<Int64>> integer_settings;
|
2020-06-26 13:58:28 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
struct QueryPipelineSettings
|
|
|
|
{
|
|
|
|
QueryPlan::ExplainPipelineOptions query_pipeline_options;
|
2020-06-26 15:07:49 +00:00
|
|
|
bool graph = false;
|
2020-06-26 17:56:33 +00:00
|
|
|
bool compact = true;
|
2020-06-26 15:07:49 +00:00
|
|
|
|
2020-06-26 13:58:28 +00:00
|
|
|
constexpr static char name[] = "PIPELINE";
|
|
|
|
|
|
|
|
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
|
|
|
|
{
|
|
|
|
{"header", query_pipeline_options.header},
|
2020-06-26 15:07:49 +00:00
|
|
|
{"graph", graph},
|
2020-06-26 17:56:33 +00:00
|
|
|
{"compact", compact},
|
2020-06-26 13:58:28 +00:00
|
|
|
};
|
2022-07-14 11:20:16 +00:00
|
|
|
|
|
|
|
std::unordered_map<std::string, std::reference_wrapper<Int64>> integer_settings;
|
2020-06-26 13:58:28 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
template <typename Settings>
|
|
|
|
struct ExplainSettings : public Settings
|
|
|
|
{
|
|
|
|
using Settings::boolean_settings;
|
2022-07-14 11:20:16 +00:00
|
|
|
using Settings::integer_settings;
|
2020-06-23 13:52:36 +00:00
|
|
|
|
2020-06-27 15:30:19 +00:00
|
|
|
bool has(const std::string & name_) const
|
2022-07-14 11:20:16 +00:00
|
|
|
{
|
|
|
|
return hasBooleanSetting(name_) || hasIntegerSetting(name_);
|
|
|
|
}
|
|
|
|
|
|
|
|
bool hasBooleanSetting(const std::string & name_) const
|
2020-06-23 13:52:36 +00:00
|
|
|
{
|
2020-06-27 15:30:19 +00:00
|
|
|
return boolean_settings.count(name_) > 0;
|
2020-06-23 13:52:36 +00:00
|
|
|
}
|
|
|
|
|
2022-07-14 11:20:16 +00:00
|
|
|
bool hasIntegerSetting(const std::string & name_) const
|
|
|
|
{
|
|
|
|
return integer_settings.count(name_) > 0;
|
|
|
|
}
|
|
|
|
|
2020-06-27 15:30:19 +00:00
|
|
|
void setBooleanSetting(const std::string & name_, bool value)
|
2020-06-23 13:52:36 +00:00
|
|
|
{
|
2020-06-27 15:30:19 +00:00
|
|
|
auto it = boolean_settings.find(name_);
|
2020-06-23 13:52:36 +00:00
|
|
|
if (it == boolean_settings.end())
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown setting for ExplainSettings: {}", name_);
|
2020-06-23 13:52:36 +00:00
|
|
|
|
|
|
|
it->second.get() = value;
|
|
|
|
}
|
2020-06-23 11:26:06 +00:00
|
|
|
|
2022-07-14 11:20:16 +00:00
|
|
|
void setIntegerSetting(const std::string & name_, Int64 value)
|
|
|
|
{
|
|
|
|
auto it = integer_settings.find(name_);
|
|
|
|
if (it == integer_settings.end())
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown setting for ExplainSettings: {}", name_);
|
2022-07-14 11:20:16 +00:00
|
|
|
|
|
|
|
it->second.get() = value;
|
|
|
|
}
|
|
|
|
|
2020-06-23 13:52:36 +00:00
|
|
|
std::string getSettingsList() const
|
2020-06-23 11:26:06 +00:00
|
|
|
{
|
|
|
|
std::string res;
|
2020-06-23 13:52:36 +00:00
|
|
|
for (const auto & setting : boolean_settings)
|
2020-06-23 11:26:06 +00:00
|
|
|
{
|
|
|
|
if (!res.empty())
|
|
|
|
res += ", ";
|
|
|
|
|
2020-06-23 13:52:36 +00:00
|
|
|
res += setting.first;
|
2020-06-23 11:26:06 +00:00
|
|
|
}
|
2022-07-14 11:20:16 +00:00
|
|
|
for (const auto & setting : integer_settings)
|
|
|
|
{
|
|
|
|
if (!res.empty())
|
|
|
|
res += ", ";
|
|
|
|
|
|
|
|
res += setting.first;
|
|
|
|
}
|
2020-06-23 11:26:06 +00:00
|
|
|
|
|
|
|
return res;
|
2020-06-23 13:52:36 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2023-01-13 12:18:58 +00:00
|
|
|
struct QuerySyntaxSettings
|
|
|
|
{
|
|
|
|
bool oneline = false;
|
|
|
|
|
|
|
|
constexpr static char name[] = "SYNTAX";
|
|
|
|
|
|
|
|
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
|
|
|
|
{
|
|
|
|
{"oneline", oneline},
|
|
|
|
};
|
|
|
|
|
|
|
|
std::unordered_map<std::string, std::reference_wrapper<Int64>> integer_settings;
|
|
|
|
};
|
|
|
|
|
2020-06-26 13:58:28 +00:00
|
|
|
template <typename Settings>
|
|
|
|
ExplainSettings<Settings> checkAndGetSettings(const ASTPtr & ast_settings)
|
2020-06-23 13:52:36 +00:00
|
|
|
{
|
|
|
|
if (!ast_settings)
|
|
|
|
return {};
|
|
|
|
|
2020-06-26 13:58:28 +00:00
|
|
|
ExplainSettings<Settings> settings;
|
2020-06-23 11:26:06 +00:00
|
|
|
const auto & set_query = ast_settings->as<ASTSetQuery &>();
|
|
|
|
|
|
|
|
for (const auto & change : set_query.changes)
|
|
|
|
{
|
2020-06-23 13:52:36 +00:00
|
|
|
if (!settings.has(change.name))
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting \"{}\" for EXPLAIN {} query. "
|
|
|
|
"Supported settings: {}", change.name, Settings::name, settings.getSettingsList());
|
2020-06-23 11:26:06 +00:00
|
|
|
|
|
|
|
if (change.value.getType() != Field::Types::UInt64)
|
2021-09-06 15:59:46 +00:00
|
|
|
throw Exception(ErrorCodes::INVALID_SETTING_VALUE,
|
2022-07-14 11:20:16 +00:00
|
|
|
"Invalid type {} for setting \"{}\" only integer settings are supported",
|
2021-09-06 15:59:46 +00:00
|
|
|
change.value.getTypeName(), change.name);
|
2020-06-23 11:26:06 +00:00
|
|
|
|
2022-07-14 11:20:16 +00:00
|
|
|
if (settings.hasBooleanSetting(change.name))
|
|
|
|
{
|
|
|
|
auto value = change.value.get<UInt64>();
|
|
|
|
if (value > 1)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid value {} for setting \"{}\". "
|
|
|
|
"Expected boolean type", value, change.name);
|
2020-06-23 11:26:06 +00:00
|
|
|
|
2022-07-14 11:20:16 +00:00
|
|
|
settings.setBooleanSetting(change.name, value);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
auto value = change.value.get<UInt64>();
|
|
|
|
settings.setIntegerSetting(change.name, value);
|
|
|
|
}
|
2020-06-23 11:26:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return settings;
|
|
|
|
}
|
|
|
|
|
2020-06-26 13:58:28 +00:00
|
|
|
}
|
|
|
|
|
2021-09-15 19:35:48 +00:00
|
|
|
QueryPipeline InterpreterExplainQuery::executeImpl()
|
2018-09-21 15:20:23 +00:00
|
|
|
{
|
2021-07-10 23:14:03 +00:00
|
|
|
const auto & ast = query->as<const ASTExplainQuery &>();
|
2020-06-22 14:37:42 +00:00
|
|
|
|
2021-07-09 10:29:45 +00:00
|
|
|
Block sample_block = getSampleBlock(ast.getKind());
|
2019-02-11 19:53:55 +00:00
|
|
|
MutableColumns res_columns = sample_block.cloneEmptyColumns();
|
2018-09-21 15:20:23 +00:00
|
|
|
|
2020-11-09 16:05:40 +00:00
|
|
|
WriteBufferFromOwnString buf;
|
2021-04-16 16:36:59 +00:00
|
|
|
bool single_line = false;
|
2021-12-16 08:26:37 +00:00
|
|
|
bool insert_buf = true;
|
2018-09-21 15:20:23 +00:00
|
|
|
|
2022-04-07 11:43:49 +00:00
|
|
|
SelectQueryOptions options;
|
|
|
|
options.setExplain();
|
|
|
|
|
2021-12-16 08:26:37 +00:00
|
|
|
switch (ast.getKind())
|
2019-02-11 19:53:55 +00:00
|
|
|
{
|
2021-12-16 08:26:37 +00:00
|
|
|
case ASTExplainQuery::ParsedAST:
|
|
|
|
{
|
2022-03-10 10:04:35 +00:00
|
|
|
auto settings = checkAndGetSettings<QueryASTSettings>(ast.getSettings());
|
2022-07-19 12:40:50 +00:00
|
|
|
if (settings.optimize)
|
2022-07-06 14:21:59 +00:00
|
|
|
{
|
|
|
|
ExplainAnalyzedSyntaxVisitor::Data data(getContext());
|
|
|
|
ExplainAnalyzedSyntaxVisitor(data).visit(query);
|
|
|
|
}
|
|
|
|
|
2022-03-10 10:04:35 +00:00
|
|
|
if (settings.graph)
|
|
|
|
dumpASTInDotFormat(*ast.getExplainedQuery(), buf);
|
|
|
|
else
|
|
|
|
dumpAST(*ast.getExplainedQuery(), buf);
|
2021-12-16 08:26:37 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTExplainQuery::AnalyzedSyntax:
|
|
|
|
{
|
2023-01-13 12:18:58 +00:00
|
|
|
auto settings = checkAndGetSettings<QuerySyntaxSettings>(ast.getSettings());
|
2020-06-26 15:07:49 +00:00
|
|
|
|
2021-12-16 08:26:37 +00:00
|
|
|
ExplainAnalyzedSyntaxVisitor::Data data(getContext());
|
|
|
|
ExplainAnalyzedSyntaxVisitor(data).visit(query);
|
2020-02-05 08:22:25 +00:00
|
|
|
|
2023-01-13 12:18:58 +00:00
|
|
|
ast.getExplainedQuery()->format(IAST::FormatSettings(buf, settings.oneline));
|
2021-12-16 08:26:37 +00:00
|
|
|
break;
|
|
|
|
}
|
2022-07-14 11:20:16 +00:00
|
|
|
case ASTExplainQuery::QueryTree:
|
|
|
|
{
|
2023-03-29 14:59:01 +00:00
|
|
|
if (!getContext()->getSettingsRef().allow_experimental_analyzer)
|
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
|
|
|
"EXPLAIN QUERY TREE is only supported with a new analyzer. Set allow_experimental_analyzer = 1.");
|
|
|
|
|
2022-07-14 11:20:16 +00:00
|
|
|
if (ast.getExplainedQuery()->as<ASTSelectWithUnionQuery>() == nullptr)
|
2022-10-19 16:49:17 +00:00
|
|
|
throw Exception(ErrorCodes::INCORRECT_QUERY, "Only SELECT is supported for EXPLAIN QUERY TREE query");
|
2022-07-14 11:20:16 +00:00
|
|
|
|
|
|
|
auto settings = checkAndGetSettings<QueryTreeSettings>(ast.getSettings());
|
2022-09-27 15:04:03 +00:00
|
|
|
auto query_tree = buildQueryTree(ast.getExplainedQuery(), getContext());
|
2022-07-14 11:20:16 +00:00
|
|
|
|
|
|
|
if (settings.run_passes)
|
|
|
|
{
|
|
|
|
auto query_tree_pass_manager = QueryTreePassManager(getContext());
|
|
|
|
addQueryTreePasses(query_tree_pass_manager);
|
|
|
|
|
2022-07-19 10:54:45 +00:00
|
|
|
size_t pass_index = settings.passes < 0 ? query_tree_pass_manager.getPasses().size() : static_cast<size_t>(settings.passes);
|
|
|
|
|
|
|
|
if (settings.dump_passes)
|
2022-07-14 11:20:16 +00:00
|
|
|
{
|
|
|
|
query_tree_pass_manager.dump(buf, pass_index);
|
|
|
|
if (pass_index > 0)
|
|
|
|
buf << '\n';
|
|
|
|
}
|
|
|
|
|
2022-07-19 10:54:45 +00:00
|
|
|
query_tree_pass_manager.run(query_tree, pass_index);
|
|
|
|
|
|
|
|
query_tree->dumpTree(buf);
|
2022-07-14 11:20:16 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2022-07-19 10:54:45 +00:00
|
|
|
query_tree->dumpTree(buf);
|
2022-07-14 11:20:16 +00:00
|
|
|
}
|
|
|
|
|
2022-07-19 10:54:45 +00:00
|
|
|
if (settings.dump_ast)
|
2022-07-14 11:20:16 +00:00
|
|
|
{
|
2022-08-25 18:35:16 +00:00
|
|
|
buf << '\n';
|
2022-07-14 11:20:16 +00:00
|
|
|
buf << '\n';
|
|
|
|
query_tree->toAST()->format(IAST::FormatSettings(buf, false));
|
|
|
|
}
|
|
|
|
|
|
|
|
break;
|
|
|
|
}
|
2021-12-16 08:26:37 +00:00
|
|
|
case ASTExplainQuery::QueryPlan:
|
|
|
|
{
|
|
|
|
if (!dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::INCORRECT_QUERY, "Only SELECT is supported for EXPLAIN query");
|
2020-06-22 14:37:42 +00:00
|
|
|
|
2021-12-16 08:26:37 +00:00
|
|
|
auto settings = checkAndGetSettings<QueryPlanSettings>(ast.getSettings());
|
|
|
|
QueryPlan plan;
|
2020-06-22 14:37:42 +00:00
|
|
|
|
2022-11-04 15:44:31 +00:00
|
|
|
ContextPtr context;
|
|
|
|
|
2022-10-25 10:35:25 +00:00
|
|
|
if (getContext()->getSettingsRef().allow_experimental_analyzer)
|
2022-08-15 16:34:10 +00:00
|
|
|
{
|
2022-12-23 17:45:28 +00:00
|
|
|
InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), getContext(), options);
|
2022-11-04 15:44:31 +00:00
|
|
|
context = interpreter.getContext();
|
2022-08-15 16:34:10 +00:00
|
|
|
plan = std::move(interpreter).extractQueryPlan();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), options);
|
|
|
|
interpreter.buildQueryPlan(plan);
|
2022-11-04 15:44:31 +00:00
|
|
|
context = interpreter.getContext();
|
2022-08-15 16:34:10 +00:00
|
|
|
}
|
2020-06-22 14:37:42 +00:00
|
|
|
|
2021-12-16 08:26:37 +00:00
|
|
|
if (settings.optimize)
|
2022-11-04 15:44:31 +00:00
|
|
|
plan.optimize(QueryPlanOptimizationSettings::fromContext(context));
|
2020-07-28 14:53:02 +00:00
|
|
|
|
2021-12-16 08:26:37 +00:00
|
|
|
if (settings.json)
|
|
|
|
{
|
|
|
|
/// Add extra layers to make plan look more like from postgres.
|
|
|
|
auto plan_map = std::make_unique<JSONBuilder::JSONMap>();
|
|
|
|
plan_map->add("Plan", plan.explainPlan(settings.query_plan_options));
|
|
|
|
auto plan_array = std::make_unique<JSONBuilder::JSONArray>();
|
|
|
|
plan_array->add(std::move(plan_map));
|
2021-04-14 14:07:56 +00:00
|
|
|
|
2022-09-09 20:15:38 +00:00
|
|
|
auto format_settings = getFormatSettings(getContext());
|
2021-12-16 08:26:37 +00:00
|
|
|
format_settings.json.quote_64bit_integers = false;
|
2021-04-14 14:07:56 +00:00
|
|
|
|
2021-12-16 08:26:37 +00:00
|
|
|
JSONBuilder::FormatSettings json_format_settings{.settings = format_settings};
|
|
|
|
JSONBuilder::FormatContext format_context{.out = buf};
|
2021-04-14 14:07:56 +00:00
|
|
|
|
2021-12-16 08:26:37 +00:00
|
|
|
plan_array->format(json_format_settings, format_context);
|
2021-04-16 16:36:59 +00:00
|
|
|
|
2021-12-16 08:26:37 +00:00
|
|
|
single_line = true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
plan.explainPlan(buf, settings.query_plan_options);
|
|
|
|
break;
|
2021-04-09 14:44:58 +00:00
|
|
|
}
|
2021-12-16 08:26:37 +00:00
|
|
|
case ASTExplainQuery::QueryPipeline:
|
|
|
|
{
|
|
|
|
if (dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
|
|
|
|
{
|
|
|
|
auto settings = checkAndGetSettings<QueryPipelineSettings>(ast.getSettings());
|
|
|
|
QueryPlan plan;
|
2022-11-04 15:44:31 +00:00
|
|
|
ContextPtr context;
|
2021-12-16 08:26:37 +00:00
|
|
|
|
2022-10-25 10:35:25 +00:00
|
|
|
if (getContext()->getSettingsRef().allow_experimental_analyzer)
|
2022-08-15 16:34:10 +00:00
|
|
|
{
|
2022-12-23 17:45:28 +00:00
|
|
|
InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), getContext(), options);
|
2022-11-04 15:44:31 +00:00
|
|
|
context = interpreter.getContext();
|
2022-08-15 16:34:10 +00:00
|
|
|
plan = std::move(interpreter).extractQueryPlan();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), options);
|
|
|
|
interpreter.buildQueryPlan(plan);
|
2022-11-04 15:44:31 +00:00
|
|
|
context = interpreter.getContext();
|
2022-08-15 16:34:10 +00:00
|
|
|
}
|
|
|
|
|
2021-12-16 08:26:37 +00:00
|
|
|
auto pipeline = plan.buildQueryPipeline(
|
2022-11-04 15:44:31 +00:00
|
|
|
QueryPlanOptimizationSettings::fromContext(context),
|
|
|
|
BuildQueryPipelineSettings::fromContext(context));
|
2021-12-16 08:26:37 +00:00
|
|
|
|
|
|
|
if (settings.graph)
|
|
|
|
{
|
|
|
|
/// Pipe holds QueryPlan, should not go out-of-scope
|
2022-05-23 13:46:57 +00:00
|
|
|
QueryPlanResourceHolder resources;
|
|
|
|
auto pipe = QueryPipelineBuilder::getPipe(std::move(*pipeline), resources);
|
2021-12-16 08:26:37 +00:00
|
|
|
const auto & processors = pipe.getProcessors();
|
|
|
|
|
2023-03-11 15:43:19 +00:00
|
|
|
if (settings.compact)
|
|
|
|
printPipelineCompact(processors, buf, settings.query_pipeline_options.header);
|
|
|
|
else
|
|
|
|
printPipeline(processors, buf);
|
2021-12-16 08:26:37 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
plan.explainPipeline(buf, settings.query_pipeline_options);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else if (dynamic_cast<const ASTInsertQuery *>(ast.getExplainedQuery().get()))
|
|
|
|
{
|
|
|
|
InterpreterInsertQuery insert(ast.getExplainedQuery(), getContext());
|
|
|
|
auto io = insert.execute();
|
|
|
|
printPipeline(io.pipeline.getProcessors(), buf);
|
|
|
|
}
|
|
|
|
else
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::INCORRECT_QUERY, "Only SELECT and INSERT is supported for EXPLAIN PIPELINE query");
|
2021-12-16 08:26:37 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTExplainQuery::QueryEstimates:
|
2021-09-03 17:29:36 +00:00
|
|
|
{
|
2021-12-16 08:26:37 +00:00
|
|
|
if (!dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::INCORRECT_QUERY, "Only SELECT is supported for EXPLAIN ESTIMATE query");
|
2021-12-16 08:26:37 +00:00
|
|
|
|
|
|
|
auto settings = checkAndGetSettings<QueryPlanSettings>(ast.getSettings());
|
2021-09-03 17:29:36 +00:00
|
|
|
QueryPlan plan;
|
2022-11-04 15:44:31 +00:00
|
|
|
ContextPtr context;
|
2020-06-25 09:39:17 +00:00
|
|
|
|
2021-09-03 17:29:36 +00:00
|
|
|
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions());
|
|
|
|
interpreter.buildQueryPlan(plan);
|
2022-11-04 15:44:31 +00:00
|
|
|
context = interpreter.getContext();
|
2023-08-14 23:24:41 +00:00
|
|
|
// Collect the selected marks, rows, parts during build query pipeline.
|
|
|
|
// Hold on to the returned QueryPipelineBuilderPtr because `plan` may have pointers into
|
|
|
|
// it (through QueryPlanResourceHolder).
|
|
|
|
auto builder = plan.buildQueryPipeline(
|
2022-11-04 15:44:31 +00:00
|
|
|
QueryPlanOptimizationSettings::fromContext(context),
|
|
|
|
BuildQueryPipelineSettings::fromContext(context));
|
2020-06-25 09:39:17 +00:00
|
|
|
|
2021-12-16 08:26:37 +00:00
|
|
|
plan.explainEstimate(res_columns);
|
|
|
|
insert_buf = false;
|
|
|
|
break;
|
2021-09-03 17:29:36 +00:00
|
|
|
}
|
2021-12-16 08:26:37 +00:00
|
|
|
case ASTExplainQuery::TableOverride:
|
2020-06-26 15:07:49 +00:00
|
|
|
{
|
2021-12-16 08:26:37 +00:00
|
|
|
if (auto * table_function = ast.getTableFunction()->as<ASTFunction>(); !table_function || table_function->name != "mysql")
|
|
|
|
{
|
|
|
|
throw Exception(ErrorCodes::INCORRECT_QUERY, "EXPLAIN TABLE OVERRIDE is not supported for the {}() table function", table_function->name);
|
|
|
|
}
|
|
|
|
auto storage = getContext()->getQueryContext()->executeTableFunction(ast.getTableFunction());
|
|
|
|
auto metadata_snapshot = storage->getInMemoryMetadata();
|
|
|
|
TableOverrideAnalyzer::Result override_info;
|
|
|
|
TableOverrideAnalyzer override_analyzer(ast.getTableOverride());
|
|
|
|
override_analyzer.analyze(metadata_snapshot, override_info);
|
|
|
|
override_info.appendTo(buf);
|
|
|
|
break;
|
2020-06-26 15:07:49 +00:00
|
|
|
}
|
2022-02-14 19:47:17 +00:00
|
|
|
case ASTExplainQuery::CurrentTransaction:
|
|
|
|
{
|
|
|
|
if (ast.getSettings())
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::UNKNOWN_SETTING, "Settings are not supported for EXPLAIN CURRENT TRANSACTION query.");
|
2022-02-14 19:47:17 +00:00
|
|
|
|
|
|
|
if (auto txn = getContext()->getCurrentTransaction())
|
|
|
|
{
|
|
|
|
String dump = txn->dumpDescription();
|
|
|
|
buf.write(dump.data(), dump.size());
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
writeCString("<no current transaction>", buf);
|
|
|
|
}
|
|
|
|
|
|
|
|
break;
|
|
|
|
}
|
2021-07-09 10:29:45 +00:00
|
|
|
}
|
2021-12-16 08:26:37 +00:00
|
|
|
if (insert_buf)
|
2021-07-09 10:29:45 +00:00
|
|
|
{
|
|
|
|
if (single_line)
|
|
|
|
res_columns[0]->insertData(buf.str().data(), buf.str().size());
|
|
|
|
else
|
|
|
|
fillColumn(*res_columns[0], buf.str());
|
|
|
|
}
|
2018-09-21 15:20:23 +00:00
|
|
|
|
2021-09-15 19:35:48 +00:00
|
|
|
return QueryPipeline(std::make_shared<SourceFromSingleChunk>(sample_block.cloneWithColumns(std::move(res_columns))));
|
2018-09-21 15:20:23 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|