Merge pull request #13337 from ClickHouse/refactor-pipes-3

Refactor Pipe
This commit is contained in:
alexey-milovidov 2020-08-14 04:12:22 +03:00 committed by GitHub
commit eeefcd6493
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
116 changed files with 1412 additions and 1440 deletions

View File

@ -22,6 +22,7 @@
#include <Processors/Pipe.h>
#include <Processors/ISink.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <Processors/ConcatProcessor.h>
#if !defined(ARCADIA_BUILD)
# include <Common/config_version.h>
@ -581,10 +582,13 @@ void Connection::sendExternalTablesData(ExternalTablesData & data)
PipelineExecutorPtr executor;
auto on_cancel = [& executor]() { executor->cancel(); };
auto sink = std::make_shared<ExternalTableDataSink>(elem->pipe->getHeader(), *this, *elem, std::move(on_cancel));
DB::connect(elem->pipe->getPort(), sink->getPort());
if (elem->pipe->numOutputPorts() > 1)
elem->pipe->addTransform(std::make_shared<ConcatProcessor>(elem->pipe->getHeader(), elem->pipe->numOutputPorts()));
auto processors = std::move(*elem->pipe).detachProcessors();
auto sink = std::make_shared<ExternalTableDataSink>(elem->pipe->getHeader(), *this, *elem, std::move(on_cancel));
DB::connect(*elem->pipe->getOutputPort(0), sink->getPort());
auto processors = Pipe::detachProcessors(std::move(*elem->pipe));
processors.push_back(sink);
executor = std::make_shared<PipelineExecutor>(processors);

View File

@ -11,6 +11,7 @@
#include <Processors/Pipe.h>
#include <Processors/Sources/SinkToOutputStream.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <Processors/ConcatProcessor.h>
#include <Core/ExternalTable.h>
#include <Poco/Net/MessageHeader.h>
#include <common/find_symbols.h>
@ -170,10 +171,13 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header,
BlockOutputStreamPtr output = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), context);
/// Write data
auto sink = std::make_shared<SinkToOutputStream>(std::move(output));
connect(data->pipe->getPort(), sink->getPort());
if (data->pipe->numOutputPorts() > 1)
data->pipe->addTransform(std::make_shared<ConcatProcessor>(data->pipe->getHeader(), data->pipe->numOutputPorts()));
auto processors = std::move(*data->pipe).detachProcessors();
auto sink = std::make_shared<SinkToOutputStream>(std::move(output));
connect(*data->pipe->getOutputPort(0), sink->getPort());
auto processors = Pipe::detachProcessors(std::move(*data->pipe));
processors.push_back(std::move(sink));
auto executor = std::make_shared<PipelineExecutor>(processors);

View File

@ -42,7 +42,7 @@ void BlockIO::reset()
in.reset();
if (process_list_entry)
process_list_entry->get().releaseQueryStreams();
pipeline = QueryPipeline();
pipeline.reset();
process_list_entry.reset();
/// TODO Do we need also reset callbacks? In which order?

View File

@ -2,10 +2,8 @@
#include <Columns/ColumnConst.h>
#include <Common/CurrentThread.h>
#include <Common/Throttler.h>
#include <Processors/Pipe.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <Processors/ConcatProcessor.h>
#include <Storages/IStorage.h>
#include <Interpreters/castColumn.h>
#include <Interpreters/Cluster.h>
@ -322,9 +320,7 @@ void RemoteQueryExecutor::sendExternalTables()
auto metadata_snapshot = cur->getInMemoryMetadataPtr();
QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage(context);
Pipes pipes;
pipes = cur->read(
Pipe pipe = cur->read(
metadata_snapshot->getColumns().getNamesOfPhysical(),
metadata_snapshot, {}, context,
read_from_table_stage, DEFAULT_BLOCK_SIZE, 1);
@ -332,16 +328,11 @@ void RemoteQueryExecutor::sendExternalTables()
auto data = std::make_unique<ExternalTableData>();
data->table_name = table.first;
if (pipes.empty())
if (pipe.empty())
data->pipe = std::make_unique<Pipe>(
std::make_shared<SourceFromSingleChunk>(metadata_snapshot->getSampleBlock(), Chunk()));
else if (pipes.size() == 1)
data->pipe = std::make_unique<Pipe>(std::move(pipes.front()));
else
{
auto concat = std::make_shared<ConcatProcessor>(pipes.front().getHeader(), pipes.size());
data->pipe = std::make_unique<Pipe>(std::move(pipes), std::move(concat));
}
data->pipe = std::make_unique<Pipe>(std::move(pipe));
res.emplace_back(std::move(data));
}

View File

@ -23,29 +23,40 @@ namespace
}
}
Pipes narrowPipes(Pipes pipes, size_t width)
void narrowPipe(Pipe & pipe, size_t width)
{
size_t size = pipes.size();
size_t size = pipe.numOutputPorts();
if (size <= width)
return pipes;
return;
std::vector<Pipes> partitions(width);
std::vector<std::vector<OutputPort *>> partitions(width);
auto distribution = getDistribution(size, width);
for (size_t i = 0; i < size; ++i)
partitions[distribution[i]].emplace_back(std::move(pipes[i]));
Pipes res;
res.reserve(width);
for (size_t i = 0; i < width; ++i)
pipe.transform([&](OutputPortRawPtrs ports)
{
auto processor = std::make_shared<ConcatProcessor>(partitions[i].at(0).getHeader(), partitions[i].size());
res.emplace_back(std::move(partitions[i]), std::move(processor));
}
for (size_t i = 0; i < size; ++i)
partitions[distribution[i]].emplace_back(ports[i]);
return res;
Processors concats;
concats.reserve(width);
for (size_t i = 0; i < width; ++i)
{
auto concat = std::make_shared<ConcatProcessor>(partitions[i].at(0)->getHeader(),
partitions[i].size());
size_t next_port = 0;
for (auto & port : concat->getInputs())
{
connect(*partitions[i][next_port], port);
++next_port;
}
concats.emplace_back(std::move(concat));
}
return concats;
});
}
}

View File

@ -7,7 +7,6 @@ namespace DB
{
class Pipe;
using Pipes = std::vector<Pipe>;
/** If the number of sources of `inputs` is greater than `width`,
* then glues the sources to each other (using ConcatBlockInputStream),
@ -16,6 +15,6 @@ using Pipes = std::vector<Pipe>;
* Trying to glue the sources with each other uniformly randomly.
* (to avoid overweighting if the distribution of the amount of data in different sources is subject to some pattern)
*/
Pipes narrowPipes(Pipes pipes, size_t width);
void narrowPipe(Pipe & pipe, size_t width);
}

View File

@ -72,7 +72,7 @@ int main(int argc, char ** argv)
QueryPipeline pipeline;
pipeline.init(std::move(source));
pipeline.addPipe({std::make_shared<MergeSortingTransform>(pipeline.getHeader(), sort_descr, n, 0, 0, 0, nullptr, 0)});
pipeline.addTransform(std::make_shared<MergeSortingTransform>(pipeline.getHeader(), sort_descr, n, 0, 0, 0, nullptr, 0));
SortDescription sort_descr_final;
sort_descr_final.emplace_back("col1", 1, 1);

View File

@ -31,7 +31,7 @@ static Block getBlockWithSize(const std::vector<std::string> & columns, size_t r
}
static Pipes getInputStreams(const std::vector<std::string> & column_names, const std::vector<std::tuple<size_t, size_t, size_t>> & block_sizes)
static Pipe getInputStreams(const std::vector<std::string> & column_names, const std::vector<std::tuple<size_t, size_t, size_t>> & block_sizes)
{
Pipes pipes;
for (auto [block_size_in_bytes, blocks_count, stride] : block_sizes)
@ -42,12 +42,12 @@ static Pipes getInputStreams(const std::vector<std::string> & column_names, cons
blocks.push_back(getBlockWithSize(column_names, block_size_in_bytes, stride, start));
pipes.emplace_back(std::make_shared<SourceFromInputStream>(std::make_shared<BlocksListBlockInputStream>(std::move(blocks))));
}
return pipes;
return Pipe::unitePipes(std::move(pipes));
}
static Pipes getInputStreamsEqualStride(const std::vector<std::string> & column_names, const std::vector<std::tuple<size_t, size_t, size_t>> & block_sizes)
static Pipe getInputStreamsEqualStride(const std::vector<std::string> & column_names, const std::vector<std::tuple<size_t, size_t, size_t>> & block_sizes)
{
Pipes pipes;
size_t i = 0;
@ -60,7 +60,7 @@ static Pipes getInputStreamsEqualStride(const std::vector<std::string> & column_
pipes.emplace_back(std::make_shared<SourceFromInputStream>(std::make_shared<BlocksListBlockInputStream>(std::move(blocks))));
i++;
}
return pipes;
return Pipe::unitePipes(std::move(pipes));
}
@ -79,15 +79,17 @@ TEST(MergingSortedTest, SimpleBlockSizeTest)
{
std::vector<std::string> key_columns{"K1", "K2", "K3"};
auto sort_description = getSortDescription(key_columns);
auto pipes = getInputStreams(key_columns, {{5, 1, 1}, {10, 1, 2}, {21, 1, 3}});
auto pipe = getInputStreams(key_columns, {{5, 1, 1}, {10, 1, 2}, {21, 1, 3}});
EXPECT_EQ(pipes.size(), 3);
EXPECT_EQ(pipe.numOutputPorts(), 3);
auto transform = std::make_shared<MergingSortedTransform>(pipes.front().getHeader(), pipes.size(), sort_description,
auto transform = std::make_shared<MergingSortedTransform>(pipe.getHeader(), pipe.numOutputPorts(), sort_description,
DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true);
pipe.addTransform(std::move(transform));
QueryPipeline pipeline;
pipeline.init(Pipe(std::move(pipes), std::move(transform)));
pipeline.init(std::move(pipe));
pipeline.setMaxThreads(1);
auto stream = std::make_shared<PipelineExecutingBlockInputStream>(std::move(pipeline));
@ -122,15 +124,17 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes)
{
std::vector<std::string> key_columns{"K1", "K2", "K3"};
auto sort_description = getSortDescription(key_columns);
auto pipes = getInputStreamsEqualStride(key_columns, {{1000, 1, 3}, {1500, 1, 3}, {1400, 1, 3}});
auto pipe = getInputStreamsEqualStride(key_columns, {{1000, 1, 3}, {1500, 1, 3}, {1400, 1, 3}});
EXPECT_EQ(pipes.size(), 3);
EXPECT_EQ(pipe.numOutputPorts(), 3);
auto transform = std::make_shared<MergingSortedTransform>(pipes.front().getHeader(), pipes.size(), sort_description,
auto transform = std::make_shared<MergingSortedTransform>(pipe.getHeader(), pipe.numOutputPorts(), sort_description,
DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true);
pipe.addTransform(std::move(transform));
QueryPipeline pipeline;
pipeline.init(Pipe(std::move(pipes), std::move(transform)));
pipeline.init(std::move(pipe));
pipeline.setMaxThreads(1);
auto stream = std::make_shared<PipelineExecutingBlockInputStream>(std::move(pipeline));

View File

@ -113,7 +113,7 @@ void SelectStreamFactory::createForShard(
const String &, const ASTPtr & query_ast,
const Context & context, const ThrottlerPtr & throttler,
const SelectQueryInfo &,
Pipes & res)
Pipes & pipes)
{
bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState;
bool add_totals = false;
@ -130,7 +130,7 @@ void SelectStreamFactory::createForShard(
auto emplace_local_stream = [&]()
{
res.emplace_back(createLocalStream(modified_query_ast, header, context, processed_stage).getPipe());
pipes.emplace_back(QueryPipeline::getPipe(createLocalStream(modified_query_ast, header, context, processed_stage)));
};
String modified_query = formattedAST(modified_query_ast);
@ -143,7 +143,7 @@ void SelectStreamFactory::createForShard(
if (!table_func_ptr)
remote_query_executor->setMainTable(main_table);
res.emplace_back(createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes));
pipes.emplace_back(createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes));
};
const auto & settings = context.getSettingsRef();
@ -270,7 +270,7 @@ void SelectStreamFactory::createForShard(
}
if (try_results.empty() || local_delay < max_remote_delay)
return createLocalStream(modified_query_ast, header, context, stage).getPipe();
return QueryPipeline::getPipe(createLocalStream(modified_query_ast, header, context, stage));
else
{
std::vector<IConnectionPool::Entry> connections;
@ -285,7 +285,7 @@ void SelectStreamFactory::createForShard(
}
};
res.emplace_back(createDelayedPipe(header, lazily_create_stream, add_totals, add_extremes));
pipes.emplace_back(createDelayedPipe(header, lazily_create_stream, add_totals, add_extremes));
}
else
emplace_remote_stream();

View File

@ -39,7 +39,7 @@ public:
const String & query, const ASTPtr & query_ast,
const Context & context, const ThrottlerPtr & throttler,
const SelectQueryInfo & query_info,
Pipes & res) override;
Pipes & pipes) override;
private:
const Block header;

View File

@ -74,7 +74,7 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin
return new_context;
}
Pipes executeQuery(
Pipe executeQuery(
IStreamFactory & stream_factory, const ClusterPtr & cluster, Poco::Logger * log,
const ASTPtr & query_ast, const Context & context, const Settings & settings, const SelectQueryInfo & query_info)
{
@ -106,7 +106,7 @@ Pipes executeQuery(
for (const auto & shard_info : cluster->getShardsInfo())
stream_factory.createForShard(shard_info, query, query_ast, new_context, throttler, query_info, res);
return res;
return Pipe::unitePipes(std::move(res));
}
}

View File

@ -12,7 +12,6 @@ class Cluster;
struct SelectQueryInfo;
class Pipe;
using Pipes = std::vector<Pipe>;
namespace ClusterProxy
{
@ -26,7 +25,7 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin
/// Execute a distributed query, creating a vector of BlockInputStreams, from which the result can be read.
/// `stream_factory` object encapsulates the logic of creating streams for a different type of query
/// (currently SELECT, DESCRIBE).
Pipes executeQuery(
Pipe executeQuery(
IStreamFactory & stream_factory, const ClusterPtr & cluster, Poco::Logger * log,
const ASTPtr & query_ast, const Context & context, const Settings & settings, const SelectQueryInfo & query_info);

View File

@ -87,9 +87,9 @@ BlockIO InterpreterAlterQuery::execute()
if (!partition_commands.empty())
{
table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, context.getSettingsRef());
auto partition_commands_pipes = table->alterPartition(query_ptr, metadata_snapshot, partition_commands, context);
if (!partition_commands_pipes.empty())
res.pipeline.init(std::move(partition_commands_pipes));
auto partition_commands_pipe = table->alterPartition(query_ptr, metadata_snapshot, partition_commands, context);
if (!partition_commands_pipe.empty())
res.pipeline.init(std::move(partition_commands_pipe));
}
if (!live_view_commands.empty())

View File

@ -269,10 +269,12 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
if (settings.graph)
{
auto processors = Pipe::detachProcessors(QueryPipeline::getPipe(std::move(*pipeline)));
if (settings.compact)
printPipelineCompact(pipeline->getProcessors(), buffer, settings.query_pipeline_options.header);
printPipelineCompact(processors, buffer, settings.query_pipeline_options.header);
else
printPipeline(pipeline->getProcessors(), buffer);
printPipeline(processors, buffer);
}
else
{

View File

@ -190,7 +190,7 @@ BlockIO InterpreterInsertQuery::execute()
}
}
res.pipeline.unitePipelines(std::move(pipelines), {});
res.pipeline = QueryPipeline::unitePipelines(std::move(pipelines), {});
}
}

View File

@ -516,7 +516,7 @@ void MergeJoin::mergeInMemoryRightBlocks()
pipeline.init(std::move(source));
/// TODO: there should be no split keys by blocks for RIGHT|FULL JOIN
pipeline.addPipe({std::make_shared<MergeSortingTransform>(pipeline.getHeader(), right_sort_description, max_rows_in_right_block, 0, 0, 0, nullptr, 0)});
pipeline.addTransform(std::make_shared<MergeSortingTransform>(pipeline.getHeader(), right_sort_description, max_rows_in_right_block, 0, 0, 0, nullptr, 0));
auto sorted_input = PipelineExecutingBlockInputStream(std::move(pipeline));

View File

@ -287,9 +287,6 @@ public:
size_t getStream() const { return stream_number; }
constexpr static size_t NO_STREAM = std::numeric_limits<size_t>::max();
void enableQuota() { has_quota = true; }
bool hasQuota() const { return has_quota; }
/// Step of QueryPlan from which processor was created.
void setQueryPlanStep(IQueryPlanStep * step, size_t group = 0)
{
@ -310,8 +307,6 @@ private:
size_t stream_number = NO_STREAM;
bool has_quota = false;
IQueryPlanStep * query_plan_step = nullptr;
size_t query_plan_step_group = 0;
};

View File

