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

[BUG] Allow for Parquet reading from files with differing schemas #2514

Merged
merged 16 commits into from
Jul 18, 2024
Merged
Show file tree
Hide file tree
Changes from 8 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
9 changes: 7 additions & 2 deletions src/daft-csv/src/read.rs
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,11 @@ fn tables_concat(mut tables: Vec<Table>) -> DaftResult<Table> {
Series::concat(series_to_cat.as_slice())
})
.collect::<DaftResult<Vec<_>>>()?;
Table::new(first_table.schema.clone(), new_series)
Table::new(
first_table.schema.clone(),
new_series,
tables.iter().map(|t| t.len()).sum(),
)
}

async fn read_csv_single_into_table(
Expand Down Expand Up @@ -508,7 +512,8 @@ fn parse_into_column_array_chunk_stream(
)
})
.collect::<DaftResult<Vec<Series>>>()?;
Ok(Table::new_unchecked(read_schema, chunk))
let num_rows = chunk.first().map(|s| s.len()).unwrap_or(0);
Ok(Table::new_unchecked(read_schema, chunk, num_rows))
})();
let _ = send.send(result);
});
Expand Down
1 change: 1 addition & 0 deletions src/daft-execution/src/task/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -243,6 +243,7 @@ mod tests {
.boxed(),
)
.unwrap()],
input_meta.num_rows.unwrap(),
)
.unwrap()]),
None,
Expand Down
1 change: 1 addition & 0 deletions src/daft-execution/src/test/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ pub(crate) fn mock_micropartition(num_rows: usize) -> MicroPartition {
.boxed(),
)
.unwrap()],
num_rows,
)
.unwrap()]),
None,
Expand Down
5 changes: 4 additions & 1 deletion src/daft-json/src/local.rs
Original file line number Diff line number Diff line change
Expand Up @@ -193,6 +193,7 @@ impl<'a> JsonReader<'a> {
})
.collect::<IndexMap<_, _>>();

let mut num_rows = 0;
for record in iter {
let value = record.map_err(|e| super::Error::JsonDeserializationError {
string: e.to_string(),
Expand Down Expand Up @@ -221,6 +222,8 @@ impl<'a> JsonReader<'a> {
.into());
}
}

num_rows += 1;
}
let columns = columns
.into_values()
Expand All @@ -234,7 +237,7 @@ impl<'a> JsonReader<'a> {
})
.collect::<DaftResult<Vec<_>>>()?;

let tbl = Table::new_unchecked(self.schema.clone(), columns);
let tbl = Table::new_unchecked(self.schema.clone(), columns, num_rows);

if let Some(pred) = &self.predicate {
tbl.filter(&[pred.clone()])
Expand Down
13 changes: 11 additions & 2 deletions src/daft-json/src/read.rs
Original file line number Diff line number Diff line change
Expand Up @@ -163,7 +163,11 @@ pub(crate) fn tables_concat(mut tables: Vec<Table>) -> DaftResult<Table> {
Series::concat(series_to_cat.as_slice())
})
.collect::<DaftResult<Vec<_>>>()?;
Table::new(first_table.schema.clone(), new_series)
Table::new(
first_table.schema.clone(),
new_series,
tables.iter().map(|t| t.len()).sum(),
)
}

async fn read_json_single_into_table(
Expand Down Expand Up @@ -424,6 +428,7 @@ fn parse_into_column_array_chunk_stream(
let schema = schema.clone();
let daft_schema = daft_schema.clone();
let daft_fields = daft_fields.clone();
let num_rows = records.len();
tokio::spawn(async move {
let (send, recv) = tokio::sync::oneshot::channel();
rayon::spawn(move || {
Expand All @@ -450,7 +455,11 @@ fn parse_into_column_array_chunk_stream(
)
})
.collect::<DaftResult<Vec<_>>>()?;
Ok(Table::new_unchecked(daft_schema.clone(), all_series))
Ok(Table::new_unchecked(
daft_schema.clone(),
all_series,
num_rows,
))
})();
let _ = send.send(result);
});
Expand Down
38 changes: 12 additions & 26 deletions src/daft-parquet/src/file.rs
Original file line number Diff line number Diff line change
Expand Up @@ -220,30 +220,8 @@ impl ParquetReaderBuilder {
&self.metadata
}

pub fn parquet_schema(&self) -> &parquet2::metadata::SchemaDescriptor {
self.metadata().schema()
}

pub fn prune_columns<S: ToString + AsRef<str>>(mut self, columns: &[S]) -> super::Result<Self> {
let avail_names = self
.parquet_schema()
.fields()
.iter()
.map(|f| f.name())
.collect::<HashSet<_>>();
let mut names_to_keep = HashSet::new();
for col_name in columns {
if avail_names.contains(col_name.as_ref()) {
names_to_keep.insert(col_name.to_string());
} else {
return Err(super::Error::FieldNotFound {
field: col_name.to_string(),
available_fields: avail_names.iter().map(|v| v.to_string()).collect(),
path: self.uri,
});
}
}
self.selected_columns = Some(names_to_keep);
self.selected_columns = Some(HashSet::from_iter(columns.iter().map(|s| s.to_string())));
Ok(self)
}

