mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge pull request #31888 from ClickHouse/fix-31114
Remove LowCardinality from window function args.
This commit is contained in:
commit
71df622b1f
@ -4,10 +4,12 @@
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <base/arithmeticOverflow.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
|
||||
@ -1005,6 +1007,12 @@ static void assertSameColumns(const Columns & left_all,
|
||||
assert(left_column);
|
||||
assert(right_column);
|
||||
|
||||
if (const auto * left_lc = typeid_cast<const ColumnLowCardinality *>(left_column))
|
||||
left_column = left_lc->getDictionary().getNestedColumn().get();
|
||||
|
||||
if (const auto * right_lc = typeid_cast<const ColumnLowCardinality *>(right_column))
|
||||
right_column = right_lc->getDictionary().getNestedColumn().get();
|
||||
|
||||
assert(typeid(*left_column).hash_code()
|
||||
== typeid(*right_column).hash_code());
|
||||
|
||||
@ -1056,10 +1064,13 @@ void WindowTransform::appendChunk(Chunk & chunk)
|
||||
// Another problem with Const columns is that the aggregate functions
|
||||
// can't work with them, so we have to materialize them like the
|
||||
// Aggregator does.
|
||||
// Likewise, aggregate functions can't work with LowCardinality,
|
||||
// so we have to materialize them too.
|
||||
// Just materialize everything.
|
||||
auto columns = chunk.detachColumns();
|
||||
block.original_input_columns = columns;
|
||||
for (auto & column : columns)
|
||||
column = std::move(column)->convertToFullColumnIfConst();
|
||||
column = recursiveRemoveLowCardinality(std::move(column)->convertToFullColumnIfConst());
|
||||
block.input_columns = std::move(columns);
|
||||
|
||||
// Initialize output columns.
|
||||
@ -1302,7 +1313,7 @@ IProcessor::Status WindowTransform::prepare()
|
||||
// Output the ready block.
|
||||
const auto i = next_output_block_number - first_block_number;
|
||||
auto & block = blocks[i];
|
||||
auto columns = block.input_columns;
|
||||
auto columns = block.original_input_columns;
|
||||
for (auto & res : block.output_columns)
|
||||
{
|
||||
columns.push_back(ColumnPtr(std::move(res)));
|
||||
|
@ -39,6 +39,7 @@ struct WindowFunctionWorkspace
|
||||
|
||||
struct WindowTransformBlock
|
||||
{
|
||||
Columns original_input_columns;
|
||||
Columns input_columns;
|
||||
MutableColumns output_columns;
|
||||
|
||||
|
@ -0,0 +1,13 @@
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
a\0aa 1
|
||||
a\0aa 1
|
||||
a\0aa 1
|
38
tests/queries/0_stateless/02126_lc_window_functions.sql
Normal file
38
tests/queries/0_stateless/02126_lc_window_functions.sql
Normal file
@ -0,0 +1,38 @@
|
||||
SELECT max(id) OVER () AS aid
|
||||
FROM
|
||||
(
|
||||
SELECT materialize(toLowCardinality('aaaa')) AS id
|
||||
FROM numbers_mt(1000000)
|
||||
)
|
||||
FORMAT `Null`;
|
||||
|
||||
SELECT max(id) OVER (PARTITION BY id) AS id
|
||||
FROM
|
||||
(
|
||||
SELECT materialize('aaaa') AS id
|
||||
FROM numbers_mt(1000000)
|
||||
)
|
||||
FORMAT `Null`;
|
||||
|
||||
SELECT countIf(sym = 'Red') OVER () AS res
|
||||
FROM
|
||||
(
|
||||
SELECT CAST(CAST(number % 5, 'Enum8(\'Red\' = 0, \'Blue\' = 1, \'Yellow\' = 2, \'Black\' = 3, \'White\' = 4)'), 'LowCardinality(String)') AS sym
|
||||
FROM numbers(10)
|
||||
);
|
||||
|
||||
SELECT materialize(toLowCardinality('a\0aa')), countIf(toLowCardinality('aaaaaaa\0aaaaaaa\0aaaaaaa\0aaaaaaa\0aaaaaaa\0aaaaaaa\0aaaaaaa\0aaaaaaa\0aaaaaaa\0aaaaaaa\0aaaaaaa\0aaaaaaa\0aaaaaaa\0aaaaaaa\0aaaaaaa\0aaaaaaa\0'), sym = 'Red') OVER (Range BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS res FROM (SELECT CAST(CAST(number % 5, 'Enum8(\'Red\' = 0, \'Blue\' = 1, \'Yellow\' = 2, \'Black\' = 3, \'White\' = 4)'), 'LowCardinality(String)') AS sym FROM numbers(3));
|
||||
|
||||
SELECT
|
||||
NULL,
|
||||
id,
|
||||
max(id) OVER (Rows BETWEEN 10 PRECEDING AND UNBOUNDED FOLLOWING) AS aid
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
NULL,
|
||||
max(id) OVER (),
|
||||
materialize(toLowCardinality('')) AS id
|
||||
FROM numbers_mt(0, 1)
|
||||
)
|
||||
FORMAT `Null`;
|
Loading…
Reference in New Issue
Block a user