ClickHouse/dbms/Processors/Executors/TreeExecutorBlockInputStream.cpp

352 lines
11 KiB
C++
Raw Normal View History

2019-10-21 16:26:29 +00:00
#include <Processors/Executors/TreeExecutorBlockInputStream.h>
2019-10-04 15:40:05 +00:00
#include <Processors/Sources/SourceWithProgress.h>
2020-01-28 16:59:28 +00:00
#include <Interpreters/ProcessList.h>
2019-09-13 12:34:05 +00:00
#include <stack>
2020-02-14 14:00:39 +00:00
#include <Processors/Sources/SourceFromInputStream.h>
2020-02-18 09:03:17 +00:00
#include <Processors/Transforms/AggregatingTransform.h>
2020-02-18 10:37:57 +00:00
#include <Processors/LimitTransform.h>
#include <Processors/Transforms/PartialSortingTransform.h>
2019-09-13 12:34:05 +00:00
namespace DB
{
2020-02-25 18:10:48 +00:00
namespace ErrorCodes
{
extern const int QUERY_WAS_CANCELLED;
extern const int LOGICAL_ERROR;
}
2019-09-13 12:34:05 +00:00
static void checkProcessorHasSingleOutput(IProcessor * processor)
{
2020-02-14 14:00:39 +00:00
/// SourceFromInputStream may have totals port. Skip this check.
if (typeid_cast<const SourceFromInputStream *>(processor))
return;
2019-09-13 12:34:05 +00:00
size_t num_outputs = processor->getOutputs().size();
if (num_outputs != 1)
2019-10-21 16:26:29 +00:00
throw Exception("All processors in TreeExecutorBlockInputStream must have single output, "
2019-09-13 12:34:05 +00:00
"but processor with name " + processor->getName() + " has " + std::to_string(num_outputs),
ErrorCodes::LOGICAL_ERROR);
}
2019-10-10 14:16:15 +00:00
/// Check tree invariants (described in TreeExecutor.h).
/// Collect sources with progress.
2020-02-03 15:35:35 +00:00
static void validateTree(const Processors & processors, IProcessor * root, IProcessor * totals_root, std::vector<ISourceWithProgress *> & sources)
2019-09-13 12:34:05 +00:00
{
std::unordered_map<IProcessor *, size_t> index;
for (auto & processor : processors)
{
bool is_inserted = index.try_emplace(processor.get(), index.size()).second;
if (!is_inserted)
2019-10-21 16:26:29 +00:00
throw Exception("Duplicate processor in TreeExecutorBlockInputStream with name " + processor->getName(),
2019-09-13 12:34:05 +00:00
ErrorCodes::LOGICAL_ERROR);
}
std::vector<bool> is_visited(processors.size(), false);
std::stack<IProcessor *> stack;
stack.push(root);
2020-02-03 15:35:35 +00:00
if (totals_root)
stack.push(totals_root);
2019-09-13 12:34:05 +00:00
while (!stack.empty())
{
IProcessor * node = stack.top();
stack.pop();
auto it = index.find(node);
if (it == index.end())
throw Exception("Processor with name " + node->getName() + " "
2019-10-21 16:26:29 +00:00
"was not mentioned in list passed to TreeExecutorBlockInputStream, "
2019-09-13 12:34:05 +00:00
"but was traversed to from other processors.", ErrorCodes::LOGICAL_ERROR);
size_t position = it->second;
if (is_visited[position])
2020-02-14 14:05:31 +00:00
{
/// SourceFromInputStream may have totals port. Skip this check.
if (typeid_cast<const SourceFromInputStream *>(node))
continue;
throw Exception("Processor with name " + node->getName() +
" was visited twice while traverse in TreeExecutorBlockInputStream. "
2019-09-13 12:34:05 +00:00
"Passed processors are not tree.", ErrorCodes::LOGICAL_ERROR);
2020-02-14 14:05:31 +00:00
}
2019-09-13 12:34:05 +00:00
2019-10-01 18:30:23 +00:00
is_visited[position] = true;
2019-09-13 12:34:05 +00:00
checkProcessorHasSingleOutput(node);
auto & children = node->getInputs();
for (auto & child : children)
stack.push(&child.getOutputPort().getProcessor());
2019-10-04 15:40:05 +00:00
/// Fill sources array.
if (children.empty())
{
if (auto * source = dynamic_cast<ISourceWithProgress *>(node))
sources.push_back(source);
}
2019-09-13 12:34:05 +00:00
}
for (size_t i = 0; i < is_visited.size(); ++i)
if (!is_visited[i])
throw Exception("Processor with name " + processors[i]->getName() +
2019-10-21 16:26:29 +00:00
" was not visited by traverse in TreeExecutorBlockInputStream.", ErrorCodes::LOGICAL_ERROR);
2019-09-13 12:34:05 +00:00
}
2019-10-21 16:26:29 +00:00
void TreeExecutorBlockInputStream::init()
2019-09-13 12:34:05 +00:00
{
if (processors.empty())
2019-10-21 16:26:29 +00:00
throw Exception("No processors were passed to TreeExecutorBlockInputStream.", ErrorCodes::LOGICAL_ERROR);
2019-09-13 12:34:05 +00:00
root = &output_port.getProcessor();
2020-02-03 15:35:35 +00:00
IProcessor * totals_root = nullptr;
2019-09-13 12:34:05 +00:00
2020-02-03 15:35:35 +00:00
if (totals_port)
totals_root = &totals_port->getProcessor();
validateTree(processors, root, totals_root, sources_with_progress);
2019-09-13 12:34:05 +00:00
input_port = std::make_unique<InputPort>(getHeader(), root);
connect(output_port, *input_port);
input_port->setNeeded();
2020-02-03 15:35:35 +00:00
if (totals_port)
{
input_totals_port = std::make_unique<InputPort>(totals_port->getHeader(), root);
connect(*totals_port, *input_totals_port);
input_totals_port->setNeeded();
}
2020-03-19 13:45:19 +00:00
initRowsBeforeLimit();
2019-09-13 12:34:05 +00:00
}
2020-02-03 15:35:35 +00:00
void TreeExecutorBlockInputStream::execute(bool on_totals)
2019-09-13 12:34:05 +00:00
{
std::stack<IProcessor *> stack;
2020-02-03 15:35:35 +00:00
if (on_totals)
stack.push(&totals_port->getProcessor());
else
stack.push(root);
2019-09-13 12:34:05 +00:00
auto prepare_processor = [](IProcessor * processor)
{
try
{
return processor->prepare();
}
catch (Exception & exception)
{
exception.addMessage(" While executing processor " + processor->getName());
throw;
}
};
2019-09-13 12:34:05 +00:00
while (!stack.empty())
{
IProcessor * node = stack.top();
auto status = prepare_processor(node);
2019-09-13 12:34:05 +00:00
switch (status)
{
case IProcessor::Status::NeedData:
{
auto & inputs = node->getInputs();
if (inputs.empty())
throw Exception("Processors " + node->getName() + " with empty input "
2019-10-21 16:26:29 +00:00
"has returned NeedData in TreeExecutorBlockInputStream", ErrorCodes::LOGICAL_ERROR);
2019-09-13 12:34:05 +00:00
bool all_finished = true;
for (auto & input : inputs)
{
if (input.isFinished())
continue;
all_finished = false;
stack.push(&input.getOutputPort().getProcessor());
}
if (all_finished)
2019-10-21 16:26:29 +00:00
throw Exception("Processors " + node->getName() + " has returned NeedData in TreeExecutorBlockInputStream, "
2019-09-13 12:34:05 +00:00
"but all it's inputs are finished.", ErrorCodes::LOGICAL_ERROR);
break;
}
case IProcessor::Status::PortFull:
case IProcessor::Status::Finished:
{
stack.pop();
break;
}
case IProcessor::Status::Ready:
{
node->work();
2020-01-28 16:59:28 +00:00
/// This is handled inside PipelineExecutor now,
/// and doesn't checked by processors as in IInputStream before.
if (process_list_element && process_list_element->isKilled())
throw Exception("Query was cancelled", ErrorCodes::QUERY_WAS_CANCELLED);
2019-09-13 12:34:05 +00:00
break;
}
case IProcessor::Status::Async:
case IProcessor::Status::Wait:
case IProcessor::Status::ExpandPipeline:
{
throw Exception("Processor with name " + node->getName() + " "
"returned status " + IProcessor::statusToName(status) + " "
2019-10-21 16:26:29 +00:00
"which is not supported in TreeExecutorBlockInputStream.", ErrorCodes::LOGICAL_ERROR);
2019-09-13 12:34:05 +00:00
}
}
}
}
2020-03-19 13:45:19 +00:00
void TreeExecutorBlockInputStream::initRowsBeforeLimit()
2020-02-18 10:37:57 +00:00
{
2020-03-19 13:53:25 +00:00
std::vector<LimitTransform *> limit_transforms;
2020-03-19 13:45:19 +00:00
std::vector<SourceFromInputStream *> sources;
struct StackEntry
{
IProcessor * processor;
bool visited_limit;
};
2020-02-18 10:37:57 +00:00
2020-03-19 13:45:19 +00:00
std::stack<StackEntry> stack;
stack.push({root, false});
2020-02-18 10:37:57 +00:00
while (!stack.empty())
{
2020-03-19 13:45:19 +00:00
auto processor = stack.top().processor;
bool visited_limit = stack.top().visited_limit;
2020-02-18 10:37:57 +00:00
stack.pop();
2020-03-19 13:45:19 +00:00
if (!visited_limit)
2020-02-18 13:29:40 +00:00
{
2020-02-18 10:37:57 +00:00
2020-03-19 13:45:19 +00:00
if (auto * limit = typeid_cast<LimitTransform *>(processor))
2020-02-18 10:37:57 +00:00
{
2020-03-19 13:45:19 +00:00
visited_limit = true;
2020-03-19 13:53:25 +00:00
limit_transforms.emplace_back(limit);
2020-02-18 10:37:57 +00:00
}
2020-03-19 13:45:19 +00:00
if (auto * source = typeid_cast<SourceFromInputStream *>(processor))
sources.emplace_back(source);
2020-02-18 10:37:57 +00:00
}
2020-03-19 17:18:33 +00:00
else if (auto * sorting = typeid_cast<PartialSortingTransform *>(processor))
2020-02-18 10:37:57 +00:00
{
2020-03-19 13:45:19 +00:00
if (!rows_before_limit_at_least)
rows_before_limit_at_least = std::make_shared<RowsBeforeLimitCounter>();
sorting->setRowsBeforeLimitCounter(rows_before_limit_at_least);
2020-02-18 10:37:57 +00:00
/// Don't go to children. Take rows_before_limit from last PartialSortingTransform.
continue;
}
for (auto & child_port : processor->getInputs())
{
auto * child_processor = &child_port.getOutputPort().getProcessor();
2020-03-19 13:45:19 +00:00
stack.push({child_processor, visited_limit});
2020-02-18 10:37:57 +00:00
}
}
2020-03-19 14:16:49 +00:00
if (!rows_before_limit_at_least && (!limit_transforms.empty() || !sources.empty()))
2020-03-19 13:45:19 +00:00
{
rows_before_limit_at_least = std::make_shared<RowsBeforeLimitCounter>();
2020-03-19 13:53:25 +00:00
for (auto & limit : limit_transforms)
2020-03-19 13:45:19 +00:00
limit->setRowsBeforeLimitCounter(rows_before_limit_at_least);
for (auto & source : sources)
source->setRowsBeforeLimitCounter(rows_before_limit_at_least);
}
2020-03-19 17:18:33 +00:00
/// If there is a limit, then enable rows_before_limit_at_least
/// It is needed when zero rows is read, but we still want rows_before_limit_at_least in result.
if (!limit_transforms.empty())
rows_before_limit_at_least->add(0);
2020-02-18 10:37:57 +00:00
}
2019-10-21 16:26:29 +00:00
Block TreeExecutorBlockInputStream::readImpl()
2019-09-13 12:34:05 +00:00
{
while (true)
{
if (input_port->isFinished())
2020-02-03 15:35:35 +00:00
{
if (totals_port && !input_totals_port->isFinished())
{
execute(true);
if (input_totals_port->hasData())
totals = getHeader().cloneWithColumns(input_totals_port->pull().detachColumns());
}
2020-03-19 13:45:19 +00:00
if (rows_before_limit_at_least && rows_before_limit_at_least->hasAppliedLimit())
info.setRowsBeforeLimit(rows_before_limit_at_least->get());
2020-02-18 10:37:57 +00:00
2019-09-13 12:34:05 +00:00
return {};
2020-02-03 15:35:35 +00:00
}
2019-09-13 12:34:05 +00:00
if (input_port->hasData())
2020-02-18 09:03:17 +00:00
{
auto chunk = input_port->pull();
Block block = getHeader().cloneWithColumns(chunk.detachColumns());
2020-02-18 09:05:49 +00:00
if (auto & chunk_info = chunk.getChunkInfo())
2020-02-18 09:03:17 +00:00
{
2020-02-18 09:05:49 +00:00
if (auto * agg_info = typeid_cast<const AggregatedChunkInfo *>(chunk_info.get()))
2020-02-18 09:03:17 +00:00
{
block.info.bucket_num = agg_info->bucket_num;
block.info.is_overflows = agg_info->is_overflows;
}
}
return block;
}
2019-09-13 12:34:05 +00:00
2020-02-03 15:35:35 +00:00
execute(false);
2019-09-13 12:34:05 +00:00
}
}
2019-10-21 16:26:29 +00:00
void TreeExecutorBlockInputStream::setProgressCallback(const ProgressCallback & callback)
2019-10-04 15:40:05 +00:00
{
for (auto & source : sources_with_progress)
source->setProgressCallback(callback);
}
2019-10-21 16:26:29 +00:00
void TreeExecutorBlockInputStream::setProcessListElement(QueryStatus * elem)
2019-10-04 15:40:05 +00:00
{
2020-01-28 16:59:28 +00:00
process_list_element = elem;
2019-10-04 15:40:05 +00:00
for (auto & source : sources_with_progress)
source->setProcessListElement(elem);
}
2019-10-21 16:26:29 +00:00
void TreeExecutorBlockInputStream::setLimits(const IBlockInputStream::LocalLimits & limits_)
2019-10-04 15:40:05 +00:00
{
for (auto & source : sources_with_progress)
source->setLimits(limits_);
}
void TreeExecutorBlockInputStream::setQuota(const std::shared_ptr<const EnabledQuota> & quota_)
2019-10-04 15:40:05 +00:00
{
for (auto & source : sources_with_progress)
source->setQuota(quota_);
}
2019-10-21 16:26:29 +00:00
void TreeExecutorBlockInputStream::addTotalRowsApprox(size_t value)
2019-10-04 15:40:05 +00:00
{
/// Add only for one source.
if (!sources_with_progress.empty())
sources_with_progress.front()->addTotalRowsApprox(value);
}
2019-09-13 12:34:05 +00:00
}