Skip to content

Commit fee5abf

Browse files
committed
Comment and refactor
1 parent 1ee086c commit fee5abf

File tree

2 files changed

+8
-5
lines changed

2 files changed

+8
-5
lines changed

crates/iceberg/src/scan.rs

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -364,9 +364,8 @@ impl TableScan {
364364

365365
let manifest_list = self.plan_context.get_manifest_list().await?;
366366

367-
// get the [`ManifestFile`]s from the [`ManifestList`], filtering out any
368-
// whose content type is not Data or whose partitions cannot match this
369-
// scan's filter
367+
// get the [`ManifestFile`]s from the [`ManifestList`], filtering out
368+
// partitions cannot match the scan's filter
370369
let manifest_file_contexts = self
371370
.plan_context
372371
.build_manifest_file_contexts(manifest_list, manifest_entry_ctx_tx)?;
@@ -634,7 +633,7 @@ impl PlanContext {
634633
// TODO: Ideally we could ditch this intermediate Vec as we return an iterator.
635634
let mut filtered_mfcs = vec![];
636635
if self.predicate.is_some() {
637-
for manifest_file in entries.iter() {
636+
for manifest_file in entries {
638637
let partition_bound_predicate = self.get_partition_filter(manifest_file)?;
639638

640639
// evaluate the ManifestFile against the partition filter. Skip
@@ -656,7 +655,7 @@ impl PlanContext {
656655
}
657656
}
658657
} else {
659-
for manifest_file in entries.iter() {
658+
for manifest_file in entries {
660659
let mfc = self.create_manifest_file_context(manifest_file, None, sender.clone());
661660
filtered_mfcs.push(Ok(mfc));
662661
}

crates/integration_tests/testdata/spark/provision.py

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,10 @@
1818
from pyspark.sql import SparkSession
1919
from pyspark.sql.functions import current_date, date_add, expr
2020

21+
# The configuration is important, otherwise we get many small
22+
# parquet files with a single row. When a positional delete
23+
# hits the Parquet file with one row, the parquet file gets
24+
# dropped instead of having a merge-on-read delete file.
2125
spark = (
2226
SparkSession
2327
.builder

0 commit comments

Comments
 (0)