ClickHouse/src/Processors/LimitTransform.h

73 lines
2.5 KiB
C++
Raw Normal View History

#pragma once
#include <Processors/IProcessor.h>
#include <Processors/RowsBeforeLimitCounter.h>
2019-08-27 17:48:42 +00:00
#include <Core/SortDescription.h>
namespace DB
{
2020-03-12 16:59:49 +00:00
/// Implementation for LIMIT N OFFSET M
/// This processor support multiple inputs and outputs (the same number).
/// Each pair of input and output port works independently.
/// The reason to have multiple ports is to be able to stop all sources when limit is reached, in a query like:
2020-03-14 14:51:32 +00:00
/// SELECT * FROM system.numbers_mt WHERE number = 1000000 LIMIT 1
2020-03-12 16:59:49 +00:00
///
/// always_read_till_end - read all data from input ports even if limit was reached.
/// with_ties, description - implementation of LIMIT WITH TIES. It works only for single port.
class LimitTransform : public IProcessor
{
private:
size_t limit;
size_t offset;
bool always_read_till_end;
2019-08-27 17:48:42 +00:00
bool with_ties;
const SortDescription description;
2020-02-28 20:20:39 +00:00
2020-03-02 12:56:47 +00:00
Chunk previous_row_chunk; /// for WITH TIES, contains only sort columns
2019-08-27 17:48:42 +00:00
std::vector<size_t> sort_column_positions;
2020-03-12 15:29:35 +00:00
2020-03-12 16:59:49 +00:00
size_t rows_read = 0; /// including the last read block
RowsBeforeLimitCounterPtr rows_before_limit_at_least;
2020-03-12 16:59:49 +00:00
2020-03-12 19:04:12 +00:00
/// State of port's pair.
/// Chunks from different port pairs are not mixed for better cache locality.
2020-03-12 16:59:49 +00:00
struct PortsData
{
Chunk current_chunk;
InputPort * input_port = nullptr;
OutputPort * output_port = nullptr;
bool is_finished = false;
};
2020-03-12 15:29:35 +00:00
2020-03-12 16:59:49 +00:00
std::vector<PortsData> ports_data;
2020-03-12 15:29:35 +00:00
size_t num_finished_port_pairs = 0;
2020-03-02 12:56:47 +00:00
Chunk makeChunkWithPreviousRow(const Chunk & current_chunk, size_t row_num) const;
2020-02-28 20:20:39 +00:00
ColumnRawPtrs extractSortColumns(const Columns & columns) const;
2020-03-02 12:56:47 +00:00
bool sortColumnsEqualAt(const ColumnRawPtrs & current_chunk_sort_columns, size_t current_chunk_row_num) const;
2019-08-27 17:48:42 +00:00
public:
2019-04-09 10:17:25 +00:00
LimitTransform(
2020-03-12 15:29:35 +00:00
const Block & header_, size_t limit_, size_t offset_, size_t num_streams = 1,
2019-08-27 17:48:42 +00:00
bool always_read_till_end_ = false, bool with_ties_ = false,
SortDescription description_ = {});
String getName() const override { return "Limit"; }
2020-03-12 12:49:42 +00:00
Status prepare(const PortNumbers & /*updated_input_ports*/, const PortNumbers & /*updated_output_ports*/) override;
2020-03-13 18:43:21 +00:00
Status prepare() override; /// Compatibility for TreeExecutor.
2020-03-12 16:59:49 +00:00
Status preparePair(PortsData & data);
void splitChunk(PortsData & data);
2020-03-12 12:49:42 +00:00
2020-03-13 13:50:34 +00:00
InputPort & getInputPort() { return inputs.front(); }
OutputPort & getOutputPort() { return outputs.front(); }
void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) { rows_before_limit_at_least.swap(counter); }
2020-03-12 12:49:42 +00:00
};
}