Skip to content

Conversation

@psvri
Copy link

@psvri psvri commented Jan 17, 2026

Closes #15069

I made changes in this PR Based on the iceberg spec -NaN < -Infinity < -value < -0 < 0 < value < Infinity < NaN .

When we have nanValueCount > 0 in dataFile, we make hasValue fn return false. This in turn will make Aggregator.isValid return false there by on spark side we wont push down aggregation.

sql(
"INSERT INTO %s VALUES (1, float('nan')),"
+ "(1, float('nan')), "
+ "(1, 10.0), "
Copy link
Author

Choose a reason for hiding this comment

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

The bug is replicated with this change in test class.

Once this approach is okay, I shall update the testClasses in other spark versions.

Copy link
Member

Choose a reason for hiding this comment

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

Could you explain why the bug is triggered with the addition of this row?

Copy link
Author

Choose a reason for hiding this comment

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

Basically the bug is triggered when we have data file containing nan count, upper bound and lower bound.

Previously without that line only nan count is created, with this change upper and lower bound is generated.

Copy link
Contributor

Choose a reason for hiding this comment

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

Interesting, it's a bit unclear to me why the additional row would trigger collecting lower/upper bounds. I'd have to double check if there's some minimum threshold of rows or some other condition that controls whether lower/upper is written when the footer is written. Looking at this test without the change I would've expected a lower/upper bounds of 1.0 and 2.0 respectively.

Copy link
Contributor

Choose a reason for hiding this comment

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

the floating-point nature of the 10.0 value?
Seems some unrelated bug of its own.

@psvri psvri changed the title Fix aggregate pushdown Spark: Fix aggregate pushdown Jan 17, 2026
Long nanCount = safeGet(file.nanValueCounts(), fieldId);
if (nanCount != null && nanCount > 0) {
return false;
}
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: Can we extract a small helper (e.g., hasNaNs) to keep this logic in one place?

@amogh-jahagirdar amogh-jahagirdar added this to the Iceberg 1.11.0 milestone Jan 30, 2026
@amogh-jahagirdar
Copy link
Contributor

cc @psvri @RussellSpitzer @huaxingao I went ahead and added this to the 1.11 milestone since it does look like a correctness issue when there are NaNs. I'm stepping through the debugger why the existing NaN test didn't really catch the problem.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Spark aggregate pushdown gives incorrect results when column contains NaN

5 participants