@ -1,44 +1,22 @@
#include <Processors/Pipe.h>
#include <IO/WriteHelpers.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include <Processors/ResizeProcessor.h>
#include <Processors/ConcatProcessor.h>
#include <Processors/LimitTransform.h>
#include <Processors/NullSink.h>
#include <Processors/Transforms/ExtremesTransform.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Processors/Sources/NullSource.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
static void checkSingleInput(const IProcessor & transform)
{
if (transform.getInputs().size() != 1)
throw Exception("Processor for pipe should have single input, "
"but " + transform.getName() + " has " +
toString(transform.getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR);
}
static void checkMultipleInputs(const IProcessor & transform, size_t num_inputs)
{
if (transform.getInputs().size() != num_inputs)
throw Exception("Processor for pipe should have " + toString(num_inputs) + " inputs, "
"but " + transform.getName() + " has " +
toString(transform.getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR);
}
static void checkSingleOutput(const IProcessor & transform)
{
if (transform.getOutputs().size() != 1)
throw Exception("Processor for pipe should have single output, "
"but " + transform.getName() + " has " +
toString(transform.getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR);
}
static void checkSimpleTransform(const IProcessor & transform)
{
checkSingleInput(transform);
checkSingleOutput(transform);
}
static void checkSource(const IProcessor & source)
{
if (!source.getInputs().empty())
@ -54,67 +32,664 @@ static void checkSource(const IProcessor & source)
toString(source.getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR);
}
static OutputPort * uniteExtremes(const OutputPortRawPtrs & ports, const Block & header, Processors & processors)
{
if (ports.empty())
return nullptr;
if (ports.size() == 1)
return ports.front();
/// Here we calculate extremes for extremes in case we unite several pipelines.
/// Example: select number from numbers(2) union all select number from numbers(3)
/// ->> Resize -> Extremes --(output port)----> Empty
/// --(extremes port)--> ...
auto resize = std::make_shared<ResizeProcessor>(header, ports.size(), 1);
auto extremes = std::make_shared<ExtremesTransform>(header);
auto sink = std::make_shared<EmptySink>(header);
auto * extremes_port = &extremes->getExtremesPort();
auto in = resize->getInputs().begin();
for (const auto & port : ports)
connect(*port, *(in++));
connect(resize->getOutputs().front(), extremes->getInputPort());
connect(extremes->getOutputPort(), sink->getPort());
processors.emplace_back(std::move(resize));
processors.emplace_back(std::move(extremes));
processors.emplace_back(std::move(sink));
return extremes_port;
}
static OutputPort * uniteTotals(const OutputPortRawPtrs & ports, const Block & header, Processors & processors)
{
if (ports.empty())
return nullptr;
if (ports.size() == 1)
return ports.front();
/// Calculate totals fro several streams.
/// Take totals from first sources which has any, skip others.
/// ->> Concat -> Limit
auto concat = std::make_shared<ConcatProcessor>(header, ports.size());
auto limit = std::make_shared<LimitTransform>(header, 1, 0);
auto * totals_port = &limit->getOutputPort();
auto in = concat->getInputs().begin();
for (const auto & port : ports)
connect(*port, *(in++));
connect(concat->getOutputs().front(), limit->getInputPort());
processors.emplace_back(std::move(concat));
processors.emplace_back(std::move(limit));
return totals_port;
}
Pipe::Holder & Pipe::Holder::operator=(Holder && rhs)
{
table_locks.insert(table_locks.end(), rhs.table_locks.begin(), rhs.table_locks.end());
storage_holders.insert(storage_holders.end(), rhs.storage_holders.begin(), rhs.storage_holders.end());
interpreter_context.insert(interpreter_context.end(),
rhs.interpreter_context.begin(), rhs.interpreter_context.end());
return *this;
}
Pipe::Pipe(ProcessorPtr source, OutputPort * output, OutputPort * totals, OutputPort * extremes)
{
if (!source->getInputs().empty())
throw Exception("Source for pipe shouldn't have any input, but " + source->getName() + " has " +
toString(source->getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR);
if (!output)
throw Exception("Cannot create Pipe from source because specified output port is nullptr",
ErrorCodes::LOGICAL_ERROR);
if (output == totals || output == extremes || (totals && totals == extremes))
throw Exception("Cannot create Pipe from source because some of specified ports are the same",
ErrorCodes::LOGICAL_ERROR);
header = output->getHeader();
/// Check that ports belong to source and all ports from source were specified.
{
auto & outputs = source->getOutputs();
size_t num_specified_ports = 0;
auto check_port_from_source = [&](OutputPort * port, std::string name)
{
if (!port)
return;
assertBlocksHaveEqualStructure(header, port->getHeader(), name);
++num_specified_ports;
auto it = std::find_if(outputs.begin(), outputs.end(), [port](const OutputPort & p) { return &p == port; });
if (it == outputs.end())
throw Exception("Cannot create Pipe because specified " + name + " port does not belong to source",
ErrorCodes::LOGICAL_ERROR);
};
check_port_from_source(output, "output");
check_port_from_source(totals, "totals");
check_port_from_source(extremes, "extremes");
if (num_specified_ports != outputs.size())
throw Exception("Cannot create Pipe from source because it has " + std::to_string(outputs.size()) +
" output ports, but " + std::to_string(num_specified_ports) + " were specified",
ErrorCodes::LOGICAL_ERROR);
}
totals_port = totals;
extremes_port = extremes;
output_ports.push_back(output);
processors.emplace_back(std::move(source));
max_parallel_streams = 1;
}
Pipe::Pipe(ProcessorPtr source)
{
if (auto * source_from_input_stream = typeid_cast<SourceFromInputStream *>(source.get()))
{
totals = source_from_input_stream->getTotalsPort();
extremes = source_from_input_stream->getExtremesPort();
/// Special case for SourceFromInputStream. Will remove it later.
totals_port = source_from_input_stream->getTotalsPort();
extremes_port = source_from_input_stream->getExtremesPort();
}
else if (source->getOutputs().size() != 1)
checkSource(*source);
output_port = &source->getOutputs().front();
if (collected_processors)
collected_processors->emplace_back(source);
output_ports.push_back(&source->getOutputs().front());
header = output_ports.front()->getHeader();
processors.emplace_back(std::move(source));
max_parallel_streams = 1;
}
Pipe::Pipe(Processors processors_, OutputPort * output_port_, OutputPort * totals_, OutputPort * extremes_)
: processors(std::move(processors_)), output_port(output_port_), totals(totals_), extremes(extremes_)
Pipe::Pipe(Processors processors_) : processors(std::move(processors_))
{
/// Create hash table with processors.
std::unordered_set<const IProcessor *> set;
for (const auto & processor : processors)
set.emplace(processor.get());
for (auto & processor : processors)
{
for (const auto & port : processor->getInputs())
{
if (!port.isConnected())
throw Exception("Cannot create Pipe because processor " + processor->getName() +
" has not connected input port", ErrorCodes::LOGICAL_ERROR);
const auto * connected_processor = &port.getOutputPort().getProcessor();
if (set.count(connected_processor) == 0)
throw Exception("Cannot create Pipe because processor " + processor->getName() +
" has input port which is connected with unknown processor " +
connected_processor->getName(), ErrorCodes::LOGICAL_ERROR);
}
for (auto & port : processor->getOutputs())
{
if (!port.isConnected())
{
output_ports.push_back(&port);
continue;
}
const auto * connected_processor = &port.getInputPort().getProcessor();
if (set.count(connected_processor) == 0)
throw Exception("Cannot create Pipe because processor " + processor->getName() +
" has output port which is connected with unknown processor " +
connected_processor->getName(), ErrorCodes::LOGICAL_ERROR);
}
}
if (output_ports.empty())
throw Exception("Cannot create Pipe because processors don't have any not-connected output ports",
ErrorCodes::LOGICAL_ERROR);
header = output_ports.front()->getHeader();
for (size_t i = 1; i < output_ports.size(); ++i)
assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipe");
max_parallel_streams = output_ports.size();
if (collected_processors)
for (const auto & processor : processors)
collected_processors->emplace_back(processor);
}
Pipe::Pipe(Pipes && pipes, ProcessorPtr transform)
static Pipes removeEmptyPipes(Pipes pipes)
{
checkSingleOutput(*transform);
checkMultipleInputs(*transform, pipes.size());
auto it = transform->getInputs().begin();
Pipes res;
res.reserve(pipes.size());
for (auto & pipe : pipes)
{
connect(*pipe.output_port, *it);
++it;
max_parallel_streams += pipe.max_parallel_streams;
processors.insert(processors.end(), pipe.processors.begin(), pipe.processors.end());
std::move(pipe.table_locks.begin(), pipe.table_locks.end(), std::back_inserter(table_locks));
std::move(pipe.interpreter_context.begin(), pipe.interpreter_context.end(), std::back_inserter(interpreter_context));
std::move(pipe.storage_holders.begin(), pipe.storage_holders.end(), std::back_inserter(storage_holders));
if (!pipe.empty())
res.emplace_back(std::move(pipe));
}
output_port = &transform->getOutputs().front();
return res;
}
Pipe Pipe::unitePipes(Pipes pipes)
{
return Pipe::unitePipes(std::move(pipes), nullptr);
}
Pipe Pipe::unitePipes(Pipes pipes, Processors * collected_processors)
{
Pipe res;
for (auto & pipe : pipes)
res.holder = std::move(pipe.holder); /// see move assignment for Pipe::Holder.
pipes = removeEmptyPipes(std::move(pipes));
if (pipes.empty())
return res;
if (pipes.size() == 1)
{
pipes[0].holder = std::move(res.holder);
return std::move(pipes[0]);
}
OutputPortRawPtrs totals;
OutputPortRawPtrs extremes;
res.header = pipes.front().header;
res.collected_processors = collected_processors;
for (auto & pipe : pipes)
{
assertBlocksHaveEqualStructure(res.header, pipe.header, "Pipe::unitePipes");
res.processors.insert(res.processors.end(), pipe.processors.begin(), pipe.processors.end());
res.output_ports.insert(res.output_ports.end(), pipe.output_ports.begin(), pipe.output_ports.end());
res.max_parallel_streams += pipe.max_parallel_streams;
if (pipe.totals_port)
totals.emplace_back(pipe.totals_port);
if (pipe.extremes_port)
extremes.emplace_back(pipe.extremes_port);
}
size_t num_processors = res.processors.size();
res.totals_port = uniteTotals(totals, res.header, res.processors);
res.extremes_port = uniteExtremes(extremes, res.header, res.processors);
if (res.collected_processors)
{
for (; num_processors < res.processors.size(); ++num_processors)
res.collected_processors->emplace_back(res.processors[num_processors]);
}
return res;
}
void Pipe::addSource(ProcessorPtr source)
{
checkSource(*source);
const auto & source_header = source->getOutputs().front().getHeader();
if (output_ports.empty())
header = source_header;
else
assertBlocksHaveEqualStructure(header, source_header, "Pipes");
if (collected_processors)
collected_processors->emplace_back(source);
output_ports.push_back(&source->getOutputs().front());
processors.emplace_back(std::move(source));
max_parallel_streams = std::max<size_t>(max_parallel_streams, output_ports.size());
}
void Pipe::addTotalsSource(ProcessorPtr source)
{
if (output_ports.empty())
throw Exception("Cannot add totals source to empty Pipe.", ErrorCodes::LOGICAL_ERROR);
if (totals_port)
throw Exception("Totals source was already added to Pipe.", ErrorCodes::LOGICAL_ERROR);
checkSource(*source);
const auto & source_header = output_ports.front()->getHeader();
assertBlocksHaveEqualStructure(header, source_header, "Pipes");
if (collected_processors)
collected_processors->emplace_back(source);
totals_port = &source->getOutputs().front();
processors.emplace_back(std::move(source));
}
void Pipe::addExtremesSource(ProcessorPtr source)
{
if (output_ports.empty())
throw Exception("Cannot add extremes source to empty Pipe.", ErrorCodes::LOGICAL_ERROR);
if (extremes_port)
throw Exception("Extremes source was already added to Pipe.", ErrorCodes::LOGICAL_ERROR);
checkSource(*source);
const auto & source_header = output_ports.front()->getHeader();
assertBlocksHaveEqualStructure(header, source_header, "Pipes");
if (collected_processors)
collected_processors->emplace_back(source);
extremes_port = &source->getOutputs().front();
processors.emplace_back(std::move(source));
}
static void dropPort(OutputPort *& port, Processors & processors, Processors * collected_processors)
{
if (port == nullptr)
return;
auto null_sink = std::make_shared<NullSink>(port->getHeader());
connect(*port, null_sink->getPort());
if (collected_processors)
collected_processors->emplace_back(null_sink);
processors.emplace_back(std::move(null_sink));
port = nullptr;
}
void Pipe::dropTotals()
{
dropPort(totals_port, processors, collected_processors);
}
void Pipe::dropExtremes()
{
dropPort(extremes_port, processors, collected_processors);
}
void Pipe::addTransform(ProcessorPtr transform)
{
addTransform(std::move(transform), nullptr, nullptr);
}
void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort * extremes)
{
if (output_ports.empty())
throw Exception("Cannot add transform to empty Pipe.", ErrorCodes::LOGICAL_ERROR);
auto & inputs = transform->getInputs();
if (inputs.size() != output_ports.size())
throw Exception("Cannot add transform " + transform->getName() + " to Pipes because "
"Processor has " + std::to_string(inputs.size()) + " input ports, "
"but " + std::to_string(output_ports.size()) + " expected", ErrorCodes::LOGICAL_ERROR);
if (totals && totals_port)
throw Exception("Cannot add transform with totals to Pipe because it already has totals.",
ErrorCodes::LOGICAL_ERROR);
if (extremes && extremes_port)
throw Exception("Cannot add transform with totals to Pipe because it already has totals.",
ErrorCodes::LOGICAL_ERROR);
if (totals)
totals_port = totals;
if (extremes)
extremes_port = extremes;
size_t next_output = 0;
for (auto & input : inputs)
{
connect(*output_ports[next_output], input);
++next_output;
}
auto & outputs = transform->getOutputs();
output_ports.clear();
output_ports.reserve(outputs.size());
bool found_totals = false;
bool found_extremes = false;
for (auto & output : outputs)
{
if (&output == totals)
found_totals = true;
else if (&output == extremes)
found_extremes = true;
else
output_ports.emplace_back(&output);
}
if (totals && !found_totals)
throw Exception("Cannot add transform " + transform->getName() + " to Pipes because "
"specified totals port does not belong to it", ErrorCodes::LOGICAL_ERROR);
if (extremes && !found_extremes)
throw Exception("Cannot add transform " + transform->getName() + " to Pipes because "
"specified extremes port does not belong to it", ErrorCodes::LOGICAL_ERROR);
if (output_ports.empty())
throw Exception("Cannot add transform " + transform->getName() + " to Pipes because it has no outputs",
ErrorCodes::LOGICAL_ERROR);
header = output_ports.front()->getHeader();
for (size_t i = 1; i < output_ports.size(); ++i)
assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipes");
// Temporarily skip this check. TotaslHavingTransform may return finalized totals but not finalized data.
// if (totals_port)
// assertBlocksHaveEqualStructure(header, totals_port->getHeader(), "Pipes");
if (extremes_port)
assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes");
if (collected_processors)
collected_processors->emplace_back(transform);
processors.emplace_back(std::move(transform));
max_parallel_streams = std::max<size_t>(max_parallel_streams, output_ports.size());
}
Pipe::Pipe(OutputPort * port) : output_port(port)
void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter)
{
if (output_ports.empty())
throw Exception("Cannot add simple transform to empty Pipe.", ErrorCodes::LOGICAL_ERROR);
Block new_header;
auto add_transform = [&](OutputPort *& port, StreamType stream_type)
{
if (!port)
return;
auto transform = getter(port->getHeader(), stream_type);
if (transform)
{
if (transform->getInputs().size() != 1)
throw Exception("Processor for query pipeline transform should have single input, "
"but " + transform->getName() + " has " +
toString(transform->getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR);
if (transform->getOutputs().size() != 1)
throw Exception("Processor for query pipeline transform should have single output, "
"but " + transform->getName() + " has " +
toString(transform->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR);
}
const auto & out_header = transform ? transform->getOutputs().front().getHeader()
: port->getHeader();
if (new_header)
assertBlocksHaveEqualStructure(new_header, out_header, "QueryPipeline");
else
new_header = out_header;
if (transform)
{
connect(*port, transform->getInputs().front());
port = &transform->getOutputs().front();
if (collected_processors)
collected_processors->emplace_back(transform);
processors.emplace_back(std::move(transform));
}
};
for (auto & port : output_ports)
add_transform(port, StreamType::Main);
add_transform(totals_port, StreamType::Totals);
add_transform(extremes_port, StreamType::Extremes);
header = std::move(new_header);
}
void Pipe::addProcessors(const Processors & processors_)
void Pipe::addSimpleTransform(const ProcessorGetter & getter)
{
processors.insert(processors.end(), processors_.begin(), processors_.end());
addSimpleTransform([&](const Block & stream_header, StreamType) { return getter(stream_header); });
}
void Pipe::addSimpleTransform(ProcessorPtr transform)
void Pipe::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter)
{
checkSimpleTransform(*transform);
connect(*output_port, transform->getInputs().front());
output_port = &transform->getOutputs().front();
processors.emplace_back(std::move(transform));
if (output_ports.empty())
throw Exception("Cannot set sink to empty Pipe.", ErrorCodes::LOGICAL_ERROR);
auto add_transform = [&](OutputPort *& stream, Pipe::StreamType stream_type)
{
if (!stream)
return;
auto transform = getter(stream->getHeader(), stream_type);
if (transform)
{
if (transform->getInputs().size() != 1)
throw Exception("Sink for query pipeline transform should have single input, "
"but " + transform->getName() + " has " +
toString(transform->getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR);
if (!transform->getOutputs().empty())
throw Exception("Sink for query pipeline transform should have no outputs, "
"but " + transform->getName() + " has " +
toString(transform->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR);
}
if (!transform)
transform = std::make_shared<NullSink>(stream->getHeader());
connect(*stream, transform->getInputs().front());
processors.emplace_back(std::move(transform));
};
for (auto & port : output_ports)
add_transform(port, StreamType::Main);
add_transform(totals_port, StreamType::Totals);
add_transform(extremes_port, StreamType::Extremes);
output_ports.clear();
header.clear();
}
void Pipe::setOutputFormat(ProcessorPtr output)
{
if (output_ports.empty())
throw Exception("Cannot set output format to empty Pipe.", ErrorCodes::LOGICAL_ERROR);
if (output_ports.size() != 1)
throw Exception("Cannot set output format to Pipe because single output port is expected, "
"but it has " + std::to_string(output_ports.size()) + " ports", ErrorCodes::LOGICAL_ERROR);
auto * format = dynamic_cast<IOutputFormat * >(output.get());
if (!format)
throw Exception("IOutputFormat processor expected for QueryPipeline::setOutputFormat.",
ErrorCodes::LOGICAL_ERROR);
auto & main = format->getPort(IOutputFormat::PortKind::Main);
auto & totals = format->getPort(IOutputFormat::PortKind::Totals);
auto & extremes = format->getPort(IOutputFormat::PortKind::Extremes);
if (!totals_port)
addTotalsSource(std::make_shared<NullSource>(totals.getHeader()));
if (!extremes_port)
addExtremesSource(std::make_shared<NullSource>(extremes.getHeader()));
if (collected_processors)
collected_processors->emplace_back(output);
processors.emplace_back(std::move(output));
connect(*output_ports.front(), main);
connect(*totals_port, totals);
connect(*extremes_port, extremes);
output_ports.clear();
header.clear();
}
void Pipe::transform(const Transformer & transformer)
{
if (output_ports.empty())
throw Exception("Cannot transform empty Pipe.", ErrorCodes::LOGICAL_ERROR);
auto new_processors = transformer(output_ports);
/// Create hash table with new processors.
std::unordered_set<const IProcessor *> set;
for (const auto & processor : new_processors)
set.emplace(processor.get());
for (const auto & port : output_ports)
{
if (!port->isConnected())
throw Exception("Transformation of Pipe is not valid because output port (" +
port->getHeader().dumpStructure() + ") is not connected", ErrorCodes::LOGICAL_ERROR);
set.emplace(&port->getProcessor());
}
output_ports.clear();
for (const auto & processor : new_processors)
{
for (const auto & port : processor->getInputs())
{
if (!port.isConnected())
throw Exception("Transformation of Pipe is not valid because processor " + processor->getName() +
" has not connected input port", ErrorCodes::LOGICAL_ERROR);
const auto * connected_processor = &port.getOutputPort().getProcessor();
if (set.count(connected_processor) == 0)
throw Exception("Transformation of Pipe is not valid because processor " + processor->getName() +
" has input port which is connected with unknown processor " +
connected_processor->getName(), ErrorCodes::LOGICAL_ERROR);
}
for (auto & port : processor->getOutputs())
{
if (!port.isConnected())
{
output_ports.push_back(&port);
continue;
}
const auto * connected_processor = &port.getInputPort().getProcessor();
if (set.count(connected_processor) == 0)
throw Exception("Transformation of Pipe is not valid because processor " + processor->getName() +
" has output port which is connected with unknown processor " +
connected_processor->getName(), ErrorCodes::LOGICAL_ERROR);
}
}
if (output_ports.empty())
throw Exception("Transformation of Pipe is not valid because processors don't have any "
"not-connected output ports", ErrorCodes::LOGICAL_ERROR);
header = output_ports.front()->getHeader();
for (size_t i = 1; i < output_ports.size(); ++i)
assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipe");
if (totals_port)
assertBlocksHaveEqualStructure(header, totals_port->getHeader(), "Pipes");
if (extremes_port)
assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes");
if (collected_processors)
{
for (const auto & processor : processors)
collected_processors->emplace_back(processor);
}
processors.insert(processors.end(), new_processors.begin(), new_processors.end());
max_parallel_streams = std::max<size_t>(max_parallel_streams, output_ports.size());
}
void Pipe::setLimits(const ISourceWithProgress::LocalLimits & limits)
@ -135,22 +710,4 @@ void Pipe::setQuota(const std::shared_ptr<const EnabledQuota> & quota)
}
}
void Pipe::pinSources(size_t executor_number)
{
for (auto & processor : processors)
{
if (auto * source = dynamic_cast<ISource *>(processor.get()))
source->setStream(executor_number);
}
}
void Pipe::enableQuota()
{
for (auto & processor : processors)
{
if (auto * source = dynamic_cast<ISource *>(processor.get()))
source->enableQuota();
}
}
}

View File

@ -11,89 +11,135 @@ using Pipes = std::vector<Pipe>;
class IStorage;
using StoragePtr = std::shared_ptr<IStorage>;
/// Pipe is a set of processors which represents the part of pipeline with single output.
/// All processors in pipe are connected. All ports are connected except the output one.
using OutputPortRawPtrs = std::vector<OutputPort *>;
/// Pipe is a set of processors which represents the part of pipeline.
/// Pipe contains a list of output ports, with specified port for totals and specified port for extremes.
/// All output ports have same header.
/// All other ports are connected, all connections are inside processors set.
class Pipe
{
public:
/// Create from source. It must have no input ports and single output.
/// Default constructor creates empty pipe. Generally, you cannot do anything with it except to check it is empty().
/// You cannot get empty pipe in any other way. All transforms check that result pipe is not empty.
Pipe() = default;
/// Create from source. Source must have no input ports and single output.
explicit Pipe(ProcessorPtr source);
/// Connect several pipes together with specified transform.
/// Transform must have the number of inputs equals to the number of pipes. And single output.
/// Will connect pipes outputs with transform inputs automatically.
Pipe(Pipes && pipes, ProcessorPtr transform);
/// Create pipe from output port. If pipe was created that way, it possibly will not have tree shape.
explicit Pipe(OutputPort * port);
/// Create from source with specified totals end extremes (may be nullptr). Ports should be owned by source.
explicit Pipe(ProcessorPtr source, OutputPort * output, OutputPort * totals, OutputPort * extremes);
/// Create from processors. Use all not-connected output ports as output_ports. Check invariants.
explicit Pipe(Processors processors_);
Pipe(const Pipe & other) = delete;
Pipe(Pipe && other) = default;
Pipe & operator=(const Pipe & other) = delete;
Pipe & operator=(Pipe && other) = default;
/// Append processors to pipe. After this, it possibly will not have tree shape.
void addProcessors(const Processors & processors_);
const Block & getHeader() const { return header; }
bool empty() const { return processors.empty(); }
size_t numOutputPorts() const { return output_ports.size(); }
size_t maxParallelStreams() const { return max_parallel_streams; }
OutputPort * getOutputPort(size_t pos) const { return output_ports[pos]; }
OutputPort * getTotalsPort() const { return totals_port; }
OutputPort * getExtremesPort() const { return extremes_port; }
OutputPort & getPort() const { return *output_port; }
const Block & getHeader() const { return output_port->getHeader(); }
/// Add processor to list, add it output ports to output_ports.
/// Processor shouldn't have input ports, output ports shouldn't be connected.
/// Output headers should have same structure and be compatible with current header (if not empty()).
void addSource(ProcessorPtr source);
/// Add transform to pipe. It must have single input and single output (is checked).
/// Input will be connected with current output port, output port will be updated.
void addSimpleTransform(ProcessorPtr transform);
/// Add totals and extremes.
void addTotalsSource(ProcessorPtr source);
void addExtremesSource(ProcessorPtr source);
Processors detachProcessors() && { return std::move(processors); }
/// Drop totals and extremes (create NullSink for them).
void dropTotals();
void dropExtremes();
/// Add processor to list. It should have size() input ports with compatible header.
/// Output ports should have same headers.
/// If totals or extremes are not empty, transform shouldn't change header.
void addTransform(ProcessorPtr transform);
void addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort * extremes);
enum class StreamType
{
Main = 0, /// Stream for query data. There may be several streams of this type.
Totals, /// Stream for totals. No more then one.
Extremes, /// Stream for extremes. No more then one.
};
using ProcessorGetter = std::function<ProcessorPtr(const Block & header)>;
using ProcessorGetterWithStreamKind = std::function<ProcessorPtr(const Block & header, StreamType stream_type)>;
/// Add transform with single input and single output for each port.
void addSimpleTransform(const ProcessorGetter & getter);
void addSimpleTransform(const ProcessorGetterWithStreamKind & getter);
using Transformer = std::function<Processors(OutputPortRawPtrs ports)>;
/// Transform Pipe in general way.
void transform(const Transformer & transformer);
/// Unite several pipes together. They should have same header.
static Pipe unitePipes(Pipes pipes);
/// Get processors from Pipe. Use it with cautious, it is easy to loss totals and extremes ports.
static Processors detachProcessors(Pipe pipe) { return std::move(pipe.processors); }
/// Specify quotas and limits for every ISourceWithProgress.
void setLimits(const SourceWithProgress::LocalLimits & limits);
void setQuota(const std::shared_ptr<const EnabledQuota> & quota);
/// Set information about preferred executor number for sources.
void pinSources(size_t executor_number);
void enableQuota();
/// Totals and extremes port.
void setTotalsPort(OutputPort * totals_) { totals = totals_; }
void setExtremesPort(OutputPort * extremes_) { extremes = extremes_; }
OutputPort * getTotalsPort() const { return totals; }
OutputPort * getExtremesPort() const { return extremes; }
size_t maxParallelStreams() const { return max_parallel_streams; }
/// Do not allow to change the table while the processors of pipe are alive.
/// TODO: move it to pipeline.
void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); }
void addTableLock(const TableLockHolder & lock) { holder.table_locks.push_back(lock); }
/// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible.
void addInterpreterContext(std::shared_ptr<Context> context) { interpreter_context.emplace_back(std::move(context)); }
void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); }
const std::vector<TableLockHolder> & getTableLocks() const { return table_locks; }
const std::vector<std::shared_ptr<Context>> & getContexts() const { return interpreter_context; }
const std::vector<StoragePtr> & getStorageHolders() const { return storage_holders; }
void addInterpreterContext(std::shared_ptr<Context> context) { holder.interpreter_context.emplace_back(std::move(context)); }
void addStorageHolder(StoragePtr storage) { holder.storage_holders.emplace_back(std::move(storage)); }
private:
Processors processors;
OutputPort * output_port = nullptr;
OutputPort * totals = nullptr;
OutputPort * extremes = nullptr;
/// Destruction order: processors, header, locks, temporary storages, local contexts
/// It is the max number of processors which can be executed in parallel for each step. See QueryPipeline::Streams.
struct Holder
{
Holder() = default;
Holder(Holder &&) = default;
/// Custom mode assignment does not destroy data from lhs. It appends data from rhs to lhs.
Holder& operator=(Holder &&);
/// Some processors may implicitly use Context or temporary Storage created by Interpreter.
/// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here,
/// because QueryPipeline is alive until query is finished.
std::vector<std::shared_ptr<Context>> interpreter_context;
std::vector<StoragePtr> storage_holders;
std::vector<TableLockHolder> table_locks;
};
Holder holder;
/// Header is common for all output below.
Block header;
Processors processors;
/// Output ports. Totals and extremes are allowed to be empty.
OutputPortRawPtrs output_ports;
OutputPort * totals_port = nullptr;
OutputPort * extremes_port = nullptr;
/// It is the max number of processors which can be executed in parallel for each step.
/// Usually, it's the same as the number of output ports.
size_t max_parallel_streams = 0;
std::vector<TableLockHolder> table_locks;
/// If is set, all newly created processors will be added to this too.
/// It is needed for debug. See QueryPipelineProcessorsCollector.
Processors * collected_processors = nullptr;
/// Some processors may implicitly use Context or temporary Storage created by Interpreter.
/// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here,
/// because QueryPipeline is alive until query is finished.
std::vector<std::shared_ptr<Context>> interpreter_context;
std::vector<StoragePtr> storage_holders;
/// This private constructor is used only from QueryPipeline.
/// It is not public, because QueryPipeline checks that processors are connected and have single output,
/// and therefore we can skip those checks.
/// Note that Pipe represents a tree if it was created using public interface. But this constructor can't assert it.
/// So, it's possible that TreeExecutorBlockInputStream could be unable to convert such Pipe to IBlockInputStream.
explicit Pipe(Processors processors_, OutputPort * output_port, OutputPort * totals, OutputPort * extremes);
/// This methods are for QueryPipeline. It is allowed to complete graph only there.
/// So, we may be sure that Pipe always has output port if not empty.
bool isCompleted() const { return !empty() && output_ports.empty(); }
static Pipe unitePipes(Pipes pipes, Processors * collected_processors);
void setSinks(const Pipe::ProcessorGetterWithStreamKind & getter);
void setOutputFormat(ProcessorPtr output);
friend class QueryPipeline;
};

