Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

fix: Reading a table with positional deletes should fail #826

Merged
merged 3 commits into from
Dec 20, 2024

Conversation

Fokko
Copy link
Contributor

@Fokko Fokko commented Dec 19, 2024

Here we fail:

if manifest_entry_context.manifest_entry.content_type() != DataContentType::Data {
return Err(Error::new(
ErrorKind::FeatureUnsupported,
"Only Data files currently supported",
));
}

But then the manifest is already filtered:

.filter(|manifest_file| manifest_file.content == ManifestContentType::Data);

So it is unreachable, and it allows to return invalid data. Instead we should raise an error.

As always, don't hold back on my rust code :)

@Fokko Fokko force-pushed the fd-positional-deletes branch from a5570ea to fe2c359 Compare December 19, 2024 17:18
@Fokko Fokko force-pushed the fd-positional-deletes branch from fe2c359 to 87bb216 Compare December 19, 2024 17:36
spark = (
SparkSession
.builder
.config("spark.sql.shuffle.partitions", "1")
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is important, otherwise we get many small parquet files with a single row. When a positional delete hits the Parquet file with one row, the parquet file gets dropped instead of having a merge-on-read delete file.

Choose a reason for hiding this comment

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

nit: add this comment alongside the code itself

also curious why its needed here, we use the same logic in pyiceberg https://github.com/apache/iceberg-python/blob/1278e8880c4767287dc69208ced20bd444c37228/dev/provision.py#L25

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I recall that I ran into the same thing with PyIceberg, but not sure how I fixed it there. Maybe good to add it there as well. Having a lot of partitions doesn't make any sense anyway, and will slow down the tests.

Choose a reason for hiding this comment

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

yep! 2 birds 1 stone :)

apache/iceberg-python#1417 (comment)

Copy link
Contributor

Choose a reason for hiding this comment

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

+1 for adding this comment along code.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I forgot to git add the comment 🤦 It is there now

@Fokko Fokko force-pushed the fd-positional-deletes branch 6 times, most recently from 87e111d to bac9253 Compare December 19, 2024 18:40
@Fokko Fokko force-pushed the fd-positional-deletes branch from bac9253 to 1ee086c Compare December 19, 2024 18:43
kevinjqliu
kevinjqliu previously approved these changes Dec 19, 2024
Copy link

@kevinjqliu kevinjqliu left a comment

Choose a reason for hiding this comment

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

Great catch! Added a few nit comments

spark = (
SparkSession
.builder
.config("spark.sql.shuffle.partitions", "1")

Choose a reason for hiding this comment

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

nit: add this comment alongside the code itself

also curious why its needed here, we use the same logic in pyiceberg https://github.com/apache/iceberg-python/blob/1278e8880c4767287dc69208ced20bd444c37228/dev/provision.py#L25

crates/iceberg/src/scan.rs Show resolved Hide resolved
liurenjie1024
liurenjie1024 previously approved these changes Dec 20, 2024
Copy link
Contributor

@liurenjie1024 liurenjie1024 left a comment

Choose a reason for hiding this comment

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

Thnaks @Fokko for this fix! Just left some nits, others LGTM!

crates/iceberg/src/scan.rs Outdated Show resolved Hide resolved
crates/iceberg/src/scan.rs Outdated Show resolved Hide resolved
spark = (
SparkSession
.builder
.config("spark.sql.shuffle.partitions", "1")
Copy link
Contributor

Choose a reason for hiding this comment

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

+1 for adding this comment along code.

@Fokko Fokko dismissed stale reviews from liurenjie1024 and kevinjqliu via fee5abf December 20, 2024 07:10
Copy link
Member

@Xuanwo Xuanwo left a comment

Choose a reason for hiding this comment

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

Thank you @Fokko to get this fixed, and also thank you @kevinjqliu and @liurenjie1024 for review.

@Xuanwo Xuanwo changed the title Reading a table with positional deletes should fail fix: Reading a table with positional deletes should fail Dec 20, 2024
@Xuanwo Xuanwo marked this pull request as ready for review December 20, 2024 07:21
@Xuanwo Xuanwo merged commit 0777fa7 into apache:main Dec 20, 2024
16 checks passed
@Fokko Fokko deleted the fd-positional-deletes branch December 20, 2024 07:48
@Fokko Fokko added this to the 0.4.0 Release milestone Dec 20, 2024
sungwy pushed a commit that referenced this pull request Dec 20, 2024
* A table with positional deletes shoulds fail

* Add possible fix

* Comment and refactor
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants