diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index f2780cd469a9..c2bccb6378ca 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -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; @@ -151,12 +152,12 @@ impl FileFormat for ParquetFormat { store: &Arc, objects: &[ObjectMeta], ) -> Result { - 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 + .buffered(32) + .try_collect() + .await?; let schema = if self.skip_metadata(state.config_options()) { Schema::try_merge(clear_metadata(schemas))