Skip to content

Fix aggregate_functions_null_for_empty for non-Nullable result types#99839

Merged
alexey-milovidov merged 2 commits intomasterfrom
fix-aggregate-functions-null-for-empty-non-nullable
Mar 18, 2026
Merged

Fix aggregate_functions_null_for_empty for non-Nullable result types#99839
alexey-milovidov merged 2 commits intomasterfrom
fix-aggregate-functions-null-for-empty-non-nullable

Conversation

@alexey-milovidov
Copy link
Copy Markdown
Member

@alexey-milovidov alexey-milovidov commented Mar 18, 2026

Closes #38738. Closes #38786.

Some aggregate functions (e.g., groupArray, sumMap) return types like Array or Map that cannot be inside Nullable. The aggregate_functions_null_for_empty setting was unconditionally wrapping all result types in Nullable via the OrNull combinator, causing an exception like:

Code: 43. DB::Exception: Nested type Array(UInt64) cannot be inside Nullable type.

Now the OrNull combinator checks canBeInsideNullable before wrapping, and skips the Nullable wrapper for incompatible types, returning the default value (e.g., empty array) instead.

Changelog category (leave one):

  • Bug Fix (user-visible misbehavior in an official stable release)

Changelog entry (a user-readable short description of the changes that goes into CHANGELOG.md):

Fix aggregate_functions_null_for_empty setting to work with aggregate functions returning non-Nullable types such as Array or Map (e.g., groupArray, sumMap).

Documentation entry for user-facing changes

  • Documentation is written (mandatory for new features)

…Nullable types

Some aggregate functions (e.g., `groupArray`, `sumMap`) return types like
Array or Map that cannot be inside Nullable. The `aggregate_functions_null_for_empty`
setting was unconditionally wrapping all result types in Nullable via the
OrNull combinator, causing an exception.

Now the OrNull combinator checks `canBeInsideNullable` before wrapping, and
skips the Nullable wrapper for incompatible types, returning the default value
(e.g., empty array) instead.

Co-Authored-By: Claude Opus 4.6 (1M context) <noreply@anthropic.com>
@clickhouse-gh
Copy link
Copy Markdown
Contributor

clickhouse-gh bot commented Mar 18, 2026

Workflow [PR], commit [778f908]

Summary:


AI Review

Summary

This PR fixes aggregate_functions_null_for_empty for aggregate functions whose result types cannot be wrapped into Nullable (for example Array/Map) by avoiding invalid wrapping in AggregateFunctionOrFill. The change is focused, aligns with the stated bug, and the added stateless coverage exercises both non-nullable-compatible and regular nullable-compatible result paths.

Missing context

  • ⚠️ No CI logs/results were provided in this review context.

ClickHouse Rules

Item Status Notes
Deletion logging
Serialization versioning
Core-area scrutiny
No test removal
Experimental gate
No magic constants
Backward compatibility
SettingsChangesHistory.cpp
Safe rollout
Compilation time

Final Verdict

  • Status: ✅ Approve

@clickhouse-gh clickhouse-gh bot added the pr-bugfix Pull request with bugfix, not backported by default label Mar 18, 2026
…hrowing

The `OrNull` combinator now skips `Nullable` wrapping for types that
cannot be inside `Nullable`, so `topKOrNullState` no longer throws
`ILLEGAL_TYPE_OF_ARGUMENT`.

https://s3.amazonaws.com/clickhouse-test-reports/json.html?PR=99839&sha=7b20176882cb107649b50cc440adc8898e64a3c0&name_0=PR&name_1=Fast%20test

Co-Authored-By: Claude Opus 4.6 (1M context) <noreply@anthropic.com>
Copy link
Copy Markdown
Member Author

@alexey-milovidov alexey-milovidov left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good change, good test.

@qoega qoega self-assigned this Mar 18, 2026
@clickhouse-gh
Copy link
Copy Markdown
Contributor

clickhouse-gh bot commented Mar 18, 2026

LLVM Coverage Report

Metric Baseline Current Δ
Lines 83.70% 83.80% +0.10%
Functions 23.90% 23.90% +0.00%
Branches 76.20% 76.30% +0.10%

PR changed lines: PR changed-lines coverage: 90.00% (18/20, 0 noise lines excluded)
Diff coverage report
Uncovered code