View File

@ -2,13 +2,12 @@
#include <Processors/ResizeProcessor.h>
#include <Processors/ConcatProcessor.h>
#include <Processors/NullSink.h>
#include <Processors/LimitTransform.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/Transforms/TotalsHavingTransform.h>
#include <Processors/Transforms/ExtremesTransform.h>
#include <Processors/Transforms/CreatingSetsTransform.h>
#include <Processors/Transforms/ConvertingTransform.h>
#include <Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include <Processors/Executors/PipelineExecutor.h>
@ -39,11 +38,11 @@ void QueryPipeline::checkInitializedAndNotCompleted()
{
checkInitialized();
if (streams.empty())
if (pipe.isCompleted())
throw Exception("QueryPipeline was already completed.", ErrorCodes::LOGICAL_ERROR);
}
void QueryPipeline::checkSource(const ProcessorPtr & source, bool can_have_totals)
static void checkSource(const ProcessorPtr & source, bool can_have_totals)
{
if (!source->getInputs().empty())
throw Exception("Source for query pipeline shouldn't have any input, but " + source->getName() + " has " +
@ -62,289 +61,45 @@ void QueryPipeline::checkSource(const ProcessorPtr & source, bool can_have_total
toString(source->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR);
}
void QueryPipeline::init(Pipe pipe)
{
Pipes pipes;
pipes.emplace_back(std::move(pipe));
init(std::move(pipes));
}
static OutputPort * uniteExtremes(const std::vector<OutputPort *> & ports, const Block & header,
QueryPipeline::ProcessorsContainer & processors)
{
/// Here we calculate extremes for extremes in case we unite several pipelines.
/// Example: select number from numbers(2) union all select number from numbers(3)
/// ->> Resize -> Extremes --(output port)----> Null
/// --(extremes port)--> ...
auto resize = std::make_shared<ResizeProcessor>(header, ports.size(), 1);
auto extremes = std::make_shared<ExtremesTransform>(header);
auto sink = std::make_shared<EmptySink>(header);
auto * extremes_port = &extremes->getExtremesPort();
auto in = resize->getInputs().begin();
for (const auto & port : ports)
connect(*port, *(in++));
connect(resize->getOutputs().front(), extremes->getInputPort());
connect(extremes->getOutputPort(), sink->getPort());
processors.emplace(std::move(resize));
processors.emplace(std::move(extremes));
processors.emplace(std::move(sink));
return extremes_port;
}
static OutputPort * uniteTotals(const std::vector<OutputPort *> & ports, const Block & header,
QueryPipeline::ProcessorsContainer & processors)
{
/// Calculate totals from several streams.
/// Take totals from first sources which has any, skip others.
/// ->> Concat -> Limit
auto concat = std::make_shared<ConcatProcessor>(header, ports.size());
auto limit = std::make_shared<LimitTransform>(header, 1, 0);
auto * totals_port = &limit->getOutputPort();
auto in = concat->getInputs().begin();
for (const auto & port : ports)
connect(*port, *(in++));
connect(concat->getOutputs().front(), limit->getInputPort());
processors.emplace(std::move(concat));
processors.emplace(std::move(limit));
return totals_port;
}
void QueryPipeline::init(Pipes pipes)
void QueryPipeline::init(Pipe pipe_)
{
if (initialized())
throw Exception("Pipeline has already been initialized.", ErrorCodes::LOGICAL_ERROR);
if (pipes.empty())
throw Exception("Can't initialize pipeline with empty pipes list.", ErrorCodes::LOGICAL_ERROR);
if (pipe_.empty())
throw Exception("Can't initialize pipeline with empty pipe.", ErrorCodes::LOGICAL_ERROR);
/// Move locks from pipes to pipeline class.
for (auto & pipe : pipes)
{
for (const auto & lock : pipe.getTableLocks())
table_locks.emplace_back(lock);
for (const auto & context : pipe.getContexts())
interpreter_context.emplace_back(context);
for (const auto & storage : pipe.getStorageHolders())
storage_holders.emplace_back(storage);
}
std::vector<OutputPort *> totals;
std::vector<OutputPort *> extremes;
for (auto & pipe : pipes)
{
const auto & header = pipe.getHeader();
if (current_header)
assertBlocksHaveEqualStructure(current_header, header, "QueryPipeline");
else
current_header = header;
if (auto * totals_port = pipe.getTotalsPort())
{
assertBlocksHaveEqualStructure(current_header, totals_port->getHeader(), "QueryPipeline");
totals.emplace_back(totals_port);
}
if (auto * port = pipe.getExtremesPort())
{
assertBlocksHaveEqualStructure(current_header, port->getHeader(), "QueryPipeline");
extremes.emplace_back(port);
}
streams.addStream(&pipe.getPort(), pipe.maxParallelStreams());
processors.emplace(std::move(pipe).detachProcessors());
}
if (!totals.empty())
{
if (totals.size() == 1)
totals_having_port = totals.back();
else
totals_having_port = uniteTotals(totals, current_header, processors);
}
if (!extremes.empty())
{
if (extremes.size() == 1)
extremes_port = extremes.back();
else
extremes_port = uniteExtremes(extremes, current_header, processors);
}
pipe = std::move(pipe_);
}
static ProcessorPtr callProcessorGetter(
const Block & header, const QueryPipeline::ProcessorGetter & getter, QueryPipeline::StreamType)
void QueryPipeline::reset()
{
return getter(header);
Pipe pipe_to_destroy(std::move(pipe));
*this = QueryPipeline();
}
static ProcessorPtr callProcessorGetter(
const Block & header, const QueryPipeline::ProcessorGetterWithStreamKind & getter, QueryPipeline::StreamType kind)
{
return getter(header, kind);
}
template <typename TProcessorGetter>
void QueryPipeline::addSimpleTransformImpl(const TProcessorGetter & getter)
void QueryPipeline::addSimpleTransform(const Pipe::ProcessorGetter & getter)
{
checkInitializedAndNotCompleted();
Block header;
auto add_transform = [&](OutputPort *& stream, StreamType stream_type)
{
if (!stream)
return;
auto transform = callProcessorGetter(stream->getHeader(), getter, stream_type);
if (transform)
{
if (transform->getInputs().size() != 1)
throw Exception("Processor for query pipeline transform should have single input, "
"but " + transform->getName() + " has " +
toString(transform->getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR);
if (transform->getOutputs().size() != 1)
throw Exception("Processor for query pipeline transform should have single output, "
"but " + transform->getName() + " has " +
toString(transform->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR);
}
auto & out_header = transform ? transform->getOutputs().front().getHeader()
: stream->getHeader();
if (header)
assertBlocksHaveEqualStructure(header, out_header, "QueryPipeline");
else
header = out_header;
if (transform)
{
connect(*stream, transform->getInputs().front());
stream = &transform->getOutputs().front();
processors.emplace(std::move(transform));
}
};
for (auto & stream : streams)
add_transform(stream, StreamType::Main);
add_transform(totals_having_port, StreamType::Totals);
add_transform(extremes_port, StreamType::Extremes);
current_header = std::move(header);
pipe.addSimpleTransform(getter);
}
void QueryPipeline::addSimpleTransform(const ProcessorGetter & getter)
{
addSimpleTransformImpl(getter);
}
void QueryPipeline::addSimpleTransform(const ProcessorGetterWithStreamKind & getter)
{
addSimpleTransformImpl(getter);
}
void QueryPipeline::setSinks(const ProcessorGetterWithStreamKind & getter)
void QueryPipeline::addSimpleTransform(const Pipe::ProcessorGetterWithStreamKind & getter)
{
checkInitializedAndNotCompleted();
auto add_transform = [&](OutputPort *& stream, StreamType stream_type)
{
if (!stream)
return;
auto transform = getter(stream->getHeader(), stream_type);
if (transform)
{
if (transform->getInputs().size() != 1)
throw Exception("Sink for query pipeline transform should have single input, "
"but " + transform->getName() + " has " +
toString(transform->getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR);
if (!transform->getOutputs().empty())
throw Exception("Sink for query pipeline transform should have no outputs, "
"but " + transform->getName() + " has " +
toString(transform->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR);
}
if (!transform)
transform = std::make_shared<NullSink>(stream->getHeader());
connect(*stream, transform->getInputs().front());
processors.emplace(std::move(transform));
};
for (auto & stream : streams)
add_transform(stream, StreamType::Main);
add_transform(totals_having_port, StreamType::Totals);
add_transform(extremes_port, StreamType::Extremes);
streams.clear();
current_header.clear();
pipe.addSimpleTransform(getter);
}
void QueryPipeline::addPipe(Processors pipe)
void QueryPipeline::addTransform(ProcessorPtr transform)
{
checkInitializedAndNotCompleted();
pipe.addTransform(std::move(transform));
}
if (pipe.empty())
throw Exception("Can't add empty processors list to QueryPipeline.", ErrorCodes::LOGICAL_ERROR);
auto & first = pipe.front();
auto & last = pipe.back();
auto num_inputs = first->getInputs().size();
if (num_inputs != streams.size())
throw Exception("Can't add processors to QueryPipeline because first processor has " + toString(num_inputs) +
" input ports, but QueryPipeline has " + toString(streams.size()) + " streams.",
ErrorCodes::LOGICAL_ERROR);
auto stream = streams.begin();
for (auto & input : first->getInputs())
connect(**(stream++), input);
Block header;
streams.clear();
streams.reserve(last->getOutputs().size());
for (auto & output : last->getOutputs())
{
streams.addStream(&output, 0);
if (header)
assertBlocksHaveEqualStructure(header, output.getHeader(), "QueryPipeline");
else
header = output.getHeader();
}
if (totals_having_port)
assertBlocksHaveEqualStructure(header, totals_having_port->getHeader(), "QueryPipeline");
if (extremes_port)
assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "QueryPipeline");
processors.emplace(pipe);
current_header = std::move(header);
void QueryPipeline::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter)
{
checkInitializedAndNotCompleted();
pipe.setSinks(getter);
}
void QueryPipeline::addDelayedStream(ProcessorPtr source)
@ -352,14 +107,18 @@ void QueryPipeline::addDelayedStream(ProcessorPtr source)
checkInitializedAndNotCompleted();
checkSource(source, false);
assertBlocksHaveEqualStructure(current_header, source->getOutputs().front().getHeader(), "QueryPipeline");
assertBlocksHaveEqualStructure(getHeader(), source->getOutputs().front().getHeader(), "QueryPipeline");
IProcessor::PortNumbers delayed_streams = { streams.size() };
streams.addStream(&source->getOutputs().front(), 0);
processors.emplace(std::move(source));
IProcessor::PortNumbers delayed_streams = { pipe.numOutputPorts() };
pipe.addSource(std::move(source));
auto processor = std::make_shared<DelayedPortsProcessor>(current_header, streams.size(), delayed_streams);
addPipe({ std::move(processor) });
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);
}
void QueryPipeline::resize(size_t num_streams, bool force, bool strict)
@ -369,31 +128,14 @@ void QueryPipeline::resize(size_t num_streams, bool force, bool strict)
if (!force && num_streams == getNumStreams())
return;
has_resize = true;
ProcessorPtr resize;
if (strict)
resize = std::make_shared<StrictResizeProcessor>(current_header, getNumStreams(), num_streams);
resize = std::make_shared<StrictResizeProcessor>(getHeader(), getNumStreams(), num_streams);
else
resize = std::make_shared<ResizeProcessor>(current_header, getNumStreams(), num_streams);
resize = std::make_shared<ResizeProcessor>(getHeader(), getNumStreams(), num_streams);
auto stream = streams.begin();
for (auto & input : resize->getInputs())
connect(**(stream++), input);
streams.clear();
streams.reserve(num_streams);
for (auto & output : resize->getOutputs())
streams.addStream(&output, 0);
processors.emplace(std::move(resize));
}
void QueryPipeline::enableQuotaForCurrentStreams()
{
for (auto & stream : streams)
stream->getProcessor().enableQuota();
pipe.addTransform(std::move(resize));
}
void QueryPipeline::addTotalsHavingTransform(ProcessorPtr transform)
@ -404,28 +146,23 @@ void QueryPipeline::addTotalsHavingTransform(ProcessorPtr transform)
throw Exception("TotalsHavingTransform expected for QueryPipeline::addTotalsHavingTransform.",
ErrorCodes::LOGICAL_ERROR);
if (totals_having_port)
if (pipe.getTotalsPort())
throw Exception("Totals having transform was already added to pipeline.", ErrorCodes::LOGICAL_ERROR);
resize(1);
connect(*streams.front(), transform->getInputs().front());
auto & outputs = transform->getOutputs();
streams.assign({ &outputs.front() });
totals_having_port = &outputs.back();
current_header = outputs.front().getHeader();
processors.emplace(std::move(transform));
auto * totals_port = &transform->getOutputs().back();
pipe.addTransform(std::move(transform), totals_port, nullptr);
}
void QueryPipeline::addDefaultTotals()
{
checkInitializedAndNotCompleted();
if (totals_having_port)
if (pipe.getTotalsPort())
throw Exception("Totals having transform was already added to pipeline.", ErrorCodes::LOGICAL_ERROR);
const auto & current_header = getHeader();
Columns columns;
columns.reserve(current_header.columns());
@ -437,66 +174,26 @@ void QueryPipeline::addDefaultTotals()
}
auto source = std::make_shared<SourceFromSingleChunk>(current_header, Chunk(std::move(columns), 1));
totals_having_port = &source->getPort();
processors.emplace(std::move(source));
}
void QueryPipeline::addTotals(ProcessorPtr source)
{
checkInitializedAndNotCompleted();
if (totals_having_port)
throw Exception("Totals having transform was already added to pipeline.", ErrorCodes::LOGICAL_ERROR);
checkSource(source, false);
assertBlocksHaveEqualStructure(current_header, source->getOutputs().front().getHeader(), "QueryPipeline");
totals_having_port = &source->getOutputs().front();
processors.emplace(std::move(source));
pipe.addTotalsSource(std::move(source));
}
void QueryPipeline::dropTotalsAndExtremes()
{
auto drop_port = [&](OutputPort *& port)
{
auto null_sink = std::make_shared<NullSink>(port->getHeader());
connect(*port, null_sink->getPort());
processors.emplace(std::move(null_sink));
port = nullptr;
};
if (totals_having_port)
drop_port(totals_having_port);
if (extremes_port)
drop_port(extremes_port);
pipe.dropTotals();
pipe.dropExtremes();
}
void QueryPipeline::addExtremesTransform()
{
checkInitializedAndNotCompleted();
if (extremes_port)
if (pipe.getExtremesPort())
throw Exception("Extremes transform was already added to pipeline.", ErrorCodes::LOGICAL_ERROR);
std::vector<OutputPort *> extremes;
extremes.reserve(streams.size());
for (auto & stream : streams)
{
auto transform = std::make_shared<ExtremesTransform>(current_header);
connect(*stream, transform->getInputPort());
stream = &transform->getOutputPort();
extremes.push_back(&transform->getExtremesPort());
processors.emplace(std::move(transform));
}
if (extremes.size() == 1)
extremes_port = extremes.front();
else
extremes_port = uniteExtremes(extremes, current_header, processors);
resize(1);
auto transform = std::make_shared<ExtremesTransform>(getHeader());
auto * port = &transform->getExtremesPort();
pipe.addTransform(std::move(transform), nullptr, port);
}
void QueryPipeline::addCreatingSetsTransform(ProcessorPtr transform)
@ -509,94 +206,49 @@ void QueryPipeline::addCreatingSetsTransform(ProcessorPtr transform)
resize(1);
auto concat = std::make_shared<ConcatProcessor>(current_header, 2);
connect(transform->getOutputs().front(), concat->getInputs().front());
connect(*streams.back(), concat->getInputs().back());
streams.assign({ &concat->getOutputs().front() });
processors.emplace(std::move(transform));
processors.emplace(std::move(concat));
/// Order is important for concat. Connect manually.
pipe.transform([&](OutputPortRawPtrs ports) -> Processors
{
auto concat = std::make_shared<ConcatProcessor>(getHeader(), 2);
connect(transform->getOutputs().front(), concat->getInputs().front());
connect(*ports.back(), concat->getInputs().back());
return { std::move(concat), std::move(transform) };
});
}
void QueryPipeline::setOutputFormat(ProcessorPtr output)
{
checkInitializedAndNotCompleted();
auto * format = dynamic_cast<IOutputFormat * >(output.get());
if (!format)
throw Exception("IOutputFormat processor expected for QueryPipeline::setOutputFormat.", ErrorCodes::LOGICAL_ERROR);
if (output_format)
throw Exception("QueryPipeline already has output.", ErrorCodes::LOGICAL_ERROR);
output_format = format;
resize(1);
auto & main = format->getPort(IOutputFormat::PortKind::Main);
auto & totals = format->getPort(IOutputFormat::PortKind::Totals);
auto & extremes = format->getPort(IOutputFormat::PortKind::Extremes);
if (!totals_having_port)
{
auto null_source = std::make_shared<NullSource>(totals.getHeader());
totals_having_port = &null_source->getPort();
processors.emplace(std::move(null_source));
}
if (!extremes_port)
{
auto null_source = std::make_shared<NullSource>(extremes.getHeader());
extremes_port = &null_source->getPort();
processors.emplace(std::move(null_source));
}
processors.emplace(std::move(output));
connect(*streams.front(), main);
connect(*totals_having_port, totals);
connect(*extremes_port, extremes);
streams.clear();
current_header.clear();
extremes_port = nullptr;
totals_having_port = nullptr;
output_format = dynamic_cast<IOutputFormat * >(output.get());
pipe.setOutputFormat(std::move(output));
initRowsBeforeLimit();
}
void QueryPipeline::unitePipelines(
std::vector<std::unique_ptr<QueryPipeline>> pipelines, const Block & common_header, size_t max_threads_limit)
QueryPipeline QueryPipeline::unitePipelines(
std::vector<std::unique_ptr<QueryPipeline>> pipelines,
const Block & common_header,
size_t max_threads_limit,
Processors * collected_processors)
{
/// 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.
bool will_limit_max_threads = !initialized() || max_threads != 0;
if (initialized())
{
addSimpleTransform([&](const Block & header)
{
return std::make_shared<ConvertingTransform>(
header, common_header, ConvertingTransform::MatchColumnsMode::Position);
});
}
std::vector<OutputPort *> extremes;
std::vector<OutputPort *> totals;
if (extremes_port)
extremes.push_back(extremes_port);
if (totals_having_port)
totals.push_back(totals_having_port);
bool will_limit_max_threads = true;
size_t max_threads = 0;
Pipes pipes;
for (auto & pipeline_ptr : pipelines)
{
auto & pipeline = *pipeline_ptr;
pipeline.checkInitialized();
pipeline.processors.setCollectedProcessors(processors.getCollectedProcessors());
pipeline.pipe.collected_processors = collected_processors;
if (!pipeline.isCompleted())
{
@ -607,36 +259,7 @@ void QueryPipeline::unitePipelines(
});
}
if (pipeline.extremes_port)
{
auto converting = std::make_shared<ConvertingTransform>(
pipeline.current_header, common_header, ConvertingTransform::MatchColumnsMode::Position);
connect(*pipeline.extremes_port, converting->getInputPort());
extremes.push_back(&converting->getOutputPort());
processors.emplace(std::move(converting));
}
/// Take totals only from first port.
if (pipeline.totals_having_port)
{
auto converting = std::make_shared<ConvertingTransform>(
pipeline.current_header, common_header, ConvertingTransform::MatchColumnsMode::Position);
connect(*pipeline.totals_having_port, converting->getInputPort());
totals.push_back(&converting->getOutputPort());
processors.emplace(std::move(converting));
}
auto * collector = processors.setCollectedProcessors(nullptr);
processors.emplace(pipeline.processors.detach());
processors.setCollectedProcessors(collector);
streams.addStreams(pipeline.streams);
table_locks.insert(table_locks.end(), std::make_move_iterator(pipeline.table_locks.begin()), std::make_move_iterator(pipeline.table_locks.end()));
interpreter_context.insert(interpreter_context.end(), pipeline.interpreter_context.begin(), pipeline.interpreter_context.end());
storage_holders.insert(storage_holders.end(), pipeline.storage_holders.begin(), pipeline.storage_holders.end());
pipes.emplace_back(std::move(pipeline.pipe));
max_threads += pipeline.max_threads;
will_limit_max_threads = will_limit_max_threads && pipeline.max_threads != 0;
@ -647,33 +270,21 @@ void QueryPipeline::unitePipelines(
max_threads_limit = pipeline.max_threads;
}
if (!will_limit_max_threads)
max_threads = 0;
else
limitMaxThreads(max_threads_limit);
QueryPipeline pipeline;
pipeline.init(Pipe::unitePipes(std::move(pipes), collected_processors));
if (!extremes.empty())
if (will_limit_max_threads)
{
if (extremes.size() == 1)
extremes_port = extremes.back();
else
extremes_port = uniteExtremes(extremes, common_header, processors);
pipeline.setMaxThreads(max_threads);
pipeline.limitMaxThreads(max_threads_limit);
}
if (!totals.empty())
{
if (totals.size() == 1)
totals_having_port = totals.back();
else
totals_having_port = uniteTotals(totals, common_header, processors);
}
current_header = common_header;
return pipeline;
}
void QueryPipeline::setProgressCallback(const ProgressCallback & callback)
{
for (auto & processor : processors.get())
for (auto & processor : pipe.processors)
{
if (auto * source = dynamic_cast<ISourceWithProgress *>(processor.get()))
source->setProgressCallback(callback);
@ -687,7 +298,7 @@ void QueryPipeline::setProcessListElement(QueryStatus * elem)
{
process_list_element = elem;
for (auto & processor : processors.get())
for (auto & processor : pipe.processors)
{
if (auto * source = dynamic_cast<ISourceWithProgress *>(processor.get()))
source->setProcessListElement(elem);
@ -791,109 +402,29 @@ void QueryPipeline::initRowsBeforeLimit()
output_format->setRowsBeforeLimitCounter(rows_before_limit_at_least);
}
Pipe QueryPipeline::getPipe() &&
{
resize(1);
return std::move(std::move(*this).getPipes()[0]);
}
Pipes QueryPipeline::getPipes() &&
{
Pipe pipe(processors.detach(), streams.at(0), totals_having_port, extremes_port);
pipe.max_parallel_streams = streams.maxParallelStreams();
for (auto & lock : table_locks)
pipe.addTableLock(lock);
for (auto & context : interpreter_context)
pipe.addInterpreterContext(context);
for (auto & storage : storage_holders)
pipe.addStorageHolder(storage);
if (totals_having_port)
pipe.setTotalsPort(totals_having_port);
if (extremes_port)
pipe.setExtremesPort(extremes_port);
Pipes pipes;
pipes.emplace_back(std::move(pipe));
for (size_t i = 1; i < streams.size(); ++i)
pipes.emplace_back(Pipe(streams[i]));
return pipes;
}
PipelineExecutorPtr QueryPipeline::execute()
{
if (!isCompleted())
throw Exception("Cannot execute pipeline because it is not completed.", ErrorCodes::LOGICAL_ERROR);
return std::make_shared<PipelineExecutor>(processors.get(), process_list_element);
return std::make_shared<PipelineExecutor>(pipe.processors, process_list_element);
}
QueryPipeline & QueryPipeline::operator= (QueryPipeline && rhs)
void QueryPipeline::setCollectedProcessors(Processors * processors)
{
/// Reset primitive fields
process_list_element = rhs.process_list_element;
rhs.process_list_element = nullptr;
max_threads = rhs.max_threads;
rhs.max_threads = 0;
output_format = rhs.output_format;
rhs.output_format = nullptr;
has_resize = rhs.has_resize;
rhs.has_resize = false;
extremes_port = rhs.extremes_port;
rhs.extremes_port = nullptr;
totals_having_port = rhs.totals_having_port;
rhs.totals_having_port = nullptr;
/// Move these fields in destruction order (it's important)
streams = std::move(rhs.streams);
processors = std::move(rhs.processors);
current_header = std::move(rhs.current_header);
table_locks = std::move(rhs.table_locks);
storage_holders = std::move(rhs.storage_holders);
interpreter_context = std::move(rhs.interpreter_context);
return *this;
pipe.collected_processors = processors;
}
void QueryPipeline::ProcessorsContainer::emplace(ProcessorPtr processor)
{
if (collected_processors)
collected_processors->emplace_back(processor);
processors.emplace_back(std::move(processor));
}
void QueryPipeline::ProcessorsContainer::emplace(Processors processors_)
{
for (auto & processor : processors_)
emplace(std::move(processor));
}
Processors * QueryPipeline::ProcessorsContainer::setCollectedProcessors(Processors * collected_processors_)
{
if (collected_processors && collected_processors_)
throw Exception("Cannot set collected processors to QueryPipeline because "
"another one object was already created for current pipeline." , ErrorCodes::LOGICAL_ERROR);
std::swap(collected_processors, collected_processors_);
return collected_processors_;
}
QueryPipelineProcessorsCollector::QueryPipelineProcessorsCollector(QueryPipeline & pipeline_, IQueryPlanStep * step_)
: pipeline(pipeline_), step(step_)
{
pipeline.processors.setCollectedProcessors(&processors);
pipeline.setCollectedProcessors(&processors);
}
QueryPipelineProcessorsCollector::~QueryPipelineProcessorsCollector()
{
pipeline.processors.setCollectedProcessors(nullptr);
pipeline.setCollectedProcessors(nullptr);
}
Processors QueryPipelineProcessorsCollector::detachProcessors(size_t group)

View File

@ -12,119 +12,40 @@
namespace DB
{
using TableLockHolders = std::vector<TableLockHolder>;
class Context;
class IOutputFormat;
class QueryPipelineProcessorsCollector;
struct AggregatingTransformParams;
using AggregatingTransformParamsPtr = std::shared_ptr<AggregatingTransformParams>;
class QueryPipeline
{
private:
/// It's a wrapper over std::vector<OutputPort *>
/// Is needed to support invariant for max_parallel_streams (see comment below).
class Streams
{
public:
auto size() const { return data.size(); }
bool empty() const { return size() == 0; }
auto begin() { return data.begin(); }
auto end() { return data.end(); }
auto & front() { return data.front(); }
auto & back() { return data.back(); }
auto & at(size_t pos) { return data.at(pos); }
auto & operator[](size_t pos) { return data[pos]; }
void clear() { data.clear(); }
void reserve(size_t size_) { data.reserve(size_); }
void addStream(OutputPort * port, size_t port_max_parallel_streams)
{
data.push_back(port);
max_parallel_streams = std::max<size_t>(max_parallel_streams + port_max_parallel_streams, data.size());
}
void addStreams(Streams & other)
{
data.insert(data.end(), other.begin(), other.end());
max_parallel_streams = std::max<size_t>(max_parallel_streams + other.max_parallel_streams, data.size());
}
void assign(std::initializer_list<OutputPort *> list)
{
data = list;
max_parallel_streams = std::max<size_t>(max_parallel_streams, data.size());
}
size_t maxParallelStreams() const { return max_parallel_streams; }
private:
std::vector<OutputPort *> data;
/// It is the max number of processors which can be executed in parallel for each step.
/// Logically, it is the upper limit on the number of threads needed to execute this pipeline.
/// Initially, it is the number of sources. It may be increased after resize, aggregation, etc.
/// This number is never decreased, and it is calculated as max(streams.size()) over all streams while building.
size_t max_parallel_streams = 0;
};
public:
class ProcessorsContainer
{
public:
bool empty() const { return processors.empty(); }
void emplace(ProcessorPtr processor);
void emplace(Processors processors_);
Processors * getCollectedProcessors() const { return collected_processors; }
Processors * setCollectedProcessors(Processors * collected_processors);
Processors & get() { return processors; }
const Processors & get() const { return processors; }
Processors detach() { return std::move(processors); }
private:
/// All added processors.
Processors processors;
/// If is set, all newly created processors will be added to this too.
/// It is needed for debug. See QueryPipelineProcessorsCollector below.
Processors * collected_processors = nullptr;
};
QueryPipeline() = default;
QueryPipeline(QueryPipeline &&) = default;
~QueryPipeline() = default;
QueryPipeline(QueryPipeline &&) = default;
QueryPipeline(const QueryPipeline &) = delete;
QueryPipeline & operator= (QueryPipeline && rhs) = default;
QueryPipeline & operator= (const QueryPipeline & rhs) = delete;
QueryPipeline & operator= (QueryPipeline && rhs);
/// All pipes must have same header.
void init(Pipes pipes);
void init(Pipe pipe); /// Simple init for single pipe
bool initialized() { return !processors.empty(); }
bool isCompleted() { return initialized() && streams.empty(); }
void init(Pipe pipe);
/// Clear and release all resources.
void reset();
/// Type of logical data stream for simple transform.
/// Sometimes it's important to know which part of pipeline we are working for.
/// Example: ExpressionTransform need special logic for totals.
enum class StreamType
{
Main = 0, /// Stream for query data. There may be several streams of this type.
Totals, /// Stream for totals. No more then one.
Extremes, /// Stream for extremes. No more then one.
};
bool initialized() { return !pipe.empty(); }
bool isCompleted() { return pipe.isCompleted(); }
using ProcessorGetter = std::function<ProcessorPtr(const Block & header)>;
using ProcessorGetterWithStreamKind = std::function<ProcessorPtr(const Block & header, StreamType stream_type)>;
using StreamType = Pipe::StreamType;
/// Add transform with simple input and simple output for each port.
void addSimpleTransform(const ProcessorGetter & getter);
void addSimpleTransform(const ProcessorGetterWithStreamKind & getter);
/// Add several processors. They must have same header for inputs and same for outputs.
/// Total number of inputs must be the same as the number of streams. Output ports will become new streams.
void addPipe(Processors pipe);
void addSimpleTransform(const Pipe::ProcessorGetter & getter);
void addSimpleTransform(const Pipe::ProcessorGetterWithStreamKind & getter);
/// Add transform with getNumStreams() input ports.
void addTransform(ProcessorPtr transform);
/// Add TotalsHavingTransform. Resize pipeline to single input. Adds totals port.
void addTotalsHavingTransform(ProcessorPtr transform);
/// Add transform which calculates extremes. This transform adds extremes port and doesn't change inputs number.
@ -137,44 +58,41 @@ public:
IOutputFormat * getOutputFormat() const { return output_format; }
/// Sink is a processor with single input port and no output ports. Creates sink for each output port.
/// Pipeline will be completed after this transformation.
void setSinks(const ProcessorGetterWithStreamKind & getter);
void setSinks(const Pipe::ProcessorGetterWithStreamKind & getter);
/// Add totals which returns one chunk with single row with defaults.
void addDefaultTotals();
/// Add already calculated totals.
void addTotals(ProcessorPtr source);
/// Forget about current totals and extremes. It is needed before aggregation, cause they will be calculated again.
void dropTotalsAndExtremes();
/// Will read from this stream after all data was read from other streams.
void addDelayedStream(ProcessorPtr source);
/// Check if resize transform was used. (In that case another distinct transform will be added).
bool hasMixedStreams() const { return has_resize || hasMoreThanOneStream(); }
void addMergingAggregatedMemoryEfficientTransform(AggregatingTransformParamsPtr params, size_t num_merging_processors);
/// Changes the number of input ports if needed. Adds ResizeTransform.
void resize(size_t num_streams, bool force = false, bool strict = false);
void enableQuotaForCurrentStreams();
/// Unite several pipelines together. Result pipeline would have common_header structure.
/// If collector is used, it will collect only newly-added processors, but not processors from pipelines.
void unitePipelines(std::vector<std::unique_ptr<QueryPipeline>> pipelines, const Block & common_header, size_t max_threads_limit = 0);
static QueryPipeline unitePipelines(
std::vector<std::unique_ptr<QueryPipeline>> pipelines,
const Block & common_header,
size_t max_threads_limit = 0,
Processors * collected_processors = nullptr);
PipelineExecutorPtr execute();
size_t getNumStreams() const { return streams.size(); }
size_t getNumStreams() const { return pipe.numOutputPorts(); }
bool hasMoreThanOneStream() const { return getNumStreams() > 1; }
bool hasTotals() const { return totals_having_port != nullptr; }
bool hasTotals() const { return pipe.getTotalsPort() != nullptr; }
const Block & getHeader() const { return current_header; }
const Block & getHeader() const { return pipe.getHeader(); }
void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); }
void addInterpreterContext(std::shared_ptr<Context> context) { interpreter_context.emplace_back(std::move(context)); }
void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); }
void addTableLock(const TableLockHolder & lock) { pipe.addTableLock(lock); }
void addInterpreterContext(std::shared_ptr<Context> context) { pipe.addInterpreterContext(std::move(context)); }
void addStorageHolder(StoragePtr storage) { pipe.addStorageHolder(std::move(storage)); }
/// For compatibility with IBlockInputStream.
void setProgressCallback(const ProgressCallback & callback);
@ -183,7 +101,7 @@ public:
/// Recommend number of threads for pipeline execution.
size_t getNumThreads() const
{
auto num_threads = streams.maxParallelStreams();
auto num_threads = pipe.maxParallelStreams();
if (max_threads)
num_threads = std::min(num_threads, max_threads);
@ -201,38 +119,12 @@ public:
max_threads = max_threads_;
}
/// Convert query pipeline to single or several pipes.
Pipe getPipe() &&;
Pipes getPipes() &&;
/// Get internal processors.
const Processors & getProcessors() const { return processors.get(); }
/// Convert query pipeline to pipe.
static Pipe getPipe(QueryPipeline pipeline) { return std::move(pipeline.pipe); }
private:
/// Destruction order: processors, header, locks, temporary storages, local contexts
/// Some Streams (or Processors) may implicitly use Context or temporary Storage created by Interpreter.
/// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here,
/// because QueryPipeline is alive until query is finished.
std::vector<std::shared_ptr<Context>> interpreter_context;
std::vector<StoragePtr> storage_holders;
TableLockHolders table_locks;
/// Common header for each stream.
Block current_header;
ProcessorsContainer processors;
/// Port for each independent "stream".
Streams streams;
/// Special ports for extremes and totals having.
OutputPort * totals_having_port = nullptr;
OutputPort * extremes_port = nullptr;
/// If resize processor was added to pipeline.
bool has_resize = false;
Pipe pipe;
IOutputFormat * output_format = nullptr;
/// Limit on the number of threads. Zero means no limit.
@ -243,13 +135,11 @@ private:
void checkInitialized();
void checkInitializedAndNotCompleted();
static void checkSource(const ProcessorPtr & source, bool can_have_totals);
template <typename TProcessorGetter>
void addSimpleTransformImpl(const TProcessorGetter & getter);
void initRowsBeforeLimit();
void setCollectedProcessors(Processors * processors);
friend class QueryPipelineProcessorsCollector;
};