Expand Down Expand Up @@ -286,6 +264,7 @@ impl ParquetReaderBuilder {
.fields
.retain(|f| names_to_keep.contains(f.name.as_str()));
}

let daft_schema =
Schema::try_from(&arrow_schema).with_context(|_| UnableToConvertSchemaToDaftSnafu {
path: self.uri.to_string(),
Expand Down Expand Up @@ -570,13 +549,17 @@ impl ParquetFileReader {
.collect::<DaftResult<Vec<_>>>()?;
let daft_schema = daft_core::schema::Schema::try_from(self.arrow_schema.as_ref())?;

Table::new(daft_schema, all_series)
Table::new(
daft_schema,
all_series,
self.row_ranges.as_ref().iter().map(|rr| rr.num_rows).sum(),
)
}

pub async fn read_from_ranges_into_arrow_arrays(
self,
ranges: Arc<RangesContainer>,
) -> DaftResult<Vec<Vec<Box<dyn arrow2::array::Array>>>> {
) -> DaftResult<(Vec<Vec<Box<dyn arrow2::array::Array>>>, usize)> {
let metadata = self.metadata;
let all_handles = self
.arrow_schema
Expand Down Expand Up @@ -723,6 +706,9 @@ impl ParquetFileReader {
})?
.into_iter()
.collect::<DaftResult<Vec<_>>>()?;
Ok(all_field_arrays)
Ok((
all_field_arrays,
self.row_ranges.as_ref().iter().map(|rr| rr.num_rows).sum(),
))
}
}
13 changes: 2 additions & 11 deletions src/daft-parquet/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -82,29 +82,20 @@ pub enum Error {
))]
UnableToConvertSchemaToDaft { path: String, source: DaftError },

#[snafu(display(
"Field: {} not found in Parquet File: {} Available Fields: {:?}",
field,
path,
available_fields
))]
FieldNotFound {
field: String,
available_fields: Vec<String>,
path: String,
},
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 removed this because our Parquet readers will now not complain if a requested field is not found.

Instead, the readers will return a best-effort Table, with all the columns it can find from the ones requested.

#[snafu(display(
"File: {} is not a valid parquet file. Has incorrect footer: {:?}",
path,
footer
))]
InvalidParquetFile { path: String, footer: Vec<u8> },

#[snafu(display(
"File: {} is not a valid parquet file and is only {} bytes, smaller than the minimum size of 12 bytes",
path,
file_size
))]
FileTooSmall { path: String, file_size: usize },

#[snafu(display(
"File: {} has a footer size: {} greater than the file size: {}",
path,
Expand Down
35 changes: 21 additions & 14 deletions src/daft-parquet/src/read.rs
Original file line number Diff line number Diff line change
Expand Up @@ -208,8 +208,11 @@ async fn read_parquet_single(
metadata_num_columns
};

