2019-11-05 17:33:03 +00:00
|
|
|
#pragma once
|
2019-10-20 09:12:42 +00:00
|
|
|
#include <Processors/IProcessor.h>
|
2019-11-05 17:33:03 +00:00
|
|
|
#include <Processors/Sources/SourceWithProgress.h>
|
2019-10-20 09:12:42 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
class Pipe;
|
2020-08-03 11:33:11 +00:00
|
|
|
using Pipes = std::vector<Pipe>;
|
2019-10-20 09:12:42 +00:00
|
|
|
|
2020-01-29 18:18:12 +00:00
|
|
|
class IStorage;
|
|
|
|
using StoragePtr = std::shared_ptr<IStorage>;
|
|
|
|
|
2020-08-03 11:33:11 +00:00
|
|
|
using OutputPortRawPtrs = std::vector<OutputPort *>;
|
|
|
|
|
2020-07-31 16:54:54 +00:00
|
|
|
/// 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.
|
2020-08-03 11:33:11 +00:00
|
|
|
class Pipe
|
2020-07-31 16:54:54 +00:00
|
|
|
{
|
|
|
|
public:
|
2020-08-03 11:33:11 +00:00
|
|
|
/// 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;
|
2020-07-31 16:54:54 +00:00
|
|
|
/// Create from source. Source must have no input ports and single output.
|
2020-08-03 11:33:11 +00:00
|
|
|
explicit Pipe(ProcessorPtr source);
|
2020-07-31 16:54:54 +00:00
|
|
|
/// Create from processors. Use all not-connected output ports as output_ports. Check invariants.
|
2020-08-03 11:33:11 +00:00
|
|
|
explicit Pipe(Processors processors_);
|
2020-07-31 16:54:54 +00:00
|
|
|
|
2020-08-03 11:33:11 +00:00
|
|
|
Pipe(const Pipe & other) = delete;
|
|
|
|
Pipe(Pipe && other) = default;
|
|
|
|
Pipe & operator=(const Pipe & other) = delete;
|
|
|
|
Pipe & operator=(Pipe && other) = default;
|
2020-07-31 16:54:54 +00:00
|
|
|
|
|
|
|
const Block & getHeader() const { return header; }
|
|
|
|
bool empty() const { return output_ports.empty(); }
|
2020-08-03 11:33:11 +00:00
|
|
|
size_t numOutputPorts() const { return output_ports.size(); }
|
2020-07-31 16:54:54 +00:00
|
|
|
OutputPort * getOutputPort(size_t pos) const { return output_ports[pos]; }
|
|
|
|
OutputPort * getTotalsPort() const { return totals_port; }
|
|
|
|
OutputPort * getExtremesPort() const { return extremes_port; }
|
|
|
|
|
2020-08-03 11:33:11 +00:00
|
|
|
/// Add processors form other pipe. It should have same header.
|
|
|
|
//void addPipes(Pipes pipes);
|
|
|
|
|
2020-07-31 16:54:54 +00:00
|
|
|
/// 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);
|
|
|
|
|
2020-08-03 11:33:11 +00:00
|
|
|
/// Add totals and extremes.
|
|
|
|
void addTotalsSource(ProcessorPtr source);
|
|
|
|
void addExtremesSource(ProcessorPtr source);
|
|
|
|
|
2020-07-31 16:54:54 +00:00
|
|
|
/// 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.
|
|
|
|
};
|
|
|
|
|
2020-08-03 13:54:14 +00:00
|
|
|
using ProcessorGetter = std::function<ProcessorPtr(const Block & header)>;
|
|
|
|
using ProcessorGetterWithStreamKind = std::function<ProcessorPtr(const Block & header, StreamType stream_type)>;
|
2020-07-31 16:54:54 +00:00
|
|
|
|
|
|
|
/// Add transform with single input and single output for each port.
|
|
|
|
void addSimpleTransform(const ProcessorGetter & port);
|
2020-08-03 13:54:14 +00:00
|
|
|
void addSimpleTransform(const ProcessorGetterWithStreamKind & port);
|
2020-07-31 16:54:54 +00:00
|
|
|
|
2020-08-03 11:33:11 +00:00
|
|
|
using Transformer = std::function<Processors(OutputPortRawPtrs ports)>;
|
|
|
|
|
|
|
|
/// Transform Pipe in general way.
|
|
|
|
void transform(const Transformer & transformer);
|
|
|
|
|
|
|
|
/// Unite several pipes together. They should have same header.
|
|
|
|
static Pipe unitePipes(Pipes pipes);
|
2020-07-31 16:54:54 +00:00
|
|
|
|
2020-08-03 13:54:14 +00:00
|
|
|
/// 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> context) { interpreter_context.emplace_back(std::move(context)); }
|
|
|
|
void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); }
|
|
|
|
|
2020-07-31 16:54:54 +00:00
|
|
|
private:
|
|
|
|
Processors processors;
|
|
|
|
|
|
|
|
/// Header is common for all output below.
|
|
|
|
Block header;
|
|
|
|
|
|
|
|
/// Output ports. Totals and extremes are allowed to be empty.
|
2020-08-03 11:33:11 +00:00
|
|
|
OutputPortRawPtrs output_ports;
|
2020-07-31 16:54:54 +00:00
|
|
|
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<TableLockHolder> 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<std::shared_ptr<Context>> interpreter_context;
|
|
|
|
std::vector<StoragePtr> storage_holders;
|
|
|
|
|
2020-08-03 11:33:11 +00:00
|
|
|
/// Destroy pipes and get processors.
|
|
|
|
static Processors detachProcessors(Pipe pipe) { return std::move(pipe.processors); }
|
|
|
|
};
|
2020-07-31 16:54:54 +00:00
|
|
|
|
2020-08-03 11:33:11 +00:00
|
|
|
/*
|
2019-10-20 09:12:42 +00:00
|
|
|
/// 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:
|
2019-10-23 03:45:43 +00:00
|
|
|
/// Create from source. It must have no input ports and single output.
|
2019-10-20 09:12:42 +00:00
|
|
|
explicit Pipe(ProcessorPtr source);
|
2019-10-23 03:45:43 +00:00
|
|
|
/// 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.
|
2019-10-20 09:12:42 +00:00
|
|
|
Pipe(Pipes && pipes, ProcessorPtr transform);
|
2020-04-22 13:52:07 +00:00
|
|
|
/// Create pipe from output port. If pipe was created that way, it possibly will not have tree shape.
|
2020-05-26 13:19:24 +00:00
|
|
|
explicit Pipe(OutputPort * port);
|
2019-10-20 09:12:42 +00:00
|
|
|
|
|
|
|
Pipe(const Pipe & other) = delete;
|
|
|
|
Pipe(Pipe && other) = default;
|
|
|
|
|
|
|
|
Pipe & operator=(const Pipe & other) = delete;
|
|
|
|
Pipe & operator=(Pipe && other) = default;
|
|
|
|
|
2020-04-22 13:52:07 +00:00
|
|
|
/// Append processors to pipe. After this, it possibly will not have tree shape.
|
|
|
|
void addProcessors(const Processors & processors_);
|
|
|
|
|
2019-10-20 09:12:42 +00:00
|
|
|
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); }
|
|
|
|
|
2019-11-05 17:33:03 +00:00
|
|
|
/// Specify quotas and limits for every ISourceWithProgress.
|
|
|
|
void setLimits(const SourceWithProgress::LocalLimits & limits);
|
2020-03-07 17:37:38 +00:00
|
|
|
void setQuota(const std::shared_ptr<const EnabledQuota> & quota);
|
2019-11-05 17:33:03 +00:00
|
|
|
|
|
|
|
/// Set information about preferred executor number for sources.
|
|
|
|
void pinSources(size_t executor_number);
|
|
|
|
|
2019-12-26 16:15:31 +00:00
|
|
|
void enableQuota();
|
|
|
|
|
2020-04-08 12:40:04 +00:00
|
|
|
/// Totals and extremes port.
|
2019-11-05 17:33:03 +00:00
|
|
|
void setTotalsPort(OutputPort * totals_) { totals = totals_; }
|
2020-04-08 12:40:04 +00:00
|
|
|
void setExtremesPort(OutputPort * extremes_) { extremes = extremes_; }
|
2019-11-05 17:33:03 +00:00
|
|
|
OutputPort * getTotalsPort() const { return totals; }
|
2020-04-08 12:40:04 +00:00
|
|
|
OutputPort * getExtremesPort() const { return extremes; }
|
2019-11-05 17:33:03 +00:00
|
|
|
|
2020-03-31 14:30:43 +00:00
|
|
|
size_t maxParallelStreams() const { return max_parallel_streams; }
|
|
|
|
|
2020-01-29 15:55:13 +00:00
|
|
|
/// Do not allow to change the table while the processors of pipe are alive.
|
|
|
|
/// TODO: move it to pipeline.
|
2020-06-18 16:10:47 +00:00
|
|
|
void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); }
|
2020-01-29 16:59:11 +00:00
|
|
|
/// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible.
|
|
|
|
void addInterpreterContext(std::shared_ptr<Context> context) { interpreter_context.emplace_back(std::move(context)); }
|
|
|
|
void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); }
|
|
|
|
|
2020-06-18 16:10:47 +00:00
|
|
|
const std::vector<TableLockHolder> & getTableLocks() const { return table_locks; }
|
2020-01-29 16:59:11 +00:00
|
|
|
const std::vector<std::shared_ptr<Context>> & getContexts() const { return interpreter_context; }
|
|
|
|
const std::vector<StoragePtr> & getStorageHolders() const { return storage_holders; }
|
2020-01-29 15:55:13 +00:00
|
|
|
|
2019-10-20 09:12:42 +00:00
|
|
|
private:
|
|
|
|
Processors processors;
|
|
|
|
OutputPort * output_port = nullptr;
|
2019-11-05 17:33:03 +00:00
|
|
|
OutputPort * totals = nullptr;
|
2020-04-08 12:40:04 +00:00
|
|
|
OutputPort * extremes = nullptr;
|
2020-01-29 15:55:13 +00:00
|
|
|
|
2020-03-31 14:30:43 +00:00
|
|
|
/// 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;
|
|
|
|
|
2020-06-18 16:10:47 +00:00
|
|
|
std::vector<TableLockHolder> table_locks;
|
2020-01-29 16:59:11 +00:00
|
|
|
|
|
|
|
/// Some processors may implicitly use Context or temporary Storage created by Interpreter.
|
|
|
|
/// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here,
|
|
|
|
/// because QueryPipeline is alive until query is finished.
|
|
|
|
std::vector<std::shared_ptr<Context>> interpreter_context;
|
|
|
|
std::vector<StoragePtr> storage_holders;
|
|
|
|
|
|
|
|
/// This private constructor is used only from QueryPipeline.
|
|
|
|
/// It is not public, because QueryPipeline checks that processors are connected and have single output,
|
|
|
|
/// and therefore we can skip those checks.
|
|
|
|
/// Note that Pipe represents a tree if it was created using public interface. But this constructor can't assert it.
|
|
|
|
/// So, it's possible that TreeExecutorBlockInputStream could be unable to convert such Pipe to IBlockInputStream.
|
2020-04-08 12:40:04 +00:00
|
|
|
explicit Pipe(Processors processors_, OutputPort * output_port, OutputPort * totals, OutputPort * extremes);
|
2020-01-29 16:59:11 +00:00
|
|
|
|
|
|
|
friend class QueryPipeline;
|
2019-10-20 09:12:42 +00:00
|
|
|
};
|
2020-08-03 11:33:11 +00:00
|
|
|
*/
|
2019-10-20 09:12:42 +00:00
|
|
|
}
|