@alexey-milovidov alexey-milovidov added this pull request to the merge queue Mar 18, 2026
Merged via the queue into master with commit f9849ac Mar 18, 2026
163 checks passed
@alexey-milovidov alexey-milovidov deleted the fix-aggregate-functions-null-for-empty-non-nullable branch March 18, 2026 12:07
@robot-ch-test-poll4 robot-ch-test-poll4 added the pr-synced-to-cloud The PR is synced to the cloud repo label Mar 18, 2026
alexey-milovidov added a commit that referenced this pull request Mar 22, 2026
The `transform` function casts WHEN values to the expression type. When
WHEN values are Nullable (e.g. `CAST(NULL AS Nullable(Int32))`) but the
expression is non-Nullable, the cast fails with "Cannot convert NULL
value to non-Nullable type".

This became visible after #99839 made `aggregate_functions_null_for_empty`
produce Nullable result types for more aggregates, causing SQLLogic tests
to hit this path.

Fix by wrapping the entire `transform` attempt (array construction,
build, and execute) in a try-catch that falls through to the `multiIf`
path on type-related errors. The `multiIf` path handles Nullable WHEN
values correctly via `caseWhenEquals`.

https://s3.amazonaws.com/clickhouse-test-reports/json.html?PR=100293&sha=f067c3bec9fe4a7a07e810a27c9587ea9f777bfb&name_0=PR&name_1=SQLLogic%20test

Co-Authored-By: Claude Opus 4.6 (1M context) <noreply@anthropic.com>
alexey-milovidov added a commit that referenced this pull request Mar 22, 2026
The `transform` function casts WHEN values to the expression type. When
WHEN values are Nullable (e.g. `CAST(NULL AS Nullable(Int32))`) but the
expression is non-Nullable, the cast fails with "Cannot convert NULL
value to non-Nullable type".

This became visible after #99839 made `aggregate_functions_null_for_empty`
produce Nullable result types for more aggregates, causing SQLLogic tests
to hit this path.

Fix by wrapping the entire `transform` attempt (array construction,
build, and execute) in a try-catch that falls through to the `multiIf`
path on type-related errors. The `multiIf` path handles Nullable WHEN
values correctly via `caseWhenEquals`.

https://s3.amazonaws.com/clickhouse-test-reports/json.html?PR=100293&sha=f067c3bec9fe4a7a07e810a27c9587ea9f777bfb&name_0=PR&name_1=SQLLogic%20test

Co-Authored-By: Claude Opus 4.6 (1M context) <noreply@anthropic.com>
alexey-milovidov added a commit that referenced this pull request Mar 24, 2026
The `transform` function casts WHEN values to the expression type. When
WHEN values are Nullable (e.g. `CAST(NULL AS Nullable(Int32))`) but the
expression is non-Nullable, the cast fails with "Cannot convert NULL
value to non-Nullable type".

This became visible after #99839 made `aggregate_functions_null_for_empty`
produce Nullable result types for more aggregates, causing SQLLogic tests
to hit this path.

Fix by wrapping the entire `transform` attempt (array construction,
build, and execute) in a try-catch that falls through to the `multiIf`
path on type-related errors. The `multiIf` path handles Nullable WHEN
values correctly via `caseWhenEquals`.

https://s3.amazonaws.com/clickhouse-test-reports/json.html?PR=100293&sha=f067c3bec9fe4a7a07e810a27c9587ea9f777bfb&name_0=PR&name_1=SQLLogic%20test

Co-Authored-By: Claude Opus 4.6 (1M context) <noreply@anthropic.com>
Desel72 pushed a commit to Desel72/ClickHouse that referenced this pull request Mar 30, 2026
The `transform` function casts WHEN values to the expression type. When
WHEN values are Nullable (e.g. `CAST(NULL AS Nullable(Int32))`) but the
expression is non-Nullable, the cast fails with "Cannot convert NULL
value to non-Nullable type".

This became visible after ClickHouse#99839 made `aggregate_functions_null_for_empty`
produce Nullable result types for more aggregates, causing SQLLogic tests
to hit this path.

Fix by wrapping the entire `transform` attempt (array construction,
build, and execute) in a try-catch that falls through to the `multiIf`
path on type-related errors. The `multiIf` path handles Nullable WHEN
values correctly via `caseWhenEquals`.

https://s3.amazonaws.com/clickhouse-test-reports/json.html?PR=100293&sha=f067c3bec9fe4a7a07e810a27c9587ea9f777bfb&name_0=PR&name_1=SQLLogic%20test
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

pr-bugfix Pull request with bugfix, not backported by default pr-synced-to-cloud The PR is synced to the cloud repo

Projects

None yet

3 participants