View File

@ -101,7 +101,7 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline)
group_by_sort_description,
max_block_size);
pipeline.addPipe({ std::move(transform) });
pipeline.addTransform(std::move(transform));
aggregating_sorted = collector.detachProcessors(1);
}
else
@ -120,8 +120,6 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline)
});
finalizing = collector.detachProcessors(2);
pipeline.enableQuotaForCurrentStreams();
return;
}
}
@ -156,8 +154,6 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline)
aggregating = collector.detachProcessors(0);
}
pipeline.enableQuotaForCurrentStreams();
}
void AggregatingStep::describeActions(FormatSettings & settings) const

View File

@ -63,11 +63,9 @@ void FinishSortingStep::transformPipeline(QueryPipeline & pipeline)
prefix_description,
max_block_size, limit_for_merging);
pipeline.addPipe({ std::move(transform) });
pipeline.addTransform(std::move(transform));
}
pipeline.enableQuotaForCurrentStreams();
if (need_finish_sorting)
{
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr

View File

@ -47,7 +47,7 @@ void LimitStep::transformPipeline(QueryPipeline & pipeline)
auto transform = std::make_shared<LimitTransform>(
pipeline.getHeader(), limit, offset, pipeline.getNumStreams(), always_read_till_end, with_ties, description);
pipeline.addPipe({std::move(transform)});
pipeline.addTransform(std::move(transform));
}
void LimitStep::describeActions(FormatSettings & settings) const

