mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +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 DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int LOGICAL_ERROR;
|
||||||
|
}
|
||||||
|
|
||||||
/** Cursor allows to compare rows in different blocks (and parts).
|
/** Cursor allows to compare rows in different blocks (and parts).
|
||||||
* Cursor moves inside single block.
|
* Cursor moves inside single block.
|
||||||
* It is used in priority queue.
|
* It is used in priority queue.
|
||||||
@ -83,21 +88,27 @@ struct SortCursorImpl
|
|||||||
SortCursorImpl(
|
SortCursorImpl(
|
||||||
const Block & header,
|
const Block & header,
|
||||||
const Columns & columns,
|
const Columns & columns,
|
||||||
|
size_t num_rows,
|
||||||
const SortDescription & desc_,
|
const SortDescription & desc_,
|
||||||
size_t order_ = 0,
|
size_t order_ = 0,
|
||||||
IColumn::Permutation * perm = nullptr)
|
IColumn::Permutation * perm = nullptr)
|
||||||
: desc(desc_), sort_columns_size(desc.size()), order(order_), need_collation(desc.size())
|
: 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; }
|
bool empty() const { return rows == 0; }
|
||||||
|
|
||||||
/// Set the cursor to the beginning of the new block.
|
/// 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.
|
/// 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();
|
all_columns.clear();
|
||||||
sort_columns.clear();
|
sort_columns.clear();
|
||||||
@ -125,7 +136,7 @@ struct SortCursorImpl
|
|||||||
}
|
}
|
||||||
|
|
||||||
pos = 0;
|
pos = 0;
|
||||||
rows = all_columns[0]->size();
|
rows = num_rows;
|
||||||
permutation = perm;
|
permutation = perm;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -24,7 +24,12 @@ void IMergingAlgorithmWithDelayedChunk::initializeQueue(Inputs inputs)
|
|||||||
continue;
|
continue;
|
||||||
|
|
||||||
cursors[source_num] = SortCursorImpl(
|
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);
|
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);
|
last_chunk_sort_columns = std::move(cursors[source_num].sort_columns);
|
||||||
|
|
||||||
current_input.swap(input);
|
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);
|
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.skip_last_row = inputs[source_num].skip_last_row;
|
||||||
source.chunk = chunk_allocator.alloc(inputs[source_num].chunk);
|
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->all_columns = cursors[source_num].all_columns;
|
||||||
source.chunk->sort_columns = cursors[source_num].sort_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];
|
auto & source = sources[source_num];
|
||||||
source.skip_last_row = input.skip_last_row;
|
source.skip_last_row = input.skip_last_row;
|
||||||
source.chunk = chunk_allocator.alloc(input.chunk);
|
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->all_columns = cursors[source_num].all_columns;
|
||||||
source.chunk->sort_columns = cursors[source_num].sort_columns;
|
source.chunk->sort_columns = cursors[source_num].sort_columns;
|
||||||
|
@ -59,7 +59,7 @@ void MergingSortedAlgorithm::initialize(Inputs inputs)
|
|||||||
if (!chunk)
|
if (!chunk)
|
||||||
continue;
|
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)
|
if (sorting_queue_strategy == SortingQueueStrategy::Default)
|
||||||
@ -84,7 +84,7 @@ void MergingSortedAlgorithm::consume(Input & input, size_t source_num)
|
|||||||
{
|
{
|
||||||
removeConstAndSparse(input);
|
removeConstAndSparse(input);
|
||||||
current_inputs[source_num].swap(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)
|
if (sorting_queue_strategy == SortingQueueStrategy::Default)
|
||||||
{
|
{
|
||||||
|
@ -394,7 +394,7 @@ void FullMergeJoinCursor::setChunk(Chunk && chunk)
|
|||||||
convertToFullIfSparse(chunk);
|
convertToFullIfSparse(chunk);
|
||||||
|
|
||||||
current_chunk = std::move(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
|
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
|
/// Convert to full column, because some cursors expect non-contant columns
|
||||||
convertToFullIfConst(chunk);
|
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;
|
has_collation |= cursors.back().has_collation;
|
||||||
|
|
||||||
nonempty_chunks.emplace_back(std::move(chunk));
|
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