ClickHouse/src/Processors/QueryPipeline.cpp

468 lines
15 KiB
C++
Raw Normal View History

2019-03-26 18:28:37 +00:00
#include <Processors/QueryPipeline.h>
#include <Processors/ResizeProcessor.h>
#include <Processors/ConcatProcessor.h>
2019-04-08 14:55:20 +00:00
#include <Processors/LimitTransform.h>
2019-03-26 18:28:37 +00:00
#include <Processors/Transforms/TotalsHavingTransform.h>
2019-04-05 11:34:11 +00:00
#include <Processors/Transforms/ExtremesTransform.h>
2019-03-26 18:28:37 +00:00
#include <Processors/Transforms/CreatingSetsTransform.h>
#include <Processors/Transforms/ConvertingTransform.h>
2020-08-04 13:06:59 +00:00
#include <Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h>
2019-03-26 18:28:37 +00:00
#include <Processors/Formats/IOutputFormat.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <Processors/Transforms/PartialSortingTransform.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
2019-03-26 18:28:37 +00:00
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Common/typeid_cast.h>
#include <Common/CurrentThread.h>
#include <Processors/DelayedPortsProcessor.h>
#include <Processors/RowsBeforeLimitCounter.h>
2020-06-03 19:50:11 +00:00
#include <Processors/Sources/RemoteSource.h>
2019-03-26 18:28:37 +00:00
namespace DB
{
2020-02-25 18:10:48 +00:00
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
2019-03-26 18:28:37 +00:00
void QueryPipeline::checkInitialized()
{
if (!initialized())
throw Exception("QueryPipeline wasn't initialized.", ErrorCodes::LOGICAL_ERROR);
}
2020-05-27 18:20:26 +00:00
void QueryPipeline::checkInitializedAndNotCompleted()
{
checkInitialized();
2020-08-04 13:06:59 +00:00
if (pipe.isCompleted())
2020-05-27 18:20:26 +00:00
throw Exception("QueryPipeline was already completed.", ErrorCodes::LOGICAL_ERROR);
}
2020-08-04 13:06:59 +00:00
static void checkSource(const ProcessorPtr & source, bool can_have_totals)
2019-03-26 18:28:37 +00:00
{
if (!source->getInputs().empty())
throw Exception("Source for query pipeline shouldn't have any input, but " + source->getName() + " has " +
toString(source->getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR);
2019-04-17 14:38:16 +00:00
if (source->getOutputs().empty())
throw Exception("Source for query pipeline should have single output, but it doesn't have any",
ErrorCodes::LOGICAL_ERROR);
if (!can_have_totals && source->getOutputs().size() != 1)
2019-03-26 18:28:37 +00:00
throw Exception("Source for query pipeline should have single output, but " + source->getName() + " has " +
toString(source->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR);
2019-04-17 14:38:16 +00:00
if (source->getOutputs().size() > 2)
throw Exception("Source for query pipeline should have 1 or 2 outputs, but " + source->getName() + " has " +
toString(source->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR);
2019-03-26 18:28:37 +00:00
}
2020-08-04 13:06:59 +00:00
void QueryPipeline::init(Pipe pipe_)
2019-03-26 18:28:37 +00:00
{
if (initialized())
throw Exception("Pipeline has already been initialized.", ErrorCodes::LOGICAL_ERROR);
2020-08-06 12:24:05 +00:00
if (pipe_.empty())
2020-08-04 13:06:59 +00:00
throw Exception("Can't initialize pipeline with empty pipe.", ErrorCodes::LOGICAL_ERROR);
2019-04-09 10:17:25 +00:00
2020-08-04 13:06:59 +00:00
pipe = std::move(pipe_);
2019-04-09 10:17:25 +00:00
}
2020-08-06 12:24:05 +00:00
void QueryPipeline::reset()
{
Pipe pipe_to_destroy(std::move(pipe));
*this = QueryPipeline();
}
2020-08-04 13:06:59 +00:00
void QueryPipeline::addSimpleTransform(const Pipe::ProcessorGetter & getter)
2019-03-26 18:28:37 +00:00
{
2020-05-27 18:20:26 +00:00
checkInitializedAndNotCompleted();
2020-08-04 13:06:59 +00:00
pipe.addSimpleTransform(getter);
2019-03-26 18:28:37 +00:00
}
2020-08-04 13:06:59 +00:00
void QueryPipeline::addSimpleTransform(const Pipe::ProcessorGetterWithStreamKind & getter)
2019-04-09 10:17:25 +00:00
{
2020-08-04 13:06:59 +00:00
checkInitializedAndNotCompleted();
pipe.addSimpleTransform(getter);
2019-04-09 10:17:25 +00:00
}
2020-08-04 13:06:59 +00:00
void QueryPipeline::addTransform(ProcessorPtr transform)
2020-05-27 18:20:26 +00:00
{
checkInitializedAndNotCompleted();
2020-08-04 13:06:59 +00:00
pipe.addTransform(std::move(transform));
2020-05-27 18:20:26 +00:00
}
2020-08-04 13:06:59 +00:00
void QueryPipeline::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter)
2019-03-26 18:28:37 +00:00
{
2020-05-27 18:20:26 +00:00
checkInitializedAndNotCompleted();
2020-08-04 13:06:59 +00:00
pipe.setSinks(getter);
2019-03-26 18:28:37 +00:00
}
void QueryPipeline::addDelayedStream(ProcessorPtr source)
{
2020-05-27 18:20:26 +00:00
checkInitializedAndNotCompleted();
2019-03-26 18:28:37 +00:00
2019-04-17 14:58:34 +00:00
checkSource(source, false);
2020-08-04 13:06:59 +00:00
assertBlocksHaveEqualStructure(getHeader(), source->getOutputs().front().getHeader(), "QueryPipeline");
2019-03-26 18:28:37 +00:00
2020-08-04 13:06:59 +00:00
IProcessor::PortNumbers delayed_streams = { pipe.numOutputPorts() };
pipe.addSource(std::move(source));
2019-04-08 09:31:49 +00:00
2020-08-04 13:06:59 +00:00
auto processor = std::make_shared<DelayedPortsProcessor>(getHeader(), pipe.numOutputPorts(), delayed_streams);
addTransform(std::move(processor));
}
void QueryPipeline::addMergingAggregatedMemoryEfficientTransform(AggregatingTransformParamsPtr params, size_t num_merging_processors)
{
DB::addMergingAggregatedMemoryEfficientTransform(pipe, std::move(params), num_merging_processors);
2019-03-26 18:28:37 +00:00
}
2020-01-13 12:04:02 +00:00
void QueryPipeline::resize(size_t num_streams, bool force, bool strict)
2019-03-26 18:28:37 +00:00
{
2020-05-27 18:20:26 +00:00
checkInitializedAndNotCompleted();
2019-03-26 18:28:37 +00:00
if (!force && num_streams == getNumStreams())
2019-03-26 18:28:37 +00:00
return;
2020-01-13 12:04:02 +00:00
ProcessorPtr resize;
if (strict)
2020-08-04 13:06:59 +00:00
resize = std::make_shared<StrictResizeProcessor>(getHeader(), getNumStreams(), num_streams);
2020-01-13 12:04:02 +00:00
else
2020-08-04 13:06:59 +00:00
resize = std::make_shared<ResizeProcessor>(getHeader(), getNumStreams(), num_streams);
2019-03-26 18:28:37 +00:00
2020-08-04 13:06:59 +00:00
pipe.addTransform(std::move(resize));
2019-12-26 16:52:15 +00:00
}
2019-03-26 18:28:37 +00:00
void QueryPipeline::addTotalsHavingTransform(ProcessorPtr transform)
{
2020-05-27 18:20:26 +00:00
checkInitializedAndNotCompleted();
2019-03-26 18:28:37 +00:00
if (!typeid_cast<const TotalsHavingTransform *>(transform.get()))
throw Exception("TotalsHavingTransform expected for QueryPipeline::addTotalsHavingTransform.",
ErrorCodes::LOGICAL_ERROR);
2020-08-04 13:06:59 +00:00
if (pipe.getTotalsPort())
2019-03-26 18:28:37 +00:00
throw Exception("Totals having transform was already added to pipeline.", ErrorCodes::LOGICAL_ERROR);
resize(1);
2020-08-04 13:06:59 +00:00
auto * totals_port = &transform->getOutputs().back();
2020-08-04 15:51:56 +00:00
pipe.addTransform(std::move(transform), totals_port, nullptr);
2019-03-26 18:28:37 +00:00
}
2019-04-09 14:51:38 +00:00
void QueryPipeline::addDefaultTotals()
{
2020-05-27 18:20:26 +00:00
checkInitializedAndNotCompleted();
2019-04-09 14:51:38 +00:00
2020-08-04 13:06:59 +00:00
if (pipe.getTotalsPort())
2019-04-09 14:51:38 +00:00
throw Exception("Totals having transform was already added to pipeline.", ErrorCodes::LOGICAL_ERROR);
2020-08-04 13:06:59 +00:00
const auto & current_header = getHeader();
2019-04-09 14:51:38 +00:00
Columns columns;
columns.reserve(current_header.columns());
for (size_t i = 0; i < current_header.columns(); ++i)
{
auto column = current_header.getByPosition(i).type->createColumn();
column->insertDefault();
columns.emplace_back(std::move(column));
}
auto source = std::make_shared<SourceFromSingleChunk>(current_header, Chunk(std::move(columns), 1));
2020-08-04 13:06:59 +00:00
pipe.addTotalsSource(std::move(source));
2019-04-10 16:28:37 +00:00
}
void QueryPipeline::dropTotalsAndExtremes()
2019-04-17 15:35:22 +00:00
{
2020-08-04 15:51:56 +00:00
pipe.dropTotals();
pipe.dropExtremes();
2019-04-17 15:35:22 +00:00
}
2020-04-08 12:40:04 +00:00
void QueryPipeline::addExtremesTransform()
2019-03-26 18:28:37 +00:00
{
2020-05-27 18:20:26 +00:00
checkInitializedAndNotCompleted();
2019-03-26 18:28:37 +00:00
2020-08-04 15:51:56 +00:00
if (pipe.getExtremesPort())
2019-03-26 18:28:37 +00:00
throw Exception("Extremes transform was already added to pipeline.", ErrorCodes::LOGICAL_ERROR);
2020-08-04 15:51:56 +00:00
resize(1);
auto transform = std::make_shared<ExtremesTransform>(getHeader());
auto * port = &transform->getExtremesPort();
pipe.addTransform(std::move(transform), nullptr, port);
2019-03-26 18:28:37 +00:00
}
void QueryPipeline::addCreatingSetsTransform(SubqueriesForSets subqueries_for_sets, const SizeLimits & network_transfer_limits, const Context & context)
2019-03-26 18:28:37 +00:00
{
2020-05-27 18:20:26 +00:00
checkInitializedAndNotCompleted();
2019-03-26 18:28:37 +00:00
Pipes sources;
for (auto & subquery : subqueries_for_sets)
{
if (!subquery.second.source.empty())
{
auto & source = sources.emplace_back(std::move(subquery.second.source));
if (source.numOutputPorts() > 1)
source.addTransform(std::make_shared<ResizeProcessor>(source.getHeader(), source.numOutputPorts(), 1));
source.dropExtremes();
auto creating_sets = std::make_shared<CreatingSetsTransform>(
source.getHeader(),
getHeader(),
std::move(subquery.second),
network_transfer_limits,
context);
InputPort * totals = nullptr;
if (source.getTotalsPort())
totals = creating_sets->addTotalsPort();
source.addTransform(std::move(creating_sets), totals, nullptr);
}
}
if (sources.empty())
return;
auto * collected_processors = pipe.collected_processors;
/// We unite all sources together.
/// Set collected_processors to attach all newly-added processors to current query plan step.
auto source = Pipe::unitePipes(std::move(sources), collected_processors);
if (source.numOutputPorts() > 1)
source.addTransform(std::make_shared<ResizeProcessor>(source.getHeader(), source.numOutputPorts(), 1));
source.collected_processors = nullptr;
2019-03-26 18:28:37 +00:00
resize(1);
2020-09-04 08:36:47 +00:00
Pipes pipes;
pipes.emplace_back(std::move(source));
pipes.emplace_back(std::move(pipe));
pipe = Pipe::unitePipes(std::move(pipes), collected_processors);
pipe.addTransform(std::make_shared<ConcatProcessor>(getHeader(), 2));
2019-03-26 18:28:37 +00:00
}
2020-05-27 18:20:26 +00:00
void QueryPipeline::setOutputFormat(ProcessorPtr output)
2019-03-26 18:28:37 +00:00
{
2020-05-27 18:20:26 +00:00
checkInitializedAndNotCompleted();
2019-03-26 18:28:37 +00:00
2019-04-08 14:55:20 +00:00
if (output_format)
2019-03-26 18:28:37 +00:00
throw Exception("QueryPipeline already has output.", ErrorCodes::LOGICAL_ERROR);
resize(1);
2020-08-04 15:51:56 +00:00
output_format = dynamic_cast<IOutputFormat * >(output.get());
pipe.setOutputFormat(std::move(output));
2020-05-27 18:20:26 +00:00
initRowsBeforeLimit();
2019-03-26 18:28:37 +00:00
}
2020-08-04 15:51:56 +00:00
QueryPipeline QueryPipeline::unitePipelines(
std::vector<std::unique_ptr<QueryPipeline>> pipelines,
const Block & common_header,
size_t max_threads_limit,
Processors * collected_processors)
2019-03-26 18:28:37 +00:00
{
/// Should we limit the number of threads for united pipeline. True if all pipelines have max_threads != 0.
/// If true, result max_threads will be sum(max_threads).
/// Note: it may be > than settings.max_threads, so we should apply this limit again.
2020-08-04 15:51:56 +00:00
bool will_limit_max_threads = true;
size_t max_threads = 0;
Pipes pipes;
2019-03-26 18:28:37 +00:00
for (auto & pipeline_ptr : pipelines)
2019-03-26 18:28:37 +00:00
{
auto & pipeline = *pipeline_ptr;
2019-03-26 18:28:37 +00:00
pipeline.checkInitialized();
2020-08-04 15:51:56 +00:00
pipeline.pipe.collected_processors = collected_processors;
2019-03-26 18:28:37 +00:00
2020-05-27 18:20:26 +00:00
if (!pipeline.isCompleted())
2019-04-05 11:34:11 +00:00
{
2020-05-27 18:20:26 +00:00
pipeline.addSimpleTransform([&](const Block & header)
{
return std::make_shared<ConvertingTransform>(
header, common_header, ConvertingTransform::MatchColumnsMode::Position);
});
}
2019-03-26 18:28:37 +00:00
2020-08-04 15:51:56 +00:00
pipes.emplace_back(std::move(pipeline.pipe));
2020-01-01 10:42:46 +00:00
max_threads += pipeline.max_threads;
2020-07-03 13:38:35 +00:00
will_limit_max_threads = will_limit_max_threads && pipeline.max_threads != 0;
/// If one of pipelines uses more threads then current limit, will keep it.
/// It may happen if max_distributed_connections > max_threads
if (pipeline.max_threads > max_threads_limit)
max_threads_limit = pipeline.max_threads;
2019-03-26 18:28:37 +00:00
}
2020-08-04 15:51:56 +00:00
QueryPipeline pipeline;
pipeline.init(Pipe::unitePipes(std::move(pipes), collected_processors));
2020-08-04 15:51:56 +00:00
if (will_limit_max_threads)
2019-03-26 18:28:37 +00:00
{
2020-08-04 15:51:56 +00:00
pipeline.setMaxThreads(max_threads);
pipeline.limitMaxThreads(max_threads_limit);
2020-04-08 12:40:04 +00:00
}
2019-03-26 18:28:37 +00:00
2020-08-04 15:51:56 +00:00
return pipeline;
2019-03-26 18:28:37 +00:00
}
void QueryPipeline::setProgressCallback(const ProgressCallback & callback)
{
2020-08-04 15:51:56 +00:00
for (auto & processor : pipe.processors)
2019-04-10 12:38:57 +00:00
{
if (auto * source = dynamic_cast<ISourceWithProgress *>(processor.get()))
2019-10-04 17:46:48 +00:00
source->setProgressCallback(callback);
2019-04-10 12:38:57 +00:00
}
2019-03-26 18:28:37 +00:00
}
void QueryPipeline::setProcessListElement(QueryStatus * elem)
{
process_list_element = elem;
2020-08-04 15:51:56 +00:00
for (auto & processor : pipe.processors)
2019-04-10 12:38:57 +00:00
{
2019-10-04 17:46:48 +00:00
if (auto * source = dynamic_cast<ISourceWithProgress *>(processor.get()))
source->setProcessListElement(elem);
2019-04-10 12:38:57 +00:00
}
2019-03-26 18:28:37 +00:00
}
void QueryPipeline::initRowsBeforeLimit()
2019-04-08 14:55:20 +00:00
{
RowsBeforeLimitCounterPtr rows_before_limit_at_least;
2019-04-29 16:52:50 +00:00
2020-06-03 19:50:11 +00:00
/// TODO: add setRowsBeforeLimitCounter as virtual method to IProcessor.
std::vector<LimitTransform *> limits;
std::vector<SourceFromInputStream *> sources;
2020-06-03 19:50:11 +00:00
std::vector<RemoteSource *> remote_sources;
2019-04-08 14:55:20 +00:00
2019-04-29 16:52:50 +00:00
std::unordered_set<IProcessor *> visited;
struct QueuedEntry
2019-04-08 14:55:20 +00:00
{
2019-04-29 16:52:50 +00:00
IProcessor * processor;
bool visited_limit;
};
2019-04-12 15:20:24 +00:00
2019-04-29 16:52:50 +00:00
std::queue<QueuedEntry> queue;
queue.push({ output_format, false });
visited.emplace(output_format);
while (!queue.empty())
{
2020-04-22 06:34:20 +00:00
auto * processor = queue.front().processor;
2019-04-29 16:52:50 +00:00
auto visited_limit = queue.front().visited_limit;
queue.pop();
if (!visited_limit)
2019-04-12 15:20:24 +00:00
{
if (auto * limit = typeid_cast<LimitTransform *>(processor))
2019-04-12 15:20:24 +00:00
{
visited_limit = true;
limits.emplace_back(limit);
2019-04-12 15:20:24 +00:00
}
2019-04-08 14:55:20 +00:00
2019-04-29 16:52:50 +00:00
if (auto * source = typeid_cast<SourceFromInputStream *>(processor))
sources.emplace_back(source);
2020-06-03 19:50:11 +00:00
if (auto * source = typeid_cast<RemoteSource *>(processor))
remote_sources.emplace_back(source);
2019-04-29 16:52:50 +00:00
}
2020-03-19 17:18:33 +00:00
else if (auto * sorting = typeid_cast<PartialSortingTransform *>(processor))
2019-04-08 14:55:20 +00:00
{
if (!rows_before_limit_at_least)
rows_before_limit_at_least = std::make_shared<RowsBeforeLimitCounter>();
sorting->setRowsBeforeLimitCounter(rows_before_limit_at_least);
2019-04-29 16:52:50 +00:00
/// Don't go to children. Take rows_before_limit from last PartialSortingTransform.
2020-03-19 17:18:33 +00:00
continue;
2019-04-29 16:52:50 +00:00
}
2019-05-13 12:08:02 +00:00
/// Skip totals and extremes port for output format.
2019-05-13 13:04:52 +00:00
if (auto * format = dynamic_cast<IOutputFormat *>(processor))
2019-05-13 12:08:02 +00:00
{
auto * child_processor = &format->getPort(IOutputFormat::PortKind::Main).getOutputPort().getProcessor();
if (visited.emplace(child_processor).second)
queue.push({ child_processor, visited_limit });
continue;
}
2019-04-29 16:52:50 +00:00
for (auto & child_port : processor->getInputs())
{
auto * child_processor = &child_port.getOutputPort().getProcessor();
if (visited.emplace(child_processor).second)
queue.push({ child_processor, visited_limit });
2019-04-08 14:55:20 +00:00
}
}
2020-06-03 19:50:11 +00:00
if (!rows_before_limit_at_least && (!limits.empty() || !sources.empty() || !remote_sources.empty()))
{
rows_before_limit_at_least = std::make_shared<RowsBeforeLimitCounter>();
for (auto & limit : limits)
limit->setRowsBeforeLimitCounter(rows_before_limit_at_least);
for (auto & source : sources)
source->setRowsBeforeLimitCounter(rows_before_limit_at_least);
2020-06-03 19:50:11 +00:00
for (auto & source : remote_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 (!limits.empty())
rows_before_limit_at_least->add(0);
if (rows_before_limit_at_least)
output_format->setRowsBeforeLimitCounter(rows_before_limit_at_least);
2019-04-08 14:55:20 +00:00
}
2019-03-26 18:28:37 +00:00
PipelineExecutorPtr QueryPipeline::execute()
2019-03-26 18:28:37 +00:00
{
2020-05-27 18:20:26 +00:00
if (!isCompleted())
throw Exception("Cannot execute pipeline because it is not completed.", ErrorCodes::LOGICAL_ERROR);
2019-03-26 18:28:37 +00:00
2020-08-04 15:51:56 +00:00
return std::make_shared<PipelineExecutor>(pipe.processors, process_list_element);
2020-06-25 09:39:17 +00:00
}
2020-08-04 15:51:56 +00:00
void QueryPipeline::setCollectedProcessors(Processors * processors)
2020-06-25 09:39:17 +00:00
{
2020-08-04 15:51:56 +00:00
pipe.collected_processors = processors;
2020-06-25 09:39:17 +00:00
}
2020-08-07 08:28:12 +00:00
2020-06-25 09:39:17 +00:00
QueryPipelineProcessorsCollector::QueryPipelineProcessorsCollector(QueryPipeline & pipeline_, IQueryPlanStep * step_)
: pipeline(pipeline_), step(step_)
{
2020-08-04 15:51:56 +00:00
pipeline.setCollectedProcessors(&processors);
2020-06-25 09:39:17 +00:00
}
QueryPipelineProcessorsCollector::~QueryPipelineProcessorsCollector()
{
2020-08-04 15:51:56 +00:00
pipeline.setCollectedProcessors(nullptr);
2020-06-25 09:39:17 +00:00
}
Processors QueryPipelineProcessorsCollector::detachProcessors(size_t group)
{
for (auto & processor : processors)
processor->setQueryPlanStep(step, group);
2020-06-26 17:56:33 +00:00
Processors res;
res.swap(processors);
2020-06-25 09:39:17 +00:00
return res;
}
2019-03-26 18:28:37 +00:00
}