View File

@ -59,16 +59,8 @@ void MergingAggregatedStep::transformPipeline(QueryPipeline & pipeline)
? static_cast<size_t>(memory_efficient_merge_threads)
: static_cast<size_t>(max_threads);
auto pipe = createMergingAggregatedMemoryEfficientPipe(
pipeline.getHeader(),
params,
pipeline.getNumStreams(),
num_merge_threads);
pipeline.addPipe(std::move(pipe));
pipeline.addMergingAggregatedMemoryEfficientTransform(params, num_merge_threads);
}
pipeline.enableQuotaForCurrentStreams();
}
void MergingAggregatedStep::describeActions(FormatSettings & settings) const

View File

@ -58,9 +58,7 @@ void MergingSortedStep::transformPipeline(QueryPipeline & pipeline)
sort_description,
max_block_size, limit);
pipeline.addPipe({ std::move(transform) });
pipeline.enableQuotaForCurrentStreams();
pipeline.addTransform(std::move(transform));
}
}

View File

@ -33,7 +33,7 @@ void OffsetStep::transformPipeline(QueryPipeline & pipeline)
auto transform = std::make_shared<OffsetTransform>(
pipeline.getHeader(), offset, pipeline.getNumStreams());
pipeline.addPipe({std::move(transform)});
pipeline.addTransform(std::move(transform));
}
void OffsetStep::describeActions(FormatSettings & settings) const

View File

@ -7,7 +7,6 @@
#include <Processors/Pipe.h>
#include <Processors/QueryPipeline.h>
#include <Storages/IStorage.h>
#include <Processors/Transforms/ConvertingTransform.h>
namespace DB
{
@ -37,34 +36,44 @@ ReadFromStorageStep::ReadFromStorageStep(
/// Note: we read from storage in constructor of step because we don't know real header before reading.
/// It will be fixed when storage return QueryPlanStep itself.
Pipes pipes = storage->read(required_columns, metadata_snapshot, query_info, *context, processing_stage, max_block_size, max_streams);
Pipe pipe = storage->read(required_columns, metadata_snapshot, query_info, *context, processing_stage, max_block_size, max_streams);
if (pipes.empty())
if (pipe.empty())
{
Pipe pipe(std::make_shared<NullSource>(metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID())));
pipe = Pipe(std::make_shared<NullSource>(metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID())));
if (query_info.prewhere_info)
{
if (query_info.prewhere_info->alias_actions)
pipe.addSimpleTransform(std::make_shared<ExpressionTransform>(
pipe.getHeader(), query_info.prewhere_info->alias_actions));
{
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<ExpressionTransform>(header, query_info.prewhere_info->alias_actions);
});
}
pipe.addSimpleTransform(std::make_shared<FilterTransform>(
pipe.getHeader(),
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<FilterTransform>(
header,
query_info.prewhere_info->prewhere_actions,
query_info.prewhere_info->prewhere_column_name,
query_info.prewhere_info->remove_prewhere_column));
query_info.prewhere_info->remove_prewhere_column);
});
// To remove additional columns
// In some cases, we did not read any marks so that the pipeline.streams is empty
// Thus, some columns in prewhere are not removed as expected
// This leads to mismatched header in distributed table
if (query_info.prewhere_info->remove_columns_actions)
pipe.addSimpleTransform(std::make_shared<ExpressionTransform>(
pipe.getHeader(), query_info.prewhere_info->remove_columns_actions));
{
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<ExpressionTransform>(
header, query_info.prewhere_info->remove_columns_actions);
});
}
}
pipes.emplace_back(std::move(pipe));
}
pipeline = std::make_unique<QueryPipeline>();
@ -104,20 +113,14 @@ ReadFromStorageStep::ReadFromStorageStep(
auto quota = context->getQuota();
for (auto & pipe : pipes)
{
if (!options.ignore_limits)
pipe.setLimits(limits);
if (!options.ignore_limits)
pipe.setLimits(limits);
if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete))
pipe.setQuota(quota);
}
if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete))
pipe.setQuota(quota);
}
for (auto & pipe : pipes)
pipe.enableQuota();
pipeline->init(std::move(pipes));
pipeline->init(std::move(pipe));
pipeline->addInterpreterContext(std::move(context));
pipeline->addStorageHolder(std::move(storage));

View File

@ -30,7 +30,7 @@ QueryPipelinePtr UnionStep::updatePipeline(QueryPipelines pipelines)
return pipeline;
}
pipeline->unitePipelines(std::move(pipelines), output_stream->header ,max_threads);
*pipeline = QueryPipeline::unitePipelines(std::move(pipelines), output_stream->header ,max_threads);
processors = collector.detachProcessors();
return pipeline;

View File

@ -1,6 +1,7 @@
#include <Processors/Sources/DelayedSource.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/NullSink.h>
#include <Processors/ResizeProcessor.h>
namespace DB
{
@ -111,15 +112,27 @@ void synchronizePorts(OutputPort *& pipe_port, OutputPort * source_port, const B
void DelayedSource::work()
{
auto pipe = creator();
const auto & header = main->getHeader();
main_output = &pipe.getPort();
if (pipe.empty())
{
auto source = std::make_shared<NullSource>(header);
main_output = &source->getPort();
processors.emplace_back(std::move(source));
return;
}
if (pipe.numOutputPorts() > 1)
pipe.addTransform(std::make_shared<ResizeProcessor>(header, pipe.numOutputPorts(), 1));
main_output = pipe.getOutputPort(0);
totals_output = pipe.getTotalsPort();
extremes_output = pipe.getExtremesPort();
processors = std::move(pipe).detachProcessors();
processors = Pipe::detachProcessors(std::move(pipe));
synchronizePorts(totals_output, totals, main->getHeader(), processors);
synchronizePorts(extremes_output, extremes, main->getHeader(), processors);
synchronizePorts(totals_output, totals, header, processors);
synchronizePorts(extremes_output, extremes, header, processors);
}
Processors DelayedSource::expandPipeline()
@ -144,12 +157,11 @@ Pipe createDelayedPipe(const Block & header, DelayedSource::Creator processors_c
{
auto source = std::make_shared<DelayedSource>(header, std::move(processors_creator), add_totals_port, add_extremes_port);
Pipe pipe(&source->getPort());
pipe.setTotalsPort(source->getTotalsPort());
pipe.setExtremesPort(source->getExtremesPort());
auto * main = &source->getPort();
auto * totals = source->getTotalsPort();
auto * extremes = source->getExtremesPort();
pipe.addProcessors({std::move(source)});
return pipe;
return Pipe(std::move(source), main, totals, extremes);
}
}

View File

@ -113,18 +113,10 @@ Pipe createRemoteSourcePipe(
Pipe pipe(std::make_shared<RemoteSource>(query_executor, add_aggregation_info));
if (add_totals)
{
auto totals_source = std::make_shared<RemoteTotalsSource>(query_executor);
pipe.setTotalsPort(&totals_source->getPort());
pipe.addProcessors({std::move(totals_source)});
}
pipe.addTotalsSource(std::make_shared<RemoteTotalsSource>(query_executor));
if (add_extremes)
{
auto extremes_source = std::make_shared<RemoteExtremesSource>(query_executor);
pipe.setExtremesPort(&extremes_source->getPort());
pipe.addProcessors({std::move(extremes_source)});
}
pipe.addExtremesSource(std::make_shared<RemoteExtremesSource>(query_executor));
return pipe;
}

View File

@ -3,6 +3,7 @@
#include <Common/ClickHouseRevision.h>
#include <DataStreams/NativeBlockInputStream.h>
#include <Processors/ISource.h>
#include <Processors/Pipe.h>
#include <Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h>
@ -585,23 +586,24 @@ void AggregatingTransform::initGenerate()
}
}
auto header = params->aggregator.getHeader(false);
const auto & files = params->aggregator.getTemporaryFiles();
BlockInputStreams input_streams;
for (const auto & file : files.files)
processors.emplace_back(std::make_unique<SourceFromNativeStream>(header, file->path()));
Pipe pipe;
{
auto header = params->aggregator.getHeader(false);
Pipes pipes;
for (const auto & file : files.files)
pipes.emplace_back(Pipe(std::make_unique<SourceFromNativeStream>(header, file->path())));
pipe = Pipe::unitePipes(std::move(pipes));
}
LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), ReadableSize(files.sum_size_compressed), ReadableSize(files.sum_size_uncompressed));
auto pipe = createMergingAggregatedMemoryEfficientPipe(
header, params, files.files.size(), temporary_data_merge_threads);
addMergingAggregatedMemoryEfficientTransform(pipe, params, temporary_data_merge_threads);
auto input = pipe.front()->getInputs().begin();
for (auto & processor : processors)
connect(processor->getOutputs().front(), *(input++));
processors.insert(processors.end(), pipe.begin(), pipe.end());
processors = Pipe::detachProcessors(std::move(pipe));
}
}

View File

