Skip to content

Commit

Permalink
Comment and refactor
Browse files Browse the repository at this point in the history
  • Loading branch information
Fokko committed Dec 20, 2024
1 parent 1ee086c commit fee5abf
Show file tree
Hide file tree
Showing 2 changed files with 8 additions and 5 deletions.
9 changes: 4 additions & 5 deletions crates/iceberg/src/scan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -364,9 +364,8 @@ impl TableScan {

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

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

// evaluate the ManifestFile against the partition filter. Skip
Expand All @@ -656,7 +655,7 @@ impl PlanContext {
}
}
} else {
for manifest_file in entries.iter() {
for manifest_file in entries {
let mfc = self.create_manifest_file_context(manifest_file, None, sender.clone());
filtered_mfcs.push(Ok(mfc));
}
Expand Down
4 changes: 4 additions & 0 deletions crates/integration_tests/testdata/spark/provision.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,10 @@
from pyspark.sql import SparkSession
from pyspark.sql.functions import current_date, date_add, expr

# The configuration 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.
spark = (
SparkSession
.builder
Expand Down

0 comments on commit fee5abf

Please sign in to comment.