mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 18:12:02 +00:00
Query Cache: Try to fix bad cast from ColumnConst to ColumnVector<char8_t>
Resolves #49445 The query cache buffers query result chunks and eventually squashes them before insertion into the cache. Here, squashing failed because not all chunks were of the same type. Looks like chunks of the same underlying type (e.g. UInt8) in a query result be of mixed const, sparse or low-cardinality type. Fix this by always materializing the data regardless of the compression. Strange thing is that the failing query in the stress test (*) isn't able to reproduce the bug, and I haven't managed to trigger the issue otherwise, so no test case is added. (*) SELECT 1 UNION ALL SELECT 1 INTERSECT SELECT 1 E.g. here: https://s3.amazonaws.com/clickhouse-test-reports/0/18817517ed6f8849e3d979e10fbb273e0edf0eaa/stress_test__debug_/fatal_messages.txt
This commit is contained in:
parent
b7ef782335
commit
a1c131c0da
@ -263,23 +263,23 @@ void QueryCache::Writer::finalizeWrite()
|
||||
|
||||
if (auto entry = cache.getWithKey(key); entry.has_value() && !IsStale()(entry->key))
|
||||
{
|
||||
/// same check as in ctor because a parallel Writer could have inserted the current key in the meantime
|
||||
/// Same check as in ctor because a parallel Writer could have inserted the current key in the meantime
|
||||
LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.queryStringFromAst());
|
||||
return;
|
||||
}
|
||||
|
||||
if (squash_partial_results)
|
||||
{
|
||||
// Squash partial result chunks to chunks of size 'max_block_size' each. This costs some performance but provides a more natural
|
||||
// compression of neither too small nor big blocks. Also, it will look like 'max_block_size' is respected when the query result is
|
||||
// served later on from the query cache.
|
||||
/// Squash partial result chunks to chunks of size 'max_block_size' each. This costs some performance but provides a more natural
|
||||
/// compression of neither too small nor big blocks. Also, it will look like 'max_block_size' is respected when the query result is
|
||||
/// served later on from the query cache.
|
||||
|
||||
Chunks squashed_chunks;
|
||||
size_t rows_remaining_in_squashed = 0; /// how many further rows can the last squashed chunk consume until it reaches max_block_size
|
||||
|
||||
for (auto & chunk : query_result->chunks)
|
||||
{
|
||||
convertToFullIfSparse(chunk);
|
||||
convertToFullIfNeeded(chunk);
|
||||
|
||||
const size_t rows_chunk = chunk.getNumRows();
|
||||
if (rows_chunk == 0)
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Columns/ColumnSparse.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -202,6 +203,24 @@ const ChunkMissingValues::RowsBitMask & ChunkMissingValues::getDefaultsBitmask(s
|
||||
return none;
|
||||
}
|
||||
|
||||
void convertToFullIfConst(Chunk & chunk)
|
||||
{
|
||||
size_t num_rows = chunk.getNumRows();
|
||||
auto columns = chunk.detachColumns();
|
||||
for (auto & column : columns)
|
||||
column = column->convertToFullColumnIfConst();
|
||||
chunk.setColumns(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
void convertToFullIfLowCardinality(Chunk & chunk)
|
||||
{
|
||||
size_t num_rows = chunk.getNumRows();
|
||||
auto columns = chunk.detachColumns();
|
||||
for (auto & column : columns)
|
||||
column = recursiveRemoveLowCardinality(column);
|
||||
chunk.setColumns(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
void convertToFullIfSparse(Chunk & chunk)
|
||||
{
|
||||
size_t num_rows = chunk.getNumRows();
|
||||
@ -212,4 +231,11 @@ void convertToFullIfSparse(Chunk & chunk)
|
||||
chunk.setColumns(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
void convertToFullIfNeeded(Chunk & chunk)
|
||||
{
|
||||
convertToFullIfSparse(chunk);
|
||||
convertToFullIfConst(chunk);
|
||||
convertToFullIfLowCardinality(chunk);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -149,6 +149,9 @@ private:
|
||||
/// It's needed, when you have to access to the internals of the column,
|
||||
/// or when you need to perform operation with two columns
|
||||
/// and their structure must be equal (e.g. compareAt).
|
||||
void convertToFullIfConst(Chunk & chunk);
|
||||
void convertToFullIfLowCardinality(Chunk & chunk);
|
||||
void convertToFullIfSparse(Chunk & chunk);
|
||||
void convertToFullIfNeeded(Chunk & chunk);
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user