@ -1,8 +1,9 @@
#include <Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h>
#include <Processors/ISimpleTransform.h>
#include <Interpreters/Aggregator.h>
#include <Processors/ISimpleTransform.h>
#include <Processors/ResizeProcessor.h>
#include <Processors/Pipe.h>
namespace DB
{
@ -492,51 +493,32 @@ IProcessor::Status SortingAggregatedTransform::prepare()
}
Processors createMergingAggregatedMemoryEfficientPipe(
Block header,
AggregatingTransformParamsPtr params,
size_t num_inputs,
size_t num_merging_processors)
void addMergingAggregatedMemoryEfficientTransform(
Pipe & pipe,
AggregatingTransformParamsPtr params,
size_t num_merging_processors)
{
Processors processors;
processors.reserve(num_merging_processors + 2);
auto grouping = std::make_shared<GroupingAggregatedTransform>(header, num_inputs, params);
processors.emplace_back(std::move(grouping));
pipe.addTransform(std::make_shared<GroupingAggregatedTransform>(pipe.getHeader(), pipe.numOutputPorts(), params));
if (num_merging_processors <= 1)
{
/// --> GroupingAggregated --> MergingAggregatedBucket -->
auto transform = std::make_shared<MergingAggregatedBucketTransform>(params);
connect(processors.back()->getOutputs().front(), transform->getInputPort());
processors.emplace_back(std::move(transform));
return processors;
pipe.addTransform(std::make_shared<MergingAggregatedBucketTransform>(params));
return;
}
/// --> --> MergingAggregatedBucket -->
/// --> GroupingAggregated --> ResizeProcessor --> MergingAggregatedBucket --> SortingAggregated -->
/// --> --> MergingAggregatedBucket -->
auto resize = std::make_shared<ResizeProcessor>(Block(), 1, num_merging_processors);
connect(processors.back()->getOutputs().front(), resize->getInputs().front());
processors.emplace_back(std::move(resize));
pipe.addTransform(std::make_shared<ResizeProcessor>(Block(), 1, num_merging_processors));
auto sorting = std::make_shared<SortingAggregatedTransform>(num_merging_processors, params);
auto out = processors.back()->getOutputs().begin();
auto in = sorting->getInputs().begin();
for (size_t i = 0; i < num_merging_processors; ++i, ++in, ++out)
pipe.addSimpleTransform([params](const Block &)
{
auto transform = std::make_shared<MergingAggregatedBucketTransform>(params);
transform->setStream(i);
connect(*out, transform->getInputPort());
connect(transform->getOutputPort(), *in);
processors.emplace_back(std::move(transform));
}
return std::make_shared<MergingAggregatedBucketTransform>(params);
});
processors.emplace_back(std::move(sorting));
return processors;
pipe.addTransform(std::make_shared<SortingAggregatedTransform>(num_merging_processors, params));
}
}

View File

@ -136,12 +136,12 @@ private:
void addChunk(Chunk chunk, size_t from_input);
};
/// Creates piece of pipeline which performs memory efficient merging of partially aggregated data from several sources.
/// First processor will have num_inputs, last - single output. You should connect them to create pipeline.
Processors createMergingAggregatedMemoryEfficientPipe(
Block header,
class Pipe;
/// Adds processors to pipe which performs memory efficient merging of partially aggregated data from several sources.
void addMergingAggregatedMemoryEfficientTransform(
Pipe & pipe,
AggregatingTransformParamsPtr params,
size_t num_inputs,
size_t num_merging_processors);
}

View File

