From 2ae94f4570596224d3702d2ec06cef771add4827 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 31 Jul 2020 19:54:54 +0300 Subject: [PATCH 1/9] Refactor Pipes part 1. --- src/DataStreams/narrowBlockInputStreams.cpp | 24 +++- src/Processors/Pipe.cpp | 143 ++++++++++++++++++++ src/Processors/Pipe.h | 75 +++++++++- src/Processors/QueryPipeline.h | 4 - 4 files changed, 234 insertions(+), 12 deletions(-) diff --git a/src/DataStreams/narrowBlockInputStreams.cpp b/src/DataStreams/narrowBlockInputStreams.cpp index 8464adb5bb8..55460517232 100644 --- a/src/DataStreams/narrowBlockInputStreams.cpp +++ b/src/DataStreams/narrowBlockInputStreams.cpp @@ -29,23 +29,33 @@ Pipes narrowPipes(Pipes pipes, size_t width) if (size <= width) return pipes; - std::vector partitions(width); + std::vector> partitions(width); auto distribution = getDistribution(size, width); for (size_t i = 0; i < size; ++i) - partitions[distribution[i]].emplace_back(std::move(pipes[i])); + partitions[distribution[i]].emplace_back(pipes.getOutputPort(i)); - Pipes res; - res.reserve(width); + Processors concats; + concats.reserve(width); for (size_t i = 0; i < width; ++i) { - auto processor = std::make_shared(partitions[i].at(0).getHeader(), partitions[i].size()); - res.emplace_back(std::move(partitions[i]), std::move(processor)); + auto concat = std::make_shared(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 res; + auto processors = Pipes::detachProcessors(std::move(pipes)); + processors.insert(processors.end(), concats.begin(), concats.end()); + + return Pipes(std::move(processors)); } } diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index b18bb139215..a7e31876b1d 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -54,6 +54,149 @@ static void checkSource(const IProcessor & source) toString(source.getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); } +Pipes::Pipes(ProcessorPtr source) +{ + checkSource(*source); + output_ports.push_back(&source->getOutputs().front()); + header = output_ports.front()->getHeader(); + processors.emplace_back(std::move(source)); + max_parallel_streams = 1; +} + +Pipes::Pipes(Processors processors_) : processors(std::move(processors_)) +{ + /// Create hash table with processors. + std::unordered_set 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 Pipes 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 Pipes 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 Pipes 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 Pipes 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(), "Pipes"); + + max_parallel_streams = output_ports.size(); +} + +void Pipes::addTransform(ProcessorPtr transform) +{ + 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); + + size_t next_output = 0; + for (auto & input : inputs) + { + connect(*output_ports[next_output], input); + ++next_output; + } + + auto & outputs = transform->getOutputs(); + if (outputs.empty()) + throw Exception("Cannot add transform " + transform->getName() + " to Pipes because it has no outputs", + ErrorCodes::LOGICAL_ERROR); + + output_ports.clear(); + output_ports.reserve(outputs.size()); + + for (auto & output : outputs) + output_ports.emplace_back(&output); + + header = output_ports.front()->getHeader(); + for (size_t i = 1; i < output_ports.size(); ++i) + assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipes"); + + if (totals_port) + assertBlocksHaveEqualStructure(header, totals_port->getHeader(), "Pipes"); + + if (extremes_port) + assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); +} + +void Pipes::addSimpleTransform(const ProcessorGetter & getter) +{ + 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(); + 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); +} Pipe::Pipe(ProcessorPtr source) { diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 085016c3588..ade93a9ed06 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -6,11 +6,84 @@ namespace DB { class Pipe; -using Pipes = std::vector; class IStorage; using StoragePtr = std::shared_ptr; +/// Pipes is a set of processors which represents the part of pipeline. +/// Pipes 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 Pipes +{ +public: + /// Create from source. Source must have no input ports and single output. + explicit Pipes(ProcessorPtr source); + /// Create from processors. Use all not-connected output ports as output_ports. Check invariants. + explicit Pipes(Processors processors_); + + Pipes(const Pipes & other) = delete; + Pipes(Pipes && other) = default; + Pipes & operator=(const Pipes & other) = delete; + Pipes & operator=(Pipes && other) = default; + + const Block & getHeader() const { return header; } + bool empty() const { return output_ports.empty(); } + size_t size() const { return output_ports.size(); } + OutputPort * getOutputPort(size_t pos) const { return output_ports[pos]; } + OutputPort * getTotalsPort() const { return totals_port; } + OutputPort * getExtremesPort() const { return extremes_port; } + + /// 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 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); + + 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; + + /// Add transform with single input and single output for each port. + void addSimpleTransform(const ProcessorGetter & port); + + /// Destroy pipes and get processors. + static Processors detachProcessors(Pipes pipes) { return std::move(pipes.processors); } + +private: + Processors processors; + + /// Header is common for all output below. + Block header; + + /// Output ports. Totals and extremes are allowed to be empty. + std::vector 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. See QueryPipeline::Streams. + /// Usually, it's the same as the number of output ports. + size_t max_parallel_streams = 0; + + std::vector table_locks; + + /// 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> interpreter_context; + std::vector storage_holders; +}; + + /// 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. class Pipe diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 7bd16ff62fd..a2f00a2343d 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -151,9 +151,6 @@ public: /// 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(); } - /// Changes the number of input ports if needed. Adds ResizeTransform. void resize(size_t num_streams, bool force = false, bool strict = false); @@ -167,7 +164,6 @@ public: size_t getNumStreams() const { return streams.size(); } - bool hasMoreThanOneStream() const { return getNumStreams() > 1; } bool hasTotals() const { return totals_having_port != nullptr; } const Block & getHeader() const { return current_header; } From e411916bde096c69b4b95b7dc6b43f57f65d5623 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 3 Aug 2020 14:33:11 +0300 Subject: [PATCH 2/9] Refactor Pipe [part 1]. --- src/DataStreams/narrowBlockInputStreams.cpp | 45 +-- src/DataStreams/narrowBlockInputStreams.h | 3 +- .../ClusterProxy/SelectStreamFactory.cpp | 8 +- .../ClusterProxy/SelectStreamFactory.h | 2 +- .../ClusterProxy/executeQuery.cpp | 2 +- src/Interpreters/ClusterProxy/executeQuery.h | 3 +- src/Processors/Pipe.cpp | 310 +++++++++++++++++- src/Processors/Pipe.h | 47 ++- src/Processors/QueryPipeline.cpp | 8 +- src/Processors/QueryPipeline.h | 4 +- src/Processors/Sources/RemoteSource.cpp | 12 +- src/Storages/IStorage.cpp | 22 ++ src/Storages/IStorage.h | 21 +- src/Storages/Kafka/StorageKafka.cpp | 4 +- src/Storages/Kafka/StorageKafka.h | 2 +- src/Storages/LiveView/StorageLiveView.cpp | 6 +- src/Storages/LiveView/StorageLiveView.h | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 241 +++++++------- .../MergeTree/MergeTreeDataSelectExecutor.h | 10 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 2 +- src/Storages/PartitionCommands.h | 1 - 21 files changed, 531 insertions(+), 224 deletions(-) diff --git a/src/DataStreams/narrowBlockInputStreams.cpp b/src/DataStreams/narrowBlockInputStreams.cpp index 55460517232..17e44e38293 100644 --- a/src/DataStreams/narrowBlockInputStreams.cpp +++ b/src/DataStreams/narrowBlockInputStreams.cpp @@ -23,39 +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> partitions(width); auto distribution = getDistribution(size, width); - for (size_t i = 0; i < size; ++i) - partitions[distribution[i]].emplace_back(pipes.getOutputPort(i)); - - Processors concats; - concats.reserve(width); - - for (size_t i = 0; i < width; ++i) + pipe.transform([&](OutputPortRawPtrs ports) { - auto concat = std::make_shared(partitions[i].at(0)->getHeader(), partitions[i].size()); - size_t next_port = 0; - for (auto & port : concat->getInputs()) + for (size_t i = 0; i < size; ++i) + partitions[distribution[i]].emplace_back(ports[i]); + + Processors concats; + concats.reserve(width); + + for (size_t i = 0; i < width; ++i) { - connect(*partitions[i][next_port], port); - ++next_port; + auto concat = std::make_shared(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)); } - concats.emplace_back(std::move(concat)); - } - - auto processors = Pipes::detachProcessors(std::move(pipes)); - processors.insert(processors.end(), concats.begin(), concats.end()); - - return Pipes(std::move(processors)); + return concats; + }); } } diff --git a/src/DataStreams/narrowBlockInputStreams.h b/src/DataStreams/narrowBlockInputStreams.h index 0477d72b3de..97e9c164ddc 100644 --- a/src/DataStreams/narrowBlockInputStreams.h +++ b/src/DataStreams/narrowBlockInputStreams.h @@ -7,7 +7,6 @@ namespace DB { class Pipe; -using Pipes = std::vector; /** 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; * 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); } diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 51f7e93552a..97d3c71019b 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -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(createLocalStream(modified_query_ast, header, context, processed_stage).getPipe()); }; 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(); @@ -285,7 +285,7 @@ void SelectStreamFactory::createForShard( } }; - res.emplace_back(createDelayedPipe(header, lazily_create_stream)); + pipes.emplace_back(createDelayedPipe(header, lazily_create_stream)); } else emplace_remote_stream(); diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index da1d18b6dd9..80f72fd0024 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -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; diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 451a8873f41..d7495a8e0c0 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -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) { diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index dcbbe0c7e95..f0d9539770d 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -12,7 +12,6 @@ class Cluster; struct SelectQueryInfo; class Pipe; -using Pipes = std::vector; 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); diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index a7e31876b1d..b5ca7f24654 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -1,6 +1,11 @@ #include #include #include +#include +#include +#include +#include +#include namespace DB { @@ -54,16 +59,88 @@ static void checkSource(const IProcessor & source) toString(source.getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); } -Pipes::Pipes(ProcessorPtr source) +static OutputPort * uniteExtremes(const OutputPortRawPtrs & ports, const Block & header, Processors & processors) { - checkSource(*source); + 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(header, ports.size(), 1); + auto extremes = std::make_shared(header); + auto sink = std::make_shared(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(header, ports.size()); + auto limit = std::make_shared(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::Pipe(ProcessorPtr source) +{ + if (auto * source_from_input_stream = typeid_cast(source.get())) + { + /// 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_ports.push_back(&source->getOutputs().front()); header = output_ports.front()->getHeader(); processors.emplace_back(std::move(source)); max_parallel_streams = 1; } -Pipes::Pipes(Processors processors_) : processors(std::move(processors_)) +Pipe::Pipe(Processors processors_) : processors(std::move(processors_)) { /// Create hash table with processors. std::unordered_set set; @@ -75,12 +152,12 @@ Pipes::Pipes(Processors processors_) : processors(std::move(processors_)) for (const auto & port : processor->getInputs()) { if (!port.isConnected()) - throw Exception("Cannot create Pipes because processor " + processor->getName() + + 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 Pipes because processor " + processor->getName() + + throw Exception("Cannot create Pipe because processor " + processor->getName() + " has input port which is connected with unknown processor " + connected_processor->getName(), ErrorCodes::LOGICAL_ERROR); } @@ -95,25 +172,160 @@ Pipes::Pipes(Processors processors_) : processors(std::move(processors_)) const auto * connected_processor = &port.getInputPort().getProcessor(); if (set.count(connected_processor) == 0) - throw Exception("Cannot create Pipes because processor " + processor->getName() + + 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 Pipes because processors don't have any not-connected output ports", + 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(), "Pipes"); + assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipe"); max_parallel_streams = output_ports.size(); } -void Pipes::addTransform(ProcessorPtr transform) +static Pipes removeEmptyPipes(Pipes pipes) { + Pipes res; + res.reserve(pipes.size()); + + for (auto & pipe : pipes) + { + if (!pipe.empty()) + res.emplace_back(std::move(pipe)); + } + + return res; +} + +Pipe Pipe::unitePipes(Pipes pipes) +{ + pipes = removeEmptyPipes(std::move(pipes)); + + if (pipes.empty()) + return {}; + + if (pipes.size() == 1) + return std::move(pipes[0]); + + Pipe res; + OutputPortRawPtrs totals; + OutputPortRawPtrs extremes; + res.header = pipes.front().header; + + 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.table_locks.insert(res.table_locks.end(), pipe.table_locks.begin(), pipe.table_locks.end()); + res.storage_holders.insert(res.storage_holders.end(), pipe.storage_holders.begin(), pipe.storage_holders.end()); + res.interpreter_context.insert(res.interpreter_context.end(), + pipe.interpreter_context.begin(), pipe.interpreter_context.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); + } + + res.totals_port = uniteTotals(totals, res.header, res.processors); + res.extremes_port = uniteExtremes(extremes, res.header, res.processors); +} + +//void Pipe::addPipes(Pipe pipes) +//{ +// if (processors.empty()) +// { +// *this = std::move(pipes); +// return; +// } +// +// if (pipes.processors.empty()) +// return; +// +// assertBlocksHaveEqualStructure(header, pipes.header, "Pipe"); +// +// max_parallel_streams += pipes.max_parallel_streams; +// processors.insert(processors.end(), pipes.processors.begin(), pipes.processors.end()); +// +// OutputPortRawPtrs totals; +// if (totals_port) +// totals.emplace_back(totals_port); +// if (pipes.totals_port) +// totals.emplace_back(pipes.totals_port); +// if (!totals.empty()) +// totals_port = uniteTotals(totals, header, processors); +// +// OutputPortRawPtrs extremes; +// if (extremes_port) +// extremes.emplace_back(extremes_port); +// if (pipes.extremes_port) +// extremes.emplace_back(pipes.extremes_port); +// if (!extremes.empty()) +// extremes_port = uniteExtremes(extremes, header, processors); +//} + +//void Pipe::addSource(ProcessorPtr source) +//{ +// checkSource(*source); +// const auto & source_header = output_ports.front()->getHeader(); +// +// assertBlocksHaveEqualStructure(header, source_header, "Pipes"); !!!! +// +// output_ports.push_back(&source->getOutputs().front()); +// processors.emplace_back(std::move(source)); +// +// max_parallel_streams = std::max(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"); + + 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"); + + extremes_port = &source->getOutputs().front(); + processors.emplace_back(std::move(source)); +} + +void Pipe::addTransform(ProcessorPtr transform) +{ + 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 " @@ -147,10 +359,15 @@ void Pipes::addTransform(ProcessorPtr transform) if (extremes_port) assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); + + max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } -void Pipes::addSimpleTransform(const ProcessorGetter & getter) +void Pipe::addSimpleTransform(const ProcessorGetter & 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) @@ -198,6 +415,77 @@ void Pipes::addSimpleTransform(const ProcessorGetter & getter) header = std::move(new_header); } +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 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()); + } + + OutputPortRawPtrs new_output_ports; + 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()) + { + new_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"); + + max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); +} + +/* Pipe::Pipe(ProcessorPtr source) { if (auto * source_from_input_stream = typeid_cast(source.get())) @@ -295,5 +583,5 @@ void Pipe::enableQuota() source->enableQuota(); } } - +*/ } diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index ade93a9ed06..316d971330f 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -6,39 +6,52 @@ namespace DB { class Pipe; +using Pipes = std::vector; class IStorage; using StoragePtr = std::shared_ptr; +using OutputPortRawPtrs = std::vector; + /// Pipes is a set of processors which represents the part of pipeline. /// Pipes 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 Pipes +class Pipe { public: + /// 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 Pipes(ProcessorPtr source); + explicit Pipe(ProcessorPtr source); /// Create from processors. Use all not-connected output ports as output_ports. Check invariants. - explicit Pipes(Processors processors_); + explicit Pipe(Processors processors_); - Pipes(const Pipes & other) = delete; - Pipes(Pipes && other) = default; - Pipes & operator=(const Pipes & other) = delete; - Pipes & operator=(Pipes && other) = default; + Pipe(const Pipe & other) = delete; + Pipe(Pipe && other) = default; + Pipe & operator=(const Pipe & other) = delete; + Pipe & operator=(Pipe && other) = default; const Block & getHeader() const { return header; } bool empty() const { return output_ports.empty(); } - size_t size() const { return output_ports.size(); } + size_t numOutputPorts() const { return output_ports.size(); } OutputPort * getOutputPort(size_t pos) const { return output_ports[pos]; } OutputPort * getTotalsPort() const { return totals_port; } OutputPort * getExtremesPort() const { return extremes_port; } + /// Add processors form other pipe. It should have same header. + //void addPipes(Pipes pipes); + /// 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 totals and extremes. + void addTotalsSource(ProcessorPtr source); + void addExtremesSource(ProcessorPtr source); + /// 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. @@ -56,8 +69,13 @@ public: /// Add transform with single input and single output for each port. void addSimpleTransform(const ProcessorGetter & port); - /// Destroy pipes and get processors. - static Processors detachProcessors(Pipes pipes) { return std::move(pipes.processors); } + using Transformer = std::function; + + /// Transform Pipe in general way. + void transform(const Transformer & transformer); + + /// Unite several pipes together. They should have same header. + static Pipe unitePipes(Pipes pipes); private: Processors processors; @@ -66,7 +84,7 @@ private: Block header; /// Output ports. Totals and extremes are allowed to be empty. - std::vector output_ports; + OutputPortRawPtrs output_ports; OutputPort * totals_port = nullptr; OutputPort * extremes_port = nullptr; @@ -81,9 +99,12 @@ private: /// because QueryPipeline is alive until query is finished. std::vector> interpreter_context; std::vector storage_holders; + + /// Destroy pipes and get processors. + static Processors detachProcessors(Pipe pipe) { return std::move(pipe.processors); } }; - +/* /// 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. class Pipe @@ -170,5 +191,5 @@ private: friend class QueryPipeline; }; - +*/ } diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index c914eab4e29..1c9ddb722f1 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -793,13 +793,7 @@ void QueryPipeline::initRowsBeforeLimit() 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); + Pipes pipes(processors.detach(), streams.at(0), totals_having_port, extremes_port); pipe.max_parallel_streams = streams.maxParallelStreams(); for (auto & lock : table_locks) diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index a2f00a2343d..f66d2a220cf 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -102,7 +102,6 @@ public: /// 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(); } @@ -197,9 +196,8 @@ public: max_threads = max_threads_; } - /// Convert query pipeline to single or several pipes. + /// Convert query pipeline to pipe. Pipe getPipe() &&; - Pipes getPipes() &&; /// Get internal processors. const Processors & getProcessors() const { return processors.get(); } diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 2f76e0c87d4..13ce2d1f6ee 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -113,18 +113,10 @@ Pipe createRemoteSourcePipe( Pipe pipe(std::make_shared(query_executor, add_aggregation_info)); if (add_totals) - { - auto totals_source = std::make_shared(query_executor); - pipe.setTotalsPort(&totals_source->getPort()); - pipe.addProcessors({std::move(totals_source)}); - } + pipe.addTotalsSource(std::make_shared(query_executor)); if (add_extremes) - { - auto extremes_source = std::make_shared(query_executor); - pipe.setExtremesPort(&extremes_source->getPort()); - pipe.addProcessors({std::move(extremes_source)}); - } + pipe.addExtremesSource(std::make_shared(query_executor)); return pipe; } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index d8f0a0fac7e..3554be67951 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -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 &) { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 1c75a661339..b67f5b4ef23 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -50,7 +50,6 @@ using ProcessorPtr = std::shared_ptr; using Processors = std::vector; class Pipe; -using Pipes = std::vector; class StoragePolicy; using StoragePolicyPtr = std::shared_ptr; @@ -136,8 +135,6 @@ public: using ColumnSizeByName = std::unordered_map; virtual ColumnSizeByName getColumnSizes() const { return {}; } -public: - /// Get mutable version (snapshot) of storage metadata. Metadata object is /// multiversion, so it can be concurrently chaged, but returned copy can be /// used without any locks. @@ -183,7 +180,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 SettingSeconds & acquire_timeout) const; @@ -276,17 +273,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. @@ -355,10 +349,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; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index c0f7ef8e44d..87de9714e73 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -198,7 +198,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 */, @@ -227,7 +227,7 @@ Pipes StorageKafka::read( } LOG_DEBUG(log, "Starting reading {} streams", pipes.size()); - return pipes; + return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index b7e6ea2a7e0..2986f46ebcc 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -37,7 +37,7 @@ public: void startup() override; void shutdown() override; - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 3a6dd86e63e..54ac5bcc791 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -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(blocks_ptr, getHeader())); + return Pipe(std::make_shared(blocks_ptr, getHeader())); } - return pipes; } BlockInputStreams StorageLiveView::watch( diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 13386c7a4e6..43afd169a92 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -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, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index acd4602f3a6..da22303a379 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -147,7 +147,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, @@ -162,7 +162,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, @@ -647,7 +647,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. @@ -721,31 +721,37 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( if (use_sampling) { - for (auto & pipe : res) - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), filter_expression, filter_function->getColumnName(), false)); + res.addSimpleTransform([&filter_expression, &filter_function](const Block & header, Pipe::StreamType) + { + return std::make_shared( + header, filter_expression, filter_function->getColumnName(), false); + }); } if (result_projection) { - for (auto & pipe : res) - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), result_projection)); + res.addSimpleTransform([&result_projection](const Block & header, Pipe::StreamType) + { + return std::make_shared(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>( - pipe.getHeader(), std::make_shared(), used_sample_factor, "_sample_factor")); + res.addSimpleTransform([used_sample_factor](const Block & header, Pipe::StreamType) + { + return std::make_shared>( + header, std::make_shared(), 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( - pipe.getHeader(), query_info.prewhere_info->remove_columns_actions)); + res.addSimpleTransform([&query_info](const Block & header, Pipe::StreamType) + { + return std::make_shared(header, query_info.prewhere_info->remove_columns_actions); + }); } return res; @@ -774,7 +780,7 @@ size_t roundRowsOrBytesToMarks( } -Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( +Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( RangesInDataParts && parts, size_t num_streams, const Names & column_names, @@ -822,13 +828,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) @@ -867,10 +873,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) { @@ -882,18 +891,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(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(pipe.getHeader(), pipe.numOutputPorts())); - return res; + return pipe; + } } static ExpressionActionsPtr createProjection(const Pipe & pipe, const MergeTreeData & data) @@ -904,7 +910,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, @@ -956,7 +962,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) @@ -1101,40 +1107,45 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( reader_settings, virt_columns, part.part_index_in_query)); - - pipes.back().addSimpleTransform(std::make_shared(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, Pipe::StreamType) + { + return std::make_shared(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(pipe.getHeader(), sorting_key_prefix_expr)); + out_projection = createProjection(pipe, data); + pipe.addSimpleTransform([sorting_key_prefix_expr](const Block & header, Pipe::StreamType) + { + return std::make_shared(header, sorting_key_prefix_expr); + }); - auto merging_sorted = std::make_shared( - pipes.back().getHeader(), pipes.size(), sort_description, max_block_size); + pipe.addTransform(std::make_shared( + 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, @@ -1172,25 +1183,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( - 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( + 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(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, Pipe::StreamType) + { + return std::make_shared(header, metadata_snapshot->getSortingKey().expression); + }); + Names sort_columns = metadata_snapshot->getSortingKeyColumns(); SortDescription sort_description; size_t sort_columns_size = sort_columns.size(); @@ -1198,7 +1219,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); @@ -1208,28 +1229,28 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( { case MergeTreeData::MergingParams::Ordinary: { - return std::make_shared(header, pipes.size(), + return std::make_shared(header, pipe.numOutputPorts(), sort_description, max_block_size); } case MergeTreeData::MergingParams::Collapsing: - return std::make_shared(header, pipes.size(), + return std::make_shared(header, pipe.numOutputPorts(), sort_description, data.merging_params.sign_column, true, max_block_size); case MergeTreeData::MergingParams::Summing: - return std::make_shared(header, pipes.size(), + return std::make_shared(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(header, pipes.size(), + return std::make_shared(header, pipe.numOutputPorts(), sort_description, max_block_size); case MergeTreeData::MergingParams::Replacing: - return std::make_shared(header, pipes.size(), + return std::make_shared(header, pipe.numOutputPorts(), sort_description, data.merging_params.version_column, max_block_size); case MergeTreeData::MergingParams::VersionedCollapsing: - return std::make_shared(header, pipes.size(), + return std::make_shared(header, pipe.numOutputPorts(), sort_description, data.merging_params.sign_column, max_block_size); case MergeTreeData::MergingParams::Graphite: @@ -1244,12 +1265,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; @@ -1263,63 +1280,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 & header, Pipe::StreamType) { - auto selector = std::make_shared(pipe.getHeader(), num_streams, key_columns); - auto copier = std::make_shared(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(header, num_streams, key_columns); + }); - Processors merges; - std::vector 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 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(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. diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 9e37d150bd1..5894d6e044b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -24,7 +24,7 @@ public: */ using PartitionIdToMaxBlock = std::unordered_map; - 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, diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 0e1555b0ccd..c13f540ad34 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -18,7 +18,7 @@ class StorageFromMergeTreeDataPart final : public ext::shared_ptr_helper; struct PartitionCommand { From 2cca4d5fcf96f2dd90d7ab3b4e73c7c5299c52a3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 3 Aug 2020 16:54:14 +0300 Subject: [PATCH 3/9] Refactor Pipe [part 2]. --- .../ClusterProxy/executeQuery.cpp | 2 +- src/Processors/Pipe.cpp | 7 +- src/Processors/Pipe.h | 10 ++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 22 +++--- src/Storages/PartitionCommands.cpp | 8 +- src/Storages/PartitionCommands.h | 2 +- src/Storages/StorageBuffer.cpp | 79 +++++++++++-------- src/Storages/StorageBuffer.h | 2 +- src/Storages/StorageDictionary.cpp | 7 +- src/Storages/StorageDictionary.h | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageDistributed.h | 2 +- src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageFile.h | 2 +- src/Storages/StorageGenerateRandom.cpp | 4 +- src/Storages/StorageGenerateRandom.h | 2 +- src/Storages/StorageHDFS.cpp | 4 +- src/Storages/StorageHDFS.h | 2 +- src/Storages/StorageInput.cpp | 8 +- src/Storages/StorageInput.h | 2 +- src/Storages/StorageJoin.cpp | 7 +- src/Storages/StorageJoin.h | 2 +- src/Storages/StorageLog.cpp | 4 +- src/Storages/StorageLog.h | 2 +- src/Storages/StorageMaterializedView.cpp | 12 ++- src/Storages/StorageMaterializedView.h | 4 +- src/Storages/StorageMemory.cpp | 4 +- src/Storages/StorageMemory.h | 2 +- src/Storages/StorageMerge.cpp | 77 ++++++++---------- src/Storages/StorageMerge.h | 2 +- src/Storages/StorageMergeTree.cpp | 6 +- src/Storages/StorageMergeTree.h | 4 +- src/Storages/StorageMongoDB.cpp | 7 +- src/Storages/StorageMongoDB.h | 2 +- src/Storages/StorageMySQL.cpp | 7 +- src/Storages/StorageMySQL.h | 2 +- src/Storages/StorageNull.h | 6 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.h | 4 +- src/Storages/StorageS3.cpp | 6 +- src/Storages/StorageS3.h | 2 +- src/Storages/StorageStripeLog.cpp | 7 +- src/Storages/StorageStripeLog.h | 2 +- src/Storages/StorageTinyLog.cpp | 8 +- src/Storages/StorageTinyLog.h | 2 +- src/Storages/StorageURL.cpp | 8 +- src/Storages/StorageURL.h | 2 +- src/Storages/StorageValues.cpp | 8 +- src/Storages/StorageValues.h | 2 +- src/Storages/StorageView.cpp | 11 +-- src/Storages/StorageView.h | 2 +- src/Storages/StorageXDBC.cpp | 3 +- src/Storages/StorageXDBC.h | 2 +- 53 files changed, 184 insertions(+), 212 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index d7495a8e0c0..1ebd3009ff7 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -106,7 +106,7 @@ Pipe 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)); } } diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index b5ca7f24654..4bd282f4802 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -363,7 +363,7 @@ void Pipe::addTransform(ProcessorPtr transform) max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } -void Pipe::addSimpleTransform(const ProcessorGetter & getter) +void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter) { if (output_ports.empty()) throw Exception("Cannot add simple transform to empty Pipe.", ErrorCodes::LOGICAL_ERROR); @@ -415,6 +415,11 @@ void Pipe::addSimpleTransform(const ProcessorGetter & getter) header = std::move(new_header); } +void Pipe::addSimpleTransform(const ProcessorGetter & getter) +{ + addSimpleTransform([&](const Block & stream_header, StreamType) { return getter(stream_header); }); +} + void Pipe::transform(const Transformer & transformer) { if (output_ports.empty()) diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 316d971330f..c6d42f8b269 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -64,10 +64,12 @@ public: Extremes, /// Stream for extremes. No more then one. }; - using ProcessorGetter = std::function; + using ProcessorGetter = std::function; + using ProcessorGetterWithStreamKind = std::function; /// Add transform with single input and single output for each port. void addSimpleTransform(const ProcessorGetter & port); + void addSimpleTransform(const ProcessorGetterWithStreamKind & port); using Transformer = std::function; @@ -77,6 +79,12 @@ public: /// Unite several pipes together. They should have same header. static Pipe unitePipes(Pipes pipes); + /// Do not allow to change the table while the processors of pipe are alive. + void addTableLock(const TableLockHolder & lock) { 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) { interpreter_context.emplace_back(std::move(context)); } + void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); } + private: Processors processors; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index da22303a379..c4e27a11e7a 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -721,7 +721,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (use_sampling) { - res.addSimpleTransform([&filter_expression, &filter_function](const Block & header, Pipe::StreamType) + res.addSimpleTransform([&filter_expression, &filter_function](const Block & header) { return std::make_shared( header, filter_expression, filter_function->getColumnName(), false); @@ -730,7 +730,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (result_projection) { - res.addSimpleTransform([&result_projection](const Block & header, Pipe::StreamType) + res.addSimpleTransform([&result_projection](const Block & header) { return std::make_shared(header, result_projection); }); @@ -739,7 +739,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( /// 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) { - res.addSimpleTransform([used_sample_factor](const Block & header, Pipe::StreamType) + res.addSimpleTransform([used_sample_factor](const Block & header) { return std::make_shared>( header, std::make_shared(), used_sample_factor, "_sample_factor"); @@ -748,7 +748,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) { - res.addSimpleTransform([&query_info](const Block & header, Pipe::StreamType) + res.addSimpleTransform([&query_info](const Block & header) { return std::make_shared(header, query_info.prewhere_info->remove_columns_actions); }); @@ -1114,7 +1114,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( if (input_order_info->direction == 1) { - pipe.addSimpleTransform([](const Block & header, Pipe::StreamType) + pipe.addSimpleTransform([](const Block & header) { return std::make_shared(header); }); @@ -1129,10 +1129,10 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( /// Drop temporary columns, added by 'sorting_key_prefix_expr' out_projection = createProjection(pipe, data); - pipe.addSimpleTransform([sorting_key_prefix_expr](const Block & header, Pipe::StreamType) - { - return std::make_shared(header, sorting_key_prefix_expr); - }); + pipe.addSimpleTransform([sorting_key_prefix_expr](const Block & header) + { + return std::make_shared(header, sorting_key_prefix_expr); + }); pipe.addTransform(std::make_shared( pipe.getHeader(), pipe.numOutputPorts(), sort_description, max_block_size)); @@ -1207,7 +1207,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( if (!out_projection) out_projection = createProjection(pipe, data); - pipe.addSimpleTransform([&metadata_snapshot](const Block & header, Pipe::StreamType) + pipe.addSimpleTransform([&metadata_snapshot](const Block & header) { return std::make_shared(header, metadata_snapshot->getSortingKey().expression); }); @@ -1280,7 +1280,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( key_columns.emplace_back(desc.column_number); } - pipe.addSimpleTransform([&](const Block & header, Pipe::StreamType) + pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, num_streams, key_columns); }); diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index e3f542695cb..991a23ea9a3 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -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(), "command_type"), @@ -180,11 +180,7 @@ Pipes convertCommandsResultToSource(const PartitionCommandsResultInfo & commands } Chunk chunk(std::move(res_columns), commands_result.size()); - - Pipe pipe(std::make_shared(std::move(header), std::move(chunk))); - Pipes result; - result.emplace_back(std::move(pipe)); - return result; + return Pipe(std::make_shared(std::move(header), std::move(chunk))); } } diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index f4d4aebb58c..cea4b38c6a3 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -101,6 +101,6 @@ using PartitionCommandsResultInfo = std::vector; /// 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); } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index c4dd94b40a8..4a728841c50 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -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,75 @@ 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 & header) { - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), header_after_adding_defaults, metadata_snapshot->getColumns().getDefaults(), context)); + return std::make_shared( + header, header_after_adding_defaults, metadata_snapshot->getColumns().getDefaults(), context); + }); - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), header, ConvertingTransform::MatchColumnsMode::Name)); - } + pipe_from_dst.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared( + 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(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(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( - pipe.getHeader(), - pipes_from_buffers.front().getHeader(), - ConvertingTransform::MatchColumnsMode::Name)); + pipe_from_buffers.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + 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 = InterpreterSelectQuery(query_info.query, context, std::move(pipe_from_buffers), SelectQueryOptions(processed_stage)).execute().pipeline.getPipe(); if (query_info.prewhere_info) { - for (auto & pipe : pipes_from_buffers) - pipe.addSimpleTransform(std::make_shared(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( + 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(pipe.getHeader(), query_info.prewhere_info->alias_actions)); - + pipe_from_buffers.addSimpleTransform([&](const Block & header) + { + return std::make_shared(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; + return Pipe::unitePipes({std::move(pipe_from_dst), std::move(pipe_from_buffers)}); } diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 34625e66a0c..4b7fa763678 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -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, diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 4ea028c7ca8..5d92b9cec55 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -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(stream); /// TODO: update dictionary interface for processors. - Pipes pipes; - pipes.emplace_back(std::move(source)); - return pipes; + return Pipe(std::make_shared(stream)); } diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index f152f8c9932..d822552124d 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -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, diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 8187b40588e..f536c6ee763 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -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, diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 4caf063ce81..b7663676d52 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -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, diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 221e9db9428..471a9f522ab 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -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( this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults())); - return pipes; + return Pipe::unitePipes(pipes); } diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 2c4a43eb979..ea70dcd5311 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -24,7 +24,7 @@ class StorageFile final : public ext::shared_ptr_helper, 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, diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 0d5011de4b1..8020c54d8cd 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -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(max_block_size, max_array_length, max_string_length, generate(), block_header, context)); - return pipes; + return Pipe::unitePipes(pipes); } } diff --git a/src/Storages/StorageGenerateRandom.h b/src/Storages/StorageGenerateRandom.h index 0d068eb951e..e0f037f9a08 100644 --- a/src/Storages/StorageGenerateRandom.h +++ b/src/Storages/StorageGenerateRandom.h @@ -15,7 +15,7 @@ class StorageGenerateRandom final : public ext::shared_ptr_helper( sources_info, uri_without_path, format_name, compression_method, metadata_snapshot->getSampleBlock(), context_, max_block_size)); - return pipes; + return Pipe::unitePipes(pipes); } BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) diff --git a/src/Storages/StorageHDFS.h b/src/Storages/StorageHDFS.h index 62425cc518f..fdeaf4ae1b3 100644 --- a/src/Storages/StorageHDFS.h +++ b/src/Storages/StorageHDFS.h @@ -19,7 +19,7 @@ class StorageHDFS final : public ext::shared_ptr_helper, 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, diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index dc8d7ec1581..5e525210548 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -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(query_context, metadata_snapshot->getSampleBlock())); - return pipes; + return Pipe(std::make_shared(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(input_stream)); - return pipes; + return Pipe(std::make_shared(input_stream)); } } diff --git a/src/Storages/StorageInput.h b/src/Storages/StorageInput.h index f4425ee8cd5..c19b19e4703 100644 --- a/src/Storages/StorageInput.h +++ b/src/Storages/StorageInput.h @@ -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, diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index af1a8bf10d7..32edae20495 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -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(*join, max_block_size, metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); - - return pipes; + return Pipe(std::make_shared(*join, max_block_size, metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); } } diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index fb8ffc1c353..386282708c9 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -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, diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 9cfc906108a..50b31a1f128 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -604,7 +604,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*/, @@ -647,7 +647,7 @@ Pipes StorageLog::read( max_read_buffer_size)); } - return pipes; + return Pipe::unitePipes(pipes); } BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 96acb1668e2..49fc9a576c5 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -24,7 +24,7 @@ class StorageLog final : public ext::shared_ptr_helper, 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, diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 5fd1e7792e4..a2e3fae0951 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -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(); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 315d4cf01a2..1ee4246c7f1 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -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, diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 3a10b2c7e7d..1c690221967 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -101,7 +101,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*/, @@ -132,7 +132,7 @@ Pipes StorageMemory::read( pipes.emplace_back(std::make_shared(column_names, begin, end, *this, metadata_snapshot)); } - return pipes; + return Pipe::unitePipes(pipes); } diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 851fe7fc70a..24eee7897bf 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -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, diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index c9858969579..15e9fd65448 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -19,7 +19,6 @@ #include #include #include -#include #include #include #include @@ -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; + Pipe pipe; bool has_table_virtual_column = false; Names real_column_names; @@ -211,22 +210,19 @@ Pipes StorageMerge::read( auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); - auto source_pipes = createSources( + 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)); } - if (res.empty()) - return res; + if (!pipe.empty()) + narrowPipe(pipe, num_streams); - return narrowPipes(std::move(res), 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 +241,17 @@ Pipes StorageMerge::createSources( VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", table_name); - Pipes pipes; + Pipe pipe; if (!storage) { - auto pipe = InterpreterSelectQuery( + pipe = InterpreterSelectQuery( modified_query_info.query, *modified_context, std::make_shared(header), SelectQueryOptions(processed_stage).analyze()).execute().pipeline.getPipe(); 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 +262,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 +274,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 = interpreter.execute().pipeline.getPipe(); /** 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(pipes.back().getHeader())); + pipe.addSimpleTransform([](const Block & header) { return std::make_shared(header); }); } - if (!pipes.empty()) + if (!pipe.empty()) { - if (concat_streams && pipes.size() > 1) - { - auto concat = std::make_shared(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(pipe.getHeader(), pipe.numOutputPorts())); - pipes = Pipes(); - pipes.emplace_back(std::move(pipe)); + if (has_table_virtual_column) + { + pipe.addSimpleTransform([name = table_name](const Block & header) + { + return std::make_shared>( + header, std::make_shared(), name, "_table"); + }); } - for (auto & pipe : pipes) - { - if (has_table_virtual_column) - pipe.addSimpleTransform(std::make_shared>( - pipe.getHeader(), std::make_shared(), 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 +440,10 @@ void StorageMerge::convertingSourceStream( QueryProcessingStage::Enum processed_stage) { Block before_block_header = pipe.getHeader(); - pipe.addSimpleTransform(std::make_shared(before_block_header, header, ConvertingTransform::MatchColumnsMode::Name)); + pipe.addSimpleTransform([&before_block_header](const Block & header) + { + return std::make_shared(before_block_header, header, ConvertingTransform::MatchColumnsMode::Name); + }); auto where_expression = query->as()->where(); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index f2af25d3f3d..4fa2eaed27d 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -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, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9ef89d62919..898cfa7f766 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -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, @@ -1052,7 +1052,7 @@ bool StorageMergeTree::optimize( return true; } -Pipes StorageMergeTree::alterPartition( +Pipe StorageMergeTree::alterPartition( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, @@ -1135,7 +1135,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) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index e12e646f04e..18fc521b067 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -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, diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index ee6296a6c8f..210dc09ba86 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -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( + return Pipe(std::make_shared( std::make_shared(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size, true))); - - return pipes; } void registerStorageMongoDB(StorageFactory & factory) diff --git a/src/Storages/StorageMongoDB.h b/src/Storages/StorageMongoDB.h index 50dc794f070..a8bd2f4d160 100644 --- a/src/Storages/StorageMongoDB.h +++ b/src/Storages/StorageMongoDB.h @@ -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, diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 000fb8030d7..b2e622663c0 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -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( + return Pipe(std::make_shared( std::make_shared(pool.get(), query, sample_block, max_block_size_))); - - return pipes; } diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index 287c65db6f3..a7f98c4379b 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -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, diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index 072a5e3bc32..226eb05ccf9 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -22,7 +22,7 @@ class StorageNull final : public ext::shared_ptr_helper, 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(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); - return pipes; } BlockOutputStreamPtr write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &) override diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e2a76f45b82..75bbf5488ff 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3418,7 +3418,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, @@ -3864,7 +3864,7 @@ void StorageReplicatedMergeTree::alter( } } -Pipes StorageReplicatedMergeTree::alterPartition( +Pipe StorageReplicatedMergeTree::alterPartition( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 84fbae73fa5..108d4e77d29 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -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, diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index fd27229e530..c08bc4e46ec 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -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*/) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 991145139cd..a172d951642 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -41,7 +41,7 @@ public: return name; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index ae8162d5f1b..66d7c79e434 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -290,7 +290,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*/, @@ -310,8 +310,7 @@ Pipes StorageStripeLog::read( String index_file = table_path + "index.mrk"; if (!disk->exists(index_file)) { - pipes.emplace_back(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); - return pipes; + Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); } CompressedReadBufferFromFile index_in(disk->readFile(index_file, INDEX_BUFFER_SIZE)); @@ -335,7 +334,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(pipes); } diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index dfdf4c381b6..f88120a932e 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -25,7 +25,7 @@ class StorageStripeLog final : public ext::shared_ptr_helper, public: String getName() const override { return "StripeLog"; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index b68ac6ae5f1..e7fe7e2d5f9 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -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( + return Pipe(std::make_shared( max_block_size, Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)), *this, context.getSettingsRef().max_read_buffer_size)); - - return pipes; } diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index 60dacf6e162..dc6ff101503 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -24,7 +24,7 @@ class StorageTinyLog final : public ext::shared_ptr_helper, publ public: String getName() const override { return "TinyLog"; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 55138bf6650..c2f7bfd18d2 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -13,7 +13,6 @@ #include #include -#include #include #include @@ -178,7 +177,7 @@ std::function 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( + return Pipe(std::make_shared( 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*/) diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 69b22e5a1e4..7983ad71520 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -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, diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index d5585edde3b..387d2065f92 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -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(res_block.cloneEmpty(), std::move(chunk))); - - return pipes; + return Pipe(std::make_shared(res_block.cloneEmpty(), std::move(chunk))); } } diff --git a/src/Storages/StorageValues.h b/src/Storages/StorageValues.h index 88fb023fb2b..8a1a06eeb54 100644 --- a/src/Storages/StorageValues.h +++ b/src/Storages/StorageValues.h @@ -15,7 +15,7 @@ class StorageValues final : public ext::shared_ptr_helper, 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, diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 6e66b770486..2c68be4fdb1 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include @@ -15,12 +14,8 @@ #include #include -#include #include #include -#include -#include - 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 std::move(pipeline).getPipe(); } static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query) diff --git a/src/Storages/StorageView.h b/src/Storages/StorageView.h index 9de1f3f0bd8..682c7424b98 100644 --- a/src/Storages/StorageView.h +++ b/src/Storages/StorageView.h @@ -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, diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index fc4bbefe74c..3350a4352db 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include @@ -85,7 +84,7 @@ std::function 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, diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index 44931af4643..508edf22684 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -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, From d9ca3d8d40880e01f183520fd7540b846220d9fc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 3 Aug 2020 18:54:53 +0300 Subject: [PATCH 4/9] Refactor Pipe [part 3]. --- src/Processors/Pipe.cpp | 105 ++++++++++++------ src/Processors/Pipe.h | 14 ++- src/Processors/QueryPipeline.cpp | 2 +- src/Processors/QueryPipeline.h | 50 +-------- .../QueryPlan/ReadFromStorageStep.cpp | 51 +++++---- 5 files changed, 113 insertions(+), 109 deletions(-) diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index 4bd282f4802..b6c3ecb5f88 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -134,6 +134,9 @@ Pipe::Pipe(ProcessorPtr source) else if (source->getOutputs().size() != 1) checkSource(*source); + if (collected_processors) + collected_processors->emplace_back(source.get()); + output_ports.push_back(&source->getOutputs().front()); header = output_ports.front()->getHeader(); processors.emplace_back(std::move(source)); @@ -187,6 +190,10 @@ Pipe::Pipe(Processors processors_) : processors(std::move(processors_)) 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.get()); } static Pipes removeEmptyPipes(Pipes pipes) @@ -204,6 +211,11 @@ static Pipes removeEmptyPipes(Pipes pipes) } Pipe Pipe::unitePipes(Pipes pipes) +{ + return Pipe::unitePipes(std::move(pipes), nullptr); +} + +Pipe Pipe::unitePipes(Pipes pipes, Processors * collected_processors) { pipes = removeEmptyPipes(std::move(pipes)); @@ -217,6 +229,7 @@ Pipe Pipe::unitePipes(Pipes pipes) OutputPortRawPtrs totals; OutputPortRawPtrs extremes; res.header = pipes.front().header; + res.collected_processors = collected_processors; for (auto & pipe : pipes) { @@ -237,8 +250,16 @@ Pipe Pipe::unitePipes(Pipes pipes) 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]); + } } //void Pipe::addPipes(Pipe pipes) @@ -300,6 +321,9 @@ void Pipe::addTotalsSource(ProcessorPtr source) assertBlocksHaveEqualStructure(header, source_header, "Pipes"); + if (collected_processors) + collected_processors->emplace_back(source.get()); + totals_port = &source->getOutputs().front(); processors.emplace_back(std::move(source)); } @@ -317,6 +341,9 @@ void Pipe::addExtremesSource(ProcessorPtr source) assertBlocksHaveEqualStructure(header, source_header, "Pipes"); + if (collected_processors) + collected_processors->emplace_back(source.get()); + extremes_port = &source->getOutputs().front(); processors.emplace_back(std::move(source)); } @@ -360,6 +387,11 @@ void Pipe::addTransform(ProcessorPtr transform) if (extremes_port) assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); + if (collected_processors) + collected_processors->emplace_back(transform.get()); + + processors.emplace_back(std::move(transform)); + max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } @@ -402,6 +434,10 @@ void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter) { connect(*port, transform->getInputs().front()); port = &transform->getOutputs().front(); + + if (collected_processors) + collected_processors->emplace_back(transform.get()); + processors.emplace_back(std::move(transform)); } }; @@ -487,9 +523,44 @@ void Pipe::transform(const Transformer & transformer) if (extremes_port) assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); + if (collected_processors) + { + for (const auto & processor : processors) + collected_processors->emplace_back(processor.get()); + } + + processors.insert(processors.end(), new_processors.begin(), new_processors.end()); + max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } +void Pipe::setLimits(const ISourceWithProgress::LocalLimits & limits) +{ + for (auto & processor : processors) + { + if (auto * source_with_progress = dynamic_cast(processor.get())) + source_with_progress->setLimits(limits); + } +} + +void Pipe::setQuota(const std::shared_ptr & quota) +{ + for (auto & processor : processors) + { + if (auto * source_with_progress = dynamic_cast(processor.get())) + source_with_progress->setQuota(quota); + } +} + +void Pipe::enableQuota() +{ + for (auto & processor : processors) + { + if (auto * source = dynamic_cast(processor.get())) + source->enableQuota(); + } +} + /* Pipe::Pipe(ProcessorPtr source) { @@ -553,40 +624,6 @@ void Pipe::addSimpleTransform(ProcessorPtr transform) processors.emplace_back(std::move(transform)); } -void Pipe::setLimits(const ISourceWithProgress::LocalLimits & limits) -{ - for (auto & processor : processors) - { - if (auto * source_with_progress = dynamic_cast(processor.get())) - source_with_progress->setLimits(limits); - } -} -void Pipe::setQuota(const std::shared_ptr & quota) -{ - for (auto & processor : processors) - { - if (auto * source_with_progress = dynamic_cast(processor.get())) - source_with_progress->setQuota(quota); - } -} - -void Pipe::pinSources(size_t executor_number) -{ - for (auto & processor : processors) - { - if (auto * source = dynamic_cast(processor.get())) - source->setStream(executor_number); - } -} - -void Pipe::enableQuota() -{ - for (auto & processor : processors) - { - if (auto * source = dynamic_cast(processor.get())) - source->enableQuota(); - } -} */ } diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index c6d42f8b269..5854f61554a 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -79,6 +79,11 @@ public: /// Unite several pipes together. They should have same header. static Pipe unitePipes(Pipes pipes); + /// Specify quotas and limits for every ISourceWithProgress. + void setLimits(const SourceWithProgress::LocalLimits & limits); + void setQuota(const std::shared_ptr & quota); + void enableQuota(); + /// Do not allow to change the table while the processors of pipe are alive. void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } /// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible. @@ -108,8 +113,13 @@ private: std::vector> interpreter_context; std::vector storage_holders; - /// Destroy pipes and get processors. - static Processors detachProcessors(Pipe pipe) { return std::move(pipe.processors); } + /// If is set, all newly created processors will be added to this too. + /// It is needed for debug. See QueryPipelineProcessorsCollector. + Processors * collected_processors = nullptr; + + static Pipe unitePipes(Pipes pipes, Processors * collected_processors); + + friend class QueryPipeline; }; /* diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 1c9ddb722f1..33d232da449 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -123,7 +123,7 @@ static OutputPort * uniteTotals(const std::vector & ports, const B 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); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index f66d2a220cf..5020a2e8964 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -22,54 +22,6 @@ class QueryPipelineProcessorsCollector; class QueryPipeline { -private: - /// It's a wrapper over std::vector - /// 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(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(max_parallel_streams + other.max_parallel_streams, data.size()); - } - - void assign(std::initializer_list list) - { - data = list; - max_parallel_streams = std::max(max_parallel_streams, data.size()); - } - - size_t maxParallelStreams() const { return max_parallel_streams; } - - private: - std::vector 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 @@ -101,7 +53,7 @@ public: QueryPipeline & operator= (QueryPipeline && rhs); /// All pipes must have same header. - void init(Pipes pipes); + void init(Pipe pipe); bool initialized() { return !processors.empty(); } bool isCompleted() { return initialized() && streams.empty(); } diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.cpp b/src/Processors/QueryPlan/ReadFromStorageStep.cpp index efe8767d27d..39b0743fe38 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromStorageStep.cpp @@ -7,7 +7,6 @@ #include #include #include -#include 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(metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID()))); + pipe = Pipe(std::make_shared(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( - pipe.getHeader(), query_info.prewhere_info->alias_actions)); + { + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, query_info.prewhere_info->alias_actions); + }); + } - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + 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( - pipe.getHeader(), query_info.prewhere_info->remove_columns_actions)); + { + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, query_info.prewhere_info->remove_columns_actions); + }); + } } - - pipes.emplace_back(std::move(pipe)); } pipeline = std::make_unique(); @@ -104,20 +113,16 @@ 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(); + pipe.enableQuota(); - pipeline->init(std::move(pipes)); + pipeline->init(std::move(pipe)); pipeline->addInterpreterContext(std::move(context)); pipeline->addStorageHolder(std::move(storage)); From 9385f3de0ea4cba50817dfafe78f80cf19e6b60e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Aug 2020 16:06:59 +0300 Subject: [PATCH 5/9] Refactor Pipe [part 4]. --- .../tests/finish_sorting_stream.cpp | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 6 +- src/Interpreters/MergeJoin.cpp | 2 +- src/Processors/IProcessor.h | 5 - src/Processors/Pipe.cpp | 221 ++++++----- src/Processors/Pipe.h | 125 +----- src/Processors/QueryPipeline.cpp | 375 +++--------------- src/Processors/QueryPipeline.h | 110 ++--- src/Processors/QueryPlan/AggregatingStep.cpp | 6 +- .../QueryPlan/FinishSortingStep.cpp | 4 +- src/Processors/QueryPlan/LimitStep.cpp | 2 +- .../QueryPlan/MergingAggregatedStep.cpp | 10 +- .../QueryPlan/MergingSortedStep.cpp | 4 +- src/Processors/QueryPlan/OffsetStep.cpp | 2 +- src/Processors/Sources/DelayedSource.cpp | 30 +- .../Transforms/AggregatingTransform.cpp | 26 +- ...gingAggregatedMemoryEfficientTransform.cpp | 46 +-- ...ergingAggregatedMemoryEfficientTransform.h | 10 +- 18 files changed, 269 insertions(+), 717 deletions(-) diff --git a/src/DataStreams/tests/finish_sorting_stream.cpp b/src/DataStreams/tests/finish_sorting_stream.cpp index ea151e01293..7ce342d3e8e 100644 --- a/src/DataStreams/tests/finish_sorting_stream.cpp +++ b/src/DataStreams/tests/finish_sorting_stream.cpp @@ -72,7 +72,7 @@ int main(int argc, char ** argv) QueryPipeline pipeline; pipeline.init(std::move(source)); - pipeline.addPipe({std::make_shared(pipeline.getHeader(), sort_descr, n, 0, 0, 0, nullptr, 0)}); + pipeline.addTransform(std::make_shared(pipeline.getHeader(), sort_descr, n, 0, 0, 0, nullptr, 0)); SortDescription sort_descr_final; sort_descr_final.emplace_back("col1", 1, 1); diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 1a94225443c..4fed6f02d5f 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -269,10 +269,12 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() if (settings.graph) { + auto processors = Pipe::detachProcessors(std::move(*pipeline).getPipe()); + 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 { diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 63e7aa36535..e279b741c91 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -516,7 +516,7 @@ void MergeJoin::mergeInMemoryRightBlocks() pipeline.init(std::move(source)); /// TODO: there should be no splitted keys by blocks for RIGHT|FULL JOIN - pipeline.addPipe({std::make_shared(pipeline.getHeader(), right_sort_description, max_rows_in_right_block, 0, 0, 0, nullptr, 0)}); + pipeline.addTransform(std::make_shared(pipeline.getHeader(), right_sort_description, max_rows_in_right_block, 0, 0, 0, nullptr, 0)); auto sorted_input = PipelineExecutingBlockInputStream(std::move(pipeline)); diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index e9148dd5075..40c3c8c8940 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -287,9 +287,6 @@ public: size_t getStream() const { return stream_number; } constexpr static size_t NO_STREAM = std::numeric_limits::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; }; diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index b6c3ecb5f88..65c2e5f2a5a 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -123,6 +123,59 @@ static OutputPort * uniteTotals(const OutputPortRawPtrs & ports, const Block & h return totals_port; } +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(source.get())) @@ -262,51 +315,24 @@ Pipe Pipe::unitePipes(Pipes pipes, Processors * collected_processors) } } -//void Pipe::addPipes(Pipe pipes) -//{ -// if (processors.empty()) -// { -// *this = std::move(pipes); -// return; -// } -// -// if (pipes.processors.empty()) -// return; -// -// assertBlocksHaveEqualStructure(header, pipes.header, "Pipe"); -// -// max_parallel_streams += pipes.max_parallel_streams; -// processors.insert(processors.end(), pipes.processors.begin(), pipes.processors.end()); -// -// OutputPortRawPtrs totals; -// if (totals_port) -// totals.emplace_back(totals_port); -// if (pipes.totals_port) -// totals.emplace_back(pipes.totals_port); -// if (!totals.empty()) -// totals_port = uniteTotals(totals, header, processors); -// -// OutputPortRawPtrs extremes; -// if (extremes_port) -// extremes.emplace_back(extremes_port); -// if (pipes.extremes_port) -// extremes.emplace_back(pipes.extremes_port); -// if (!extremes.empty()) -// extremes_port = uniteExtremes(extremes, header, processors); -//} +void Pipe::addSource(ProcessorPtr source) +{ + checkSource(*source); + const auto & source_header = output_ports.front()->getHeader(); -//void Pipe::addSource(ProcessorPtr source) -//{ -// checkSource(*source); -// const auto & source_header = output_ports.front()->getHeader(); -// -// assertBlocksHaveEqualStructure(header, source_header, "Pipes"); !!!! -// -// output_ports.push_back(&source->getOutputs().front()); -// processors.emplace_back(std::move(source)); -// -// max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); -//} + if (output_ports.empty()) + header = source_header; + else + assertBlocksHaveEqualStructure(header, source_header, "Pipes"); + + if (collected_processors) + collected_processors->emplace_back(source.get()); + + output_ports.push_back(&source->getOutputs().front()); + processors.emplace_back(std::move(source)); + + max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); +} void Pipe::addTotalsSource(ProcessorPtr source) { @@ -456,6 +482,48 @@ void Pipe::addSimpleTransform(const ProcessorGetter & getter) addSimpleTransform([&](const Block & stream_header, StreamType) { return getter(stream_header); }); } +void Pipe::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) +{ + 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(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::transform(const Transformer & transformer) { if (output_ports.empty()) @@ -561,69 +629,4 @@ void Pipe::enableQuota() } } -/* -Pipe::Pipe(ProcessorPtr source) -{ - if (auto * source_from_input_stream = typeid_cast(source.get())) - { - totals = source_from_input_stream->getTotalsPort(); - extremes = source_from_input_stream->getExtremesPort(); - } - else if (source->getOutputs().size() != 1) - checkSource(*source); - - output_port = &source->getOutputs().front(); - - 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(Pipes && pipes, ProcessorPtr transform) -{ - checkSingleOutput(*transform); - checkMultipleInputs(*transform, pipes.size()); - - auto it = transform->getInputs().begin(); - - 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)); - } - - output_port = &transform->getOutputs().front(); - processors.emplace_back(std::move(transform)); -} - -Pipe::Pipe(OutputPort * port) : output_port(port) -{ -} - -void Pipe::addProcessors(const Processors & processors_) -{ - processors.insert(processors.end(), processors_.begin(), processors_.end()); -} - -void Pipe::addSimpleTransform(ProcessorPtr transform) -{ - checkSimpleTransform(*transform); - connect(*output_port, transform->getInputs().front()); - output_port = &transform->getOutputs().front(); - processors.emplace_back(std::move(transform)); -} - - -*/ } diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 5854f61554a..e6ae375da48 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -25,6 +25,8 @@ public: Pipe() = default; /// Create from source. Source must have no input ports and single output. explicit Pipe(ProcessorPtr source); + /// 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_); @@ -34,19 +36,17 @@ public: Pipe & operator=(Pipe && other) = default; const Block & getHeader() const { return header; } - bool empty() const { return output_ports.empty(); } + 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; } - /// Add processors form other pipe. It should have same header. - //void addPipes(Pipes pipes); - /// 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); + void addSource(ProcessorPtr source); /// Add totals and extremes. void addTotalsSource(ProcessorPtr source); @@ -79,6 +79,9 @@ public: /// 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 & quota); @@ -91,123 +94,39 @@ public: void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); } private: - Processors processors; + /// Destruction order: processors, header, locks, temporary storages, local contexts + + /// 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> interpreter_context; + std::vector storage_holders; + std::vector table_locks; /// 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. See QueryPipeline::Streams. + /// 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 table_locks; - - /// 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> interpreter_context; - std::vector storage_holders; - /// If is set, all newly created processors will be added to this too. /// It is needed for debug. See QueryPipelineProcessorsCollector. Processors * collected_processors = nullptr; + /// 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); friend class QueryPipeline; }; -/* -/// 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. -class Pipe -{ -public: - /// Create from source. It 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); - - 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_); - - OutputPort & getPort() const { return *output_port; } - const Block & getHeader() const { return output_port->getHeader(); } - - /// 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); - - Processors detachProcessors() && { return std::move(processors); } - - /// Specify quotas and limits for every ISourceWithProgress. - void setLimits(const SourceWithProgress::LocalLimits & limits); - void setQuota(const std::shared_ptr & 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); } - /// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible. - void addInterpreterContext(std::shared_ptr context) { interpreter_context.emplace_back(std::move(context)); } - void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); } - - const std::vector & getTableLocks() const { return table_locks; } - const std::vector> & getContexts() const { return interpreter_context; } - const std::vector & getStorageHolders() const { return storage_holders; } - -private: - Processors processors; - OutputPort * output_port = nullptr; - OutputPort * totals = nullptr; - OutputPort * extremes = nullptr; - - /// It is the max number of processors which can be executed in parallel for each step. See QueryPipeline::Streams. - size_t max_parallel_streams = 0; - - std::vector table_locks; - - /// 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> interpreter_context; - std::vector 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); - - friend class QueryPipeline; -}; -*/ } diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 33d232da449..778d5ab093d 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -39,11 +40,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 +63,39 @@ 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 & 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(header, ports.size(), 1); - auto extremes = std::make_shared(header); - auto sink = std::make_shared(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 & ports, const Block & header, - QueryPipeline::ProcessorsContainer & processors) -{ - /// Calculate totals fro several streams. - /// Take totals from first sources which has any, skip others. - - /// ->> Concat -> Limit - - auto concat = std::make_shared(header, ports.size()); - auto limit = std::make_shared(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(Pipe pipe) +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); - - /// 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 totals; - std::vector 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); - } -} - -static ProcessorPtr callProcessorGetter( - const Block & header, const QueryPipeline::ProcessorGetter & getter, QueryPipeline::StreamType) -{ - return getter(header); -} - -static ProcessorPtr callProcessorGetter( - const Block & header, const QueryPipeline::ProcessorGetterWithStreamKind & getter, QueryPipeline::StreamType kind) -{ - return getter(header, kind); -} - -template -void QueryPipeline::addSimpleTransformImpl(const TProcessorGetter & 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); -} - -void QueryPipeline::addSimpleTransform(const ProcessorGetter & getter) -{ - addSimpleTransformImpl(getter); -} - -void QueryPipeline::addSimpleTransform(const ProcessorGetterWithStreamKind & getter) -{ - addSimpleTransformImpl(getter); -} - -void QueryPipeline::setSinks(const 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(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(); -} - -void QueryPipeline::addPipe(Processors pipe) -{ - checkInitializedAndNotCompleted(); - if (pipe.empty()) - throw Exception("Can't add empty processors list to QueryPipeline.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Can't initialize pipeline with empty pipe.", ErrorCodes::LOGICAL_ERROR); - auto & first = pipe.front(); - auto & last = pipe.back(); + pipe = std::move(pipe_); +} - auto num_inputs = first->getInputs().size(); +void QueryPipeline::addSimpleTransform(const Pipe::ProcessorGetter & getter) +{ + checkInitializedAndNotCompleted(); + pipe.addSimpleTransform(getter); +} - 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); +void QueryPipeline::addSimpleTransform(const Pipe::ProcessorGetterWithStreamKind & getter) +{ + checkInitializedAndNotCompleted(); + pipe.addSimpleTransform(getter); +} - auto stream = streams.begin(); - for (auto & input : first->getInputs()) - connect(**(stream++), input); +void QueryPipeline::addTransform(ProcessorPtr transform) +{ + checkInitializedAndNotCompleted(); + pipe.addTransform(std::move(transform)); +} - 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 +103,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(current_header, streams.size(), delayed_streams); - addPipe({ std::move(processor) }); + auto processor = std::make_shared(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 +124,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(current_header, getNumStreams(), num_streams); + resize = std::make_shared(getHeader(), getNumStreams(), num_streams); else - resize = std::make_shared(current_header, getNumStreams(), num_streams); + resize = std::make_shared(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 +142,24 @@ 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)); + pipe.totals_port = totals_port; } 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,22 +171,7 @@ void QueryPipeline::addDefaultTotals() } auto source = std::make_shared(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() @@ -837,8 +556,6 @@ QueryPipeline & QueryPipeline::operator= (QueryPipeline && rhs) 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; diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 5020a2e8964..61b4dc507fd 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -12,70 +12,37 @@ namespace DB { - -using TableLockHolders = std::vector; class Context; class IOutputFormat; class QueryPipelineProcessorsCollector; +struct AggregatingTransformParams; +using AggregatingTransformParamsPtr = std::shared_ptr; + class QueryPipeline { 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= (const QueryPipeline & rhs) = delete; - QueryPipeline & operator= (QueryPipeline && rhs); + QueryPipeline & operator= (const QueryPipeline & rhs) = delete; /// All pipes must have same header. void init(Pipe pipe); - bool initialized() { return !processors.empty(); } - bool isCompleted() { return initialized() && streams.empty(); } + bool initialized() { return !pipe.empty(); } + bool isCompleted() { return pipe.isCompleted(); } - /// 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. - }; - - using ProcessorGetter = std::function; - using ProcessorGetterWithStreamKind = std::function; + 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. @@ -88,40 +55,37 @@ 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); + 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> pipelines, const Block & common_header, size_t max_threads_limit = 0); PipelineExecutorPtr execute(); - size_t getNumStreams() const { return streams.size(); } + size_t getNumStreams() const { return pipe.numOutputPorts(); } - 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) { 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) { pipe.addInterpreterContext(std::move(context)); } + void addStorageHolder(StoragePtr storage) { pipe.addStorageHolder(std::move(storage)); } /// For compatibility with IBlockInputStream. void setProgressCallback(const ProgressCallback & callback); @@ -130,7 +94,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); @@ -151,34 +115,10 @@ public: /// Convert query pipeline to pipe. Pipe getPipe() &&; - /// Get internal processors. - const Processors & getProcessors() const { return processors.get(); } - 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> interpreter_context; - std::vector 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. @@ -189,10 +129,6 @@ private: void checkInitialized(); void checkInitializedAndNotCompleted(); - static void checkSource(const ProcessorPtr & source, bool can_have_totals); - - template - void addSimpleTransformImpl(const TProcessorGetter & getter); void initRowsBeforeLimit(); diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index a78c813ef8a..e8d4a262366 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -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 diff --git a/src/Processors/QueryPlan/FinishSortingStep.cpp b/src/Processors/QueryPlan/FinishSortingStep.cpp index 8d9db8f83a1..d883bd0e0dd 100644 --- a/src/Processors/QueryPlan/FinishSortingStep.cpp +++ b/src/Processors/QueryPlan/FinishSortingStep.cpp @@ -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 diff --git a/src/Processors/QueryPlan/LimitStep.cpp b/src/Processors/QueryPlan/LimitStep.cpp index 21ee1462ef5..565d05956e5 100644 --- a/src/Processors/QueryPlan/LimitStep.cpp +++ b/src/Processors/QueryPlan/LimitStep.cpp @@ -47,7 +47,7 @@ void LimitStep::transformPipeline(QueryPipeline & pipeline) auto transform = std::make_shared( 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 diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index 473cfb0d125..85bbbeab59a 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -59,16 +59,8 @@ void MergingAggregatedStep::transformPipeline(QueryPipeline & pipeline) ? static_cast(memory_efficient_merge_threads) : static_cast(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 diff --git a/src/Processors/QueryPlan/MergingSortedStep.cpp b/src/Processors/QueryPlan/MergingSortedStep.cpp index 98a7deebc9e..c59540b009f 100644 --- a/src/Processors/QueryPlan/MergingSortedStep.cpp +++ b/src/Processors/QueryPlan/MergingSortedStep.cpp @@ -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)); } } diff --git a/src/Processors/QueryPlan/OffsetStep.cpp b/src/Processors/QueryPlan/OffsetStep.cpp index 66f2ada2152..7ac3d3f2110 100644 --- a/src/Processors/QueryPlan/OffsetStep.cpp +++ b/src/Processors/QueryPlan/OffsetStep.cpp @@ -33,7 +33,7 @@ void OffsetStep::transformPipeline(QueryPipeline & pipeline) auto transform = std::make_shared( pipeline.getHeader(), offset, pipeline.getNumStreams()); - pipeline.addPipe({std::move(transform)}); + pipeline.addTransform(std::move(transform)); } void OffsetStep::describeActions(FormatSettings & settings) const diff --git a/src/Processors/Sources/DelayedSource.cpp b/src/Processors/Sources/DelayedSource.cpp index f9e112f5e68..f0cc6b1930c 100644 --- a/src/Processors/Sources/DelayedSource.cpp +++ b/src/Processors/Sources/DelayedSource.cpp @@ -1,6 +1,7 @@ #include #include #include +#include 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(header); + main_output = &source->getPort(); + processors.emplace_back(std::move(source)); + return; + } + + if (pipe.numOutputPorts() > 1) + pipe.addTransform(std::make_shared(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(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); } } diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index c5be62e276a..8fb6f13a85b 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -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(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(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)); } } diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp index 6496c32738e..3445b1b3312 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp @@ -1,8 +1,9 @@ #include -#include #include +#include #include +#include 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(header, num_inputs, params); - processors.emplace_back(std::move(grouping)); + pipe.addTransform(std::make_shared(pipe.getHeader(), pipe.numOutputPorts(), params)); if (num_merging_processors <= 1) { /// --> GroupingAggregated --> MergingAggregatedBucket --> - auto transform = std::make_shared(params); - connect(processors.back()->getOutputs().front(), transform->getInputPort()); - - processors.emplace_back(std::move(transform)); - return processors; + pipe.addTransform(std::make_shared(params)); + return; } /// --> --> MergingAggregatedBucket --> /// --> GroupingAggregated --> ResizeProcessor --> MergingAggregatedBucket --> SortingAggregated --> /// --> --> MergingAggregatedBucket --> - auto resize = std::make_shared(Block(), 1, num_merging_processors); - connect(processors.back()->getOutputs().front(), resize->getInputs().front()); - processors.emplace_back(std::move(resize)); + pipe.addTransform(std::make_shared(Block(), 1, num_merging_processors)); - auto sorting = std::make_shared(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(params); - transform->setStream(i); - connect(*out, transform->getInputPort()); - connect(transform->getOutputPort(), *in); - processors.emplace_back(std::move(transform)); - } + return std::make_shared(params); + }); - processors.emplace_back(std::move(sorting)); - return processors; + pipe.addTransform(std::make_shared(num_merging_processors, params)); } } diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h index 1ed1979c0d4..1f0b7ed4157 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h @@ -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); } From a153f05e10fe9ddfb7607672385c8ce6a6a826dc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Aug 2020 18:51:56 +0300 Subject: [PATCH 6/9] Refactor Pipe [part 5]. --- src/Interpreters/InterpreterAlterQuery.cpp | 4 +- src/Processors/Pipe.cpp | 119 ++++++++- src/Processors/Pipe.h | 7 +- src/Processors/QueryPipeline.cpp | 269 ++++----------------- src/Processors/QueryPipeline.h | 13 +- 5 files changed, 164 insertions(+), 248 deletions(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 2f9e93d0eee..6f1215d6af0 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -87,8 +87,8 @@ 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()) + 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_pipes)); } diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index 65c2e5f2a5a..7a0c7184882 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include namespace DB { @@ -374,7 +376,37 @@ void Pipe::addExtremesSource(ProcessorPtr source) 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(port->getHeader()); + connect(*port, null_sink->getPort()); + + if (collected_processors) + collected_processors->emplace_back(null_sink.get()); + + 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); @@ -385,6 +417,19 @@ void Pipe::addTransform(ProcessorPtr transform) "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) { @@ -393,15 +438,34 @@ void Pipe::addTransform(ProcessorPtr transform) } auto & outputs = transform->getOutputs(); - if (outputs.empty()) - throw Exception("Cannot add transform " + transform->getName() + " to Pipes because it has no outputs", - ErrorCodes::LOGICAL_ERROR); output_ports.clear(); output_ports.reserve(outputs.size()); + bool found_totals = false; + bool found_extremes = false; + for (auto & output : outputs) - output_ports.emplace_back(&output); + { + 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) @@ -524,6 +588,44 @@ void Pipe::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) 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(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(totals.getHeader())); + + if (!extremes_port) + addExtremesSource(std::make_shared(extremes.getHeader())); + + if (collected_processors) + collected_processors->emplace_back(output.get()); + + 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()) @@ -620,13 +722,4 @@ void Pipe::setQuota(const std::shared_ptr & quota) } } -void Pipe::enableQuota() -{ - for (auto & processor : processors) - { - if (auto * source = dynamic_cast(processor.get())) - source->enableQuota(); - } -} - } diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index e6ae375da48..787bade065a 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -52,10 +52,15 @@ public: void addTotalsSource(ProcessorPtr source); void addExtremesSource(ProcessorPtr source); + /// 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 { @@ -85,7 +90,6 @@ public: /// Specify quotas and limits for every ISourceWithProgress. void setLimits(const SourceWithProgress::LocalLimits & limits); void setQuota(const std::shared_ptr & quota); - void enableQuota(); /// Do not allow to change the table while the processors of pipe are alive. void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } @@ -125,6 +129,7 @@ private: 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; }; diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 778d5ab093d..97e96387f7f 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -2,9 +2,7 @@ #include #include -#include #include -#include #include #include #include @@ -148,8 +146,7 @@ void QueryPipeline::addTotalsHavingTransform(ProcessorPtr transform) resize(1); auto * totals_port = &transform->getOutputs().back(); - pipe.addTransform(std::move(transform)); - pipe.totals_port = totals_port; + pipe.addTransform(std::move(transform), totals_port, nullptr); } void QueryPipeline::addDefaultTotals() @@ -176,46 +173,21 @@ void QueryPipeline::addDefaultTotals() void QueryPipeline::dropTotalsAndExtremes() { - auto drop_port = [&](OutputPort *& port) - { - auto null_sink = std::make_shared(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 extremes; - extremes.reserve(streams.size()); - - for (auto & stream : streams) - { - auto transform = std::make_shared(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(getHeader()); + auto * port = &transform->getExtremesPort(); + pipe.addTransform(std::move(transform), nullptr, port); } void QueryPipeline::addCreatingSetsTransform(ProcessorPtr transform) @@ -228,94 +200,49 @@ void QueryPipeline::addCreatingSetsTransform(ProcessorPtr transform) resize(1); - auto concat = std::make_shared(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(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(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(totals.getHeader()); - totals_having_port = &null_source->getPort(); - processors.emplace(std::move(null_source)); - } - - if (!extremes_port) - { - auto null_source = std::make_shared(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(output.get()); + pipe.setOutputFormat(std::move(output)); initRowsBeforeLimit(); } -void QueryPipeline::unitePipelines( - std::vector> pipelines, const Block & common_header, size_t max_threads_limit) +QueryPipeline QueryPipeline::unitePipelines( + std::vector> 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( - header, common_header, ConvertingTransform::MatchColumnsMode::Position); - }); - } - - std::vector extremes; - std::vector 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()) { @@ -326,36 +253,7 @@ void QueryPipeline::unitePipelines( }); } - if (pipeline.extremes_port) - { - auto converting = std::make_shared( - 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( - 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; @@ -366,33 +264,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(processor.get())) source->setProgressCallback(callback); @@ -406,7 +292,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(processor.get())) source->setProcessListElement(elem); @@ -510,101 +396,28 @@ void QueryPipeline::initRowsBeforeLimit() output_format->setRowsBeforeLimitCounter(rows_before_limit_at_least); } -Pipe QueryPipeline::getPipe() && -{ - Pipes pipes(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(processors.get(), process_list_element); + return std::make_shared(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; - 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; -} - -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_; + pipe.collected_processors = 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) diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 61b4dc507fd..74cc692ac42 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -28,7 +28,7 @@ public: ~QueryPipeline() = default; QueryPipeline(QueryPipeline &&) = default; QueryPipeline(const QueryPipeline &) = delete; - QueryPipeline & operator= (QueryPipeline && rhs); + QueryPipeline & operator= (QueryPipeline && rhs) = default; QueryPipeline & operator= (const QueryPipeline & rhs) = delete; /// All pipes must have same header. @@ -73,7 +73,11 @@ public: /// 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> pipelines, const Block & common_header, size_t max_threads_limit = 0); + static QueryPipeline unitePipelines( + std::vector> pipelines, + const Block & common_header, + size_t max_threads_limit = 0, + Processors * collected_processors = nullptr); PipelineExecutorPtr execute(); @@ -113,10 +117,9 @@ public: } /// Convert query pipeline to pipe. - Pipe getPipe() &&; + static Pipe getPipe(QueryPipeline pipeline) { return std::move(pipeline.pipe); } private: - /// Destruction order: processors, header, locks, temporary storages, local contexts Pipe pipe; IOutputFormat * output_format = nullptr; @@ -132,6 +135,8 @@ private: void initRowsBeforeLimit(); + void setCollectedProcessors(Processors * processors); + friend class QueryPipelineProcessorsCollector; }; From 20e63d22711c00fc7fa4587211fbb9b1f65fbc8b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 6 Aug 2020 15:24:05 +0300 Subject: [PATCH 7/9] Refactor Pipe [part 6] --- src/Client/Connection.cpp | 10 ++- src/Core/ExternalTable.cpp | 10 ++- src/DataStreams/BlockIO.cpp | 2 +- src/DataStreams/RemoteQueryExecutor.cpp | 15 +--- .../gtest_blocks_size_merging_streams.cpp | 28 +++--- .../ClusterProxy/SelectStreamFactory.cpp | 4 +- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Processors/Pipe.cpp | 90 ++++++++----------- src/Processors/Pipe.h | 32 ++++--- src/Processors/QueryPipeline.cpp | 8 +- src/Processors/QueryPipeline.h | 3 + .../QueryPlan/ReadFromStorageStep.cpp | 2 - src/Processors/QueryPlan/UnionStep.cpp | 2 +- src/Storages/LiveView/StorageBlocks.h | 4 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 11 +-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 4 +- src/Storages/RabbitMQ/StorageRabbitMQ.h | 2 +- src/Storages/StorageBuffer.cpp | 17 ++-- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageGenerateRandom.cpp | 2 +- src/Storages/StorageHDFS.cpp | 2 +- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMerge.cpp | 24 ++--- src/Storages/StorageMerge.h | 2 +- src/Storages/StorageStripeLog.cpp | 4 +- src/Storages/StorageView.cpp | 2 +- src/Storages/System/IStorageSystemOneBlock.h | 7 +- src/Storages/System/StorageSystemColumns.cpp | 8 +- src/Storages/System/StorageSystemColumns.h | 2 +- .../System/StorageSystemDetachedParts.cpp | 6 +- src/Storages/System/StorageSystemDisks.cpp | 7 +- src/Storages/System/StorageSystemDisks.h | 2 +- src/Storages/System/StorageSystemNumbers.cpp | 33 +++---- src/Storages/System/StorageSystemNumbers.h | 2 +- src/Storages/System/StorageSystemOne.cpp | 7 +- src/Storages/System/StorageSystemOne.h | 2 +- .../System/StorageSystemPartsBase.cpp | 7 +- src/Storages/System/StorageSystemPartsBase.h | 2 +- src/Storages/System/StorageSystemReplicas.cpp | 8 +- src/Storages/System/StorageSystemReplicas.h | 2 +- .../System/StorageSystemStoragePolicies.cpp | 7 +- .../System/StorageSystemStoragePolicies.h | 2 +- src/Storages/System/StorageSystemTables.cpp | 7 +- src/Storages/System/StorageSystemTables.h | 2 +- src/Storages/System/StorageSystemZeros.cpp | 7 +- src/Storages/System/StorageSystemZeros.h | 2 +- 50 files changed, 209 insertions(+), 211 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 02e107db9a6..e93e84c5e94 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include @@ -581,10 +582,13 @@ void Connection::sendExternalTablesData(ExternalTablesData & data) PipelineExecutorPtr executor; auto on_cancel = [& executor]() { executor->cancel(); }; - auto sink = std::make_shared(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(elem->pipe->getHeader(), elem->pipe->numOutputPorts())); - auto processors = std::move(*elem->pipe).detachProcessors(); + auto sink = std::make_shared(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(processors); diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 3639a109b42..a6e1407c1f2 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -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(std::move(output)); - connect(data->pipe->getPort(), sink->getPort()); + if (data->pipe->numOutputPorts() > 1) + data->pipe->addTransform(std::make_shared(data->pipe->getHeader(), data->pipe->numOutputPorts())); - auto processors = std::move(*data->pipe).detachProcessors(); + auto sink = std::make_shared(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(processors); diff --git a/src/DataStreams/BlockIO.cpp b/src/DataStreams/BlockIO.cpp index 150995962bf..3bbef4471db 100644 --- a/src/DataStreams/BlockIO.cpp +++ b/src/DataStreams/BlockIO.cpp @@ -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? diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index be09cd94814..5de396ec967 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -2,10 +2,8 @@ #include #include -#include #include #include -#include #include #include #include @@ -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(); data->table_name = table.first; - if (pipes.empty()) + if (pipe.empty()) data->pipe = std::make_unique( std::make_shared(metadata_snapshot->getSampleBlock(), Chunk())); - else if (pipes.size() == 1) - data->pipe = std::make_unique(std::move(pipes.front())); else - { - auto concat = std::make_shared(pipes.front().getHeader(), pipes.size()); - data->pipe = std::make_unique(std::move(pipes), std::move(concat)); - } + data->pipe = std::make_unique(std::move(pipe)); res.emplace_back(std::move(data)); } diff --git a/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp b/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp index c0d1a4979e1..0ce450c4e6c 100644 --- a/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp @@ -31,7 +31,7 @@ static Block getBlockWithSize(const std::vector & columns, size_t r } -static Pipes getInputStreams(const std::vector & column_names, const std::vector> & block_sizes) +static Pipe getInputStreams(const std::vector & column_names, const std::vector> & 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 & column_names, cons blocks.push_back(getBlockWithSize(column_names, block_size_in_bytes, stride, start)); pipes.emplace_back(std::make_shared(std::make_shared(std::move(blocks)))); } - return pipes; + return Pipe::unitePipes(std::move(pipes)); } -static Pipes getInputStreamsEqualStride(const std::vector & column_names, const std::vector> & block_sizes) +static Pipe getInputStreamsEqualStride(const std::vector & column_names, const std::vector> & block_sizes) { Pipes pipes; size_t i = 0; @@ -60,7 +60,7 @@ static Pipes getInputStreamsEqualStride(const std::vector & column_ pipes.emplace_back(std::make_shared(std::make_shared(std::move(blocks)))); i++; } - return pipes; + return Pipe::unitePipes(std::move(pipes)); } @@ -79,15 +79,17 @@ TEST(MergingSortedTest, SimpleBlockSizeTest) { std::vector 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(pipes.front().getHeader(), pipes.size(), sort_description, + auto transform = std::make_shared(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(std::move(pipeline)); @@ -122,15 +124,17 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes) { std::vector 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(pipes.front().getHeader(), pipes.size(), sort_description, + auto transform = std::make_shared(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(std::move(pipeline)); diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 272ad7876d9..986de85d712 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -130,7 +130,7 @@ void SelectStreamFactory::createForShard( auto emplace_local_stream = [&]() { - pipes.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); @@ -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 connections; diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 6f1215d6af0..c0ed5d8a36c 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -89,7 +89,7 @@ BlockIO InterpreterAlterQuery::execute() table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, context.getSettingsRef()); 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_pipes)); + res.pipeline.init(std::move(partition_commands_pipe)); } if (!live_view_commands.empty()) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 4fed6f02d5f..9960509a5d7 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -269,7 +269,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() if (settings.graph) { - auto processors = Pipe::detachProcessors(std::move(*pipeline).getPipe()); + auto processors = Pipe::detachProcessors(QueryPipeline::getPipe(std::move(*pipeline))); if (settings.compact) printPipelineCompact(processors, buffer, settings.query_pipeline_options.header); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 36b106d5d68..31a623e82fd 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -190,7 +190,7 @@ BlockIO InterpreterInsertQuery::execute() } } - res.pipeline.unitePipelines(std::move(pipelines), {}); + res.pipeline = QueryPipeline::unitePipelines(std::move(pipelines), {}); } } diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index 7a0c7184882..e6749827efe 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -11,41 +11,12 @@ 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()) @@ -125,6 +96,16 @@ static OutputPort * uniteTotals(const OutputPortRawPtrs & ports, const Block & h 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()) @@ -190,7 +171,7 @@ Pipe::Pipe(ProcessorPtr source) checkSource(*source); if (collected_processors) - collected_processors->emplace_back(source.get()); + collected_processors->emplace_back(source); output_ports.push_back(&source->getOutputs().front()); header = output_ports.front()->getHeader(); @@ -248,7 +229,7 @@ Pipe::Pipe(Processors processors_) : processors(std::move(processors_)) if (collected_processors) for (const auto & processor : processors) - collected_processors->emplace_back(processor.get()); + collected_processors->emplace_back(processor); } static Pipes removeEmptyPipes(Pipes pipes) @@ -272,15 +253,22 @@ Pipe Pipe::unitePipes(Pipes pipes) 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 {}; + return res; if (pipes.size() == 1) + { + pipes[0].holder = std::move(res.holder); return std::move(pipes[0]); + } - Pipe res; OutputPortRawPtrs totals; OutputPortRawPtrs extremes; res.header = pipes.front().header; @@ -291,10 +279,6 @@ Pipe Pipe::unitePipes(Pipes pipes, Processors * collected_processors) 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.table_locks.insert(res.table_locks.end(), pipe.table_locks.begin(), pipe.table_locks.end()); - res.storage_holders.insert(res.storage_holders.end(), pipe.storage_holders.begin(), pipe.storage_holders.end()); - res.interpreter_context.insert(res.interpreter_context.end(), - pipe.interpreter_context.begin(), pipe.interpreter_context.end()); res.max_parallel_streams += pipe.max_parallel_streams; @@ -315,12 +299,14 @@ Pipe Pipe::unitePipes(Pipes pipes, Processors * 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 = output_ports.front()->getHeader(); + const auto & source_header = source->getOutputs().front().getHeader(); if (output_ports.empty()) header = source_header; @@ -328,7 +314,7 @@ void Pipe::addSource(ProcessorPtr source) assertBlocksHaveEqualStructure(header, source_header, "Pipes"); if (collected_processors) - collected_processors->emplace_back(source.get()); + collected_processors->emplace_back(source); output_ports.push_back(&source->getOutputs().front()); processors.emplace_back(std::move(source)); @@ -350,7 +336,7 @@ void Pipe::addTotalsSource(ProcessorPtr source) assertBlocksHaveEqualStructure(header, source_header, "Pipes"); if (collected_processors) - collected_processors->emplace_back(source.get()); + collected_processors->emplace_back(source); totals_port = &source->getOutputs().front(); processors.emplace_back(std::move(source)); @@ -370,7 +356,7 @@ void Pipe::addExtremesSource(ProcessorPtr source) assertBlocksHaveEqualStructure(header, source_header, "Pipes"); if (collected_processors) - collected_processors->emplace_back(source.get()); + collected_processors->emplace_back(source); extremes_port = &source->getOutputs().front(); processors.emplace_back(std::move(source)); @@ -385,7 +371,7 @@ static void dropPort(OutputPort *& port, Processors & processors, Processors * c connect(*port, null_sink->getPort()); if (collected_processors) - collected_processors->emplace_back(null_sink.get()); + collected_processors->emplace_back(null_sink); processors.emplace_back(std::move(null_sink)); port = nullptr; @@ -471,14 +457,15 @@ void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort for (size_t i = 1; i < output_ports.size(); ++i) assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipes"); - if (totals_port) - assertBlocksHaveEqualStructure(header, totals_port->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.get()); + collected_processors->emplace_back(transform); processors.emplace_back(std::move(transform)); @@ -526,7 +513,7 @@ void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter) port = &transform->getOutputs().front(); if (collected_processors) - collected_processors->emplace_back(transform.get()); + collected_processors->emplace_back(transform); processors.emplace_back(std::move(transform)); } @@ -614,7 +601,7 @@ void Pipe::setOutputFormat(ProcessorPtr output) addExtremesSource(std::make_shared(extremes.getHeader())); if (collected_processors) - collected_processors->emplace_back(output.get()); + collected_processors->emplace_back(output); processors.emplace_back(std::move(output)); @@ -647,7 +634,8 @@ void Pipe::transform(const Transformer & transformer) set.emplace(&port->getProcessor()); } - OutputPortRawPtrs new_output_ports; + output_ports.clear(); + for (const auto & processor : new_processors) { for (const auto & port : processor->getInputs()) @@ -667,7 +655,7 @@ void Pipe::transform(const Transformer & transformer) { if (!port.isConnected()) { - new_output_ports.push_back(&port); + output_ports.push_back(&port); continue; } @@ -696,7 +684,7 @@ void Pipe::transform(const Transformer & transformer) if (collected_processors) { for (const auto & processor : processors) - collected_processors->emplace_back(processor.get()); + collected_processors->emplace_back(processor); } processors.insert(processors.end(), new_processors.begin(), new_processors.end()); diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 787bade065a..6795d3b58b4 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -73,8 +73,8 @@ public: using ProcessorGetterWithStreamKind = std::function; /// Add transform with single input and single output for each port. - void addSimpleTransform(const ProcessorGetter & port); - void addSimpleTransform(const ProcessorGetterWithStreamKind & port); + void addSimpleTransform(const ProcessorGetter & getter); + void addSimpleTransform(const ProcessorGetterWithStreamKind & getter); using Transformer = std::function; @@ -92,20 +92,30 @@ public: void setQuota(const std::shared_ptr & quota); /// Do not allow to change the table while the processors of pipe are alive. - 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) { interpreter_context.emplace_back(std::move(context)); } - void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); } + void addInterpreterContext(std::shared_ptr context) { holder.interpreter_context.emplace_back(std::move(context)); } + void addStorageHolder(StoragePtr storage) { holder.storage_holders.emplace_back(std::move(storage)); } private: /// Destruction order: processors, header, locks, temporary storages, local contexts - /// 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> interpreter_context; - std::vector storage_holders; - std::vector table_locks; + 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> interpreter_context; + std::vector storage_holders; + std::vector table_locks; + }; + + Holder holder; /// Header is common for all output below. Block header; diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 97e96387f7f..751f93ad694 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -66,12 +66,18 @@ void QueryPipeline::init(Pipe pipe_) if (initialized()) throw Exception("Pipeline has already been initialized.", ErrorCodes::LOGICAL_ERROR); - if (pipe.empty()) + if (pipe_.empty()) throw Exception("Can't initialize pipeline with empty pipe.", ErrorCodes::LOGICAL_ERROR); pipe = std::move(pipe_); } +void QueryPipeline::reset() +{ + Pipe pipe_to_destroy(std::move(pipe)); + *this = QueryPipeline(); +} + void QueryPipeline::addSimpleTransform(const Pipe::ProcessorGetter & getter) { checkInitializedAndNotCompleted(); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 74cc692ac42..385cf77198e 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -33,6 +33,9 @@ public: /// All pipes must have same header. void init(Pipe pipe); + /// Clear and release all resources. + void reset(); + bool initialized() { return !pipe.empty(); } bool isCompleted() { return pipe.isCompleted(); } diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.cpp b/src/Processors/QueryPlan/ReadFromStorageStep.cpp index 39b0743fe38..3af011d1333 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromStorageStep.cpp @@ -120,8 +120,6 @@ ReadFromStorageStep::ReadFromStorageStep( pipe.setQuota(quota); } - pipe.enableQuota(); - pipeline->init(std::move(pipe)); pipeline->addInterpreterContext(std::move(context)); diff --git a/src/Processors/QueryPlan/UnionStep.cpp b/src/Processors/QueryPlan/UnionStep.cpp index b645eb3f3d7..1e74046b071 100644 --- a/src/Processors/QueryPlan/UnionStep.cpp +++ b/src/Processors/QueryPlan/UnionStep.cpp @@ -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; diff --git a/src/Storages/LiveView/StorageBlocks.h b/src/Storages/LiveView/StorageBlocks.h index 56fd0c620c2..37861b55568 100644 --- a/src/Storages/LiveView/StorageBlocks.h +++ b/src/Storages/LiveView/StorageBlocks.h @@ -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: diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index f2049a9c248..202603b7481 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -26,14 +26,12 @@ #include #include #include -#include #include #include #include #include #include -#include #include #include @@ -730,8 +728,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (metadata_snapshot->hasSortingKey()) { - auto expr = std::make_shared(pipe.getHeader(), metadata_snapshot->getSortingKey().expression); - pipe.addSimpleTransform(std::move(expr)); + pipe.addSimpleTransform([&metadata_snapshot](const Block & header) + { + return std::make_shared(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(std::move(pipeline)); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index b2d00be8620..6109c7ace6d 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1122,7 +1122,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( auto pipe = Pipe::unitePipes(std::move(pipes)); - if (input_order_info->direction == 1) + if (input_order_info->direction != 1) { pipe.addSimpleTransform([](const Block & header) { @@ -1290,9 +1290,9 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( key_columns.emplace_back(desc.column_number); } - pipe.addSimpleTransform([&](const Block & header) + pipe.addSimpleTransform([&](const Block & stream_header) { - return std::make_shared(header, num_streams, key_columns); + return std::make_shared(stream_header, num_streams, key_columns); }); pipe.transform([&](OutputPortRawPtrs ports) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index a3d16087e34..6052837b2cc 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -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)); } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index e1c8b33c91e..064af15c16c 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -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, diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 4a728841c50..5b9957f4ed4 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -221,10 +221,10 @@ Pipe StorageBuffer::read( columns_intersection, destination_metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - pipe_from_dst.addSimpleTransform([&](const Block & header) + pipe_from_dst.addSimpleTransform([&](const Block & stream_header) { - return std::make_shared( - header, header_after_adding_defaults, metadata_snapshot->getColumns().getDefaults(), context); + return std::make_shared(stream_header, header_after_adding_defaults, + metadata_snapshot->getColumns().getDefaults(), context); }); pipe_from_dst.addSimpleTransform([&](const Block & stream_header) @@ -252,7 +252,7 @@ Pipe StorageBuffer::read( if (!pipe_from_buffers.empty() && !pipe_from_dst.empty() && !blocksHaveEqualStructure(pipe_from_buffers.getHeader(), pipe_from_dst.getHeader())) { - pipe_from_buffers.addSimpleTransform([&](const Block & header) + pipe_from_dst.addSimpleTransform([&](const Block & header) { return std::make_shared( header, @@ -265,7 +265,9 @@ Pipe StorageBuffer::read( * then sources from the buffers must also be wrapped in the processing pipeline before the same stage. */ if (processed_stage > QueryProcessingStage::FetchColumns) - pipe_from_buffers = InterpreterSelectQuery(query_info.query, context, std::move(pipe_from_buffers), 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) { @@ -285,7 +287,10 @@ Pipe StorageBuffer::read( } } - return Pipe::unitePipes({std::move(pipe_from_dst), std::move(pipe_from_buffers)}); + Pipes pipes; + pipes.emplace_back(std::move(pipe_from_dst)); + pipes.emplace_back(std::move(pipe_from_buffers)); + return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 471a9f522ab..558216a6216 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -419,7 +419,7 @@ Pipe StorageFile::read( pipes.emplace_back(std::make_shared( this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults())); - return Pipe::unitePipes(pipes); + return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 8020c54d8cd..62d8259f705 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -467,7 +467,7 @@ Pipe StorageGenerateRandom::read( for (UInt64 i = 0; i < num_streams; ++i) pipes.emplace_back(std::make_shared(max_block_size, max_array_length, max_string_length, generate(), block_header, context)); - return Pipe::unitePipes(pipes); + return Pipe::unitePipes(std::move(pipes)); } } diff --git a/src/Storages/StorageHDFS.cpp b/src/Storages/StorageHDFS.cpp index 98106850492..65b1eed7793 100644 --- a/src/Storages/StorageHDFS.cpp +++ b/src/Storages/StorageHDFS.cpp @@ -298,7 +298,7 @@ Pipe StorageHDFS::read( pipes.emplace_back(std::make_shared( sources_info, uri_without_path, format_name, compression_method, metadata_snapshot->getSampleBlock(), context_, max_block_size)); - return Pipe::unitePipes(pipes); + return Pipe::unitePipes(std::move(pipes)); } BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 50b31a1f128..042f3abff4e 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -647,7 +647,7 @@ Pipe StorageLog::read( max_read_buffer_size)); } - return Pipe::unitePipes(pipes); + return Pipe::unitePipes(std::move(pipes)); } BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 1c690221967..e9881fe4804 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -132,7 +132,7 @@ Pipe StorageMemory::read( pipes.emplace_back(std::make_shared(column_names, begin, end, *this, metadata_snapshot)); } - return Pipe::unitePipes(pipes); + return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 15e9fd65448..a98d789a048 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -136,7 +136,7 @@ Pipe StorageMerge::read( const size_t max_block_size, unsigned num_streams) { - Pipe pipe; + Pipes pipes; bool has_table_virtual_column = false; Names real_column_names; @@ -210,12 +210,16 @@ Pipe StorageMerge::read( auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); - pipe = 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); + + pipes.emplace_back(std::move(source_pipe)); } + auto pipe = Pipe::unitePipes(std::move(pipes)); + if (!pipe.empty()) narrowPipe(pipe, num_streams); @@ -245,10 +249,10 @@ Pipe StorageMerge::createSources( if (!storage) { - pipe = InterpreterSelectQuery( + pipe = QueryPipeline::getPipe(InterpreterSelectQuery( modified_query_info.query, *modified_context, std::make_shared(header), - SelectQueryOptions(processed_stage).analyze()).execute().pipeline.getPipe(); + SelectQueryOptions(processed_stage).analyze()).execute().pipeline); pipe.addInterpreterContext(modified_context); return pipe; @@ -275,13 +279,13 @@ Pipe StorageMerge::createSources( InterpreterSelectQuery interpreter{modified_query_info.query, *modified_context, SelectQueryOptions(processed_stage)}; - pipe = interpreter.execute().pipeline.getPipe(); + 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. */ - pipe.addSimpleTransform([](const Block & header) { return std::make_shared(header); }); + pipe.addSimpleTransform([](const Block & stream_header) { return std::make_shared(stream_header); }); } if (!pipe.empty()) @@ -291,10 +295,10 @@ Pipe StorageMerge::createSources( if (has_table_virtual_column) { - pipe.addSimpleTransform([name = table_name](const Block & header) + pipe.addSimpleTransform([name = table_name](const Block & stream_header) { return std::make_shared>( - header, std::make_shared(), name, "_table"); + stream_header, std::make_shared(), name, "_table"); }); } @@ -440,9 +444,9 @@ void StorageMerge::convertingSourceStream( QueryProcessingStage::Enum processed_stage) { Block before_block_header = pipe.getHeader(); - pipe.addSimpleTransform([&before_block_header](const Block & header) + pipe.addSimpleTransform([&](const Block & stream_header) { - return std::make_shared(before_block_header, header, ConvertingTransform::MatchColumnsMode::Name); + return std::make_shared(stream_header, header, ConvertingTransform::MatchColumnsMode::Name); }); auto where_expression = query->as()->where(); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 4fa2eaed27d..5e2d56d18c0 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -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, diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 66d7c79e434..e4ba104cffc 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -310,7 +310,7 @@ Pipe StorageStripeLog::read( String index_file = table_path + "index.mrk"; if (!disk->exists(index_file)) { - Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); + return Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); } CompressedReadBufferFromFile index_in(disk->readFile(index_file, INDEX_BUFFER_SIZE)); @@ -334,7 +334,7 @@ Pipe 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 Pipe::unitePipes(pipes); + return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 2c68be4fdb1..1a95b7ea21f 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -86,7 +86,7 @@ Pipe StorageView::read( column_names, getVirtuals(), getStorageID()), ConvertingTransform::MatchColumnsMode::Name); }); - return std::move(pipeline).getPipe(); + return QueryPipeline::getPipe(std::move(pipeline)); } static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query) diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 7c2ef85f158..b30d6476b22 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -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(sample_block, std::move(chunk))); - - return pipes; + return Pipe(std::make_shared(sample_block, std::move(chunk))); } }; diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 57d8d13e79b..18cc46c0c1b 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -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(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(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)); } } diff --git a/src/Storages/System/StorageSystemColumns.h b/src/Storages/System/StorageSystemColumns.h index 7336b406183..480afe5c36c 100644 --- a/src/Storages/System/StorageSystemColumns.h +++ b/src/Storages/System/StorageSystemColumns.h @@ -17,7 +17,7 @@ class StorageSystemColumns final : public ext::shared_ptr_helpersize(); Chunk chunk(std::move(new_columns), num_rows); - Pipes pipes; - pipes.emplace_back(std::make_shared(std::move(block), std::move(chunk))); - return pipes; + return Pipe(std::make_shared(std::move(block), std::move(chunk))); } }; diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index 554b8cfd1eb..b54f3510eb6 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -27,7 +27,7 @@ StorageSystemDisks::StorageSystemDisks(const std::string & name_) setInMemoryMetadata(storage_metadata); } -Pipes StorageSystemDisks::read( +Pipe StorageSystemDisks::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, @@ -66,10 +66,7 @@ Pipes StorageSystemDisks::read( UInt64 num_rows = res_columns.at(0)->size(); Chunk chunk(std::move(res_columns), num_rows); - Pipes pipes; - pipes.emplace_back(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); - - return pipes; + return Pipe(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); } } diff --git a/src/Storages/System/StorageSystemDisks.h b/src/Storages/System/StorageSystemDisks.h index 714a0a5428c..e2fb0e03705 100644 --- a/src/Storages/System/StorageSystemDisks.h +++ b/src/Storages/System/StorageSystemDisks.h @@ -20,7 +20,7 @@ class StorageSystemDisks final : public ext::shared_ptr_helper #include #include -#include #include #include @@ -123,7 +122,7 @@ StorageSystemNumbers::StorageSystemNumbers(const StorageID & table_id, bool mult setInMemoryMetadata(storage_metadata); } -Pipes StorageSystemNumbers::read( +Pipe StorageSystemNumbers::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo &, @@ -143,8 +142,7 @@ Pipes StorageSystemNumbers::read( if (!multithreaded) num_streams = 1; - Pipes res; - res.reserve(num_streams); + Pipe pipe; if (num_streams > 1 && !even_distribution && *limit) { @@ -152,9 +150,9 @@ Pipes StorageSystemNumbers::read( UInt64 max_counter = offset + *limit; for (size_t i = 0; i < num_streams; ++i) - res.emplace_back(std::make_shared(state, max_block_size, max_counter)); + pipe.addSource(std::make_shared(state, max_block_size, max_counter)); - return res; + return pipe; } for (size_t i = 0; i < num_streams; ++i) @@ -164,17 +162,22 @@ Pipes StorageSystemNumbers::read( if (limit && i == 0) source->addTotalRowsApprox(*limit); - res.emplace_back(std::move(source)); - - if (limit) - { - /// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly. - res.back().addSimpleTransform(std::make_shared( - res.back().getHeader(), *limit * (i + 1) / num_streams - *limit * i / num_streams, 0)); - } + pipe.addSource(std::move(source)); } - return res; + if (limit) + { + size_t i = 0; + /// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly. + pipe.addSimpleTransform([&](const Block & header) + { + ++i; + return std::make_shared( + header, *limit * i / num_streams - *limit * (i - 1) / num_streams, 0); + }); + } + + return pipe; } } diff --git a/src/Storages/System/StorageSystemNumbers.h b/src/Storages/System/StorageSystemNumbers.h index 88d3651c7f9..f907f3d5f93 100644 --- a/src/Storages/System/StorageSystemNumbers.h +++ b/src/Storages/System/StorageSystemNumbers.h @@ -29,7 +29,7 @@ class StorageSystemNumbers final : public ext::shared_ptr_helperconvertToFullColumnIfConst(); Chunk chunk({ std::move(column) }, 1); - Pipes pipes; - pipes.emplace_back(std::make_shared(std::move(header), std::move(chunk))); - - return pipes; + return Pipe(std::make_shared(std::move(header), std::move(chunk))); } diff --git a/src/Storages/System/StorageSystemOne.h b/src/Storages/System/StorageSystemOne.h index 3b3e531dc86..09db9822914 100644 --- a/src/Storages/System/StorageSystemOne.h +++ b/src/Storages/System/StorageSystemOne.h @@ -21,7 +21,7 @@ class StorageSystemOne final : public ext::shared_ptr_helper, public: std::string getName() const override { return "SystemOne"; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 168b305605d..b6e94993468 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -223,7 +223,7 @@ StoragesInfo StoragesInfoStream::next() return {}; } -Pipes StorageSystemPartsBase::read( +Pipe StorageSystemPartsBase::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -254,10 +254,7 @@ Pipes StorageSystemPartsBase::read( UInt64 num_rows = res_columns.at(0)->size(); Chunk chunk(std::move(res_columns), num_rows); - Pipes pipes; - pipes.emplace_back(std::make_shared(std::move(header), std::move(chunk))); - - return pipes; + return Pipe(std::make_shared(std::move(header), std::move(chunk))); } diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 56c9a8fb0d0..040a03770ac 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -55,7 +55,7 @@ private: class StorageSystemPartsBase : public IStorage { public: - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 27a9cd0c4bb..f049b69a7ff 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -57,7 +57,7 @@ StorageSystemReplicas::StorageSystemReplicas(const std::string & name_) } -Pipes StorageSystemReplicas::read( +Pipe StorageSystemReplicas::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -139,7 +139,7 @@ Pipes StorageSystemReplicas::read( VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); if (!filtered_block.rows()) - return Pipes(); + return {}; col_database = filtered_block.getByName("database").column; col_table = filtered_block.getByName("table").column; @@ -202,9 +202,7 @@ Pipes StorageSystemReplicas::read( UInt64 num_rows = fin_columns.at(0)->size(); Chunk chunk(std::move(fin_columns), num_rows); - Pipes pipes; - pipes.emplace_back(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); - return pipes; + return Pipe(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); } diff --git a/src/Storages/System/StorageSystemReplicas.h b/src/Storages/System/StorageSystemReplicas.h index b068ebc8b0a..bc830721a23 100644 --- a/src/Storages/System/StorageSystemReplicas.h +++ b/src/Storages/System/StorageSystemReplicas.h @@ -18,7 +18,7 @@ class StorageSystemReplicas final : public ext::shared_ptr_helpersize(); Chunk chunk(std::move(res_columns), num_rows); - Pipes pipes; - pipes.emplace_back(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); - - return pipes; + return Pipe(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); } } diff --git a/src/Storages/System/StorageSystemStoragePolicies.h b/src/Storages/System/StorageSystemStoragePolicies.h index a1427da8559..73fda1e8c46 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.h +++ b/src/Storages/System/StorageSystemStoragePolicies.h @@ -20,7 +20,7 @@ class StorageSystemStoragePolicies final : public ext::shared_ptr_helper( + return Pipe(std::make_shared( std::move(columns_mask), std::move(res_block), max_block_size, std::move(filtered_databases_column), context)); - - return pipes; } } diff --git a/src/Storages/System/StorageSystemTables.h b/src/Storages/System/StorageSystemTables.h index 54551205684..766c3046dbc 100644 --- a/src/Storages/System/StorageSystemTables.h +++ b/src/Storages/System/StorageSystemTables.h @@ -18,7 +18,7 @@ class StorageSystemTables final : public ext::shared_ptr_helperaddTotalRowsApprox(*limit); - res.emplace_back(std::move(source)); + res.addSource(std::move(source)); } return res; diff --git a/src/Storages/System/StorageSystemZeros.h b/src/Storages/System/StorageSystemZeros.h index f169861122a..41de3ce6246 100644 --- a/src/Storages/System/StorageSystemZeros.h +++ b/src/Storages/System/StorageSystemZeros.h @@ -20,7 +20,7 @@ class StorageSystemZeros final : public ext::shared_ptr_helper Date: Fri, 7 Aug 2020 11:28:12 +0300 Subject: [PATCH 8/9] Push ci --- src/Processors/QueryPipeline.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 751f93ad694..41ab4ad392d 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -415,6 +415,7 @@ void QueryPipeline::setCollectedProcessors(Processors * processors) pipe.collected_processors = processors; } + QueryPipelineProcessorsCollector::QueryPipelineProcessorsCollector(QueryPipeline & pipeline_, IQueryPlanStep * step_) : pipeline(pipeline_), step(step_) { From 3b05c315e4be46a1043e497f3781b27dbae55a1e Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 14 Aug 2020 04:10:10 +0300 Subject: [PATCH 9/9] Update Pipe.h --- src/Processors/Pipe.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 6795d3b58b4..28b64937aeb 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -13,8 +13,8 @@ using StoragePtr = std::shared_ptr; using OutputPortRawPtrs = std::vector; -/// Pipes is a set of processors which represents the part of pipeline. -/// Pipes contains a list of output ports, with specified port for totals and specified port for extremes. +/// 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