|
| 1 | +// Licensed to the Apache Software Foundation (ASF) under one |
| 2 | +// or more contributor license agreements. See the NOTICE file |
| 3 | +// distributed with this work for additional information |
| 4 | +// regarding copyright ownership. The ASF licenses this file |
| 5 | +// to you under the Apache License, Version 2.0 (the |
| 6 | +// "License"); you may not use this file except in compliance |
| 7 | +// with the License. You may obtain a copy of the License at |
| 8 | +// |
| 9 | +// http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | +// |
| 11 | +// Unless required by applicable law or agreed to in writing, |
| 12 | +// software distributed under the License is distributed on an |
| 13 | +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 14 | +// KIND, either express or implied. See the License for the |
| 15 | +// specific language governing permissions and limitations |
| 16 | +// under the License. |
| 17 | + |
| 18 | +//! Parquet file data reader |
| 19 | +
|
| 20 | +use arrow_schema::SchemaRef as ArrowSchemaRef; |
| 21 | +use async_stream::try_stream; |
| 22 | +use futures::stream::StreamExt; |
| 23 | +use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask, PARQUET_FIELD_ID_META_KEY}; |
| 24 | +use parquet::schema::types::SchemaDescriptor; |
| 25 | +use std::collections::HashMap; |
| 26 | +use std::str::FromStr; |
| 27 | + |
| 28 | +use crate::arrow::arrow_schema_to_schema; |
| 29 | +use crate::io::FileIO; |
| 30 | +use crate::scan::{ArrowRecordBatchStream, FileScanTaskStream}; |
| 31 | +use crate::spec::SchemaRef; |
| 32 | +use crate::{Error, ErrorKind}; |
| 33 | + |
| 34 | +/// Builder to create ArrowReader |
| 35 | +pub struct ArrowReaderBuilder { |
| 36 | + batch_size: Option<usize>, |
| 37 | + field_ids: Vec<usize>, |
| 38 | + file_io: FileIO, |
| 39 | + schema: SchemaRef, |
| 40 | +} |
| 41 | + |
| 42 | +impl ArrowReaderBuilder { |
| 43 | + /// Create a new ArrowReaderBuilder |
| 44 | + pub fn new(file_io: FileIO, schema: SchemaRef) -> Self { |
| 45 | + ArrowReaderBuilder { |
| 46 | + batch_size: None, |
| 47 | + field_ids: vec![], |
| 48 | + file_io, |
| 49 | + schema, |
| 50 | + } |
| 51 | + } |
| 52 | + |
| 53 | + /// Sets the desired size of batches in the response |
| 54 | + /// to something other than the default |
| 55 | + pub fn with_batch_size(mut self, batch_size: usize) -> Self { |
| 56 | + self.batch_size = Some(batch_size); |
| 57 | + self |
| 58 | + } |
| 59 | + |
| 60 | + /// Sets the desired column projection with a list of field ids. |
| 61 | + pub fn with_field_ids(mut self, field_ids: impl IntoIterator<Item = usize>) -> Self { |
| 62 | + self.field_ids = field_ids.into_iter().collect(); |
| 63 | + self |
| 64 | + } |
| 65 | + |
| 66 | + /// Build the ArrowReader. |
| 67 | + pub fn build(self) -> ArrowReader { |
| 68 | + ArrowReader { |
| 69 | + batch_size: self.batch_size, |
| 70 | + field_ids: self.field_ids, |
| 71 | + schema: self.schema, |
| 72 | + file_io: self.file_io, |
| 73 | + } |
| 74 | + } |
| 75 | +} |
| 76 | + |
| 77 | +/// Reads data from Parquet files |
| 78 | +pub struct ArrowReader { |
| 79 | + batch_size: Option<usize>, |
| 80 | + field_ids: Vec<usize>, |
| 81 | + #[allow(dead_code)] |
| 82 | + schema: SchemaRef, |
| 83 | + file_io: FileIO, |
| 84 | +} |
| 85 | + |
| 86 | +impl ArrowReader { |
| 87 | + /// Take a stream of FileScanTasks and reads all the files. |
| 88 | + /// Returns a stream of Arrow RecordBatches containing the data from the files |
| 89 | + pub fn read(self, mut tasks: FileScanTaskStream) -> crate::Result<ArrowRecordBatchStream> { |
| 90 | + let file_io = self.file_io.clone(); |
| 91 | + |
| 92 | + Ok(try_stream! { |
| 93 | + while let Some(Ok(task)) = tasks.next().await { |
| 94 | + let parquet_reader = file_io |
| 95 | + .new_input(task.data().data_file().file_path())? |
| 96 | + .reader() |
| 97 | + .await?; |
| 98 | + |
| 99 | + let mut batch_stream_builder = ParquetRecordBatchStreamBuilder::new(parquet_reader) |
| 100 | + .await?; |
| 101 | + |
| 102 | + let parquet_schema = batch_stream_builder.parquet_schema(); |
| 103 | + let arrow_schema = batch_stream_builder.schema(); |
| 104 | + let projection_mask = self.get_arrow_projection_mask(parquet_schema, arrow_schema)?; |
| 105 | + batch_stream_builder = batch_stream_builder.with_projection(projection_mask); |
| 106 | + |
| 107 | + if let Some(batch_size) = self.batch_size { |
| 108 | + batch_stream_builder = batch_stream_builder.with_batch_size(batch_size); |
| 109 | + } |
| 110 | + |
| 111 | + let mut batch_stream = batch_stream_builder.build()?; |
| 112 | + |
| 113 | + while let Some(batch) = batch_stream.next().await { |
| 114 | + yield batch?; |
| 115 | + } |
| 116 | + } |
| 117 | + } |
| 118 | + .boxed()) |
| 119 | + } |
| 120 | + |
| 121 | + fn get_arrow_projection_mask( |
| 122 | + &self, |
| 123 | + parquet_schema: &SchemaDescriptor, |
| 124 | + arrow_schema: &ArrowSchemaRef, |
| 125 | + ) -> crate::Result<ProjectionMask> { |
| 126 | + if self.field_ids.is_empty() { |
| 127 | + Ok(ProjectionMask::all()) |
| 128 | + } else { |
| 129 | + // Build the map between field id and column index in Parquet schema. |
| 130 | + let mut column_map = HashMap::new(); |
| 131 | + |
| 132 | + let fields = arrow_schema.fields(); |
| 133 | + let iceberg_schema = arrow_schema_to_schema(arrow_schema)?; |
| 134 | + fields.filter_leaves(|idx, field| { |
| 135 | + let field_id = field.metadata().get(PARQUET_FIELD_ID_META_KEY); |
| 136 | + if field_id.is_none() { |
| 137 | + return false; |
| 138 | + } |
| 139 | + |
| 140 | + let field_id = i32::from_str(field_id.unwrap()); |
| 141 | + if field_id.is_err() { |
| 142 | + return false; |
| 143 | + } |
| 144 | + let field_id = field_id.unwrap(); |
| 145 | + |
| 146 | + if !self.field_ids.contains(&(field_id as usize)) { |
| 147 | + return false; |
| 148 | + } |
| 149 | + |
| 150 | + let iceberg_field = self.schema.field_by_id(field_id); |
| 151 | + let parquet_iceberg_field = iceberg_schema.field_by_id(field_id); |
| 152 | + |
| 153 | + if iceberg_field.is_none() || parquet_iceberg_field.is_none() { |
| 154 | + return false; |
| 155 | + } |
| 156 | + |
| 157 | + if iceberg_field.unwrap().field_type != parquet_iceberg_field.unwrap().field_type { |
| 158 | + return false; |
| 159 | + } |
| 160 | + |
| 161 | + column_map.insert(field_id, idx); |
| 162 | + true |
| 163 | + }); |
| 164 | + |
| 165 | + if column_map.len() != self.field_ids.len() { |
| 166 | + return Err(Error::new( |
| 167 | + ErrorKind::DataInvalid, |
| 168 | + format!( |
| 169 | + "Parquet schema {} and Iceberg schema {} do not match.", |
| 170 | + iceberg_schema, self.schema |
| 171 | + ), |
| 172 | + )); |
| 173 | + } |
| 174 | + |
| 175 | + let mut indices = vec![]; |
| 176 | + for field_id in &self.field_ids { |
| 177 | + if let Some(col_idx) = column_map.get(&(*field_id as i32)) { |
| 178 | + indices.push(*col_idx); |
| 179 | + } else { |
| 180 | + return Err(Error::new( |
| 181 | + ErrorKind::DataInvalid, |
| 182 | + format!("Field {} is not found in Parquet schema.", field_id), |
| 183 | + )); |
| 184 | + } |
| 185 | + } |
| 186 | + Ok(ProjectionMask::leaves(parquet_schema, indices)) |
| 187 | + } |
| 188 | + } |
| 189 | +} |
0 commit comments