From 1763a50d37a821299df26a2ededd9ac6097ac654 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 6 Nov 2024 23:09:15 +0000 Subject: [PATCH] Backport #71494 to 24.8: Prevent crash in SortCursor with 0 columns --- src/Core/SortCursor.h | 19 +++++++++++---- .../IMergingAlgorithmWithDelayedChunk.cpp | 9 +++++-- .../IMergingAlgorithmWithSharedChunks.cpp | 5 ++-- .../Algorithms/MergingSortedAlgorithm.cpp | 4 ++-- .../Transforms/MergeJoinTransform.cpp | 2 +- .../Transforms/SortingTransform.cpp | 2 +- .../03261_sort_cursor_crash.reference | 4 ++++ .../0_stateless/03261_sort_cursor_crash.sql | 24 +++++++++++++++++++ 8 files changed, 57 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/03261_sort_cursor_crash.reference create mode 100644 tests/queries/0_stateless/03261_sort_cursor_crash.sql diff --git a/src/Core/SortCursor.h b/src/Core/SortCursor.h index 56e50d3a94f..586a8777c89 100644 --- a/src/Core/SortCursor.h +++ b/src/Core/SortCursor.h @@ -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; } diff --git a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp index cbad6813fbc..5e271e12943 100644 --- a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp @@ -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); diff --git a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp index 47b7ddf38dc..f99f021286e 100644 --- a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp @@ -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; diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp index 3a9cf7ee141..28c6cb473e5 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp @@ -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) { diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 6abfa0fccd0..0f4e8efc448 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -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 diff --git a/src/Processors/Transforms/SortingTransform.cpp b/src/Processors/Transforms/SortingTransform.cpp index 8e7a526864d..b22f10e2030 100644 --- a/src/Processors/Transforms/SortingTransform.cpp +++ b/src/Processors/Transforms/SortingTransform.cpp @@ -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)); diff --git a/tests/queries/0_stateless/03261_sort_cursor_crash.reference b/tests/queries/0_stateless/03261_sort_cursor_crash.reference new file mode 100644 index 00000000000..7299f2f5a5f --- /dev/null +++ b/tests/queries/0_stateless/03261_sort_cursor_crash.reference @@ -0,0 +1,4 @@ +42 +43 +44 +45 diff --git a/tests/queries/0_stateless/03261_sort_cursor_crash.sql b/tests/queries/0_stateless/03261_sort_cursor_crash.sql new file mode 100644 index 00000000000..b659f3d4a92 --- /dev/null +++ b/tests/queries/0_stateless/03261_sort_cursor_crash.sql @@ -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;