ClickHouse/src/Processors/QueryPlan/QueryPlan.cpp

510 lines
15 KiB
C++
Raw Normal View History

2022-03-24 11:53:58 +00:00
#include <stack>
#include <Common/JSONBuilder.h>
2020-11-16 14:57:56 +00:00
#include <Interpreters/ActionsDAG.h>
2020-08-12 13:30:02 +00:00
#include <Interpreters/ArrayJoinAction.h>
2022-03-24 11:53:58 +00:00
#include <IO/Operators.h>
#include <IO/WriteBuffer.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <Processors/QueryPlan/IQueryPlanStep.h>
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
2021-03-04 17:38:12 +00:00
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
2022-03-24 11:53:58 +00:00
#include <Processors/QueryPlan/QueryPlan.h>
2021-07-09 10:29:45 +00:00
#include <Processors/QueryPlan/ReadFromMergeTree.h>
2022-03-24 11:53:58 +00:00
#include <QueryPipeline/QueryPipelineBuilder.h>
2020-06-08 09:14:58 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
2020-06-22 14:37:42 +00:00
QueryPlan::QueryPlan() = default;
QueryPlan::~QueryPlan() = default;
QueryPlan::QueryPlan(QueryPlan &&) noexcept = default;
QueryPlan & QueryPlan::operator=(QueryPlan &&) noexcept = default;
2020-06-08 09:14:58 +00:00
void QueryPlan::checkInitialized() const
{
if (!isInitialized())
throw Exception("QueryPlan was not initialized", ErrorCodes::LOGICAL_ERROR);
}
void QueryPlan::checkNotCompleted() const
{
if (isCompleted())
throw Exception("QueryPlan was already completed", ErrorCodes::LOGICAL_ERROR);
}
bool QueryPlan::isCompleted() const
{
return isInitialized() && !root->step->hasOutputStream();
}
const DataStream & QueryPlan::getCurrentDataStream() const
{
checkInitialized();
checkNotCompleted();
return root->step->getOutputStream();
}
2020-09-15 17:13:13 +00:00
void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vector<std::unique_ptr<QueryPlan>> plans)
{
if (isInitialized())
throw Exception("Cannot unite plans because current QueryPlan is already initialized",
ErrorCodes::LOGICAL_ERROR);
const auto & inputs = step->getInputStreams();
size_t num_inputs = step->getInputStreams().size();
if (num_inputs != plans.size())
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Cannot unite QueryPlans using {} because step has different number of inputs. Has {} plans and {} inputs",
step->getName(),
plans.size(),
num_inputs);
for (size_t i = 0; i < num_inputs; ++i)
{
const auto & step_header = inputs[i].header;
2020-09-15 17:13:13 +00:00
const auto & plan_header = plans[i]->getCurrentDataStream().header;
if (!blocksHaveEqualStructure(step_header, plan_header))
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Cannot unite QueryPlans using {} because it has incompatible header with plan {} plan header: {} step header: {}",
step->getName(),
root->step->getName(),
plan_header.dumpStructure(),
step_header.dumpStructure());
}
for (auto & plan : plans)
2020-09-15 17:13:13 +00:00
nodes.splice(nodes.end(), std::move(plan->nodes));
nodes.emplace_back(Node{.step = std::move(step)});
root = &nodes.back();
for (auto & plan : plans)
2020-09-15 17:13:13 +00:00
root->children.emplace_back(plan->root);
2020-06-19 10:45:48 +00:00
for (auto & plan : plans)
{
2020-09-15 17:13:13 +00:00
max_threads = std::max(max_threads, plan->max_threads);
2022-05-20 19:49:31 +00:00
resources = std::move(plan->resources);
}
}
2020-06-08 09:14:58 +00:00
void QueryPlan::addStep(QueryPlanStepPtr step)
{
checkNotCompleted();
size_t num_input_streams = step->getInputStreams().size();
if (num_input_streams == 0)
{
if (isInitialized())
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Cannot add step {} to QueryPlan because step has no inputs, but QueryPlan is already initialized",
step->getName());
2020-06-08 09:14:58 +00:00
nodes.emplace_back(Node{.step = std::move(step)});
root = &nodes.back();
2020-06-08 09:14:58 +00:00
return;
}
if (num_input_streams == 1)
{
if (!isInitialized())
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Cannot add step {} to QueryPlan because step has input, but QueryPlan is not initialized",
step->getName());
2020-06-08 09:14:58 +00:00
const auto & root_header = root->step->getOutputStream().header;
const auto & step_header = step->getInputStreams().front().header;
if (!blocksHaveEqualStructure(root_header, step_header))
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Cannot add step {} to QueryPlan because it has incompatible header with root step {} root header: {} step header: {}",
step->getName(),
root->step->getName(),
root_header.dumpStructure(),
step_header.dumpStructure());
2020-06-08 09:14:58 +00:00
nodes.emplace_back(Node{.step = std::move(step), .children = {root}});
root = &nodes.back();
return;
}
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Cannot add step {} to QueryPlan because it has {} inputs but {} input expected",
step->getName(),
num_input_streams,
isInitialized() ? 1 : 0);
2020-06-08 09:14:58 +00:00
}
2022-05-23 13:46:57 +00:00
QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline(
2021-03-04 17:38:12 +00:00
const QueryPlanOptimizationSettings & optimization_settings,
const BuildQueryPipelineSettings & build_pipeline_settings)
2020-06-08 09:14:58 +00:00
{
checkInitialized();
optimize(optimization_settings);
2020-06-08 09:14:58 +00:00
struct Frame
{
2021-05-08 15:35:09 +00:00
Node * node = {};
QueryPipelineBuilders pipelines = {};
2020-06-08 09:14:58 +00:00
};
QueryPipelineBuilderPtr last_pipeline;
2020-06-08 09:14:58 +00:00
std::stack<Frame> stack;
2020-06-16 07:56:22 +00:00
stack.push(Frame{.node = root});
2020-06-08 09:14:58 +00:00
while (!stack.empty())
{
auto & frame = stack.top();
if (last_pipeline)
{
2020-06-08 09:14:58 +00:00
frame.pipelines.emplace_back(std::move(last_pipeline));
last_pipeline = nullptr; //-V1048
}
2020-06-08 09:14:58 +00:00
size_t next_child = frame.pipelines.size();
if (next_child == frame.node->children.size())
{
bool limit_max_threads = frame.pipelines.empty();
2021-03-04 17:38:12 +00:00
last_pipeline = frame.node->step->updatePipeline(std::move(frame.pipelines), build_pipeline_settings);
if (limit_max_threads && max_threads)
last_pipeline->limitMaxThreads(max_threads);
2020-06-08 09:14:58 +00:00
stack.pop();
}
else
2020-06-16 07:56:22 +00:00
stack.push(Frame{.node = frame.node->children[next_child]});
2020-06-08 09:14:58 +00:00
}
2022-05-20 19:49:31 +00:00
/// last_pipeline->setProgressCallback(build_pipeline_settings.progress_callback);
last_pipeline->setProcessListElement(build_pipeline_settings.process_list_element);
2022-05-23 13:46:57 +00:00
last_pipeline->addResources(std::move(resources));
2022-05-23 13:46:57 +00:00
return last_pipeline;
2020-06-08 09:14:58 +00:00
}
static void explainStep(const IQueryPlanStep & step, JSONBuilder::JSONMap & map, const QueryPlan::ExplainPlanOptions & options)
2021-04-09 14:44:58 +00:00
{
map.add("Node Type", step.getName());
2021-04-09 14:44:58 +00:00
2021-04-13 07:51:55 +00:00
if (options.description)
{
const auto & description = step.getStepDescription();
if (!description.empty())
map.add("Description", description);
2021-04-13 07:51:55 +00:00
}
if (options.header && step.hasOutputStream())
{
auto header_array = std::make_unique<JSONBuilder::JSONArray>();
2021-04-13 07:51:55 +00:00
for (const auto & output_column : step.getOutputStream().header)
{
auto column_map = std::make_unique<JSONBuilder::JSONMap>();
column_map->add("Name", output_column.name);
2021-04-13 07:51:55 +00:00
if (output_column.type)
column_map->add("Type", output_column.type->getName());
2021-04-13 07:51:55 +00:00
header_array->add(std::move(column_map));
2021-04-13 07:51:55 +00:00
}
map.add("Header", std::move(header_array));
2021-04-13 07:51:55 +00:00
}
2021-04-09 14:44:58 +00:00
2021-04-13 07:51:55 +00:00
if (options.actions)
step.describeActions(map);
if (options.indexes)
step.describeIndexes(map);
2021-04-09 14:44:58 +00:00
}
JSONBuilder::ItemPtr QueryPlan::explainPlan(const ExplainPlanOptions & options)
2021-04-09 14:44:58 +00:00
{
checkInitialized();
struct Frame
{
2021-05-08 15:35:09 +00:00
Node * node = {};
2021-04-09 14:44:58 +00:00
size_t next_child = 0;
std::unique_ptr<JSONBuilder::JSONMap> node_map = {};
std::unique_ptr<JSONBuilder::JSONArray> children_array = {};
2021-04-09 14:44:58 +00:00
};
std::stack<Frame> stack;
stack.push(Frame{.node = root});
std::unique_ptr<JSONBuilder::JSONMap> tree;
2021-04-09 14:44:58 +00:00
while (!stack.empty())
{
auto & frame = stack.top();
if (frame.next_child == 0)
{
if (!frame.node->children.empty())
frame.children_array = std::make_unique<JSONBuilder::JSONArray>();
frame.node_map = std::make_unique<JSONBuilder::JSONMap>();
explainStep(*frame.node->step, *frame.node_map, options);
}
2021-04-09 14:44:58 +00:00
if (frame.next_child < frame.node->children.size())
{
stack.push(Frame{frame.node->children[frame.next_child]});
++frame.next_child;
}
else
{
if (frame.children_array)
frame.node_map->add("Plans", std::move(frame.children_array));
2021-04-09 14:44:58 +00:00
tree.swap(frame.node_map);
2021-04-09 14:44:58 +00:00
stack.pop();
if (!stack.empty())
stack.top().children_array->add(std::move(tree));
2021-04-09 14:44:58 +00:00
}
}
return tree;
}
2020-06-23 11:26:06 +00:00
static void explainStep(
2020-06-27 14:02:24 +00:00
const IQueryPlanStep & step,
2020-12-24 09:11:37 +00:00
IQueryPlanStep::FormatSettings & settings,
2020-06-27 14:02:24 +00:00
const QueryPlan::ExplainPlanOptions & options)
2020-06-23 11:26:06 +00:00
{
2020-06-27 14:02:24 +00:00
std::string prefix(settings.offset, ' ');
settings.out << prefix;
settings.out << step.getName();
2020-06-23 11:26:06 +00:00
const auto & description = step.getStepDescription();
2020-06-23 14:11:15 +00:00
if (options.description && !description.empty())
2020-06-27 14:02:24 +00:00
settings.out <<" (" << description << ')';
2020-06-23 11:26:06 +00:00
2020-06-27 14:02:24 +00:00
settings.out.write('\n');
2020-06-23 11:26:06 +00:00
if (options.header)
{
2020-06-27 14:02:24 +00:00
settings.out << prefix;
2020-06-23 11:26:06 +00:00
if (!step.hasOutputStream())
2020-06-27 14:02:24 +00:00
settings.out << "No header";
2020-06-23 11:26:06 +00:00
else if (!step.getOutputStream().header)
2020-06-27 14:02:24 +00:00
settings.out << "Empty header";
2020-06-23 11:26:06 +00:00
else
{
2020-06-27 14:02:24 +00:00
settings.out << "Header: ";
2020-06-23 11:26:06 +00:00
bool first = true;
for (const auto & elem : step.getOutputStream().header)
{
if (!first)
2020-06-27 14:02:24 +00:00
settings.out << "\n" << prefix << " ";
2020-06-23 11:26:06 +00:00
first = false;
elem.dumpNameAndType(settings.out);
2020-06-23 11:26:06 +00:00
}
}
2020-06-27 14:02:24 +00:00
settings.out.write('\n');
}
if (options.sorting)
{
if (step.hasOutputStream())
{
settings.out << prefix << "Sorting (" << step.getOutputStream().sort_mode << ")";
if (step.getOutputStream().sort_mode != DataStream::SortMode::None)
{
settings.out << ": ";
dumpSortDescription(step.getOutputStream().sort_description, settings.out);
}
settings.out.write('\n');
}
2020-06-23 11:26:06 +00:00
}
if (options.actions)
2020-06-27 14:02:24 +00:00
step.describeActions(settings);
2021-04-16 11:11:45 +00:00
if (options.indexes)
step.describeIndexes(settings);
2020-06-23 11:26:06 +00:00
}
2020-12-24 04:03:33 +00:00
std::string debugExplainStep(const IQueryPlanStep & step)
{
WriteBufferFromOwnString out;
2020-12-24 09:11:37 +00:00
IQueryPlanStep::FormatSettings settings{.out = out};
2020-12-24 04:03:33 +00:00
QueryPlan::ExplainPlanOptions options{.actions = true};
explainStep(step, settings, options);
return out.str();
}
2020-06-26 13:58:28 +00:00
void QueryPlan::explainPlan(WriteBuffer & buffer, const ExplainPlanOptions & options)
2020-06-22 14:37:42 +00:00
{
checkInitialized();
2020-12-24 09:11:37 +00:00
IQueryPlanStep::FormatSettings settings{.out = buffer, .write_header = options.header};
2020-06-22 14:37:42 +00:00
struct Frame
{
2021-05-08 15:35:09 +00:00
Node * node = {};
2020-06-22 14:37:42 +00:00
bool is_description_printed = false;
size_t next_child = 0;
};
std::stack<Frame> stack;
stack.push(Frame{.node = root});
while (!stack.empty())
{
auto & frame = stack.top();
if (!frame.is_description_printed)
{
2020-07-07 19:51:32 +00:00
settings.offset = (stack.size() - 1) * settings.indent;
2020-06-27 14:02:24 +00:00
explainStep(*frame.node->step, settings, options);
2020-06-22 14:37:42 +00:00
frame.is_description_printed = true;
}
if (frame.next_child < frame.node->children.size())
{
stack.push(Frame{frame.node->children[frame.next_child]});
++frame.next_child;
}
else
stack.pop();
}
}
2020-12-24 09:11:37 +00:00
static void explainPipelineStep(IQueryPlanStep & step, IQueryPlanStep::FormatSettings & settings)
2020-06-25 09:39:17 +00:00
{
settings.out << String(settings.offset, settings.indent_char) << "(" << step.getName() << ")\n";
2020-06-25 09:39:17 +00:00
size_t current_offset = settings.offset;
step.describePipeline(settings);
if (current_offset == settings.offset)
2020-07-07 19:51:32 +00:00
settings.offset += settings.indent;
2020-06-25 09:39:17 +00:00
}
2020-06-26 13:58:28 +00:00
void QueryPlan::explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptions & options)
2020-06-25 09:39:17 +00:00
{
checkInitialized();
2020-12-24 09:11:37 +00:00
IQueryPlanStep::FormatSettings settings{.out = buffer, .write_header = options.header};
2020-06-25 09:39:17 +00:00
struct Frame
{
2021-05-08 15:35:09 +00:00
Node * node = {};
2020-06-25 09:39:17 +00:00
size_t offset = 0;
bool is_description_printed = false;
size_t next_child = 0;
};
std::stack<Frame> stack;
stack.push(Frame{.node = root});
while (!stack.empty())
{
auto & frame = stack.top();
if (!frame.is_description_printed)
{
settings.offset = frame.offset;
explainPipelineStep(*frame.node->step, settings);
frame.offset = settings.offset;
frame.is_description_printed = true;
}
if (frame.next_child < frame.node->children.size())
{
stack.push(Frame{frame.node->children[frame.next_child], frame.offset});
++frame.next_child;
}
else
stack.pop();
}
}
void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_settings)
2020-07-23 10:20:38 +00:00
{
QueryPlanOptimizations::optimizeTree(optimization_settings, *root, nodes);
QueryPlanOptimizations::optimizePrimaryKeyCondition(*root);
2020-07-23 10:20:38 +00:00
}
2021-07-10 23:14:03 +00:00
void QueryPlan::explainEstimate(MutableColumns & columns)
2021-07-09 10:29:45 +00:00
{
checkInitialized();
struct EstimateCounters
{
std::string database_name;
std::string table_name;
UInt64 parts = 0;
UInt64 rows = 0;
UInt64 marks = 0;
2021-07-10 23:14:03 +00:00
2021-07-09 10:29:45 +00:00
EstimateCounters(const std::string & database, const std::string & table) : database_name(database), table_name(table)
{
}
};
using CountersPtr = std::shared_ptr<EstimateCounters>;
std::unordered_map<std::string, CountersPtr> counters;
using processNodeFuncType = std::function<void(const Node * node)>;
processNodeFuncType process_node = [&counters, &process_node] (const Node * node)
{
if (!node)
return;
if (const auto * step = dynamic_cast<ReadFromMergeTree*>(node->step.get()))
{
const auto & id = step->getStorageID();
auto key = id.database_name + "." + id.table_name;
auto it = counters.find(key);
if (it == counters.end())
{
it = counters.insert({key, std::make_shared<EstimateCounters>(id.database_name, id.table_name)}).first;
}
it->second->parts += step->getSelectedParts();
it->second->rows += step->getSelectedRows();
it->second->marks += step->getSelectedMarks();
}
for (const auto * child : node->children)
process_node(child);
};
process_node(root);
for (const auto & counter : counters)
{
size_t index = 0;
const auto & database_name = counter.second->database_name;
const auto & table_name = counter.second->table_name;
columns[index++]->insertData(database_name.c_str(), database_name.size());
columns[index++]->insertData(table_name.c_str(), table_name.size());
columns[index++]->insert(counter.second->parts);
columns[index++]->insert(counter.second->rows);
columns[index++]->insert(counter.second->marks);
}
}
2020-06-08 09:14:58 +00:00
}