@ -6,6 +6,7 @@
#include <Storages/AlterCommands.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Processors/Pipe.h>
#include <Interpreters/Context.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/quoteString.h>
@ -78,6 +79,27 @@ TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, cons
return result;
}
Pipe IStorage::read(
const Names & /*column_names*/,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & /*query_info*/,
const Context & /*context*/,
QueryProcessingStage::Enum /*processed_stage*/,
size_t /*max_block_size*/,
unsigned /*num_streams*/)
{
throw Exception("Method read is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
Pipe IStorage::alterPartition(
const ASTPtr & /* query */,
const StorageMetadataPtr & /* metadata_snapshot */,
const PartitionCommands & /* commands */,
const Context & /* context */)
{
throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
void IStorage::alter(
const AlterCommands & params, const Context & context, TableLockHolder &)
{

View File

@ -48,7 +48,6 @@ using ProcessorPtr = std::shared_ptr<IProcessor>;
using Processors = std::vector<ProcessorPtr>;
class Pipe;
using Pipes = std::vector<Pipe>;
class StoragePolicy;
using StoragePolicyPtr = std::shared_ptr<const StoragePolicy>;
@ -134,8 +133,6 @@ public:
using ColumnSizeByName = std::unordered_map<std::string, ColumnSize>;
virtual ColumnSizeByName getColumnSizes() const { return {}; }
public:
/// Get mutable version (snapshot) of storage metadata. Metadata object is
/// multiversion, so it can be concurrently changed, but returned copy can be
/// used without any locks.
@ -181,7 +178,7 @@ private:
/// Multiversion storage metadata. Allows to read/write storage metadata
/// without locks.
MultiVersionStorageMetadataPtr metadata;
private:
RWLockImpl::LockHolder tryLockTimed(
const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const std::chrono::milliseconds & acquire_timeout) const;
@ -274,17 +271,14 @@ public:
* changed during lifetime of the returned pipeline, but the snapshot is
* guaranteed to be immutable.
*/
virtual Pipes read(
virtual Pipe read(
const Names & /*column_names*/,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & /*query_info*/,
const Context & /*context*/,
QueryProcessingStage::Enum /*processed_stage*/,
size_t /*max_block_size*/,
unsigned /*num_streams*/)
{
throw Exception("Method read is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
unsigned /*num_streams*/);
/** Writes the data to a table.
* Receives a description of the query, which can contain information about the data write method.
@ -353,10 +347,11 @@ public:
/** ALTER tables with regard to its partitions.
* Should handle locks for each command on its own.
*/
virtual Pipes alterPartition(const ASTPtr & /* query */, const StorageMetadataPtr & /* metadata_snapshot */, const PartitionCommands & /* commands */, const Context & /* context */)
{
throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
virtual Pipe alterPartition(
const ASTPtr & /* query */,
const StorageMetadataPtr & /* metadata_snapshot */,
const PartitionCommands & /* commands */,
const Context & /* context */);
/// Checks that partition commands can be applied to storage.
virtual void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const;

View File

@ -197,7 +197,7 @@ String StorageKafka::getDefaultClientId(const StorageID & table_id_)
}
Pipes StorageKafka::read(
Pipe StorageKafka::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /* query_info */,
@ -226,7 +226,7 @@ Pipes StorageKafka::read(
}
LOG_DEBUG(log, "Starting reading {} streams", pipes.size());
return pipes;
return Pipe::unitePipes(std::move(pipes));
}

View File

@ -38,7 +38,7 @@ public:
void startup() override;
void shutdown() override;
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -35,7 +35,7 @@ public:
QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const ASTPtr &) const override { return to_stage; }
Pipes read(
Pipe read(
const Names & /*column_names*/,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & /*query_info*/,
@ -44,7 +44,7 @@ public:
size_t /*max_block_size*/,
unsigned /*num_streams*/) override
{
return std::move(pipes);
return Pipe::unitePipes(std::move(pipes));
}
private:

View File

@ -528,7 +528,7 @@ void StorageLiveView::refresh(const Context & context)
}
}
Pipes StorageLiveView::read(
Pipe StorageLiveView::read(
const Names & /*column_names*/,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & /*query_info*/,
@ -537,7 +537,6 @@ Pipes StorageLiveView::read(
const size_t /*max_block_size*/,
const unsigned /*num_streams*/)
{
Pipes pipes;
{
std::lock_guard lock(mutex);
if (!(*blocks_ptr))
@ -545,9 +544,8 @@ Pipes StorageLiveView::read(
if (getNewBlocks())
condition.notify_all();
}
pipes.emplace_back(std::make_shared<BlocksSource>(blocks_ptr, getHeader()));
return Pipe(std::make_shared<BlocksSource>(blocks_ptr, getHeader()));
}
return pipes;
}
BlockInputStreams StorageLiveView::watch(

View File

@ -124,7 +124,7 @@ public:
void refresh(const Context & context);
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -26,14 +26,12 @@
#include <Processors/Executors/PipelineExecutingBlockInputStream.h>
#include <Interpreters/MutationsInterpreter.h>
#include <Interpreters/Context.h>
#include <Common/SimpleIncrement.h>
#include <Common/interpolate.h>
#include <Common/typeid_cast.h>
#include <Common/escapeForFileName.h>
#include <cmath>
#include <ctime>
#include <iomanip>
#include <numeric>
#include <boost/algorithm/string/replace.hpp>
@ -730,8 +728,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
if (metadata_snapshot->hasSortingKey())
{
auto expr = std::make_shared<ExpressionTransform>(pipe.getHeader(), metadata_snapshot->getSortingKey().expression);
pipe.addSimpleTransform(std::move(expr));
pipe.addSimpleTransform([&metadata_snapshot](const Block & header)
{
return std::make_shared<ExpressionTransform>(header, metadata_snapshot->getSortingKey().expression);
});
}
pipes.emplace_back(std::move(pipe));
@ -800,7 +800,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
}
QueryPipeline pipeline;
pipeline.init(Pipe(std::move(pipes), std::move(merged_transform)));
pipeline.init(Pipe::unitePipes(std::move(pipes)));
pipeline.addTransform(std::move(merged_transform));
pipeline.setMaxThreads(1);
BlockInputStreamPtr merged_stream = std::make_shared<PipelineExecutingBlockInputStream>(std::move(pipeline));

View File

@ -149,7 +149,7 @@ static RelativeSize convertAbsoluteSampleSizeToRelative(const ASTPtr & node, siz
}
Pipes MergeTreeDataSelectExecutor::read(
Pipe MergeTreeDataSelectExecutor::read(
const Names & column_names_to_return,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
@ -164,7 +164,7 @@ Pipes MergeTreeDataSelectExecutor::read(
max_block_numbers_to_read);
}
Pipes MergeTreeDataSelectExecutor::readFromParts(
Pipe MergeTreeDataSelectExecutor::readFromParts(
MergeTreeData::DataPartsVector parts,
const Names & column_names_to_return,
const StorageMetadataPtr & metadata_snapshot,
@ -658,7 +658,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
ProfileEvents::increment(ProfileEvents::SelectedRanges, sum_ranges);
ProfileEvents::increment(ProfileEvents::SelectedMarks, sum_marks);
Pipes res;
Pipe res;
/// Projection, that needed to drop columns, which have appeared by execution
/// of some extra expressions, and to allow execute the same expressions later.
@ -732,31 +732,37 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
if (use_sampling)
{
for (auto & pipe : res)
pipe.addSimpleTransform(std::make_shared<FilterTransform>(
pipe.getHeader(), filter_expression, filter_function->getColumnName(), false));
res.addSimpleTransform([&filter_expression, &filter_function](const Block & header)
{
return std::make_shared<FilterTransform>(
header, filter_expression, filter_function->getColumnName(), false);
});
}
if (result_projection)
{
for (auto & pipe : res)
pipe.addSimpleTransform(std::make_shared<ExpressionTransform>(
pipe.getHeader(), result_projection));
res.addSimpleTransform([&result_projection](const Block & header)
{
return std::make_shared<ExpressionTransform>(header, result_projection);
});
}
/// By the way, if a distributed query or query to a Merge table is made, then the `_sample_factor` column can have different values.
if (sample_factor_column_queried)
{
for (auto & pipe : res)
pipe.addSimpleTransform(std::make_shared<AddingConstColumnTransform<Float64>>(
pipe.getHeader(), std::make_shared<DataTypeFloat64>(), used_sample_factor, "_sample_factor"));
res.addSimpleTransform([used_sample_factor](const Block & header)
{
return std::make_shared<AddingConstColumnTransform<Float64>>(
header, std::make_shared<DataTypeFloat64>(), used_sample_factor, "_sample_factor");
});
}
if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions)
{
for (auto & pipe : res)
pipe.addSimpleTransform(std::make_shared<ExpressionTransform>(
pipe.getHeader(), query_info.prewhere_info->remove_columns_actions));
res.addSimpleTransform([&query_info](const Block & header)
{
return std::make_shared<ExpressionTransform>(header, query_info.prewhere_info->remove_columns_actions);
});
}
return res;
@ -785,7 +791,7 @@ size_t roundRowsOrBytesToMarks(
}
Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams(
Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams(
RangesInDataParts && parts,
size_t num_streams,
const Names & column_names,
@ -833,13 +839,13 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams(
if (sum_marks > max_marks_to_use_cache)
use_uncompressed_cache = false;
Pipes res;
if (0 == sum_marks)
return res;
return {};
if (num_streams > 1)
{
/// Parallel query execution.
Pipes res;
/// Reduce the number of num_streams if the data is small.
if (sum_marks < num_streams * min_marks_for_concurrent_read && parts.size() < num_streams)
@ -878,10 +884,13 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams(
res.emplace_back(std::move(source));
}
return Pipe::unitePipes(std::move(res));
}
else
{
/// Sequential query execution.
Pipes res;
for (const auto & part : parts)
{
@ -893,18 +902,15 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams(
res.emplace_back(std::move(source));
}
auto pipe = Pipe::unitePipes(std::move(res));
/// Use ConcatProcessor to concat sources together.
/// It is needed to read in parts order (and so in PK order) if single thread is used.
if (res.size() > 1)
{
auto concat = std::make_shared<ConcatProcessor>(res.front().getHeader(), res.size());
Pipe pipe(std::move(res), std::move(concat));
res = Pipes();
res.emplace_back(std::move(pipe));
}
}
if (pipe.numOutputPorts() > 1)
pipe.addTransform(std::make_shared<ConcatProcessor>(pipe.getHeader(), pipe.numOutputPorts()));
return res;
return pipe;
}
}
static ExpressionActionsPtr createProjection(const Pipe & pipe, const MergeTreeData & data)
@ -915,7 +921,7 @@ static ExpressionActionsPtr createProjection(const Pipe & pipe, const MergeTreeD
return projection;
}
Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder(
Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder(
RangesInDataParts && parts,
size_t num_streams,
const Names & column_names,
@ -967,7 +973,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder(
Pipes res;
if (sum_marks == 0)
return res;
return {};
/// Let's split ranges to avoid reading much data.
auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size](const auto & ranges, int direction)
@ -1112,40 +1118,45 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder(
reader_settings,
virt_columns,
part.part_index_in_query));
pipes.back().addSimpleTransform(std::make_shared<ReverseTransform>(pipes.back().getHeader()));
}
}
if (pipes.size() > 1 && need_preliminary_merge)
auto pipe = Pipe::unitePipes(std::move(pipes));
if (input_order_info->direction != 1)
{
pipe.addSimpleTransform([](const Block & header)
{
return std::make_shared<ReverseTransform>(header);
});
}
if (pipe.numOutputPorts() > 1 && need_preliminary_merge)
{
SortDescription sort_description;
for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j)
sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j],
input_order_info->direction, 1);
input_order_info->direction, 1);
/// Drop temporary columns, added by 'sorting_key_prefix_expr'
out_projection = createProjection(pipes.back(), data);
for (auto & pipe : pipes)
pipe.addSimpleTransform(std::make_shared<ExpressionTransform>(pipe.getHeader(), sorting_key_prefix_expr));
out_projection = createProjection(pipe, data);
pipe.addSimpleTransform([sorting_key_prefix_expr](const Block & header)
{
return std::make_shared<ExpressionTransform>(header, sorting_key_prefix_expr);
});
auto merging_sorted = std::make_shared<MergingSortedTransform>(
pipes.back().getHeader(), pipes.size(), sort_description, max_block_size);
pipe.addTransform(std::make_shared<MergingSortedTransform>(
pipe.getHeader(), pipe.numOutputPorts(), sort_description, max_block_size));
}
res.emplace_back(std::move(pipes), std::move(merging_sorted));
}
else
{
for (auto && pipe : pipes)
res.emplace_back(std::move(pipe));
}
res.emplace_back(std::move(pipe));
}
return res;
return Pipe::unitePipes(std::move(res));
}
Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
RangesInDataParts && parts,
size_t num_streams,
const Names & column_names,
@ -1183,25 +1194,35 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
if (sum_marks > max_marks_to_use_cache)
use_uncompressed_cache = false;
Pipes pipes;
Pipe pipe;
for (const auto & part : parts)
{
auto source_processor = std::make_shared<MergeTreeSelectProcessor>(
data, metadata_snapshot, part.data_part, max_block_size, settings.preferred_block_size_bytes,
settings.preferred_max_column_in_block_size_bytes, column_names, part.ranges, use_uncompressed_cache,
query_info.prewhere_info, true, reader_settings,
virt_columns, part.part_index_in_query);
Pipes pipes;
Pipe pipe(std::move(source_processor));
/// Drop temporary columns, added by 'sorting_key_expr'
if (!out_projection)
out_projection = createProjection(pipe, data);
for (const auto & part : parts)
{
auto source_processor = std::make_shared<MergeTreeSelectProcessor>(
data, metadata_snapshot, part.data_part, max_block_size, settings.preferred_block_size_bytes,
settings.preferred_max_column_in_block_size_bytes, column_names, part.ranges,
use_uncompressed_cache,
query_info.prewhere_info, true, reader_settings,
virt_columns, part.part_index_in_query);
pipe.addSimpleTransform(std::make_shared<ExpressionTransform>(pipe.getHeader(), metadata_snapshot->getSortingKey().expression));
pipes.emplace_back(std::move(pipe));
pipes.emplace_back(std::move(source_processor));
}
pipe = Pipe::unitePipes(std::move(pipes));
}
/// Drop temporary columns, added by 'sorting_key_expr'
if (!out_projection)
out_projection = createProjection(pipe, data);
pipe.addSimpleTransform([&metadata_snapshot](const Block & header)
{
return std::make_shared<ExpressionTransform>(header, metadata_snapshot->getSortingKey().expression);
});
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
SortDescription sort_description;
size_t sort_columns_size = sort_columns.size();
@ -1209,7 +1230,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names;
Block header = pipes.at(0).getHeader();
Block header = pipe.getHeader();
for (size_t i = 0; i < sort_columns_size; ++i)
sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1);
@ -1219,28 +1240,28 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
{
case MergeTreeData::MergingParams::Ordinary:
{
return std::make_shared<MergingSortedTransform>(header, pipes.size(),
return std::make_shared<MergingSortedTransform>(header, pipe.numOutputPorts(),
sort_description, max_block_size);
}
case MergeTreeData::MergingParams::Collapsing:
return std::make_shared<CollapsingSortedTransform>(header, pipes.size(),
return std::make_shared<CollapsingSortedTransform>(header, pipe.numOutputPorts(),
sort_description, data.merging_params.sign_column, true, max_block_size);
case MergeTreeData::MergingParams::Summing:
return std::make_shared<SummingSortedTransform>(header, pipes.size(),
return std::make_shared<SummingSortedTransform>(header, pipe.numOutputPorts(),
sort_description, data.merging_params.columns_to_sum, partition_key_columns, max_block_size);
case MergeTreeData::MergingParams::Aggregating:
return std::make_shared<AggregatingSortedTransform>(header, pipes.size(),
return std::make_shared<AggregatingSortedTransform>(header, pipe.numOutputPorts(),
sort_description, max_block_size);
case MergeTreeData::MergingParams::Replacing:
return std::make_shared<ReplacingSortedTransform>(header, pipes.size(),
return std::make_shared<ReplacingSortedTransform>(header, pipe.numOutputPorts(),
sort_description, data.merging_params.version_column, max_block_size);
case MergeTreeData::MergingParams::VersionedCollapsing:
return std::make_shared<VersionedCollapsingTransform>(header, pipes.size(),
return std::make_shared<VersionedCollapsingTransform>(header, pipe.numOutputPorts(),
sort_description, data.merging_params.sign_column, max_block_size);
case MergeTreeData::MergingParams::Graphite:
@ -1255,12 +1276,8 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
if (num_streams <= 1 || sort_description.empty())
{
Pipe pipe(std::move(pipes), get_merging_processor());
pipes = Pipes();
pipes.emplace_back(std::move(pipe));
return pipes;
pipe.addTransform(get_merging_processor());
return pipe;
}
ColumnNumbers key_columns;
@ -1274,63 +1291,47 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
key_columns.emplace_back(desc.column_number);
}
Processors selectors;
Processors copiers;
selectors.reserve(pipes.size());
for (auto & pipe : pipes)
pipe.addSimpleTransform([&](const Block & stream_header)
{
auto selector = std::make_shared<AddingSelectorTransform>(pipe.getHeader(), num_streams, key_columns);
auto copier = std::make_shared<CopyTransform>(pipe.getHeader(), num_streams);
connect(pipe.getPort(), selector->getInputPort());
connect(selector->getOutputPort(), copier->getInputPort());
selectors.emplace_back(std::move(selector));
copiers.emplace_back(std::move(copier));
}
return std::make_shared<AddingSelectorTransform>(stream_header, num_streams, key_columns);
});
Processors merges;
std::vector<InputPorts::iterator> input_ports;
merges.reserve(num_streams);
input_ports.reserve(num_streams);
for (size_t i = 0; i < num_streams; ++i)
pipe.transform([&](OutputPortRawPtrs ports)
{
auto merge = get_merging_processor();
merge->setSelectorPosition(i);
input_ports.emplace_back(merge->getInputs().begin());
merges.emplace_back(std::move(merge));
}
Processors processors;
std::vector<OutputPorts::iterator> output_ports;
processors.reserve(ports.size() + num_streams);
output_ports.reserve(ports.size());
/// Connect outputs of i-th splitter with i-th input port of every merge.
for (auto & resize : copiers)
{
size_t input_num = 0;
for (auto & output : resize->getOutputs())
for (auto & port : ports)
{
connect(output, *input_ports[input_num]);
++input_ports[input_num];
++input_num;
auto copier = std::make_shared<CopyTransform>(header, num_streams);
connect(*port, copier->getInputPort());
output_ports.emplace_back(copier->getOutputs().begin());
processors.emplace_back(std::move(copier));
}
}
Processors processors;
for (auto & pipe : pipes)
{
auto pipe_processors = std::move(pipe).detachProcessors();
processors.insert(processors.end(), pipe_processors.begin(), pipe_processors.end());
}
for (size_t i = 0; i < num_streams; ++i)
{
auto merge = get_merging_processor();
merge->setSelectorPosition(i);
auto input = merge->getInputs().begin();
pipes.clear();
pipes.reserve(num_streams);
for (auto & merge : merges)
pipes.emplace_back(&merge->getOutputs().front());
/// Connect i-th merge with i-th input port of every copier.
for (size_t j = 0; j < ports.size(); ++j)
{
connect(*output_ports[j], *input);
++output_ports[j];
++input;
}
pipes.front().addProcessors(processors);
pipes.front().addProcessors(selectors);
pipes.front().addProcessors(copiers);
pipes.front().addProcessors(merges);
processors.emplace_back(std::move(merge));
}
return pipes;
return processors;
});
return pipe;
}
/// Calculates a set of mark ranges, that could possibly contain keys, required by condition.

View File

@ -24,7 +24,7 @@ public:
*/
using PartitionIdToMaxBlock = std::unordered_map<String, Int64>;
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
@ -33,7 +33,7 @@ public:
unsigned num_streams,
const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const;
Pipes readFromParts(
Pipe readFromParts(
MergeTreeData::DataPartsVector parts,
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
@ -48,7 +48,7 @@ private:
Poco::Logger * log;
Pipes spreadMarkRangesAmongStreams(
Pipe spreadMarkRangesAmongStreams(
RangesInDataParts && parts,
size_t num_streams,
const Names & column_names,
@ -61,7 +61,7 @@ private:
const MergeTreeReaderSettings & reader_settings) const;
/// out_projection - save projection only with columns, requested to read
Pipes spreadMarkRangesAmongStreamsWithOrder(
Pipe spreadMarkRangesAmongStreamsWithOrder(
RangesInDataParts && parts,
size_t num_streams,
const Names & column_names,
@ -75,7 +75,7 @@ private:
const MergeTreeReaderSettings & reader_settings,
ExpressionActionsPtr & out_projection) const;
Pipes spreadMarkRangesAmongStreamsFinal(
Pipe spreadMarkRangesAmongStreamsFinal(
RangesInDataParts && parts,
size_t num_streams,
const Names & column_names,

View File

@ -18,7 +18,7 @@ class StorageFromMergeTreeDataPart final : public ext::shared_ptr_helper<Storage
public:
String getName() const override { return "FromMergeTreeDataPart"; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,

View File

@ -135,7 +135,7 @@ std::string PartitionCommand::typeToString() const
__builtin_unreachable();
}
Pipes convertCommandsResultToSource(const PartitionCommandsResultInfo & commands_result)
Pipe convertCommandsResultToSource(const PartitionCommandsResultInfo & commands_result)
{
Block header {
ColumnWithTypeAndName(std::make_shared<DataTypeString>(), "command_type"),
@ -187,11 +187,7 @@ Pipes convertCommandsResultToSource(const PartitionCommandsResultInfo & commands
}
Chunk chunk(std::move(res_columns), commands_result.size());
Pipe pipe(std::make_shared<SourceFromSingleChunk>(std::move(header), std::move(chunk)));
Pipes result;
result.emplace_back(std::move(pipe));
return result;
return Pipe(std::make_shared<SourceFromSingleChunk>(std::move(header), std::move(chunk)));
}
}

View File

@ -15,7 +15,6 @@ namespace DB
class ASTAlterCommand;
class Pipe;
using Pipes = std::vector<Pipe>;
struct PartitionCommand
{
@ -104,6 +103,6 @@ using PartitionCommandsResultInfo = std::vector<PartitionCommandResultInfo>;
/// used to print info to the user. Tries to create narrowest table for given
/// results. For example, if all commands were FREEZE commands, than
/// old_part_name column will be absent.
Pipes convertCommandsResultToSource(const PartitionCommandsResultInfo & commands_result);
Pipe convertCommandsResultToSource(const PartitionCommandsResultInfo & commands_result);
}

View File

@ -139,7 +139,7 @@ void StorageRabbitMQ::loopingFunc()
}
Pipes StorageRabbitMQ::read(
Pipe StorageRabbitMQ::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /* query_info */,
@ -171,7 +171,7 @@ Pipes StorageRabbitMQ::read(
}
LOG_DEBUG(log, "Starting reading {} streams", pipes.size());
return pipes;
return Pipe::unitePipes(std::move(pipes));
}

View File

@ -31,7 +31,7 @@ public:
void startup() override;
void shutdown() override;
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,

View File

@ -146,7 +146,7 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context
}
Pipes StorageBuffer::read(
Pipe StorageBuffer::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
@ -155,7 +155,7 @@ Pipes StorageBuffer::read(
size_t max_block_size,
unsigned num_streams)
{
Pipes pipes_from_dst;
Pipe pipe_from_dst;
if (destination_id)
{
@ -182,7 +182,7 @@ Pipes StorageBuffer::read(
query_info.input_order_info = query_info.order_optimizer->getInputOrder(destination, destination_metadata_snapshot);
/// The destination table has the same structure of the requested columns and we can simply read blocks from there.
pipes_from_dst = destination->read(
pipe_from_dst = destination->read(
column_names, destination_metadata_snapshot, query_info,
context, processed_stage, max_block_size, num_streams);
}
@ -217,66 +217,80 @@ Pipes StorageBuffer::read(
}
else
{
pipes_from_dst = destination->read(
pipe_from_dst = destination->read(
columns_intersection, destination_metadata_snapshot, query_info,
context, processed_stage, max_block_size, num_streams);
for (auto & pipe : pipes_from_dst)
pipe_from_dst.addSimpleTransform([&](const Block & stream_header)
{
pipe.addSimpleTransform(std::make_shared<AddingMissedTransform>(
pipe.getHeader(), header_after_adding_defaults, metadata_snapshot->getColumns().getDefaults(), context));
return std::make_shared<AddingMissedTransform>(stream_header, header_after_adding_defaults,
metadata_snapshot->getColumns().getDefaults(), context);
});
pipe.addSimpleTransform(std::make_shared<ConvertingTransform>(
pipe.getHeader(), header, ConvertingTransform::MatchColumnsMode::Name));
}
pipe_from_dst.addSimpleTransform([&](const Block & stream_header)
{
return std::make_shared<ConvertingTransform>(
stream_header, header, ConvertingTransform::MatchColumnsMode::Name);
});
}
}
for (auto & pipe : pipes_from_dst)
pipe.addTableLock(destination_lock);
pipe_from_dst.addTableLock(destination_lock);
}
Pipes pipes_from_buffers;
pipes_from_buffers.reserve(num_shards);
for (auto & buf : buffers)
pipes_from_buffers.emplace_back(std::make_shared<BufferSource>(column_names, buf, *this, metadata_snapshot));
Pipe pipe_from_buffers;
{
Pipes pipes_from_buffers;
pipes_from_buffers.reserve(num_shards);
for (auto & buf : buffers)
pipes_from_buffers.emplace_back(std::make_shared<BufferSource>(column_names, buf, *this, metadata_snapshot));
pipe_from_buffers = Pipe::unitePipes(std::move(pipes_from_buffers));
}
/// Convert pipes from table to structure from buffer.
if (!pipes_from_buffers.empty() && !pipes_from_dst.empty()
&& !blocksHaveEqualStructure(pipes_from_buffers.front().getHeader(), pipes_from_dst.front().getHeader()))
if (!pipe_from_buffers.empty() && !pipe_from_dst.empty()
&& !blocksHaveEqualStructure(pipe_from_buffers.getHeader(), pipe_from_dst.getHeader()))
{
for (auto & pipe : pipes_from_dst)
pipe.addSimpleTransform(std::make_shared<ConvertingTransform>(
pipe.getHeader(),
pipes_from_buffers.front().getHeader(),
ConvertingTransform::MatchColumnsMode::Name));
pipe_from_dst.addSimpleTransform([&](const Block & header)
{
return std::make_shared<ConvertingTransform>(
header,
pipe_from_buffers.getHeader(),
ConvertingTransform::MatchColumnsMode::Name);
});
}
/** If the sources from the table were processed before some non-initial stage of query execution,
* then sources from the buffers must also be wrapped in the processing pipeline before the same stage.
*/
if (processed_stage > QueryProcessingStage::FetchColumns)
for (auto & pipe : pipes_from_buffers)
pipe = InterpreterSelectQuery(query_info.query, context, std::move(pipe), SelectQueryOptions(processed_stage)).execute().pipeline.getPipe();
pipe_from_buffers = QueryPipeline::getPipe(
InterpreterSelectQuery(query_info.query, context, std::move(pipe_from_buffers),
SelectQueryOptions(processed_stage)).execute().pipeline);
if (query_info.prewhere_info)
{
for (auto & pipe : pipes_from_buffers)
pipe.addSimpleTransform(std::make_shared<FilterTransform>(pipe.getHeader(), query_info.prewhere_info->prewhere_actions,
query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column));
pipe_from_buffers.addSimpleTransform([&](const Block & header)
{
return std::make_shared<FilterTransform>(
header, query_info.prewhere_info->prewhere_actions,
query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column);
});
if (query_info.prewhere_info->alias_actions)
{
for (auto & pipe : pipes_from_buffers)
pipe.addSimpleTransform(std::make_shared<ExpressionTransform>(pipe.getHeader(), query_info.prewhere_info->alias_actions));
pipe_from_buffers.addSimpleTransform([&](const Block & header)
{
return std::make_shared<ExpressionTransform>(header, query_info.prewhere_info->alias_actions);
});
}
}
for (auto & pipe : pipes_from_buffers)
pipes_from_dst.emplace_back(std::move(pipe));
return pipes_from_dst;
Pipes pipes;
pipes.emplace_back(std::move(pipe_from_dst));
pipes.emplace_back(std::move(pipe_from_buffers));
return Pipe::unitePipes(std::move(pipes));
}

View File

@ -56,7 +56,7 @@ public:
QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const ASTPtr &) const override;
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -123,7 +123,7 @@ void StorageDictionary::checkTableCanBeDropped() const
throw Exception("Cannot detach table " + getStorageID().getFullTableName() + " from a database with DICTIONARY engine", ErrorCodes::CANNOT_DETACH_DICTIONARY_AS_TABLE);
}
Pipes StorageDictionary::read(
Pipe StorageDictionary::read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & /*query_info*/,
@ -134,11 +134,8 @@ Pipes StorageDictionary::read(
{
auto dictionary = context.getExternalDictionariesLoader().getDictionary(dictionary_name);
auto stream = dictionary->getBlockInputStream(column_names, max_block_size);
auto source = std::make_shared<SourceFromInputStream>(stream);
/// TODO: update dictionary interface for processors.
Pipes pipes;
pipes.emplace_back(std::move(source));
return pipes;
return Pipe(std::make_shared<SourceFromInputStream>(stream));
}

View File

@ -16,7 +16,7 @@ public:
void checkTableCanBeDropped() const override;
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -467,7 +467,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Con
return getQueryProcessingStageImpl(context, to_stage, cluster);
}
Pipes StorageDistributed::read(
Pipe StorageDistributed::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,

View File

@ -70,7 +70,7 @@ public:
bool canForceGroupByNoMerge(const Context &, QueryProcessingStage::Enum to_stage, const ASTPtr &) const;
QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum to_stage, const ASTPtr &) const override;
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -378,7 +378,7 @@ private:
};
Pipes StorageFile::read(
Pipe StorageFile::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /*query_info*/,
@ -419,7 +419,7 @@ Pipes StorageFile::read(
pipes.emplace_back(std::make_shared<StorageFileSource>(
this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults()));
return pipes;
return Pipe::unitePipes(std::move(pipes));
}

View File

@ -24,7 +24,7 @@ class StorageFile final : public ext::shared_ptr_helper<StorageFile>, public ISt
public:
std::string getName() const override { return "File"; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -438,7 +438,7 @@ void registerStorageGenerateRandom(StorageFactory & factory)
});
}
Pipes StorageGenerateRandom::read(
Pipe StorageGenerateRandom::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /*query_info*/,
@ -467,7 +467,7 @@ Pipes StorageGenerateRandom::read(
for (UInt64 i = 0; i < num_streams; ++i)
pipes.emplace_back(std::make_shared<GenerateSource>(max_block_size, max_array_length, max_string_length, generate(), block_header, context));
return pipes;
return Pipe::unitePipes(std::move(pipes));
}
}

View File

@ -15,7 +15,7 @@ class StorageGenerateRandom final : public ext::shared_ptr_helper<StorageGenerat
public:
std::string getName() const override { return "GenerateRandom"; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -262,7 +262,7 @@ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, c
}
Pipes StorageHDFS::read(
Pipe StorageHDFS::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /*query_info*/,
@ -298,7 +298,7 @@ Pipes StorageHDFS::read(
pipes.emplace_back(std::make_shared<HDFSSource>(
sources_info, uri_without_path, format_name, compression_method, metadata_snapshot->getSampleBlock(), context_, max_block_size));
return pipes;
return Pipe::unitePipes(std::move(pipes));
}
BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/)

View File

@ -19,7 +19,7 @@ class StorageHDFS final : public ext::shared_ptr_helper<StorageHDFS>, public ISt
public:
String getName() const override { return "HDFS"; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -58,7 +58,7 @@ void StorageInput::setInputStream(BlockInputStreamPtr input_stream_)
}
Pipes StorageInput::read(
Pipe StorageInput::read(
const Names & /*column_names*/,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /*query_info*/,
@ -74,15 +74,13 @@ Pipes StorageInput::read(
{
/// Send structure to the client.
query_context.initializeInput(shared_from_this());
pipes.emplace_back(std::make_shared<StorageInputSource>(query_context, metadata_snapshot->getSampleBlock()));
return pipes;
return Pipe(std::make_shared<StorageInputSource>(query_context, metadata_snapshot->getSampleBlock()));
}
if (!input_stream)
throw Exception("Input stream is not initialized, input() must be used only in INSERT SELECT query", ErrorCodes::INVALID_USAGE_OF_INPUT);
pipes.emplace_back(std::make_shared<SourceFromInputStream>(input_stream));
return pipes;
return Pipe(std::make_shared<SourceFromInputStream>(input_stream));
}
}

View File

@ -17,7 +17,7 @@ public:
/// A table will read from this stream.
void setInputStream(BlockInputStreamPtr input_stream_);
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -436,7 +436,7 @@ private:
// TODO: multiple stream read and index read
Pipes StorageJoin::read(
Pipe StorageJoin::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /*query_info*/,
@ -447,10 +447,7 @@ Pipes StorageJoin::read(
{
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
Pipes pipes;
pipes.emplace_back(std::make_shared<JoinSource>(*join, max_block_size, metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())));
return pipes;
return Pipe(std::make_shared<JoinSource>(*join, max_block_size, metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())));
}
}

View File

@ -36,7 +36,7 @@ public:
/// Verify that the data structure is suitable for implementing this type of JOIN.
void assertCompatible(ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_) const;
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -605,7 +605,7 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount(const StorageMeta
return it->second.marks;
}
Pipes StorageLog::read(
Pipe StorageLog::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /*query_info*/,
@ -648,7 +648,7 @@ Pipes StorageLog::read(
max_read_buffer_size));
}
return pipes;
return Pipe::unitePipes(std::move(pipes));
}
BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/)

View File

@ -24,7 +24,7 @@ class StorageLog final : public ext::shared_ptr_helper<StorageLog>, public IStor
public:
String getName() const override { return "Log"; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,

View File

@ -106,7 +106,7 @@ QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(cons
return getTargetTable()->getQueryProcessingStage(context, to_stage, query_ptr);
}
Pipes StorageMaterializedView::read(
Pipe StorageMaterializedView::read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,
@ -122,12 +122,10 @@ Pipes StorageMaterializedView::read(
if (query_info.order_optimizer)
query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage, metadata_snapshot);
Pipes pipes = storage->read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
Pipe pipe = storage->read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
pipe.addTableLock(lock);
for (auto & pipe : pipes)
pipe.addTableLock(lock);
return pipes;
return pipe;
}
BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context)
@ -250,7 +248,7 @@ void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & command
}
}
Pipes StorageMaterializedView::alterPartition(
Pipe StorageMaterializedView::alterPartition(
const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context)
{
checkStatementCanBeForwarded();

View File

@ -51,7 +51,7 @@ public:
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override;
Pipes alterPartition(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) override;
Pipe alterPartition(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) override;
void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override;
@ -71,7 +71,7 @@ public:
ActionLock getActionLock(StorageActionBlockType type) override;
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -106,7 +106,7 @@ StorageMemory::StorageMemory(const StorageID & table_id_, ColumnsDescription col
}
Pipes StorageMemory::read(
Pipe StorageMemory::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /*query_info*/,
@ -142,7 +142,7 @@ Pipes StorageMemory::read(
pipes.emplace_back(std::make_shared<MemorySource>(column_names, first, last, *this, metadata_snapshot));
}
return pipes;
return Pipe::unitePipes(std::move(pipes));
}

View File

@ -28,7 +28,7 @@ public:
size_t getSize() const { return data.size(); }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -19,7 +19,6 @@
#include <ext/range.h>
#include <algorithm>
#include <Parsers/queryToString.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include <Processors/Transforms/MaterializingTransform.h>
#include <Processors/ConcatProcessor.h>
#include <Processors/Transforms/AddingConstColumnTransform.h>
@ -128,7 +127,7 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context &
}
Pipes StorageMerge::read(
Pipe StorageMerge::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
@ -137,7 +136,7 @@ Pipes StorageMerge::read(
const size_t max_block_size,
unsigned num_streams)
{
Pipes res;
Pipes pipes;
bool has_table_virtual_column = false;
Names real_column_names;
@ -211,22 +210,23 @@ Pipes StorageMerge::read(
auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr();
auto source_pipes = createSources(
auto source_pipe = createSources(
storage_metadata_snapshot, query_info, processed_stage,
max_block_size, header, table, real_column_names, modified_context,
current_streams, has_table_virtual_column);
for (auto & pipe : source_pipes)
res.emplace_back(std::move(pipe));
pipes.emplace_back(std::move(source_pipe));
}
if (res.empty())
return res;
auto pipe = Pipe::unitePipes(std::move(pipes));
return narrowPipes(std::move(res), num_streams);
if (!pipe.empty())
narrowPipe(pipe, num_streams);
return pipe;
}
Pipes StorageMerge::createSources(
Pipe StorageMerge::createSources(
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
const QueryProcessingStage::Enum & processed_stage,
@ -245,18 +245,17 @@ Pipes StorageMerge::createSources(
VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", table_name);
Pipes pipes;
Pipe pipe;
if (!storage)
{
auto pipe = InterpreterSelectQuery(
pipe = QueryPipeline::getPipe(InterpreterSelectQuery(
modified_query_info.query, *modified_context,
std::make_shared<OneBlockInputStream>(header),
SelectQueryOptions(processed_stage).analyze()).execute().pipeline.getPipe();
SelectQueryOptions(processed_stage).analyze()).execute().pipeline);
pipe.addInterpreterContext(modified_context);
pipes.emplace_back(std::move(pipe));
return pipes;
return pipe;
}
auto storage_stage = storage->getQueryProcessingStage(*modified_context, QueryProcessingStage::Complete, query_info.query);
@ -267,7 +266,7 @@ Pipes StorageMerge::createSources(
real_column_names.push_back(ExpressionActions::getSmallestColumn(metadata_snapshot->getColumns().getAllPhysical()));
pipes = storage->read(real_column_names, metadata_snapshot, modified_query_info, *modified_context, processed_stage, max_block_size, UInt32(streams_num));
pipe = storage->read(real_column_names, metadata_snapshot, modified_query_info, *modified_context, processed_stage, max_block_size, UInt32(streams_num));
}
else if (processed_stage > storage_stage)
{
@ -279,46 +278,39 @@ Pipes StorageMerge::createSources(
InterpreterSelectQuery interpreter{modified_query_info.query, *modified_context, SelectQueryOptions(processed_stage)};
{
Pipe pipe = interpreter.execute().pipeline.getPipe();
pipes.emplace_back(std::move(pipe));
}
pipe = QueryPipeline::getPipe(interpreter.execute().pipeline);
/** Materialization is needed, since from distributed storage the constants come materialized.
* If you do not do this, different types (Const and non-Const) columns will be produced in different threads,
* And this is not allowed, since all code is based on the assumption that in the block stream all types are the same.
*/
pipes.back().addSimpleTransform(std::make_shared<MaterializingTransform>(pipes.back().getHeader()));
pipe.addSimpleTransform([](const Block & stream_header) { return std::make_shared<MaterializingTransform>(stream_header); });
}
if (!pipes.empty())
if (!pipe.empty())
{
if (concat_streams && pipes.size() > 1)
{
auto concat = std::make_shared<ConcatProcessor>(pipes.at(0).getHeader(), pipes.size());
Pipe pipe(std::move(pipes), std::move(concat));
if (concat_streams && pipe.numOutputPorts() > 1)
pipe.addTransform(std::make_shared<ConcatProcessor>(pipe.getHeader(), pipe.numOutputPorts()));
pipes = Pipes();
pipes.emplace_back(std::move(pipe));
if (has_table_virtual_column)
{
pipe.addSimpleTransform([name = table_name](const Block & stream_header)
{
return std::make_shared<AddingConstColumnTransform<String>>(
stream_header, std::make_shared<DataTypeString>(), name, "_table");
});
}
for (auto & pipe : pipes)
{
if (has_table_virtual_column)
pipe.addSimpleTransform(std::make_shared<AddingConstColumnTransform<String>>(
pipe.getHeader(), std::make_shared<DataTypeString>(), table_name, "_table"));
/// Subordinary tables could have different but convertible types, like numeric types of different width.
/// We must return streams with structure equals to structure of Merge table.
convertingSourceStream(header, metadata_snapshot, *modified_context, modified_query_info.query, pipe, processed_stage);
/// Subordinary tables could have different but convertible types, like numeric types of different width.
/// We must return streams with structure equals to structure of Merge table.
convertingSourceStream(header, metadata_snapshot, *modified_context, modified_query_info.query, pipe, processed_stage);
pipe.addTableLock(struct_lock);
pipe.addInterpreterContext(modified_context);
}
pipe.addTableLock(struct_lock);
pipe.addInterpreterContext(modified_context);
}
return pipes;
return pipe;
}
@ -452,7 +444,10 @@ void StorageMerge::convertingSourceStream(
QueryProcessingStage::Enum processed_stage)
{
Block before_block_header = pipe.getHeader();
pipe.addSimpleTransform(std::make_shared<ConvertingTransform>(before_block_header, header, ConvertingTransform::MatchColumnsMode::Name));
pipe.addSimpleTransform([&](const Block & stream_header)
{
return std::make_shared<ConvertingTransform>(stream_header, header, ConvertingTransform::MatchColumnsMode::Name);
});
auto where_expression = query->as<ASTSelectQuery>()->where();

View File

@ -29,7 +29,7 @@ public:
QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const ASTPtr &) const override;
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,
@ -82,7 +82,7 @@ protected:
const Context & context,
QueryProcessingStage::Enum processed_stage);
Pipes createSources(
Pipe createSources(
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
const QueryProcessingStage::Enum & processed_stage,

View File

@ -172,7 +172,7 @@ StorageMergeTree::~StorageMergeTree()
shutdown();
}
Pipes StorageMergeTree::read(
Pipe StorageMergeTree::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
@ -1054,7 +1054,7 @@ bool StorageMergeTree::optimize(
return true;
}
Pipes StorageMergeTree::alterPartition(
Pipe StorageMergeTree::alterPartition(
const ASTPtr & query,
const StorageMetadataPtr & metadata_snapshot,
const PartitionCommands & commands,
@ -1137,7 +1137,7 @@ Pipes StorageMergeTree::alterPartition(
if (query_context.getSettingsRef().alter_partition_verbose_result)
return convertCommandsResultToSource(result);
return { };
return {};
}
void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, const Context & context)

View File

@ -37,7 +37,7 @@ public:
bool supportsIndexForIn() const override { return true; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,
@ -61,7 +61,7 @@ public:
bool deduplicate,
const Context & context) override;
Pipes alterPartition(
Pipe alterPartition(
const ASTPtr & query,
const StorageMetadataPtr & /* metadata_snapshot */,
const PartitionCommands & commands,

View File

@ -53,7 +53,7 @@ StorageMongoDB::StorageMongoDB(
}
Pipes StorageMongoDB::read(
Pipe StorageMongoDB::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /*query_info*/,
@ -79,11 +79,8 @@ Pipes StorageMongoDB::read(
sample_block.insert({ column_data.type, column_data.name });
}
Pipes pipes;
pipes.emplace_back(std::make_shared<SourceFromInputStream>(
return Pipe(std::make_shared<SourceFromInputStream>(
std::make_shared<MongoDBBlockInputStream>(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size, true)));
return pipes;
}
void registerStorageMongoDB(StorageFactory & factory)

View File

@ -34,7 +34,7 @@ public:
std::string getName() const override { return "MongoDB"; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,

View File

@ -64,7 +64,7 @@ StorageMySQL::StorageMySQL(
}
Pipes StorageMySQL::read(
Pipe StorageMySQL::read(
const Names & column_names_,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info_,
@ -94,12 +94,9 @@ Pipes StorageMySQL::read(
sample_block.insert({ column_data.type, column_data.name });
}
Pipes pipes;
/// TODO: rewrite MySQLBlockInputStream
pipes.emplace_back(std::make_shared<SourceFromInputStream>(
return Pipe(std::make_shared<SourceFromInputStream>(
std::make_shared<MySQLBlockInputStream>(pool.get(), query, sample_block, max_block_size_)));
return pipes;
}

View File

@ -37,7 +37,7 @@ public:
std::string getName() const override { return "MySQL"; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -22,7 +22,7 @@ class StorageNull final : public ext::shared_ptr_helper<StorageNull>, public ISt
public:
std::string getName() const override { return "Null"; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo &,
@ -31,10 +31,8 @@ public:
size_t,
unsigned) override
{
Pipes pipes;
pipes.emplace_back(
return Pipe(
std::make_shared<NullSource>(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())));
return pipes;
}
BlockOutputStreamPtr write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &) override

View File

@ -3424,7 +3424,7 @@ ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock StorageReplicatedMerg
return max_added_blocks;
}
Pipes StorageReplicatedMergeTree::read(
Pipe StorageReplicatedMergeTree::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
@ -3870,7 +3870,7 @@ void StorageReplicatedMergeTree::alter(
}
}
Pipes StorageReplicatedMergeTree::alterPartition(
Pipe StorageReplicatedMergeTree::alterPartition(
const ASTPtr & query,
const StorageMetadataPtr & metadata_snapshot,
const PartitionCommands & commands,

View File

@ -87,7 +87,7 @@ public:
bool supportsReplication() const override { return true; }
bool supportsDeduplication() const override { return true; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,
@ -111,7 +111,7 @@ public:
void alter(const AlterCommands & params, const Context & query_context, TableLockHolder & table_lock_holder) override;
Pipes alterPartition(
Pipe alterPartition(
const ASTPtr & query,
const StorageMetadataPtr & metadata_snapshot,
const PartitionCommands & commands,

View File

@ -284,7 +284,7 @@ Strings listFilesWithRegexpMatching(Aws::S3::S3Client & client, const S3::URI &
}
Pipes StorageS3::read(
Pipe StorageS3::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /*query_info*/,
@ -319,7 +319,9 @@ Pipes StorageS3::read(
uri.bucket,
key));
return narrowPipes(std::move(pipes), num_streams);
auto pipe = Pipe::unitePipes(std::move(pipes));
narrowPipe(pipe, num_streams);
return pipe;
}
BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/)

View File

@ -41,7 +41,7 @@ public:
return name;
}
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -292,7 +292,7 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora
}
Pipes StorageStripeLog::read(
Pipe StorageStripeLog::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /*query_info*/,
@ -312,8 +312,7 @@ Pipes StorageStripeLog::read(
String index_file = table_path + "index.mrk";
if (!disk->exists(index_file))
{
pipes.emplace_back(std::make_shared<NullSource>(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())));
return pipes;
return Pipe(std::make_shared<NullSource>(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())));
}
CompressedReadBufferFromFile index_in(disk->readFile(index_file, INDEX_BUFFER_SIZE));
@ -337,7 +336,7 @@ Pipes StorageStripeLog::read(
/// We do not keep read lock directly at the time of reading, because we read ranges of data that do not change.
return pipes;
return Pipe::unitePipes(std::move(pipes));
}

View File

@ -25,7 +25,7 @@ class StorageStripeLog final : public ext::shared_ptr_helper<StorageStripeLog>,
public:
String getName() const override { return "StripeLog"; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -420,7 +420,7 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const Storage
}
Pipes StorageTinyLog::read(
Pipe StorageTinyLog::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /*query_info*/,
@ -431,14 +431,10 @@ Pipes StorageTinyLog::read(
{
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
Pipes pipes;
// When reading, we lock the entire storage, because we only have one file
// per column and can't modify it concurrently.
pipes.emplace_back(std::make_shared<TinyLogSource>(
return Pipe(std::make_shared<TinyLogSource>(
max_block_size, Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)), *this, context.getSettingsRef().max_read_buffer_size));
return pipes;
}

View File

@ -24,7 +24,7 @@ class StorageTinyLog final : public ext::shared_ptr_helper<StorageTinyLog>, publ
public:
String getName() const override { return "TinyLog"; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -13,7 +13,6 @@
#include <Formats/FormatFactory.h>
#include <DataStreams/IBlockOutputStream.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataStreams/AddingDefaultsBlockInputStream.h>
#include <Poco/Net/HTTPRequest.h>
@ -178,7 +177,7 @@ std::function<void(std::ostream &)> IStorageURLBase::getReadPOSTDataCallback(
}
Pipes IStorageURLBase::read(
Pipe IStorageURLBase::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
@ -192,8 +191,7 @@ Pipes IStorageURLBase::read(
for (const auto & [param, value] : params)
request_uri.addQueryParameter(param, value);
Pipes pipes;
pipes.emplace_back(std::make_shared<StorageURLSource>(
return Pipe(std::make_shared<StorageURLSource>(
request_uri,
getReadMethod(),
getReadPOSTDataCallback(
@ -207,8 +205,6 @@ Pipes IStorageURLBase::read(
max_block_size,
ConnectionTimeouts::getHTTPTimeouts(context),
chooseCompressionMethod(request_uri.getPath(), compression_method)));
return pipes;
}
BlockOutputStreamPtr IStorageURLBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/)

View File

@ -19,7 +19,7 @@ namespace DB
class IStorageURLBase : public IStorage
{
public:
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -21,7 +21,7 @@ StorageValues::StorageValues(
setInMemoryMetadata(storage_metadata);
}
Pipes StorageValues::read(
Pipe StorageValues::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /*query_info*/,
@ -32,12 +32,8 @@ Pipes StorageValues::read(
{
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
Pipes pipes;
Chunk chunk(res_block.getColumns(), res_block.rows());
pipes.emplace_back(std::make_shared<SourceFromSingleChunk>(res_block.cloneEmpty(), std::move(chunk)));
return pipes;
return Pipe(std::make_shared<SourceFromSingleChunk>(res_block.cloneEmpty(), std::move(chunk)));
}
}

View File

@ -15,7 +15,7 @@ class StorageValues final : public ext::shared_ptr_helper<StorageValues>, public
public:
std::string getName() const override { return "Values"; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -6,7 +6,6 @@
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/queryToString.h>
#include <Storages/StorageView.h>
#include <Storages/StorageFactory.h>
@ -15,12 +14,8 @@
#include <Common/typeid_cast.h>
#include <Processors/Pipe.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include <Processors/Transforms/MaterializingTransform.h>
#include <Processors/Transforms/ConvertingTransform.h>
#include <DataStreams/MaterializingBlockInputStream.h>
#include <DataStreams/ConvertingBlockInputStream.h>
namespace DB
{
@ -52,7 +47,7 @@ StorageView::StorageView(
}
Pipes StorageView::read(
Pipe StorageView::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
@ -91,9 +86,7 @@ Pipes StorageView::read(
column_names, getVirtuals(), getStorageID()), ConvertingTransform::MatchColumnsMode::Name);
});
pipes = std::move(pipeline).getPipes();
return pipes;
return QueryPipeline::getPipe(std::move(pipeline));
}
static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query)

View File

@ -21,7 +21,7 @@ public:
bool supportsSampling() const override { return true; }
bool supportsFinal() const override { return true; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -9,7 +9,6 @@
#include <IO/ReadHelpers.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Path.h>
#include <Common/ShellCommand.h>
#include <DataStreams/IBlockOutputStream.h>
#include <Processors/Pipe.h>
@ -85,7 +84,7 @@ std::function<void(std::ostream &)> StorageXDBC::getReadPOSTDataCallback(
return [query](std::ostream & os) { os << "query=" << query; };
}
Pipes StorageXDBC::read(
Pipe StorageXDBC::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,

View File

@ -15,7 +15,7 @@ namespace DB
class StorageXDBC : public IStorageURLBase
{
public:
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -28,7 +28,7 @@ public:
setInMemoryMetadata(metadata_);
}
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
@ -46,10 +46,7 @@ public:
UInt64 num_rows = res_columns.at(0)->size();
Chunk chunk(std::move(res_columns), num_rows);
Pipes pipes;
pipes.emplace_back(std::make_shared<SourceFromSingleChunk>(sample_block, std::move(chunk)));
return pipes;
return Pipe(std::make_shared<SourceFromSingleChunk>(sample_block, std::move(chunk)));
}
};

View File

@ -240,7 +240,7 @@ private:
};
Pipes StorageSystemColumns::read(
Pipe StorageSystemColumns::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
@ -294,7 +294,7 @@ Pipes StorageSystemColumns::read(
if (!block_to_filter.rows())
{
pipes.emplace_back(std::make_shared<NullSource>(header));
return pipes;
return Pipe::unitePipes(std::move(pipes));
}
ColumnPtr & database_column = block_to_filter.getByName("database").column;
@ -333,7 +333,7 @@ Pipes StorageSystemColumns::read(
if (!block_to_filter.rows())
{
pipes.emplace_back(std::make_shared<NullSource>(header));
return pipes;
return Pipe::unitePipes(std::move(pipes));
}
ColumnPtr filtered_database_column = block_to_filter.getByName("database").column;
@ -344,7 +344,7 @@ Pipes StorageSystemColumns::read(
std::move(filtered_database_column), std::move(filtered_table_column),
std::move(storages), context));
return pipes;
return Pipe::unitePipes(std::move(pipes));
}
}

View File

@ -17,7 +17,7 @@ class StorageSystemColumns final : public ext::shared_ptr_helper<StorageSystemCo
public:
std::string getName() const override { return "SystemColumns"; }
Pipes read(
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
const SelectQueryInfo & query_info,

View File

@ -45,7 +45,7 @@ protected:
setInMemoryMetadata(storage_metadata);
}
Pipes read(
Pipe read(
const Names & /* column_names */,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
@ -81,9 +81,7 @@ protected:
UInt64 num_rows = new_columns.at(0)->size();
Chunk chunk(std::move(new_columns), num_rows);
Pipes pipes;
pipes.emplace_back(std::make_shared<SourceFromSingleChunk>(std::move(block), std::move(chunk)));
return pipes;
return Pipe(std::make_shared<SourceFromSingleChunk>(std::move(block), std::move(chunk)));
}
};

Some files were not shown because too many files have changed in this diff Show More