Fix possible "No column to rollback" logical error during INSERT into Buffer

In case of concurrent INSERT and SELECT, and memory pressure it is
possible to get this error, if you are very lucky.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
This commit is contained in:
Azat Khuzhin 2022-11-17 17:37:51 +01:00
parent 62d01f3943
commit aa77beabc9
3 changed files with 49 additions and 2 deletions

View File

@ -456,7 +456,24 @@ static void appendBlock(Poco::Logger * log, const Block & from, Block & to)
for (size_t column_no = 0, columns = to.columns(); column_no < columns; ++column_no)
{
const IColumn & col_from = *from.getByPosition(column_no).column.get();
last_col = IColumn::mutate(std::move(to.getByPosition(column_no).column));
{
/// Usually IColumn::mutate() here will simply move pointers,
/// however in case of parallel reading from it via SELECT, it
/// is possible for the full IColumn::clone() here, and in this
/// case it may fail due to MEMORY_LIMIT_EXCEEDED, and this
/// breaks the rollback, since the column got lost, it is
/// neither in last_col nor in "to" block.
///
/// The safest option here, is to do a full clone every time,
/// however, it is overhead. And it looks like the only
/// exception that is possible here is MEMORY_LIMIT_EXCEEDED,
/// and it is better to simply suppress it, to avoid overhead
/// for every INSERT into Buffer (Anyway we have a
/// LOGICAL_ERROR in rollback that will bail if something else
/// will happens here).
LockMemoryExceptionInThread temporarily_ignore_any_memory_limits(VariableContext::Global);
last_col = IColumn::mutate(std::move(to.getByPosition(column_no).column));
}
/// In case of ColumnAggregateFunction aggregate states will
/// be allocated from the query context but can be destroyed from the
@ -468,7 +485,10 @@ static void appendBlock(Poco::Logger * log, const Block & from, Block & to)
last_col->ensureOwnership();
last_col->insertRangeFrom(col_from, 0, rows);
to.getByPosition(column_no).column = std::move(last_col);
{
DENY_ALLOCATIONS_IN_SCOPE;
to.getByPosition(column_no).column = std::move(last_col);
}
}
CurrentMetrics::add(CurrentMetrics::StorageBufferRows, rows);
CurrentMetrics::add(CurrentMetrics::StorageBufferBytes, to.bytes() - old_bytes);

View File

@ -0,0 +1,27 @@
#!/usr/bin/env bash
# Tags: no-fasttest
# Regression test for 'Logical error: No column to rollback' in case of
# exception while commiting batch into the Buffer, see [1].
#
# [1]: https://github.com/ClickHouse/ClickHouse/issues/42740
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_buffer_string"
$CLICKHOUSE_CLIENT -q "CREATE TABLE t_buffer_string(key String) ENGINE = Buffer('', '', 1, 1, 1, 1000000000000, 1000000000000, 1000000000000, 1000000000000)"
# --continue_on_errors -- to ignore possible MEMORY_LIMIT_EXCEEDED errors
# --concurrency -- we need have SELECT and INSERT in parallel to have refcount
# of the column in the Buffer block > 1, that way we will do
# full clone and moving a column may throw.
#
# It reproduces the problem 100% with MemoryTrackerFaultInjectorInThread in the appendBlock()
$CLICKHOUSE_BENCHMARK --randomize --timelimit 10 --continue_on_errors --concurrency 10 >& /dev/null <<EOL
INSERT INTO t_buffer_string SELECT number::String from numbers(10000)
SELECT * FROM t_buffer_string
EOL
$CLICKHOUSE_CLIENT -q "DROP TABLE t_buffer_string"