Skip to content

draft: row group base parallel scan for iceberg-datafusion #1610

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

Draft
wants to merge 4 commits into
base: main
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 2 additions & 4 deletions crates/iceberg/src/arrow/delete_filter.rs
Original file line number Diff line number Diff line change
Expand Up @@ -330,8 +330,7 @@ pub(crate) mod tests {

let file_scan_tasks = vec![
FileScanTask {
start: 0,
length: 0,
file_range: None,
record_count: None,
data_file_path: format!("{}/1.parquet", table_location.to_str().unwrap()),
data_file_format: DataFileFormat::Parquet,
Expand All @@ -341,8 +340,7 @@ pub(crate) mod tests {
deletes: vec![pos_del_1, pos_del_2.clone()],
},
FileScanTask {
start: 0,
length: 0,
file_range: None,
record_count: None,
data_file_path: format!("{}/2.parquet", table_location.to_str().unwrap()),
data_file_format: DataFileFormat::Parquet,
Expand Down
161 changes: 161 additions & 0 deletions crates/iceberg/src/arrow/group_pruner.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,161 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

use std::collections::{HashMap, HashSet};
use std::sync::Arc;

use futures::{TryStreamExt, try_join};
use parquet::arrow::ParquetRecordBatchStreamBuilder;
use parquet::arrow::arrow_reader::ArrowReaderOptions;
use parquet::file::metadata::ParquetMetaData;
use parquet::schema::types::SchemaDescriptor;

use crate::Result;
use crate::arrow::{ArrowFileReader, CollectFieldIdVisitor, build_field_id_map};
use crate::expr::BoundPredicate;
use crate::expr::visitors::bound_predicate_visitor::visit;
use crate::expr::visitors::row_group_metrics_evaluator::RowGroupMetricsEvaluator;
use crate::io::FileIO;
use crate::scan::{FileScanGroup, FileScanTask, FileScanTaskStream, ParquetFileScanGroup};
use crate::spec::DataFileFormat;

/// Prune the row groups of the parquet files based on the predicate
#[derive(Clone)]
pub struct ParquetGroupPruner {
file_io: FileIO,
}

impl ParquetGroupPruner {
/// Create a new ParquetGroupPruner
pub fn new(file_io: FileIO) -> Self {
ParquetGroupPruner { file_io }
}

fn build_field_id_set_and_map(
parquet_schema: &SchemaDescriptor,
predicate: &BoundPredicate,
) -> Result<(HashSet<i32>, HashMap<i32, usize>)> {
// Collects all Iceberg field IDs referenced in the filter predicate
let mut collector = CollectFieldIdVisitor {
field_ids: HashSet::default(),
};
visit(&mut collector, predicate)?;

let iceberg_field_ids = collector.field_ids();
let field_id_map = build_field_id_map(parquet_schema)?;

Ok((iceberg_field_ids, field_id_map))
}

fn get_selected_row_group_indices(
&self,
task: &FileScanTask,
parquet_schema: &SchemaDescriptor,
parquet_metadata: &Arc<ParquetMetaData>,
) -> Result<Vec<usize>> {
if let Some(predicate) = &task.predicate {
let (_iceberg_field_ids, field_id_map) =
Self::build_field_id_set_and_map(parquet_schema, predicate)?;

let row_groups_metadata = parquet_metadata.row_groups();
let mut results = Vec::with_capacity(parquet_metadata.row_groups().len());

for (idx, row_group_metadata) in row_groups_metadata.iter().enumerate() {
if RowGroupMetricsEvaluator::eval(
predicate,
row_group_metadata,
&field_id_map,
&task.schema,
)? {
results.push(idx);
}
}

Ok(results)
} else {
Ok((0..parquet_metadata.num_row_groups()).collect())
}
}

/// Prune the row groups of the parquet files based on the predicate
pub async fn prune(&self, mut task: FileScanTask) -> Result<FileScanTask> {
// Create stream reader for parquet file meta
let parquet_file = self.file_io.new_input(&task.data_file_path)?;
let (parquet_metadata, parquet_reader) =
try_join!(parquet_file.metadata(), parquet_file.reader())?;
let parquet_file_reader = ArrowFileReader::new(parquet_metadata, parquet_reader)
.with_preload_column_index(true)
.with_preload_offset_index(true)
.with_preload_page_index(false);
let record_batch_stream_builder = ParquetRecordBatchStreamBuilder::new_with_options(
parquet_file_reader,
ArrowReaderOptions::new(),
)
.await?;

let row_group_indices = self.get_selected_row_group_indices(
&task,
record_batch_stream_builder.parquet_schema(),
record_batch_stream_builder.metadata(),
)?;

task.file_range = Some(FileScanGroup::Parquet(ParquetFileScanGroup {
row_group_indexes: row_group_indices,
}));

Ok(task)
}
}

/// Prune the row groups of the parquet files based on the predicate
pub struct GroupPruner {
parquet_pruner: ParquetGroupPruner,
}

impl GroupPruner {
/// Create a new GroupPruner
pub fn new(file_io: FileIO) -> Self {
GroupPruner {
parquet_pruner: ParquetGroupPruner::new(file_io),
}
}

/// Prune the groups of the files task based on the info of the file scan task.(E.g. predicate)
pub fn prune(&self, file_tasks_stream: FileScanTaskStream) -> FileScanTaskStream {
let parquet_pruner = self.parquet_pruner.clone();

let pruned_stream = file_tasks_stream
.map_ok(move |task| {
let parquet_pruner = parquet_pruner.clone();
async move {
match task.data_file_format {
DataFileFormat::Parquet => {
// Use the parquet pruner to prune row groups
parquet_pruner.prune(task).await
}
DataFileFormat::Avro | DataFileFormat::Orc | DataFileFormat::Puffin => {
// For other formats, return the task unchanged as no pruning is supported
Ok(task)
}
}
}
})
.try_buffer_unordered(10); // Process up to 10 tasks concurrently

Box::pin(pruned_stream)
}
}
3 changes: 3 additions & 0 deletions crates/iceberg/src/arrow/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -34,3 +34,6 @@ mod value;

pub use reader::*;
pub use value::*;

mod group_pruner;
pub use group_pruner::*;
40 changes: 30 additions & 10 deletions crates/iceberg/src/arrow/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ use crate::expr::visitors::page_index_evaluator::PageIndexEvaluator;
use crate::expr::visitors::row_group_metrics_evaluator::RowGroupMetricsEvaluator;
use crate::expr::{BoundPredicate, BoundReference};
use crate::io::{FileIO, FileMetadata, FileRead};
use crate::scan::{ArrowRecordBatchStream, FileScanTask, FileScanTaskStream};
use crate::scan::{ArrowRecordBatchStream, FileScanGroup, FileScanTask, FileScanTaskStream};
use crate::spec::{Datum, NestedField, PrimitiveType, Schema, Type};
use crate::utils::available_parallelism;
use crate::{Error, ErrorKind};
Expand All @@ -68,7 +68,7 @@ pub struct ArrowReaderBuilder {

impl ArrowReaderBuilder {
/// Create a new ArrowReaderBuilder
pub(crate) fn new(file_io: FileIO) -> Self {
pub fn new(file_io: FileIO) -> Self {
let num_cpus = available_parallelism().get();

ArrowReaderBuilder {
Expand Down Expand Up @@ -238,7 +238,10 @@ impl ArrowReader {
// equality delete files OR (there is a scan predicate AND row_selection_enabled),
// since the only implemented method of applying positional deletes is
// by using a `RowSelection`.
let mut selected_row_group_indices = None;
let mut selected_row_group_indices = task.file_range.as_ref().map(|file_range| {
let FileScanGroup::Parquet(parquet_file_scan_range) = file_range;
parquet_file_scan_range.row_group_indexes.clone()
});
let mut row_selection = None;

if let Some(predicate) = final_predicate {
Expand All @@ -261,6 +264,7 @@ impl ArrowReader {
record_batch_stream_builder.metadata(),
&field_id_map,
&task.schema,
&selected_row_group_indices,
)?;

selected_row_group_indices = Some(result);
Expand Down Expand Up @@ -649,8 +653,25 @@ impl ArrowReader {
parquet_metadata: &Arc<ParquetMetaData>,
field_id_map: &HashMap<i32, usize>,
snapshot_schema: &Schema,
selected_row_groups: &Option<Vec<usize>>,
) -> Result<Vec<usize>> {
let row_groups_metadata = parquet_metadata.row_groups();
let row_groups_metadata = if let Some(selected_row_groups) = selected_row_groups {
parquet_metadata
.row_groups()
.iter()
.enumerate()
.filter_map(|(idx, row_group_metadata)| {
if selected_row_groups.contains(&idx) {
Some(row_group_metadata)
} else {
None
}
})
.collect::<Vec<_>>()
} else {
parquet_metadata.row_groups().iter().collect::<Vec<_>>()
};

let mut results = Vec::with_capacity(row_groups_metadata.len());

for (idx, row_group_metadata) in row_groups_metadata.iter().enumerate() {
Expand Down Expand Up @@ -730,7 +751,7 @@ impl ArrowReader {
}

/// Build the map of parquet field id to Parquet column index in the schema.
fn build_field_id_map(parquet_schema: &SchemaDescriptor) -> Result<HashMap<i32, usize>> {
pub fn build_field_id_map(parquet_schema: &SchemaDescriptor) -> Result<HashMap<i32, usize>> {
let mut column_map = HashMap::new();
for (idx, field) in parquet_schema.columns().iter().enumerate() {
let field_type = field.self_type();
Expand Down Expand Up @@ -765,12 +786,12 @@ fn build_field_id_map(parquet_schema: &SchemaDescriptor) -> Result<HashMap<i32,
}

/// A visitor to collect field ids from bound predicates.
struct CollectFieldIdVisitor {
field_ids: HashSet<i32>,
pub(crate) struct CollectFieldIdVisitor {
pub field_ids: HashSet<i32>,
}

impl CollectFieldIdVisitor {
fn field_ids(self) -> HashSet<i32> {
pub fn field_ids(self) -> HashSet<i32> {
self.field_ids
}
}
Expand Down Expand Up @@ -1736,8 +1757,7 @@ message schema {
) -> Vec<Option<String>> {
let tasks = Box::pin(futures::stream::iter(
vec![Ok(FileScanTask {
start: 0,
length: 0,
file_range: None,
record_count: None,
data_file_path: format!("{}/1.parquet", table_location),
data_file_format: DataFileFormat::Parquet,
Expand Down
3 changes: 1 addition & 2 deletions crates/iceberg/src/scan/context.rs
Original file line number Diff line number Diff line change
Expand Up @@ -114,8 +114,7 @@ impl ManifestEntryContext {
.await;

Ok(FileScanTask {
start: 0,
length: self.manifest_entry.file_size_in_bytes(),
file_range: None,
record_count: Some(self.manifest_entry.record_count()),

data_file_path: self.manifest_entry.file_path().to_string(),
Expand Down
8 changes: 2 additions & 6 deletions crates/iceberg/src/scan/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1757,8 +1757,6 @@ pub mod tests {
let deserialized: FileScanTask = serde_json::from_str(&serialized).unwrap();

assert_eq!(task.data_file_path, deserialized.data_file_path);
assert_eq!(task.start, deserialized.start);
assert_eq!(task.length, deserialized.length);
assert_eq!(task.project_field_ids, deserialized.project_field_ids);
assert_eq!(task.predicate, deserialized.predicate);
assert_eq!(task.schema, deserialized.schema);
Expand All @@ -1777,8 +1775,7 @@ pub mod tests {
);
let task = FileScanTask {
data_file_path: "data_file_path".to_string(),
start: 0,
length: 100,
file_range: None,
project_field_ids: vec![1, 2, 3],
predicate: None,
schema: schema.clone(),
Expand All @@ -1791,8 +1788,7 @@ pub mod tests {
// with predicate
let task = FileScanTask {
data_file_path: "data_file_path".to_string(),
start: 0,
length: 100,
file_range: None,
project_field_ids: vec![1, 2, 3],
predicate: Some(BoundPredicate::AlwaysTrue),
schema,
Expand Down
Loading
Loading