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

Concurrent Parquet Schema Inference #6366

Merged
merged 1 commit into from
May 19, 2023
Merged
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
13 changes: 7 additions & 6 deletions datafusion/core/src/datasource/file_format/parquet.rs
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ use async_trait::async_trait;
use bytes::{BufMut, BytesMut};
use datafusion_common::DataFusionError;
use datafusion_physical_expr::PhysicalExpr;
use futures::{StreamExt, TryStreamExt};
use hashbrown::HashMap;
use object_store::{ObjectMeta, ObjectStore};
use parquet::arrow::parquet_to_arrow_schema;
Expand Down Expand Up @@ -151,12 +152,12 @@ impl FileFormat for ParquetFormat {
store: &Arc<dyn ObjectStore>,
objects: &[ObjectMeta],
) -> Result<SchemaRef> {
let mut schemas = Vec::with_capacity(objects.len());
for object in objects {
let schema =
fetch_schema(store.as_ref(), object, self.metadata_size_hint).await?;
schemas.push(schema)
}
let schemas: Vec<_> = futures::stream::iter(objects)
.map(|object| fetch_schema(store.as_ref(), object, self.metadata_size_hint))
.boxed() // Workaround https://github.com/rust-lang/rust/issues/64552
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No I cannot explain why this works... But for some unknown reason it placates the compiler

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I found using streams was a similarly frustrating experience where the compiler give you an opaque error message (and opaque is being polite)

.buffered(32)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the 32 should at least be a named constant so it is more discoverable

What do you think about potentially using the same value as in #6183:

const CONCURRENCY_LIMIT: usize = 100;

(I sort of imagine some day someone will want to make that a configuration knob rather than a constant so using the same constant in the code will make it easier to find where they are used)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'll make this change as a follow up

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

.try_collect()
.await?;

let schema = if self.skip_metadata(state.config_options()) {
Schema::try_merge(clear_metadata(schemas))
Expand Down