mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Refactor processors.
This commit is contained in:
parent
f55f70d67b
commit
7d4de44695
@ -8,44 +8,54 @@ ConcatProcessor::Status ConcatProcessor::prepare()
|
||||
{
|
||||
auto & output = outputs[0];
|
||||
|
||||
if (output.hasData())
|
||||
return Status::PortFull;
|
||||
/// Check can output.
|
||||
|
||||
if (output.isFinished())
|
||||
{
|
||||
for (; current_input != inputs.end(); ++current_input)
|
||||
current_input->close();
|
||||
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
if (!output.isNeeded())
|
||||
{
|
||||
for (auto & input : inputs)
|
||||
input.setNotNeeded();
|
||||
if (current_input != inputs.end())
|
||||
current_input->setNotNeeded();
|
||||
|
||||
return Status::Unneeded;
|
||||
return Status::PortFull;
|
||||
}
|
||||
|
||||
if (output.hasData())
|
||||
return Status::PortFull;
|
||||
|
||||
/// Check can input.
|
||||
|
||||
if (current_input == inputs.end())
|
||||
return Status::Finished;
|
||||
|
||||
if (current_input->isFinished())
|
||||
{
|
||||
++current_input;
|
||||
if (current_input == inputs.end())
|
||||
{
|
||||
output.finish();
|
||||
return Status::Finished;
|
||||
}
|
||||
}
|
||||
|
||||
auto & input = *current_input;
|
||||
|
||||
input.setNeeded();
|
||||
|
||||
if (input.hasData())
|
||||
{
|
||||
output.push(input.pull());
|
||||
}
|
||||
if (!input.hasData())
|
||||
return Status::NeedData;
|
||||
|
||||
if (input.isFinished())
|
||||
{
|
||||
input.setNotNeeded();
|
||||
/// Move data.
|
||||
output.push(input.pull());
|
||||
|
||||
++current_input;
|
||||
if (current_input == inputs.end())
|
||||
{
|
||||
output.setFinished();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
current_input->setNeeded();
|
||||
}
|
||||
|
||||
return Status::NeedData;
|
||||
/// Now, we pushed to output, and it must be full.
|
||||
return Status::PortFull;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -8,45 +8,59 @@ ForkProcessor::Status ForkProcessor::prepare()
|
||||
{
|
||||
auto & input = inputs[0];
|
||||
|
||||
bool all_outputs_unneeded = true;
|
||||
/// Check can output.
|
||||
|
||||
bool all_finished = true;
|
||||
bool all_can_push = true;
|
||||
|
||||
for (const auto & output : outputs)
|
||||
{
|
||||
if (output.isNeeded())
|
||||
if (!output.isFinished())
|
||||
{
|
||||
all_outputs_unneeded = false;
|
||||
if (output.hasData())
|
||||
return Status::PortFull;
|
||||
all_finished = false;
|
||||
|
||||
/// The order is important.
|
||||
if (!output.canPush())
|
||||
all_can_push = false;
|
||||
}
|
||||
}
|
||||
|
||||
if (all_outputs_unneeded)
|
||||
if (all_finished)
|
||||
{
|
||||
input.close();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
if (!all_can_push)
|
||||
{
|
||||
input.setNotNeeded();
|
||||
return Status::Unneeded;
|
||||
return Status::PortFull;
|
||||
}
|
||||
|
||||
/// Check can input.
|
||||
|
||||
if (input.isFinished())
|
||||
{
|
||||
for (auto & output : outputs)
|
||||
output.finish();
|
||||
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
input.setNeeded();
|
||||
|
||||
if (!input.hasData())
|
||||
{
|
||||
if (input.isFinished())
|
||||
{
|
||||
input.setNotNeeded();
|
||||
for (auto & output : outputs)
|
||||
output.setFinished();
|
||||
return Status::Finished;
|
||||
}
|
||||
else
|
||||
return Status::NeedData;
|
||||
}
|
||||
return Status::NeedData;
|
||||
|
||||
/// Move data.
|
||||
|
||||
auto data = input.pull();
|
||||
|
||||
for (auto & output : outputs)
|
||||
if (output.isNeeded())
|
||||
output.push(data);
|
||||
if (!output.isFinished()) /// Skip finished outputs.
|
||||
output.push(data); /// Can push because no full or unneeded outputs.
|
||||
|
||||
/// Now, we pulled from input. It must be empty.
|
||||
return Status::NeedData;
|
||||
}
|
||||
|
||||
|
@ -6,7 +6,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Has one input and arbitary non zero number of outputs.
|
||||
/** Has one input and arbitrary non zero number of outputs.
|
||||
* All of them have the same structure.
|
||||
*
|
||||
* Pulls data input and copies it to every output.
|
||||
@ -18,7 +18,7 @@ namespace DB
|
||||
class ForkProcessor : public IProcessor
|
||||
{
|
||||
public:
|
||||
ForkProcessor(Block header, size_t num_outputs)
|
||||
ForkProcessor(const Block & header, size_t num_outputs)
|
||||
: IProcessor(InputPorts{header}, OutputPorts(num_outputs, header))
|
||||
{
|
||||
}
|
||||
|
@ -11,25 +11,28 @@ IOutputFormat::IOutputFormat(Block header, WriteBuffer & out)
|
||||
|
||||
IOutputFormat::Status IOutputFormat::prepare()
|
||||
{
|
||||
if (current_block)
|
||||
if (has_input)
|
||||
return Status::Ready;
|
||||
|
||||
for (auto kind : {Main, Totals, Extremes})
|
||||
{
|
||||
auto & input = inputs[kind];
|
||||
|
||||
if (input.hasData())
|
||||
{
|
||||
current_block = input.pull();
|
||||
current_block_kind = kind;
|
||||
return Status::Ready;
|
||||
}
|
||||
if (kind != Main && !input.isConnected())
|
||||
continue;
|
||||
|
||||
if (!input.isFinished())
|
||||
{
|
||||
input.setNeeded();
|
||||
if (input.isFinished())
|
||||
continue;
|
||||
|
||||
input.setNeeded();
|
||||
|
||||
if (!input.hasData())
|
||||
return Status::NeedData;
|
||||
}
|
||||
|
||||
current_block = input.pull();
|
||||
current_block_kind = kind;
|
||||
has_input = true;
|
||||
return Status::Ready;
|
||||
}
|
||||
|
||||
return Status::Finished;
|
||||
@ -49,6 +52,8 @@ void IOutputFormat::work()
|
||||
consumeExtremes(std::move(current_block));
|
||||
break;
|
||||
}
|
||||
|
||||
has_input = false;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -28,6 +28,7 @@ protected:
|
||||
|
||||
Block current_block;
|
||||
PortKind current_block_kind;
|
||||
bool has_input = false;
|
||||
|
||||
virtual void consume(Block) = 0;
|
||||
virtual void consumeTotals(Block) {}
|
||||
|
@ -12,49 +12,62 @@ IAccumulatingTransform::IAccumulatingTransform(Block input_header, Block output_
|
||||
|
||||
IAccumulatingTransform::Status IAccumulatingTransform::prepare()
|
||||
{
|
||||
if (!output.isNeeded())
|
||||
return Status::Unneeded;
|
||||
/// Check can output.
|
||||
if (output.isFinished())
|
||||
{
|
||||
input.close();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
if (current_input_block)
|
||||
return Status::Ready;
|
||||
if (!output.canPush())
|
||||
{
|
||||
input.setNotNeeded();
|
||||
return Status::PortFull;
|
||||
}
|
||||
|
||||
/// Output if has data.
|
||||
if (current_output_block)
|
||||
output.push(std::move(current_output_block));
|
||||
|
||||
if (finished_generate)
|
||||
{
|
||||
if (output.hasData())
|
||||
return Status::PortFull;
|
||||
else
|
||||
output.push(std::move(current_output_block));
|
||||
output.finish();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
if (input.hasData())
|
||||
/// Check can input.
|
||||
if (!has_input)
|
||||
{
|
||||
/// Generate output block.
|
||||
if (input.isFinished())
|
||||
{
|
||||
finished_input = true;
|
||||
return Status::Ready;
|
||||
}
|
||||
|
||||
input.setNeeded();
|
||||
if (!input.hasData())
|
||||
return Status::NeedData;
|
||||
|
||||
current_input_block = input.pull();
|
||||
return Status::Ready;
|
||||
has_input = true;
|
||||
}
|
||||
|
||||
if (input.isFinished())
|
||||
{
|
||||
if (finished)
|
||||
return Status::Finished;
|
||||
|
||||
return Status::Ready;
|
||||
}
|
||||
|
||||
input.setNeeded();
|
||||
return Status::NeedData;
|
||||
return Status::Ready;
|
||||
}
|
||||
|
||||
void IAccumulatingTransform::work()
|
||||
{
|
||||
if (current_input_block)
|
||||
if (!finished_input)
|
||||
{
|
||||
consume(std::move(current_input_block));
|
||||
has_input = false;
|
||||
}
|
||||
else
|
||||
{
|
||||
current_output_block = generate();
|
||||
if (!current_output_block)
|
||||
finished = true;
|
||||
finished_generate = true;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -18,7 +18,9 @@ protected:
|
||||
|
||||
Block current_input_block;
|
||||
Block current_output_block;
|
||||
bool finished = false;
|
||||
bool has_input = false;
|
||||
bool finished_input = false;
|
||||
bool finished_generate = false;
|
||||
|
||||
virtual void consume(Block block) = 0;
|
||||
virtual Block generate() = 0;
|
||||
|
@ -122,15 +122,15 @@ public:
|
||||
/// You need to run another processor to generate required input and then call 'prepare' again.
|
||||
NeedData,
|
||||
|
||||
/// Processor cannot proceed because output port is full.
|
||||
/// Processor cannot proceed because output port is full or not isNeeded().
|
||||
/// You need to transfer data from output port to the input port of another processor and then call 'prepare' again.
|
||||
PortFull,
|
||||
|
||||
/// All work is done, nothing more to do.
|
||||
/// All work is done (all data is processed or all output are closed), nothing more to do.
|
||||
Finished,
|
||||
|
||||
/// No one needs data on output ports.
|
||||
Unneeded,
|
||||
/// Unneeded,
|
||||
|
||||
/// You may call 'work' method and processor will do some work synchronously.
|
||||
Ready,
|
||||
|
@ -12,34 +12,48 @@ ISimpleTransform::ISimpleTransform(Block input_header, Block output_header)
|
||||
|
||||
ISimpleTransform::Status ISimpleTransform::prepare()
|
||||
{
|
||||
if (!output.isNeeded())
|
||||
return Status::Unneeded;
|
||||
/// Check can output.
|
||||
|
||||
if (current_block)
|
||||
if (output.isFinished())
|
||||
{
|
||||
if (!transformed)
|
||||
return Status::Ready;
|
||||
else if (output.hasData())
|
||||
return Status::PortFull;
|
||||
else
|
||||
output.push(std::move(current_block));
|
||||
}
|
||||
|
||||
if (input.hasData())
|
||||
{
|
||||
current_block = input.pull();
|
||||
transformed = false;
|
||||
return Status::Ready;
|
||||
}
|
||||
|
||||
if (input.isFinished())
|
||||
{
|
||||
output.setFinished();
|
||||
input.close();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
input.setNeeded();
|
||||
return Status::NeedData;
|
||||
if (!output.canPush())
|
||||
{
|
||||
input.setNotNeeded();
|
||||
return Status::PortFull;
|
||||
}
|
||||
|
||||
/// Output if has data.
|
||||
if (transformed)
|
||||
{
|
||||
output.push(std::move(current_block));
|
||||
has_input = false;
|
||||
transformed = false;
|
||||
}
|
||||
|
||||
/// Check can input.
|
||||
if (!has_input)
|
||||
{
|
||||
if (input.isFinished())
|
||||
{
|
||||
output.finish();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
input.setNeeded();
|
||||
|
||||
if (!input.hasData())
|
||||
return Status::NeedData;
|
||||
|
||||
current_block = input.pull();
|
||||
has_input = true;
|
||||
}
|
||||
|
||||
/// Now transform.
|
||||
return Status::Ready;
|
||||
}
|
||||
|
||||
void ISimpleTransform::work()
|
||||
|
@ -16,6 +16,7 @@ protected:
|
||||
OutputPort & output;
|
||||
|
||||
Block current_block;
|
||||
bool has_input = false;
|
||||
bool transformed = false;
|
||||
|
||||
virtual void transform(Block & block) = 0;
|
||||
|
@ -11,20 +11,19 @@ ISink::ISink(Block header)
|
||||
|
||||
ISink::Status ISink::prepare()
|
||||
{
|
||||
if (current_block)
|
||||
if (has_input)
|
||||
return Status::Ready;
|
||||
|
||||
if (input.hasData())
|
||||
{
|
||||
current_block = input.pull();
|
||||
return Status::Ready;
|
||||
}
|
||||
|
||||
if (input.isFinished())
|
||||
return Status::Finished;
|
||||
|
||||
input.setNeeded();
|
||||
return Status::NeedData;
|
||||
if (!input.hasData())
|
||||
return Status::NeedData;
|
||||
|
||||
current_block = input.pull();
|
||||
has_input = true;
|
||||
return Status::Ready;
|
||||
}
|
||||
|
||||
void ISink::work()
|
||||
|
@ -11,6 +11,7 @@ class ISink : public IProcessor
|
||||
protected:
|
||||
InputPort & input;
|
||||
Block current_block;
|
||||
bool has_input;
|
||||
|
||||
virtual void consume(Block block) = 0;
|
||||
|
||||
|
@ -13,20 +13,25 @@ ISource::Status ISource::prepare()
|
||||
{
|
||||
if (finished)
|
||||
{
|
||||
output.setFinished();
|
||||
output.finish();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
if (output.hasData())
|
||||
/// Check can output.
|
||||
if (output.isFinished())
|
||||
return Status::Finished;
|
||||
|
||||
if (!output.canPush())
|
||||
return Status::PortFull;
|
||||
|
||||
if (!output.isNeeded())
|
||||
return Status::Unneeded;
|
||||
if (!has_input)
|
||||
return Status::Ready;
|
||||
|
||||
if (current_block)
|
||||
output.push(std::move(current_block));
|
||||
output.push(std::move(current_block));
|
||||
has_input = false;
|
||||
|
||||
return Status::Ready;
|
||||
/// Now, we pushed to output, and it must be full.
|
||||
return Status::PortFull;
|
||||
}
|
||||
|
||||
void ISource::work()
|
||||
@ -34,6 +39,8 @@ void ISource::work()
|
||||
current_block = generate();
|
||||
if (!current_block)
|
||||
finished = true;
|
||||
else
|
||||
has_input = true;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -10,6 +10,7 @@ class ISource : public IProcessor
|
||||
{
|
||||
protected:
|
||||
OutputPort & output;
|
||||
bool has_input = false;
|
||||
bool finished = false;
|
||||
Block current_block;
|
||||
|
||||
|
@ -14,57 +14,88 @@ LimitTransform::LimitTransform(Block header, size_t limit, size_t offset, bool a
|
||||
|
||||
LimitTransform::Status LimitTransform::prepare()
|
||||
{
|
||||
if (current_block)
|
||||
{
|
||||
if (output.hasData())
|
||||
return Status::PortFull;
|
||||
/// Check can output.
|
||||
|
||||
output.push(std::move(current_block));
|
||||
if (output.isFinished())
|
||||
{
|
||||
input.close();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
if (rows_read >= offset + limit)
|
||||
if (!output.canPush())
|
||||
{
|
||||
output.setFinished();
|
||||
input.setNotNeeded();
|
||||
return Status::PortFull;
|
||||
}
|
||||
|
||||
/// Push block if can.
|
||||
if (block_processed)
|
||||
{
|
||||
output.push(std::move(current_block));
|
||||
has_block = false;
|
||||
block_processed = false;
|
||||
}
|
||||
|
||||
/// Check if we are done with pushing.
|
||||
bool pushing_is_finished = rows_read >= offset + limit;
|
||||
if (pushing_is_finished)
|
||||
{
|
||||
output.finish();
|
||||
if (!always_read_till_end)
|
||||
{
|
||||
input.setNotNeeded();
|
||||
input.close();
|
||||
return Status::Finished;
|
||||
}
|
||||
}
|
||||
|
||||
if (!output.isNeeded())
|
||||
return Status::Unneeded;
|
||||
/// Check can input.
|
||||
|
||||
if (input.isFinished())
|
||||
{
|
||||
output.finish();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
input.setNeeded();
|
||||
|
||||
if (!input.hasData())
|
||||
return Status::NeedData;
|
||||
|
||||
current_block = input.pull();
|
||||
has_block = true;
|
||||
|
||||
/// Skip block (for 'always_read_till_end' case)
|
||||
if (rows_read >= offset + limit)
|
||||
/// Skip block (for 'always_read_till_end' case).
|
||||
if (pushing_is_finished)
|
||||
{
|
||||
current_block.clear();
|
||||
has_block = false;
|
||||
|
||||
/// Now, we pulled from input, and it must be empty.
|
||||
return Status::NeedData;
|
||||
}
|
||||
|
||||
/// Process block.
|
||||
|
||||
size_t rows = current_block.rows();
|
||||
rows_read += rows;
|
||||
|
||||
if (rows_read <= offset)
|
||||
{
|
||||
current_block.clear();
|
||||
has_block = false;
|
||||
|
||||
/// Now, we pulled from input, and it must be empty.
|
||||
return Status::NeedData;
|
||||
}
|
||||
|
||||
/// return the whole block
|
||||
/// Return the whole block.
|
||||
if (rows_read >= offset + rows && rows_read <= offset + limit)
|
||||
{
|
||||
if (output.hasData())
|
||||
return Status::PortFull;
|
||||
|
||||
output.push(std::move(current_block));
|
||||
has_block = false;
|
||||
|
||||
return Status::NeedData;
|
||||
}
|
||||
|
||||
@ -89,6 +120,8 @@ void LimitTransform::work()
|
||||
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
current_block.getByPosition(i).column = current_block.getByPosition(i).column->cut(start, length);
|
||||
|
||||
block_processed = true;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -17,6 +17,8 @@ private:
|
||||
size_t rows_read = 0; /// including the last read block
|
||||
bool always_read_till_end;
|
||||
|
||||
bool has_block = false;
|
||||
bool block_processed = false;
|
||||
Block current_block;
|
||||
|
||||
public:
|
||||
|
@ -21,10 +21,98 @@ class Port
|
||||
|
||||
protected:
|
||||
/// Shared state of two connected ports.
|
||||
struct State
|
||||
class State
|
||||
{
|
||||
public:
|
||||
State() = default;
|
||||
|
||||
void push(Block block)
|
||||
{
|
||||
if (finished)
|
||||
throw Exception("Cannot push block to finished port.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!needed)
|
||||
throw Exception("Cannot push block to port which is not needed.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (has_data)
|
||||
throw Exception("Cannot push block to port which already has data.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
data = std::move(block);
|
||||
has_data = true;
|
||||
}
|
||||
|
||||
Block pull()
|
||||
{
|
||||
if (!needed)
|
||||
throw Exception("Cannot pull block from port which is not needed.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!has_data)
|
||||
throw Exception("Cannot pull block from port which has no data.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
has_data = false;
|
||||
return std::move(data);
|
||||
}
|
||||
|
||||
bool hasData() const
|
||||
{
|
||||
if (finished)
|
||||
throw Exception("Finished port can't has data.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!needed)
|
||||
throw Exception("Cannot check if not needed port has data.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return has_data;
|
||||
}
|
||||
|
||||
/// Only for output port.
|
||||
/// If port still has data, it will be finished after pulling.
|
||||
void finish()
|
||||
{
|
||||
finished = true;
|
||||
}
|
||||
|
||||
/// Only for input port. Removes data if has.
|
||||
void close()
|
||||
{
|
||||
finished = true;
|
||||
has_data = false;
|
||||
data.clear();
|
||||
}
|
||||
|
||||
/// Only empty ports are finished.
|
||||
bool isFinished() const { return finished && !has_data; }
|
||||
bool isSetFinished() const { return finished; }
|
||||
|
||||
void setNeeded()
|
||||
{
|
||||
if (finished)
|
||||
throw Exception("Can't set port needed if it is finished.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
// if (has_data)
|
||||
// throw Exception("Can't set port needed if it has data.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
needed = true;
|
||||
}
|
||||
|
||||
void setNotNeeded()
|
||||
{
|
||||
// if (finished)
|
||||
// throw Exception("Can't set port not needed if it is finished.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
needed = false;
|
||||
}
|
||||
|
||||
/// Only for output port.
|
||||
bool isNeeded() const { return needed && !finished; }
|
||||
|
||||
private:
|
||||
Block data;
|
||||
/// Use special flag to check if block has data. This allows to send empty blocks between processors.
|
||||
bool has_data = false;
|
||||
/// Block is not needed right now, but may be will be needed later.
|
||||
/// This allows to pause calculations if we are not sure that we need more data.
|
||||
bool needed = false;
|
||||
/// Port was set finished or closed.
|
||||
bool finished = false;
|
||||
};
|
||||
|
||||
@ -34,8 +122,7 @@ protected:
|
||||
IProcessor * processor = nullptr;
|
||||
|
||||
public:
|
||||
Port(const Block & header)
|
||||
: header(header) {}
|
||||
Port(Block header) : header(std::move(header)) {}
|
||||
|
||||
const Block & getHeader() const { return header; }
|
||||
bool isConnected() const { return state != nullptr; }
|
||||
@ -49,13 +136,7 @@ public:
|
||||
bool hasData() const
|
||||
{
|
||||
assumeConnected();
|
||||
return state->data;
|
||||
}
|
||||
|
||||
bool isNeeded() const
|
||||
{
|
||||
assumeConnected();
|
||||
return state->needed;
|
||||
return state->hasData();
|
||||
}
|
||||
|
||||
IProcessor & getProcessor()
|
||||
@ -73,7 +154,11 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Invariants:
|
||||
/// * If you close port, it isFinished().
|
||||
/// * If port isFinished(), you can do nothing with it.
|
||||
/// * If port is not needed, you can only setNeeded() or close() it.
|
||||
/// * You can pull only if port hasData().
|
||||
class InputPort : public Port
|
||||
{
|
||||
friend void connect(OutputPort &, InputPort &);
|
||||
@ -86,28 +171,32 @@ public:
|
||||
|
||||
Block pull()
|
||||
{
|
||||
if (!hasData())
|
||||
throw Exception("Port has no data", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return std::move(state->data);
|
||||
assumeConnected();
|
||||
return state->pull();
|
||||
}
|
||||
|
||||
bool isFinished() const
|
||||
{
|
||||
assumeConnected();
|
||||
return state->finished;
|
||||
return state->isFinished();
|
||||
}
|
||||
|
||||
void setNeeded()
|
||||
{
|
||||
assumeConnected();
|
||||
state->needed = true;
|
||||
state->setNeeded();
|
||||
}
|
||||
|
||||
void setNotNeeded()
|
||||
{
|
||||
assumeConnected();
|
||||
state->needed = false;
|
||||
state->setNotNeeded();
|
||||
}
|
||||
|
||||
void close()
|
||||
{
|
||||
assumeConnected();
|
||||
state->close();
|
||||
}
|
||||
|
||||
OutputPort & getOutputPort()
|
||||
@ -124,6 +213,11 @@ public:
|
||||
};
|
||||
|
||||
|
||||
/// Invariants:
|
||||
/// * If you finish port, it isFinished().
|
||||
/// * If port isFinished(), you can do nothing with it.
|
||||
/// * If port not isNeeded(), you can only finish() it.
|
||||
/// * You can hush only if port doesn't hasData().
|
||||
class OutputPort : public Port
|
||||
{
|
||||
friend void connect(OutputPort &, InputPort &);
|
||||
@ -136,18 +230,30 @@ public:
|
||||
|
||||
void push(Block block)
|
||||
{
|
||||
if (hasData())
|
||||
throw Exception("Port already has data", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
state->data = std::move(block);
|
||||
assumeConnected();
|
||||
state->push(std::move(block));
|
||||
}
|
||||
|
||||
void setFinished()
|
||||
void finish()
|
||||
{
|
||||
assumeConnected();
|
||||
state->finished = true;
|
||||
state->finish();
|
||||
}
|
||||
|
||||
bool isNeeded() const
|
||||
{
|
||||
assumeConnected();
|
||||
return state->isNeeded();
|
||||
}
|
||||
|
||||
bool isFinished() const
|
||||
{
|
||||
assumeConnected();
|
||||
return state->isSetFinished();
|
||||
}
|
||||
|
||||
bool canPush() const { return isNeeded() && !hasData(); }
|
||||
|
||||
InputPort & getInputPort()
|
||||
{
|
||||
assumeConnected();
|
||||
|
@ -6,74 +6,112 @@ namespace DB
|
||||
|
||||
ResizeProcessor::Status ResizeProcessor::prepare()
|
||||
{
|
||||
while (true)
|
||||
auto cur_output = outputs.begin();
|
||||
bool all_outs_full_or_unneeded = true;
|
||||
bool all_outs_finished = true;
|
||||
|
||||
/// Find next output where can push.
|
||||
auto get_next_out = [&, this]() -> OutputPorts::iterator
|
||||
{
|
||||
bool all_outputs_full = true;
|
||||
bool all_outputs_unneeded = true;
|
||||
|
||||
for (const auto & output : outputs)
|
||||
while (cur_output != outputs.end())
|
||||
{
|
||||
if (!output.hasData())
|
||||
all_outputs_full = false;
|
||||
if (!cur_output->isFinished())
|
||||
{
|
||||
all_outs_finished = false;
|
||||
|
||||
if (output.isNeeded())
|
||||
all_outputs_unneeded = false;
|
||||
if (cur_output->canPush())
|
||||
{
|
||||
all_outs_full_or_unneeded = false;
|
||||
++cur_output;
|
||||
return std::prev(cur_output);
|
||||
}
|
||||
}
|
||||
|
||||
++cur_output;
|
||||
}
|
||||
|
||||
if (all_outputs_full)
|
||||
return Status::PortFull;
|
||||
return cur_output;
|
||||
};
|
||||
|
||||
if (all_outputs_unneeded)
|
||||
auto cur_input = inputs.begin();
|
||||
bool all_inputs_finished = true;
|
||||
|
||||
/// Find next input from where can pull.
|
||||
auto get_next_input = [&, this]() -> InputPorts::iterator
|
||||
{
|
||||
while (cur_input != inputs.end())
|
||||
{
|
||||
for (auto & input : inputs)
|
||||
input.setNotNeeded();
|
||||
|
||||
return Status::Unneeded;
|
||||
}
|
||||
|
||||
bool all_inputs_finished = true;
|
||||
bool all_inputs_have_no_data = true;
|
||||
|
||||
for (auto & input : inputs)
|
||||
{
|
||||
if (!input.isFinished())
|
||||
if (!cur_input->isFinished())
|
||||
{
|
||||
all_inputs_finished = false;
|
||||
|
||||
input.setNeeded();
|
||||
if (input.hasData())
|
||||
all_inputs_have_no_data = false;
|
||||
}
|
||||
}
|
||||
|
||||
if (all_inputs_have_no_data)
|
||||
{
|
||||
if (all_inputs_finished)
|
||||
{
|
||||
for (auto & output : outputs)
|
||||
output.setFinished();
|
||||
return Status::Finished;
|
||||
}
|
||||
else
|
||||
return Status::NeedData;
|
||||
}
|
||||
|
||||
for (auto & input : inputs)
|
||||
{
|
||||
if (input.hasData())
|
||||
{
|
||||
for (auto & output : outputs)
|
||||
cur_input->setNeeded();
|
||||
if (cur_input->hasData())
|
||||
{
|
||||
if (!output.hasData())
|
||||
{
|
||||
output.push(input.pull());
|
||||
break;
|
||||
}
|
||||
++cur_input;
|
||||
return std::prev(cur_input);
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
++cur_input;
|
||||
}
|
||||
|
||||
return cur_input;
|
||||
};
|
||||
|
||||
auto get_status_if_no_outputs = [&]() -> Status
|
||||
{
|
||||
if (all_outs_finished)
|
||||
{
|
||||
for (auto & in : inputs)
|
||||
in.close();
|
||||
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
if (all_outs_full_or_unneeded)
|
||||
{
|
||||
for (auto & in : inputs)
|
||||
in.setNotNeeded();
|
||||
|
||||
return Status::PortFull;
|
||||
}
|
||||
|
||||
/// Now, we pushed to output, and it must be full.
|
||||
return Status::PortFull;
|
||||
};
|
||||
|
||||
auto get_status_if_no_inputs = [&]() -> Status
|
||||
{
|
||||
if (all_inputs_finished)
|
||||
{
|
||||
for (auto & out : outputs)
|
||||
out.finish();
|
||||
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
return Status::NeedData;
|
||||
};
|
||||
|
||||
while (cur_input != inputs.end() && cur_output != outputs.end())
|
||||
{
|
||||
auto output = get_next_out();
|
||||
if (output == outputs.end())
|
||||
return get_status_if_no_outputs();
|
||||
|
||||
auto input = get_next_input();
|
||||
if (input == inputs.end())
|
||||
return get_status_if_no_inputs();
|
||||
|
||||
output->push(input->pull());
|
||||
}
|
||||
|
||||
if (cur_output == outputs.end())
|
||||
return get_status_if_no_outputs();
|
||||
|
||||
/// cur_input == inputs_end()
|
||||
return get_status_if_no_inputs();
|
||||
}
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user