mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Backport #71494 to 24.8: Prevent crash in SortCursor with 0 columns
This commit is contained in:
parent
f8bd728c73
commit
1763a50d37
@ -35,6 +35,11 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
/** Cursor allows to compare rows in different blocks (and parts).
|
||||
* Cursor moves inside single block.
|
||||
* It is used in priority queue.
|
||||
@ -83,21 +88,27 @@ struct SortCursorImpl
|
||||
SortCursorImpl(
|
||||
const Block & header,
|
||||
const Columns & columns,
|
||||
size_t num_rows,
|
||||
const SortDescription & desc_,
|
||||
size_t order_ = 0,
|
||||
IColumn::Permutation * perm = nullptr)
|
||||
: desc(desc_), sort_columns_size(desc.size()), order(order_), need_collation(desc.size())
|
||||
{
|
||||
reset(columns, header, perm);
|
||||
reset(columns, header, num_rows, perm);
|
||||
}
|
||||
|
||||
bool empty() const { return rows == 0; }
|
||||
|
||||
/// Set the cursor to the beginning of the new block.
|
||||
void reset(const Block & block, IColumn::Permutation * perm = nullptr) { reset(block.getColumns(), block, perm); }
|
||||
void reset(const Block & block, IColumn::Permutation * perm = nullptr)
|
||||
{
|
||||
if (block.getColumns().empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty column list in block");
|
||||
reset(block.getColumns(), block, block.getColumns()[0]->size(), perm);
|
||||
}
|
||||
|
||||
/// Set the cursor to the beginning of the new block.
|
||||
void reset(const Columns & columns, const Block & block, IColumn::Permutation * perm = nullptr)
|
||||
void reset(const Columns & columns, const Block & block, UInt64 num_rows, IColumn::Permutation * perm = nullptr)
|
||||
{
|
||||
all_columns.clear();
|
||||
sort_columns.clear();
|
||||
@ -125,7 +136,7 @@ struct SortCursorImpl
|
||||
}
|
||||
|
||||
pos = 0;
|
||||
rows = all_columns[0]->size();
|
||||
rows = num_rows;
|
||||
permutation = perm;
|
||||
}
|
||||
|
||||
|
@ -24,7 +24,12 @@ void IMergingAlgorithmWithDelayedChunk::initializeQueue(Inputs inputs)
|
||||
continue;
|
||||
|
||||
cursors[source_num] = SortCursorImpl(
|
||||
header, current_inputs[source_num].chunk.getColumns(), description, source_num, current_inputs[source_num].permutation);
|
||||
header,
|
||||
current_inputs[source_num].chunk.getColumns(),
|
||||
current_inputs[source_num].chunk.getNumRows(),
|
||||
description,
|
||||
source_num,
|
||||
current_inputs[source_num].permutation);
|
||||
|
||||
inputs_origin_merge_tree_part_level[source_num] = getPartLevelFromChunk(current_inputs[source_num].chunk);
|
||||
}
|
||||
@ -41,7 +46,7 @@ void IMergingAlgorithmWithDelayedChunk::updateCursor(Input & input, size_t sourc
|
||||
last_chunk_sort_columns = std::move(cursors[source_num].sort_columns);
|
||||
|
||||
current_input.swap(input);
|
||||
cursors[source_num].reset(current_input.chunk.getColumns(), header, current_input.permutation);
|
||||
cursors[source_num].reset(current_input.chunk.getColumns(), header, current_input.chunk.getNumRows(), current_input.permutation);
|
||||
|
||||
inputs_origin_merge_tree_part_level[source_num] = getPartLevelFromChunk(current_input.chunk);
|
||||
|
||||
|
@ -31,7 +31,8 @@ void IMergingAlgorithmWithSharedChunks::initialize(Inputs inputs)
|
||||
|
||||
source.skip_last_row = inputs[source_num].skip_last_row;
|
||||
source.chunk = chunk_allocator.alloc(inputs[source_num].chunk);
|
||||
cursors[source_num] = SortCursorImpl(header, source.chunk->getColumns(), description, source_num, inputs[source_num].permutation);
|
||||
cursors[source_num] = SortCursorImpl(
|
||||
header, source.chunk->getColumns(), source.chunk->getNumRows(), description, source_num, inputs[source_num].permutation);
|
||||
|
||||
source.chunk->all_columns = cursors[source_num].all_columns;
|
||||
source.chunk->sort_columns = cursors[source_num].sort_columns;
|
||||
@ -49,7 +50,7 @@ void IMergingAlgorithmWithSharedChunks::consume(Input & input, size_t source_num
|
||||
auto & source = sources[source_num];
|
||||
source.skip_last_row = input.skip_last_row;
|
||||
source.chunk = chunk_allocator.alloc(input.chunk);
|
||||
cursors[source_num].reset(source.chunk->getColumns(), header, input.permutation);
|
||||
cursors[source_num].reset(source.chunk->getColumns(), header, source.chunk->getNumRows(), input.permutation);
|
||||
|
||||
source.chunk->all_columns = cursors[source_num].all_columns;
|
||||
source.chunk->sort_columns = cursors[source_num].sort_columns;
|
||||
|
@ -59,7 +59,7 @@ void MergingSortedAlgorithm::initialize(Inputs inputs)
|
||||
if (!chunk)
|
||||
continue;
|
||||
|
||||
cursors[source_num] = SortCursorImpl(header, chunk.getColumns(), description, source_num);
|
||||
cursors[source_num] = SortCursorImpl(header, chunk.getColumns(), chunk.getNumRows(), description, source_num);
|
||||
}
|
||||
|
||||
if (sorting_queue_strategy == SortingQueueStrategy::Default)
|
||||
@ -84,7 +84,7 @@ void MergingSortedAlgorithm::consume(Input & input, size_t source_num)
|
||||
{
|
||||
removeConstAndSparse(input);
|
||||
current_inputs[source_num].swap(input);
|
||||
cursors[source_num].reset(current_inputs[source_num].chunk.getColumns(), header);
|
||||
cursors[source_num].reset(current_inputs[source_num].chunk.getColumns(), header, current_inputs[source_num].chunk.getNumRows());
|
||||
|
||||
if (sorting_queue_strategy == SortingQueueStrategy::Default)
|
||||
{
|
||||
|
@ -394,7 +394,7 @@ void FullMergeJoinCursor::setChunk(Chunk && chunk)
|
||||
convertToFullIfSparse(chunk);
|
||||
|
||||
current_chunk = std::move(chunk);
|
||||
cursor = SortCursorImpl(sample_block, current_chunk.getColumns(), desc);
|
||||
cursor = SortCursorImpl(sample_block, current_chunk.getColumns(), current_chunk.getNumRows(), desc);
|
||||
}
|
||||
|
||||
bool FullMergeJoinCursor::fullyCompleted() const
|
||||
|
@ -42,7 +42,7 @@ MergeSorter::MergeSorter(const Block & header, Chunks chunks_, SortDescription &
|
||||
/// Convert to full column, because some cursors expect non-contant columns
|
||||
convertToFullIfConst(chunk);
|
||||
|
||||
cursors.emplace_back(header, chunk.getColumns(), description, chunk_index);
|
||||
cursors.emplace_back(header, chunk.getColumns(), chunk.getNumRows(), description, chunk_index);
|
||||
has_collation |= cursors.back().has_collation;
|
||||
|
||||
nonempty_chunks.emplace_back(std::move(chunk));
|
||||
|
@ -0,0 +1,4 @@
|
||||
42
|
||||
43
|
||||
44
|
||||
45
|
24
tests/queries/0_stateless/03261_sort_cursor_crash.sql
Normal file
24
tests/queries/0_stateless/03261_sort_cursor_crash.sql
Normal file
@ -0,0 +1,24 @@
|
||||
-- https://github.com/ClickHouse/ClickHouse/issues/70779
|
||||
-- Crash in SortCursorImpl with the old analyzer, which produces a block with 0 columns and 1 row
|
||||
DROP TABLE IF EXISTS t0;
|
||||
DROP TABLE IF EXISTS t1;
|
||||
|
||||
CREATE TABLE t0 (c0 Int) ENGINE = AggregatingMergeTree() ORDER BY tuple();
|
||||
INSERT INTO TABLE t0 (c0) VALUES (1);
|
||||
SELECT 42 FROM t0 FINAL PREWHERE t0.c0 = 1;
|
||||
DROP TABLE t0;
|
||||
|
||||
CREATE TABLE t0 (c0 Int) ENGINE = SummingMergeTree() ORDER BY tuple();
|
||||
INSERT INTO TABLE t0 (c0) VALUES (1);
|
||||
SELECT 43 FROM t0 FINAL PREWHERE t0.c0 = 1;
|
||||
DROP TABLE t0;
|
||||
|
||||
CREATE TABLE t0 (c0 Int) ENGINE = ReplacingMergeTree() ORDER BY tuple();
|
||||
INSERT INTO TABLE t0 (c0) VALUES (1);
|
||||
SELECT 44 FROM t0 FINAL PREWHERE t0.c0 = 1;
|
||||
DROP TABLE t0;
|
||||
|
||||
CREATE TABLE t1 (a0 UInt8, c0 Int32, c1 UInt8) ENGINE = AggregatingMergeTree() ORDER BY tuple();
|
||||
INSERT INTO TABLE t1 (a0, c0, c1) VALUES (1, 1, 1);
|
||||
SELECT 45 FROM t1 FINAL PREWHERE t1.c0 = t1.c1;
|
||||
DROP TABLE t1;
|
Loading…
Reference in New Issue
Block a user