Skip to content

Commit 956825c

Browse files
committed
WIP: Some cargo fmt fixes
1 parent 1f450cc commit 956825c

File tree

9 files changed

+160
-52
lines changed

9 files changed

+160
-52
lines changed

datafusion-examples/examples/advanced_parquet_index.rs

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -612,7 +612,9 @@ impl AsyncFileReader for ParquetReaderWithCache {
612612

613613
fn get_metadata(
614614
&mut self,
615-
encryption_config: &Option<datafusion::parquet::file::encryption::ParquetEncryptionConfig>,
615+
encryption_config: &Option<
616+
datafusion::parquet::file::encryption::ParquetEncryptionConfig,
617+
>,
616618
) -> BoxFuture<'_, datafusion::parquet::errors::Result<Arc<ParquetMetaData>>> {
617619
println!("get_metadata: {} returning cached metadata", self.filename);
618620
assert!(encryption_config.is_none());

datafusion/common/src/file_options/mod.rs

Lines changed: 12 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,10 @@ mod tests {
3333
use super::parquet_writer::ParquetWriterOptions;
3434
use crate::{
3535
config::{ConfigFileType, TableOptions},
36-
file_options::{csv_writer::CsvWriterOptions, json_writer::JsonWriterOptions, parquet_writer::WriterPropertiesConfig},
36+
file_options::{
37+
csv_writer::CsvWriterOptions, json_writer::JsonWriterOptions,
38+
parquet_writer::WriterPropertiesConfig,
39+
},
3740
parsers::CompressionTypeVariant,
3841
Result,
3942
};
@@ -79,7 +82,10 @@ mod tests {
7982
table_config.set_config_format(ConfigFileType::PARQUET);
8083
table_config.alter_with_string_hash_map(&option_map)?;
8184

82-
let parquet_options = ParquetWriterOptions::from_table_parquet_options(&table_config.parquet, WriterPropertiesConfig::noop().as_ref())?;
85+
let parquet_options = ParquetWriterOptions::from_table_parquet_options(
86+
&table_config.parquet,
87+
WriterPropertiesConfig::noop().as_ref(),
88+
)?;
8389
let properties = parquet_options.writer_options();
8490

8591
// Verify the expected options propagated down to parquet crate WriterProperties struct
@@ -184,7 +190,10 @@ mod tests {
184190
table_config.set_config_format(ConfigFileType::PARQUET);
185191
table_config.alter_with_string_hash_map(&option_map)?;
186192

187-
let parquet_options = ParquetWriterOptions::from_table_parquet_options(&table_config.parquet, WriterPropertiesConfig::noop().as_ref())?;
193+
let parquet_options = ParquetWriterOptions::from_table_parquet_options(
194+
&table_config.parquet,
195+
WriterPropertiesConfig::noop().as_ref(),
196+
)?;
188197
let properties = parquet_options.writer_options();
189198

190199
let col1 = ColumnPath::from(vec!["col1".to_owned()]);

datafusion/common/src/file_options/parquet_writer.rs

Lines changed: 18 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -25,10 +25,13 @@ use crate::{
2525
};
2626

2727
use parquet::{
28-
basic::{BrotliLevel, GzipLevel, ZstdLevel}, file::properties::{
28+
basic::{BrotliLevel, GzipLevel, ZstdLevel},
29+
file::properties::{
2930
EnabledStatistics, WriterProperties, WriterPropertiesBuilder, WriterVersion,
3031
DEFAULT_MAX_STATISTICS_SIZE, DEFAULT_STATISTICS_ENABLED,
31-
}, format::KeyValue, schema::types::ColumnPath
32+
},
33+
format::KeyValue,
34+
schema::types::ColumnPath,
3235
};
3336

3437
/// Options for writing parquet files
@@ -51,7 +54,10 @@ impl ParquetWriterOptions {
5154
}
5255

5356
impl ParquetWriterOptions {
54-
pub fn from_table_parquet_options(parquet_table_options: &TableParquetOptions, customizer: &dyn WriterPropertiesCustomizer) -> Result<Self> {
57+
pub fn from_table_parquet_options(
58+
parquet_table_options: &TableParquetOptions,
59+
customizer: &dyn WriterPropertiesCustomizer,
60+
) -> Result<Self> {
5561
// ParquetWriterOptions will have defaults for the remaining fields (e.g. sorting_columns)
5662
let mut builder = WriterPropertiesBuilder::try_from(parquet_table_options)?;
5763
builder = customizer.adjust_write_properties(builder);
@@ -68,12 +74,15 @@ pub struct WriterPropertiesConfig {
6874

6975
impl WriterPropertiesConfig {
7076
pub fn noop() -> Arc<dyn WriterPropertiesCustomizer> {
71-
Arc::new(NoopWriterPropertiesCustomizer{})
77+
Arc::new(NoopWriterPropertiesCustomizer {})
7278
}
7379
}
7480

7581
pub trait WriterPropertiesCustomizer: Sync + Send + std::fmt::Debug {
76-
fn adjust_write_properties(&self, builder: WriterPropertiesBuilder) -> WriterPropertiesBuilder;
82+
fn adjust_write_properties(
83+
&self,
84+
builder: WriterPropertiesBuilder,
85+
) -> WriterPropertiesBuilder;
7786
fn allow_single_file_parallelism(&self) -> bool {
7887
true
7988
}
@@ -83,7 +92,10 @@ pub trait WriterPropertiesCustomizer: Sync + Send + std::fmt::Debug {
8392
pub struct NoopWriterPropertiesCustomizer;
8493

8594
impl WriterPropertiesCustomizer for NoopWriterPropertiesCustomizer {
86-
fn adjust_write_properties(&self, builder: WriterPropertiesBuilder) -> WriterPropertiesBuilder {
95+
fn adjust_write_properties(
96+
&self,
97+
builder: WriterPropertiesBuilder,
98+
) -> WriterPropertiesBuilder {
8799
builder
88100
}
89101
}

datafusion/core/src/datasource/file_format/options.rs

Lines changed: 15 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,9 @@ use crate::execution::context::{SessionConfig, SessionState};
3434

3535
use arrow::datatypes::{DataType, Schema, SchemaRef};
3636
use datafusion_common::config::TableOptions;
37-
use datafusion_common::file_options::parquet_writer::{WriterPropertiesConfig, WriterPropertiesCustomizer};
37+
use datafusion_common::file_options::parquet_writer::{
38+
WriterPropertiesConfig, WriterPropertiesCustomizer,
39+
};
3840
use datafusion_common::{
3941
DEFAULT_ARROW_EXTENSION, DEFAULT_AVRO_EXTENSION, DEFAULT_CSV_EXTENSION,
4042
DEFAULT_JSON_EXTENSION, DEFAULT_PARQUET_EXTENSION,
@@ -545,8 +547,15 @@ impl ReadOptions<'_> for CsvReadOptions<'_> {
545547
}
546548

547549
/// Retrieves `WriterPropertiesConfig` from the `SessionConfig::extensions` field, constructing a Noop customizer if not present.
548-
pub fn get_writer_properties_customizer(config: &SessionConfig) -> Arc<dyn WriterPropertiesCustomizer> {
549-
config.get_extension::<WriterPropertiesConfig>().map_or_else(|| WriterPropertiesConfig::noop(), |cfg| cfg.customizer.clone())
550+
pub fn get_writer_properties_customizer(
551+
config: &SessionConfig,
552+
) -> Arc<dyn WriterPropertiesCustomizer> {
553+
config
554+
.get_extension::<WriterPropertiesConfig>()
555+
.map_or_else(
556+
|| WriterPropertiesConfig::noop(),
557+
|cfg| cfg.customizer.clone(),
558+
)
550559
}
551560

552561
#[cfg(feature = "parquet")]
@@ -558,7 +567,9 @@ impl ReadOptions<'_> for ParquetReadOptions<'_> {
558567
table_options: TableOptions,
559568
) -> ListingOptions {
560569
let customizer = get_writer_properties_customizer(config);
561-
let mut file_format = ParquetFormat::new().with_options(table_options.parquet).with_customizer(customizer);
570+
let mut file_format = ParquetFormat::new()
571+
.with_options(table_options.parquet)
572+
.with_customizer(customizer);
562573

563574
if let Some(parquet_pruning) = self.parquet_pruning {
564575
file_format = file_format.with_enable_pruning(parquet_pruning)

datafusion/core/src/datasource/file_format/parquet.rs

Lines changed: 66 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -44,7 +44,9 @@ use crate::physical_plan::{
4444

4545
use arrow::compute::sum;
4646
use datafusion_common::config::{ConfigField, ConfigFileType, TableParquetOptions};
47-
use datafusion_common::file_options::parquet_writer::{ParquetWriterOptions, WriterPropertiesConfig, WriterPropertiesCustomizer};
47+
use datafusion_common::file_options::parquet_writer::{
48+
ParquetWriterOptions, WriterPropertiesConfig, WriterPropertiesCustomizer,
49+
};
4850
use datafusion_common::parsers::CompressionTypeVariant;
4951
use datafusion_common::stats::Precision;
5052
use datafusion_common::{
@@ -80,7 +82,9 @@ use tokio::io::{AsyncWrite, AsyncWriteExt};
8082
use tokio::sync::mpsc::{self, Receiver, Sender};
8183
use tokio::task::JoinSet;
8284

83-
use crate::datasource::physical_plan::parquet::{get_reader_options_config_or_default, MetadataFetcher, ParquetExecBuilder};
85+
use crate::datasource::physical_plan::parquet::{
86+
get_reader_options_config_or_default, MetadataFetcher, ParquetExecBuilder,
87+
};
8488
use datafusion_physical_expr_common::sort_expr::LexRequirement;
8589
use futures::{StreamExt, TryStreamExt};
8690
use object_store::path::Path;
@@ -141,7 +145,9 @@ impl FileFormatFactory for ParquetFormatFactory {
141145
let customizer = get_writer_properties_customizer(state.config());
142146

143147
Ok(Arc::new(
144-
ParquetFormat::new().with_options(parquet_options).with_customizer(customizer),
148+
ParquetFormat::new()
149+
.with_options(parquet_options)
150+
.with_customizer(customizer),
145151
))
146152
}
147153

@@ -245,7 +251,10 @@ impl ParquetFormat {
245251
}
246252

247253
/// Set WriterPropertiesCustomizer for the ParquetFormat
248-
pub fn with_customizer(mut self, customizer: Arc<dyn WriterPropertiesCustomizer>) -> Self {
254+
pub fn with_customizer(
255+
mut self,
256+
customizer: Arc<dyn WriterPropertiesCustomizer>,
257+
) -> Self {
249258
self.customizer = customizer;
250259
self
251260
}
@@ -434,7 +443,11 @@ impl FileFormat for ParquetFormat {
434443
}
435444

436445
let sink_schema = conf.output_schema().clone();
437-
let sink = Arc::new(ParquetSink::new(conf, self.options.clone(), self.customizer.clone()));
446+
let sink = Arc::new(ParquetSink::new(
447+
conf,
448+
self.options.clone(),
449+
self.customizer.clone(),
450+
));
438451

439452
Ok(Arc::new(DataSinkExec::new(
440453
input,
@@ -512,15 +525,16 @@ pub async fn fetch_parquet_metadata(
512525
}
513526
}
514527

515-
516528
/// Read and parse the schema of the Parquet file at location `path`
517529
async fn fetch_schema(
518530
store: &dyn ObjectStore,
519531
file: &ObjectMeta,
520532
metadata_size_hint: Option<usize>,
521533
metadata_fetcher: &dyn MetadataFetcher,
522534
) -> Result<Schema> {
523-
let metadata = metadata_fetcher.fetch_metadata(store, file, metadata_size_hint).await?;
535+
let metadata = metadata_fetcher
536+
.fetch_metadata(store, file, metadata_size_hint)
537+
.await?;
524538
let file_metadata = metadata.file_metadata();
525539
let schema = parquet_to_arrow_schema(
526540
file_metadata.schema_descr(),
@@ -539,7 +553,9 @@ async fn fetch_statistics(
539553
metadata_size_hint: Option<usize>,
540554
metadata_fetcher: &dyn MetadataFetcher,
541555
) -> Result<Statistics> {
542-
let metadata = metadata_fetcher.fetch_metadata(store, file, metadata_size_hint).await?;
556+
let metadata = metadata_fetcher
557+
.fetch_metadata(store, file, metadata_size_hint)
558+
.await?;
543559
statistics_from_parquet_meta_calc(&metadata, table_schema)
544560
}
545561

@@ -703,7 +719,11 @@ impl DisplayAs for ParquetSink {
703719

704720
impl ParquetSink {
705721
/// Create from config.
706-
pub fn new(config: FileSinkConfig, parquet_options: TableParquetOptions, customizer: Arc<dyn WriterPropertiesCustomizer>) -> Self {
722+
pub fn new(
723+
config: FileSinkConfig,
724+
parquet_options: TableParquetOptions,
725+
customizer: Arc<dyn WriterPropertiesCustomizer>,
726+
) -> Self {
707727
Self {
708728
config,
709729
parquet_options,
@@ -788,15 +808,19 @@ impl DataSink for ParquetSink {
788808
data: SendableRecordBatchStream,
789809
context: &Arc<TaskContext>,
790810
) -> Result<u64> {
791-
let parquet_props = ParquetWriterOptions::from_table_parquet_options(&self.parquet_options, self.customizer.as_ref())?;
811+
let parquet_props = ParquetWriterOptions::from_table_parquet_options(
812+
&self.parquet_options,
813+
self.customizer.as_ref(),
814+
)?;
792815

793816
let object_store = context
794817
.runtime_env()
795818
.object_store(&self.config.object_store_url)?;
796819

797820
let parquet_opts = &self.parquet_options;
798821
let allow_single_file_parallelism =
799-
parquet_opts.global.allow_single_file_parallelism && self.customizer.allow_single_file_parallelism();
822+
parquet_opts.global.allow_single_file_parallelism
823+
&& self.customizer.allow_single_file_parallelism();
800824

801825
let part_col = if !self.config.table_partition_cols.is_empty() {
802826
Some(self.config.table_partition_cols.clone())
@@ -1360,16 +1384,29 @@ mod tests {
13601384
let format = ParquetFormat::default().with_force_view_types(force_views);
13611385
let schema = format.infer_schema(&ctx, &store, &meta).await.unwrap();
13621386

1363-
let stats =
1364-
fetch_statistics(store.as_ref(), schema.clone(), &meta[0], None, &DefaultMetadataFetcher{}).await?;
1387+
let stats = fetch_statistics(
1388+
store.as_ref(),
1389+
schema.clone(),
1390+
&meta[0],
1391+
None,
1392+
&DefaultMetadataFetcher {},
1393+
)
1394+
.await?;
13651395

13661396
assert_eq!(stats.num_rows, Precision::Exact(3));
13671397
let c1_stats = &stats.column_statistics[0];
13681398
let c2_stats = &stats.column_statistics[1];
13691399
assert_eq!(c1_stats.null_count, Precision::Exact(1));
13701400
assert_eq!(c2_stats.null_count, Precision::Exact(3));
13711401

1372-
let stats = fetch_statistics(store.as_ref(), schema, &meta[1], None, &DefaultMetadataFetcher{}).await?;
1402+
let stats = fetch_statistics(
1403+
store.as_ref(),
1404+
schema,
1405+
&meta[1],
1406+
None,
1407+
&DefaultMetadataFetcher {},
1408+
)
1409+
.await?;
13731410
assert_eq!(stats.num_rows, Precision::Exact(3));
13741411
let c1_stats = &stats.column_statistics[0];
13751412
let c2_stats = &stats.column_statistics[1];
@@ -1561,9 +1598,14 @@ mod tests {
15611598
.await
15621599
.unwrap();
15631600

1564-
let stats =
1565-
fetch_statistics(store.upcast().as_ref(), schema.clone(), &meta[0], Some(9), &DefaultMetadataFetcher{})
1566-
.await?;
1601+
let stats = fetch_statistics(
1602+
store.upcast().as_ref(),
1603+
schema.clone(),
1604+
&meta[0],
1605+
Some(9),
1606+
&DefaultMetadataFetcher {},
1607+
)
1608+
.await?;
15671609

15681610
assert_eq!(stats.num_rows, Precision::Exact(3));
15691611
let c1_stats = &stats.column_statistics[0];
@@ -1597,7 +1639,7 @@ mod tests {
15971639
schema.clone(),
15981640
&meta[0],
15991641
Some(size_hint),
1600-
&DefaultMetadataFetcher{},
1642+
&DefaultMetadataFetcher {},
16011643
)
16021644
.await?;
16031645

@@ -2288,7 +2330,8 @@ mod tests {
22882330
overwrite: true,
22892331
keep_partition_by_columns: false,
22902332
};
2291-
let customizer: Arc<dyn WriterPropertiesCustomizer> = WriterPropertiesConfig::noop();
2333+
let customizer: Arc<dyn WriterPropertiesCustomizer> =
2334+
WriterPropertiesConfig::noop();
22922335
let parquet_sink = Arc::new(ParquetSink::new(
22932336
file_sink_config,
22942337
TableParquetOptions {
@@ -2385,7 +2428,8 @@ mod tests {
23852428
overwrite: true,
23862429
keep_partition_by_columns: false,
23872430
};
2388-
let customizer: Arc<dyn WriterPropertiesCustomizer> = WriterPropertiesConfig::noop();
2431+
let customizer: Arc<dyn WriterPropertiesCustomizer> =
2432+
WriterPropertiesConfig::noop();
23892433
let parquet_sink = Arc::new(ParquetSink::new(
23902434
file_sink_config,
23912435
TableParquetOptions::default(),
@@ -2470,7 +2514,8 @@ mod tests {
24702514
overwrite: true,
24712515
keep_partition_by_columns: false,
24722516
};
2473-
let customizer: Arc<dyn WriterPropertiesCustomizer> = WriterPropertiesConfig::noop();
2517+
let customizer: Arc<dyn WriterPropertiesCustomizer> =
2518+
WriterPropertiesConfig::noop();
24742519
let parquet_sink = Arc::new(ParquetSink::new(
24752520
file_sink_config,
24762521
TableParquetOptions {

0 commit comments

Comments
 (0)