if (!field_id_mapping_provided && table.num_columns() != expected_num_columns)
if (!field_id_mapping_provided
&& requested_columns.is_none()
&& table.num_columns() != expected_num_columns)
|| (field_id_mapping_provided && table.num_columns() > expected_num_columns)
|| (requested_columns.is_some() && table.num_columns() > expected_num_columns)
{
return Err(super::Error::ParquetNumColumnMismatch {
path: uri.into(),
Expand Down Expand Up @@ -237,8 +240,8 @@ async fn read_parquet_single_into_arrow(
) -> DaftResult<(arrow2::datatypes::SchemaRef, Vec<ArrowChunk>)> {
let field_id_mapping_provided = field_id_mapping.is_some();
let (source_type, fixed_uri) = parse_url(uri)?;
let (metadata, schema, all_arrays) = if matches!(source_type, SourceType::File) {
let (metadata, schema, all_arrays) =
let (metadata, schema, all_arrays, num_rows_read) = if matches!(source_type, SourceType::File) {
let (metadata, schema, all_arrays, num_rows_read) =
crate::stream_reader::local_parquet_read_into_arrow_async(
fixed_uri.as_ref(),
columns.map(|s| s.iter().map(|s| s.to_string()).collect_vec()),
Expand All @@ -250,7 +253,7 @@ async fn read_parquet_single_into_arrow(
metadata,
)
.await?;
(metadata, Arc::new(schema), all_arrays)
(metadata, Arc::new(schema), all_arrays, num_rows_read)
} else {
let builder = ParquetReaderBuilder::from_uri(
uri,
Expand Down Expand Up @@ -283,10 +286,10 @@ async fn read_parquet_single_into_arrow(

let schema = parquet_reader.arrow_schema().clone();
let ranges = parquet_reader.prebuffer_ranges(io_client, io_stats)?;
let all_arrays = parquet_reader
let (all_arrays, num_rows_read) = parquet_reader
.read_from_ranges_into_arrow_arrays(ranges)
.await?;
(Arc::new(metadata), schema, all_arrays)
(Arc::new(metadata), schema, all_arrays, num_rows_read)
};

let rows_per_row_groups = metadata
Expand All @@ -307,7 +310,7 @@ async fn read_parquet_single_into_arrow(
return Err(super::Error::ParquetColumnsDontHaveEqualRows { path: uri.into() }.into());
}

let table_len = *len_per_col.first().unwrap_or(&0);
let table_len = *len_per_col.first().unwrap_or(&num_rows_read);
let table_ncol = all_arrays.len();

if let Some(row_groups) = &row_groups {
Expand Down Expand Up @@ -354,8 +357,9 @@ async fn read_parquet_single_into_arrow(
metadata_num_columns
};

if (!field_id_mapping_provided && table_ncol != expected_num_columns)
if (!field_id_mapping_provided && columns.is_none() && table_ncol != expected_num_columns)
|| (field_id_mapping_provided && table_ncol > expected_num_columns)
|| (columns.is_some() && table_ncol > expected_num_columns)
{
return Err(super::Error::ParquetNumColumnMismatch {
path: uri.into(),
Expand Down Expand Up @@ -721,12 +725,15 @@ pub fn read_parquet_statistics(
)),
));

Table::from_columns(vec![
uris.clone(),
row_count_series.into_series(),
row_group_series.into_series(),
version_series.into_series(),
])
Table::from_columns(
vec![
uris.clone(),
row_count_series.into_series(),
row_group_series.into_series(),
version_series.into_series(),
],
uris.len(),
)
}

#[cfg(test)]
Expand Down
29 changes: 18 additions & 11 deletions src/daft-parquet/src/stream_reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ use rayon::iter::ParallelIterator;
fn prune_fields_from_schema(
schema: arrow2::datatypes::Schema,
columns: Option<&[String]>,
uri: &str,
) -> super::Result<arrow2::datatypes::Schema> {
if let Some(columns) = columns {
let avail_names = schema
Expand All @@ -33,12 +32,6 @@ fn prune_fields_from_schema(
for col_name in columns {
if avail_names.contains(col_name.as_str()) {
names_to_keep.insert(col_name.to_string());
} else {
return Err(super::Error::FieldNotFound {
field: col_name.to_string(),
available_fields: avail_names.iter().map(|v| v.to_string()).collect(),
path: uri.to_string(),
});
}
}
Ok(schema.filter(|_, field| names_to_keep.contains(&field.name)))
Expand All @@ -61,6 +54,7 @@ pub(crate) fn local_parquet_read_into_arrow(
Arc<parquet2::metadata::FileMetaData>,
arrow2::datatypes::Schema,
Vec<ArrowChunk>,
usize,
)> {
const LOCAL_PROTOCOL: &str = "file://";

Expand Down Expand Up @@ -96,7 +90,7 @@ pub(crate) fn local_parquet_read_into_arrow(
.with_context(|_| super::UnableToParseSchemaFromMetadataSnafu {
path: uri.to_string(),
})?;
let schema = prune_fields_from_schema(schema, columns, uri)?;
let schema = prune_fields_from_schema(schema, columns)?;
let daft_schema =
Schema::try_from(&schema).with_context(|_| UnableToConvertSchemaToDaftSnafu {
path: uri.to_string(),
Expand Down Expand Up @@ -177,7 +171,12 @@ pub(crate) fn local_parquet_read_into_arrow(
.expect("array index during scatter out of index")
.extend(v);
}
Ok((metadata, schema, all_columns))
Ok((
metadata,
schema,
all_columns,
row_ranges.iter().map(|rr| rr.num_rows).sum(),
))
}

#[allow(clippy::too_many_arguments)]
Expand Down Expand Up @@ -205,7 +204,7 @@ pub(crate) async fn local_parquet_read_async(
schema_infer_options,
metadata,
);
let (metadata, schema, arrays) = v?;
let (metadata, schema, arrays, num_rows_read) = v?;

let converted_arrays = arrays
.into_par_iter()
Expand All @@ -225,7 +224,14 @@ pub(crate) async fn local_parquet_read_async(
}
})
.collect::<Result<Vec<_>, _>>()?;
Ok((metadata, Table::from_columns(converted_arrays)?))
Ok((
metadata,
Table::new(
Schema::new(converted_arrays.iter().map(|s| s.field().clone()).collect())?,
converted_arrays,
num_rows_read,
)?,
))
})();
let _ = send.send(result);
});
Expand All @@ -247,6 +253,7 @@ pub(crate) async fn local_parquet_read_into_arrow_async(
Arc<parquet2::metadata::FileMetaData>,
arrow2::datatypes::Schema,
Vec<ArrowChunk>,
usize,
)> {
let (send, recv) = tokio::sync::oneshot::channel();
let uri = uri.to_string();
Expand Down
2 changes: 2 additions & 0 deletions src/daft-plan/src/source_info/file_info.rs
Original file line number Diff line number Diff line change
Expand Up @@ -173,9 +173,11 @@ impl FileInfos {
arrow2::array::PrimitiveArray::<i64>::from(&self.num_rows).to_boxed(),
))?,
];
let num_rows = columns.first().map(|s| s.len()).unwrap();
Table::new(
Schema::new(columns.iter().map(|s| s.field().clone()).collect())?,
columns,
num_rows,
)
}
}
Expand Down
Loading
Loading