forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathconfig.rs
More file actions
3527 lines (3130 loc) · 141 KB
/
config.rs
File metadata and controls
3527 lines (3130 loc) · 141 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
//! Runtime configuration, via [`ConfigOptions`]
use arrow_ipc::CompressionType;
#[cfg(feature = "parquet_encryption")]
use crate::encryption::{FileDecryptionProperties, FileEncryptionProperties};
use crate::error::_config_err;
use crate::format::{ExplainAnalyzeLevel, ExplainFormat};
use crate::parquet_config::DFParquetWriterVersion;
use crate::parsers::CompressionTypeVariant;
use crate::utils::get_available_parallelism;
use crate::{DataFusionError, Result};
#[cfg(feature = "parquet_encryption")]
use hex;
use std::any::Any;
use std::collections::{BTreeMap, HashMap};
use std::error::Error;
use std::fmt::{self, Display};
use std::str::FromStr;
#[cfg(feature = "parquet_encryption")]
use std::sync::Arc;
/// A macro that wraps a configuration struct and automatically derives
/// [`Default`] and [`ConfigField`] for it, allowing it to be used
/// in the [`ConfigOptions`] configuration tree.
///
/// `transform` is used to normalize values before parsing.
///
/// For example,
///
/// ```ignore
/// config_namespace! {
/// /// Amazing config
/// pub struct MyConfig {
/// /// Field 1 doc
/// field1: String, transform = str::to_lowercase, default = "".to_string()
///
/// /// Field 2 doc
/// field2: usize, default = 232
///
/// /// Field 3 doc
/// field3: Option<usize>, default = None
/// }
/// }
/// ```
///
/// Will generate
///
/// ```ignore
/// /// Amazing config
/// #[derive(Debug, Clone)]
/// #[non_exhaustive]
/// pub struct MyConfig {
/// /// Field 1 doc
/// field1: String,
/// /// Field 2 doc
/// field2: usize,
/// /// Field 3 doc
/// field3: Option<usize>,
/// }
/// impl ConfigField for MyConfig {
/// fn set(&mut self, key: &str, value: &str) -> Result<()> {
/// let (key, rem) = key.split_once('.').unwrap_or((key, ""));
/// match key {
/// "field1" => {
/// let value = str::to_lowercase(value);
/// self.field1.set(rem, value.as_ref())
/// },
/// "field2" => self.field2.set(rem, value.as_ref()),
/// "field3" => self.field3.set(rem, value.as_ref()),
/// _ => _internal_err!(
/// "Config value \"{}\" not found on MyConfig",
/// key
/// ),
/// }
/// }
///
/// fn visit<V: Visit>(&self, v: &mut V, key_prefix: &str, _description: &'static str) {
/// let key = format!("{}.field1", key_prefix);
/// let desc = "Field 1 doc";
/// self.field1.visit(v, key.as_str(), desc);
/// let key = format!("{}.field2", key_prefix);
/// let desc = "Field 2 doc";
/// self.field2.visit(v, key.as_str(), desc);
/// let key = format!("{}.field3", key_prefix);
/// let desc = "Field 3 doc";
/// self.field3.visit(v, key.as_str(), desc);
/// }
/// }
///
/// impl Default for MyConfig {
/// fn default() -> Self {
/// Self {
/// field1: "".to_string(),
/// field2: 232,
/// field3: None,
/// }
/// }
/// }
/// ```
///
/// NB: Misplaced commas may result in nonsensical errors
#[macro_export]
macro_rules! config_namespace {
(
$(#[doc = $struct_d:tt])* // Struct-level documentation attributes
$(#[deprecated($($struct_depr:tt)*)])? // Optional struct-level deprecated attribute
$(#[allow($($struct_de:tt)*)])?
$vis:vis struct $struct_name:ident {
$(
$(#[doc = $d:tt])* // Field-level documentation attributes
$(#[deprecated($($field_depr:tt)*)])? // Optional field-level deprecated attribute
$(#[allow($($field_de:tt)*)])?
$field_vis:vis $field_name:ident : $field_type:ty,
$(warn = $warn:expr,)?
$(transform = $transform:expr,)?
default = $default:expr
)*$(,)*
}
) => {
$(#[doc = $struct_d])* // Apply struct documentation
$(#[deprecated($($struct_depr)*)])? // Apply struct deprecation
$(#[allow($($struct_de)*)])?
#[derive(Debug, Clone, PartialEq)]
$vis struct $struct_name {
$(
$(#[doc = $d])* // Apply field documentation
$(#[deprecated($($field_depr)*)])? // Apply field deprecation
$(#[allow($($field_de)*)])?
$field_vis $field_name: $field_type,
)*
}
impl $crate::config::ConfigField for $struct_name {
fn set(&mut self, key: &str, value: &str) -> $crate::error::Result<()> {
let (key, rem) = key.split_once('.').unwrap_or((key, ""));
match key {
$(
stringify!($field_name) => {
// Safely apply deprecated attribute if present
// $(#[allow(deprecated)])?
{
$(let value = $transform(value);)? // Apply transformation if specified
let ret = self.$field_name.set(rem, value.as_ref());
$(if !$warn.is_empty() {
let default: $field_type = $default;
if default != self.$field_name {
log::warn!($warn);
}
})? // Log warning if specified, and the value is not the default
ret
}
},
)*
_ => return $crate::error::_config_err!(
"Config value \"{}\" not found on {}", key, stringify!($struct_name)
)
}
}
fn visit<V: $crate::config::Visit>(&self, v: &mut V, key_prefix: &str, _description: &'static str) {
$(
let key = format!(concat!("{}.", stringify!($field_name)), key_prefix);
let desc = concat!($($d),*).trim();
self.$field_name.visit(v, key.as_str(), desc);
)*
}
fn reset(&mut self, key: &str) -> $crate::error::Result<()> {
let (key, rem) = key.split_once('.').unwrap_or((key, ""));
match key {
$(
stringify!($field_name) => {
{
if rem.is_empty() {
let default_value: $field_type = $default;
self.$field_name = default_value;
Ok(())
} else {
self.$field_name.reset(rem)
}
}
},
)*
_ => $crate::error::_config_err!(
"Config value \"{}\" not found on {}",
key,
stringify!($struct_name)
),
}
}
}
impl Default for $struct_name {
fn default() -> Self {
Self {
$($field_name: $default),*
}
}
}
}
}
config_namespace! {
/// Options related to catalog and directory scanning
///
/// See also: [`SessionConfig`]
///
/// [`SessionConfig`]: https://docs.rs/datafusion/latest/datafusion/prelude/struct.SessionConfig.html
pub struct CatalogOptions {
/// Whether the default catalog and schema should be created automatically.
pub create_default_catalog_and_schema: bool, default = true
/// The default catalog name - this impacts what SQL queries use if not specified
pub default_catalog: String, default = "datafusion".to_string()
/// The default schema name - this impacts what SQL queries use if not specified
pub default_schema: String, default = "public".to_string()
/// Should DataFusion provide access to `information_schema`
/// virtual tables for displaying schema information
pub information_schema: bool, default = false
/// Location scanned to load tables for `default` schema
pub location: Option<String>, default = None
/// Type of `TableProvider` to use when loading `default` schema
pub format: Option<String>, default = None
/// Default value for `format.has_header` for `CREATE EXTERNAL TABLE`
/// if not specified explicitly in the statement.
pub has_header: bool, default = true
/// Specifies whether newlines in (quoted) CSV values are supported.
///
/// This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE`
/// if not specified explicitly in the statement.
///
/// Parsing newlines in quoted values may be affected by execution behaviour such as
/// parallel file scanning. Setting this to `true` ensures that newlines in values are
/// parsed successfully, which may reduce performance.
pub newlines_in_values: bool, default = false
}
}
config_namespace! {
/// Options related to SQL parser
///
/// See also: [`SessionConfig`]
///
/// [`SessionConfig`]: https://docs.rs/datafusion/latest/datafusion/prelude/struct.SessionConfig.html
pub struct SqlParserOptions {
/// When set to true, SQL parser will parse float as decimal type
pub parse_float_as_decimal: bool, default = false
/// When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted)
pub enable_ident_normalization: bool, default = true
/// When set to true, SQL parser will normalize options value (convert value to lowercase).
/// Note that this option is ignored and will be removed in the future. All case-insensitive values
/// are normalized automatically.
pub enable_options_value_normalization: bool, warn = "`enable_options_value_normalization` is deprecated and ignored", default = false
/// Configure the SQL dialect used by DataFusion's parser; supported values include: Generic,
/// MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks.
pub dialect: Dialect, default = Dialect::Generic
// no need to lowercase because `sqlparser::dialect_from_str`] is case-insensitive
/// If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but
/// ignore the length. If false, error if a `VARCHAR` with a length is
/// specified. The Arrow type system does not have a notion of maximum
/// string length and thus DataFusion can not enforce such limits.
pub support_varchar_with_length: bool, default = true
/// If true, string types (VARCHAR, CHAR, Text, and String) are mapped to `Utf8View` during SQL planning.
/// If false, they are mapped to `Utf8`.
/// Default is true.
pub map_string_types_to_utf8view: bool, default = true
/// When set to true, the source locations relative to the original SQL
/// query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected
/// and recorded in the logical plan nodes.
pub collect_spans: bool, default = false
/// Specifies the recursion depth limit when parsing complex SQL Queries
pub recursion_limit: usize, default = 50
/// Specifies the default null ordering for query results. There are 4 options:
/// - `nulls_max`: Nulls appear last in ascending order.
/// - `nulls_min`: Nulls appear first in ascending order.
/// - `nulls_first`: Nulls always be first in any order.
/// - `nulls_last`: Nulls always be last in any order.
///
/// By default, `nulls_max` is used to follow Postgres's behavior.
/// postgres rule: <https://www.postgresql.org/docs/current/queries-order.html>
pub default_null_ordering: String, default = "nulls_max".to_string()
}
}
/// This is the SQL dialect used by DataFusion's parser.
/// This mirrors [sqlparser::dialect::Dialect](https://docs.rs/sqlparser/latest/sqlparser/dialect/trait.Dialect.html)
/// trait in order to offer an easier API and avoid adding the `sqlparser` dependency
#[derive(Debug, Default, Clone, Copy, PartialEq, Eq)]
pub enum Dialect {
#[default]
Generic,
MySQL,
PostgreSQL,
Hive,
SQLite,
Snowflake,
Redshift,
MsSQL,
ClickHouse,
BigQuery,
Ansi,
DuckDB,
Databricks,
}
impl AsRef<str> for Dialect {
fn as_ref(&self) -> &str {
match self {
Self::Generic => "generic",
Self::MySQL => "mysql",
Self::PostgreSQL => "postgresql",
Self::Hive => "hive",
Self::SQLite => "sqlite",
Self::Snowflake => "snowflake",
Self::Redshift => "redshift",
Self::MsSQL => "mssql",
Self::ClickHouse => "clickhouse",
Self::BigQuery => "bigquery",
Self::Ansi => "ansi",
Self::DuckDB => "duckdb",
Self::Databricks => "databricks",
}
}
}
impl FromStr for Dialect {
type Err = DataFusionError;
fn from_str(s: &str) -> Result<Self, Self::Err> {
let value = match s.to_ascii_lowercase().as_str() {
"generic" => Self::Generic,
"mysql" => Self::MySQL,
"postgresql" | "postgres" => Self::PostgreSQL,
"hive" => Self::Hive,
"sqlite" => Self::SQLite,
"snowflake" => Self::Snowflake,
"redshift" => Self::Redshift,
"mssql" => Self::MsSQL,
"clickhouse" => Self::ClickHouse,
"bigquery" => Self::BigQuery,
"ansi" => Self::Ansi,
"duckdb" => Self::DuckDB,
"databricks" => Self::Databricks,
other => {
let error_message = format!(
"Invalid Dialect: {other}. Expected one of: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB, Databricks"
);
return Err(DataFusionError::Configuration(error_message));
}
};
Ok(value)
}
}
impl ConfigField for Dialect {
fn visit<V: Visit>(&self, v: &mut V, key: &str, description: &'static str) {
v.some(key, self, description)
}
fn set(&mut self, _: &str, value: &str) -> Result<()> {
*self = Self::from_str(value)?;
Ok(())
}
}
impl Display for Dialect {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
let str = self.as_ref();
write!(f, "{str}")
}
}
#[derive(Debug, Default, Clone, Copy, PartialEq, Eq)]
pub enum SpillCompression {
Zstd,
Lz4Frame,
#[default]
Uncompressed,
}
impl FromStr for SpillCompression {
type Err = DataFusionError;
fn from_str(s: &str) -> Result<Self, Self::Err> {
match s.to_ascii_lowercase().as_str() {
"zstd" => Ok(Self::Zstd),
"lz4_frame" => Ok(Self::Lz4Frame),
"uncompressed" | "" => Ok(Self::Uncompressed),
other => Err(DataFusionError::Configuration(format!(
"Invalid Spill file compression type: {other}. Expected one of: zstd, lz4_frame, uncompressed"
))),
}
}
}
impl ConfigField for SpillCompression {
fn visit<V: Visit>(&self, v: &mut V, key: &str, description: &'static str) {
v.some(key, self, description)
}
fn set(&mut self, _: &str, value: &str) -> Result<()> {
*self = SpillCompression::from_str(value)?;
Ok(())
}
}
impl Display for SpillCompression {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
let str = match self {
Self::Zstd => "zstd",
Self::Lz4Frame => "lz4_frame",
Self::Uncompressed => "uncompressed",
};
write!(f, "{str}")
}
}
impl From<SpillCompression> for Option<CompressionType> {
fn from(c: SpillCompression) -> Self {
match c {
SpillCompression::Zstd => Some(CompressionType::ZSTD),
SpillCompression::Lz4Frame => Some(CompressionType::LZ4_FRAME),
SpillCompression::Uncompressed => None,
}
}
}
config_namespace! {
/// Options related to query execution
///
/// See also: [`SessionConfig`]
///
/// [`SessionConfig`]: https://docs.rs/datafusion/latest/datafusion/prelude/struct.SessionConfig.html
pub struct ExecutionOptions {
/// Default batch size while creating new batches, it's especially useful for
/// buffer-in-memory batches since creating tiny batches would result in too much
/// metadata memory consumption
pub batch_size: usize, default = 8192
/// A perfect hash join (see `HashJoinExec` for more details) will be considered
/// if the range of keys (max - min) on the build side is < this threshold.
/// This provides a fast path for joins with very small key ranges,
/// bypassing the density check.
///
/// Currently only supports cases where build_side.num_rows() < u32::MAX.
/// Support for build_side.num_rows() >= u32::MAX will be added in the future.
pub perfect_hash_join_small_build_threshold: usize, default = 1024
/// The minimum required density of join keys on the build side to consider a
/// perfect hash join (see `HashJoinExec` for more details). Density is calculated as:
/// `(number of rows) / (max_key - min_key + 1)`.
/// A perfect hash join may be used if the actual key density > this
/// value.
///
/// Currently only supports cases where build_side.num_rows() < u32::MAX.
/// Support for build_side.num_rows() >= u32::MAX will be added in the future.
pub perfect_hash_join_min_key_density: f64, default = 0.15
/// When set to true, record batches will be examined between each operator and
/// small batches will be coalesced into larger batches. This is helpful when there
/// are highly selective filters or joins that could produce tiny output batches. The
/// target batch size is determined by the configuration setting
pub coalesce_batches: bool, default = true
/// Should DataFusion collect statistics when first creating a table.
/// Has no effect after the table is created. Applies to the default
/// `ListingTableProvider` in DataFusion. Defaults to true.
pub collect_statistics: bool, default = true
/// Number of partitions for query execution. Increasing partitions can increase
/// concurrency.
///
/// Defaults to the number of CPU cores on the system
pub target_partitions: usize, transform = ExecutionOptions::normalized_parallelism, default = get_available_parallelism()
/// The default time zone
///
/// Some functions, e.g. `now` return timestamps in this time zone
pub time_zone: Option<String>, default = None
/// Parquet options
pub parquet: ParquetOptions, default = Default::default()
/// Fan-out during initial physical planning.
///
/// This is mostly use to plan `UNION` children in parallel.
///
/// Defaults to the number of CPU cores on the system
pub planning_concurrency: usize, transform = ExecutionOptions::normalized_parallelism, default = get_available_parallelism()
/// When set to true, skips verifying that the schema produced by
/// planning the input of `LogicalPlan::Aggregate` exactly matches the
/// schema of the input plan.
///
/// When set to false, if the schema does not match exactly
/// (including nullability and metadata), a planning error will be raised.
///
/// This is used to workaround bugs in the planner that are now caught by
/// the new schema verification step.
pub skip_physical_aggregate_schema_check: bool, default = false
/// Sets the compression codec used when spilling data to disk.
///
/// Since datafusion writes spill files using the Arrow IPC Stream format,
/// only codecs supported by the Arrow IPC Stream Writer are allowed.
/// Valid values are: uncompressed, lz4_frame, zstd.
/// Note: lz4_frame offers faster (de)compression, but typically results in
/// larger spill files. In contrast, zstd achieves
/// higher compression ratios at the cost of slower (de)compression speed.
pub spill_compression: SpillCompression, default = SpillCompression::Uncompressed
/// Specifies the reserved memory for each spillable sort operation to
/// facilitate an in-memory merge.
///
/// When a sort operation spills to disk, the in-memory data must be
/// sorted and merged before being written to a file. This setting reserves
/// a specific amount of memory for that in-memory sort/merge process.
///
/// Note: This setting is irrelevant if the sort operation cannot spill
/// (i.e., if there's no `DiskManager` configured).
pub sort_spill_reservation_bytes: usize, default = 10 * 1024 * 1024
/// When sorting, below what size should data be concatenated
/// and sorted in a single RecordBatch rather than sorted in
/// batches and merged.
pub sort_in_place_threshold_bytes: usize, default = 1024 * 1024
/// Maximum size in bytes for individual spill files before rotating to a new file.
///
/// When operators spill data to disk (e.g., RepartitionExec), they write
/// multiple batches to the same file until this size limit is reached, then rotate
/// to a new file. This reduces syscall overhead compared to one-file-per-batch
/// while preventing files from growing too large.
///
/// A larger value reduces file creation overhead but may hold more disk space.
/// A smaller value creates more files but allows finer-grained space reclamation
/// as files can be deleted once fully consumed.
///
/// Now only `RepartitionExec` supports this spill file rotation feature, other spilling operators
/// may create spill files larger than the limit.
///
/// Default: 128 MB
pub max_spill_file_size_bytes: usize, default = 128 * 1024 * 1024
/// Number of files to read in parallel when inferring schema and statistics
pub meta_fetch_concurrency: usize, default = 32
/// Guarantees a minimum level of output files running in parallel.
/// RecordBatches will be distributed in round robin fashion to each
/// parallel writer. Each writer is closed and a new file opened once
/// soft_max_rows_per_output_file is reached.
pub minimum_parallel_output_files: usize, default = 4
/// Target number of rows in output files when writing multiple.
/// This is a soft max, so it can be exceeded slightly. There also
/// will be one file smaller than the limit if the total
/// number of rows written is not roughly divisible by the soft max
pub soft_max_rows_per_output_file: usize, default = 50000000
/// This is the maximum number of RecordBatches buffered
/// for each output file being worked. Higher values can potentially
/// give faster write performance at the cost of higher peak
/// memory consumption
pub max_buffered_batches_per_output_file: usize, default = 2
/// Should sub directories be ignored when scanning directories for data
/// files. Defaults to true (ignores subdirectories), consistent with
/// Hive. Note that this setting does not affect reading partitioned
/// tables (e.g. `/table/year=2021/month=01/data.parquet`).
pub listing_table_ignore_subdirectory: bool, default = true
/// Should a `ListingTable` created through the `ListingTableFactory` infer table
/// partitions from Hive compliant directories. Defaults to true (partition columns are
/// inferred and will be represented in the table schema).
pub listing_table_factory_infer_partitions: bool, default = true
/// Should DataFusion support recursive CTEs
pub enable_recursive_ctes: bool, default = true
/// Attempt to eliminate sorts by packing & sorting files with non-overlapping
/// statistics into the same file groups.
/// Currently experimental
pub split_file_groups_by_statistics: bool, default = false
/// Should DataFusion keep the columns used for partition_by in the output RecordBatches
pub keep_partition_by_columns: bool, default = false
/// Aggregation ratio (number of distinct groups / number of input rows)
/// threshold for skipping partial aggregation. If the value is greater
/// then partial aggregation will skip aggregation for further input
pub skip_partial_aggregation_probe_ratio_threshold: f64, default = 0.8
/// Number of input rows partial aggregation partition should process, before
/// aggregation ratio check and trying to switch to skipping aggregation mode
pub skip_partial_aggregation_probe_rows_threshold: usize, default = 100_000
/// Should DataFusion use row number estimates at the input to decide
/// whether increasing parallelism is beneficial or not. By default,
/// only exact row numbers (not estimates) are used for this decision.
/// Setting this flag to `true` will likely produce better plans.
/// if the source of statistics is accurate.
/// We plan to make this the default in the future.
pub use_row_number_estimates_to_optimize_partitioning: bool, default = false
/// Should DataFusion enforce batch size in joins or not. By default,
/// DataFusion will not enforce batch size in joins. Enforcing batch size
/// in joins can reduce memory usage when joining large
/// tables with a highly-selective join filter, but is also slightly slower.
pub enforce_batch_size_in_joins: bool, default = false
/// Size (bytes) of data buffer DataFusion uses when writing output files.
/// This affects the size of the data chunks that are uploaded to remote
/// object stores (e.g. AWS S3). If very large (>= 100 GiB) output files are being
/// written, it may be necessary to increase this size to avoid errors from
/// the remote end point.
pub objectstore_writer_buffer_size: usize, default = 10 * 1024 * 1024
/// Whether to enable ANSI SQL mode.
///
/// The flag is experimental and relevant only for DataFusion Spark built-in functions
///
/// When `enable_ansi_mode` is set to `true`, the query engine follows ANSI SQL
/// semantics for expressions, casting, and error handling. This means:
/// - **Strict type coercion rules:** implicit casts between incompatible types are disallowed.
/// - **Standard SQL arithmetic behavior:** operations such as division by zero,
/// numeric overflow, or invalid casts raise runtime errors rather than returning
/// `NULL` or adjusted values.
/// - **Consistent ANSI behavior** for string concatenation, comparisons, and `NULL` handling.
///
/// When `enable_ansi_mode` is `false` (the default), the engine uses a more permissive,
/// non-ANSI mode designed for user convenience and backward compatibility. In this mode:
/// - Implicit casts between types are allowed (e.g., string to integer when possible).
/// - Arithmetic operations are more lenient — for example, `abs()` on the minimum
/// representable integer value returns the input value instead of raising overflow.
/// - Division by zero or invalid casts may return `NULL` instead of failing.
///
/// # Default
/// `false` — ANSI SQL mode is disabled by default.
pub enable_ansi_mode: bool, default = false
}
}
config_namespace! {
/// Options for reading and writing parquet files
///
/// See also: [`SessionConfig`]
///
/// [`SessionConfig`]: https://docs.rs/datafusion/latest/datafusion/prelude/struct.SessionConfig.html
pub struct ParquetOptions {
// The following options affect reading parquet files
/// (reading) If true, reads the Parquet data page level metadata (the
/// Page Index), if present, to reduce the I/O and number of
/// rows decoded.
pub enable_page_index: bool, default = true
/// (reading) If true, the parquet reader attempts to skip entire row groups based
/// on the predicate in the query and the metadata (min/max values) stored in
/// the parquet file
pub pruning: bool, default = true
/// (reading) If true, the parquet reader skip the optional embedded metadata that may be in
/// the file Schema. This setting can help avoid schema conflicts when querying
/// multiple parquet files with schemas containing compatible types but different metadata
pub skip_metadata: bool, default = true
/// (reading) If specified, the parquet reader will try and fetch the last `size_hint`
/// bytes of the parquet file optimistically. If not specified, two reads are required:
/// One read to fetch the 8-byte parquet footer and
/// another to fetch the metadata length encoded in the footer
/// Default setting to 512 KiB, which should be sufficient for most parquet files,
/// it can reduce one I/O operation per parquet file. If the metadata is larger than
/// the hint, two reads will still be performed.
pub metadata_size_hint: Option<usize>, default = Some(512 * 1024)
/// (reading) If true, filter expressions are be applied during the parquet decoding operation to
/// reduce the number of rows decoded. This optimization is sometimes called "late materialization".
pub pushdown_filters: bool, default = false
/// (reading) If true, filter expressions evaluated during the parquet decoding operation
/// will be reordered heuristically to minimize the cost of evaluation. If false,
/// the filters are applied in the same order as written in the query
pub reorder_filters: bool, default = false
/// (reading) Force the use of RowSelections for filter results, when
/// pushdown_filters is enabled. If false, the reader will automatically
/// choose between a RowSelection and a Bitmap based on the number and
/// pattern of selected rows.
pub force_filter_selections: bool, default = false
/// (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`,
/// and `Binary/BinaryLarge` with `BinaryView`.
pub schema_force_view_types: bool, default = true
/// (reading) If true, parquet reader will read columns of
/// `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`.
///
/// Parquet files generated by some legacy writers do not correctly set
/// the UTF8 flag for strings, causing string columns to be loaded as
/// BLOB instead.
pub binary_as_string: bool, default = false
/// (reading) If true, parquet reader will read columns of
/// physical type int96 as originating from a different resolution
/// than nanosecond. This is useful for reading data from systems like Spark
/// which stores microsecond resolution timestamps in an int96 allowing it
/// to write values with a larger date range than 64-bit timestamps with
/// nanosecond resolution.
pub coerce_int96: Option<String>, transform = str::to_lowercase, default = None
/// (reading) Use any available bloom filters when reading parquet files
pub bloom_filter_on_read: bool, default = true
/// (reading) The maximum predicate cache size, in bytes. When
/// `pushdown_filters` is enabled, sets the maximum memory used to cache
/// the results of predicate evaluation between filter evaluation and
/// output generation. Decreasing this value will reduce memory usage,
/// but may increase IO and CPU usage. None means use the default
/// parquet reader setting. 0 means no caching.
pub max_predicate_cache_size: Option<usize>, default = None
// The following options affect writing to parquet files
// and map to parquet::file::properties::WriterProperties
/// (writing) Sets best effort maximum size of data page in bytes
pub data_pagesize_limit: usize, default = 1024 * 1024
/// (writing) Sets write_batch_size in rows
pub write_batch_size: usize, default = 1024
/// (writing) Sets parquet writer version
/// valid values are "1.0" and "2.0"
pub writer_version: DFParquetWriterVersion, default = DFParquetWriterVersion::default()
/// (writing) Skip encoding the embedded arrow metadata in the KV_meta
///
/// This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`.
/// Refer to <https://docs.rs/parquet/53.3.0/parquet/arrow/arrow_writer/struct.ArrowWriterOptions.html#method.with_skip_arrow_metadata>
pub skip_arrow_metadata: bool, default = false
/// (writing) Sets default parquet compression codec.
/// Valid values are: uncompressed, snappy, gzip(level),
/// brotli(level), lz4, zstd(level), and lz4_raw.
/// These values are not case sensitive. If NULL, uses
/// default parquet writer setting
///
/// Note that this default setting is not the same as
/// the default parquet writer setting.
pub compression: Option<String>, transform = str::to_lowercase, default = Some("zstd(3)".into())
/// (writing) Sets if dictionary encoding is enabled. If NULL, uses
/// default parquet writer setting
pub dictionary_enabled: Option<bool>, default = Some(true)
/// (writing) Sets best effort maximum dictionary page size, in bytes
pub dictionary_page_size_limit: usize, default = 1024 * 1024
/// (writing) Sets if statistics are enabled for any column
/// Valid values are: "none", "chunk", and "page"
/// These values are not case sensitive. If NULL, uses
/// default parquet writer setting
pub statistics_enabled: Option<String>, transform = str::to_lowercase, default = Some("page".into())
/// (writing) Target maximum number of rows in each row group (defaults to 1M
/// rows). Writing larger row groups requires more memory to write, but
/// can get better compression and be faster to read.
pub max_row_group_size: usize, default = 1024 * 1024
/// (writing) Sets "created by" property
pub created_by: String, default = concat!("datafusion version ", env!("CARGO_PKG_VERSION")).into()
/// (writing) Sets column index truncate length
pub column_index_truncate_length: Option<usize>, default = Some(64)
/// (writing) Sets statistics truncate length. If NULL, uses
/// default parquet writer setting
pub statistics_truncate_length: Option<usize>, default = Some(64)
/// (writing) Sets best effort maximum number of rows in data page
pub data_page_row_count_limit: usize, default = 20_000
/// (writing) Sets default encoding for any column.
/// Valid values are: plain, plain_dictionary, rle,
/// bit_packed, delta_binary_packed, delta_length_byte_array,
/// delta_byte_array, rle_dictionary, and byte_stream_split.
/// These values are not case sensitive. If NULL, uses
/// default parquet writer setting
pub encoding: Option<String>, transform = str::to_lowercase, default = None
/// (writing) Write bloom filters for all columns when creating parquet files
pub bloom_filter_on_write: bool, default = false
/// (writing) Sets bloom filter false positive probability. If NULL, uses
/// default parquet writer setting
pub bloom_filter_fpp: Option<f64>, default = None
/// (writing) Sets bloom filter number of distinct values. If NULL, uses
/// default parquet writer setting
pub bloom_filter_ndv: Option<u64>, default = None
/// (writing) Controls whether DataFusion will attempt to speed up writing
/// parquet files by serializing them in parallel. Each column
/// in each row group in each output file are serialized in parallel
/// leveraging a maximum possible core count of n_files*n_row_groups*n_columns.
pub allow_single_file_parallelism: bool, default = true
/// (writing) By default parallel parquet writer is tuned for minimum
/// memory usage in a streaming execution plan. You may see
/// a performance benefit when writing large parquet files
/// by increasing maximum_parallel_row_group_writers and
/// maximum_buffered_record_batches_per_stream if your system
/// has idle cores and can tolerate additional memory usage.
/// Boosting these values is likely worthwhile when
/// writing out already in-memory data, such as from a cached
/// data frame.
pub maximum_parallel_row_group_writers: usize, default = 1
/// (writing) By default parallel parquet writer is tuned for minimum
/// memory usage in a streaming execution plan. You may see
/// a performance benefit when writing large parquet files
/// by increasing maximum_parallel_row_group_writers and
/// maximum_buffered_record_batches_per_stream if your system
/// has idle cores and can tolerate additional memory usage.
/// Boosting these values is likely worthwhile when
/// writing out already in-memory data, such as from a cached
/// data frame.
pub maximum_buffered_record_batches_per_stream: usize, default = 2
}
}
config_namespace! {
/// Options for configuring Parquet Modular Encryption
///
/// To use Parquet encryption, you must enable the `parquet_encryption` feature flag, as it is not activated by default.
pub struct ParquetEncryptionOptions {
/// Optional file decryption properties
pub file_decryption: Option<ConfigFileDecryptionProperties>, default = None
/// Optional file encryption properties
pub file_encryption: Option<ConfigFileEncryptionProperties>, default = None
/// Identifier for the encryption factory to use to create file encryption and decryption properties.
/// Encryption factories can be registered in the runtime environment with
/// `RuntimeEnv::register_parquet_encryption_factory`.
pub factory_id: Option<String>, default = None
/// Any encryption factory specific options
pub factory_options: EncryptionFactoryOptions, default = EncryptionFactoryOptions::default()
}
}
impl ParquetEncryptionOptions {
/// Specify the encryption factory to use for Parquet modular encryption, along with its configuration
pub fn configure_factory(
&mut self,
factory_id: &str,
config: &impl ExtensionOptions,
) {
self.factory_id = Some(factory_id.to_owned());
self.factory_options.options.clear();
for entry in config.entries() {
if let Some(value) = entry.value {
self.factory_options.options.insert(entry.key, value);
}
}
}
}
config_namespace! {
/// Options related to query optimization
///
/// See also: [`SessionConfig`]
///
/// [`SessionConfig`]: https://docs.rs/datafusion/latest/datafusion/prelude/struct.SessionConfig.html
pub struct OptimizerOptions {
/// When set to true, the optimizer will push a limit operation into
/// grouped aggregations which have no aggregate expressions, as a soft limit,
/// emitting groups once the limit is reached, before all rows in the group are read.
pub enable_distinct_aggregation_soft_limit: bool, default = true
/// When set to true, the physical plan optimizer will try to add round robin
/// repartitioning to increase parallelism to leverage more CPU cores
pub enable_round_robin_repartition: bool, default = true
/// When set to true, the optimizer will attempt to perform limit operations
/// during aggregations, if possible
pub enable_topk_aggregation: bool, default = true
/// When set to true, the optimizer will attempt to push limit operations
/// past window functions, if possible
pub enable_window_limits: bool, default = true
/// When set to true, the optimizer will attempt to push down TopK dynamic filters
/// into the file scan phase.
pub enable_topk_dynamic_filter_pushdown: bool, default = true
/// When set to true, the optimizer will attempt to push down Join dynamic filters
/// into the file scan phase.
pub enable_join_dynamic_filter_pushdown: bool, default = true
/// When set to true, the optimizer will attempt to push down Aggregate dynamic filters
/// into the file scan phase.
pub enable_aggregate_dynamic_filter_pushdown: bool, default = true
/// When set to true attempts to push down dynamic filters generated by operators (TopK, Join & Aggregate) into the file scan phase.
/// For example, for a query such as `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer
/// will attempt to push down the current top 10 timestamps that the TopK operator references into the file scans.
/// This means that if we already have 10 timestamps in the year 2025
/// any files that only have timestamps in the year 2024 can be skipped / pruned at various stages in the scan.
/// The config will suppress `enable_join_dynamic_filter_pushdown`, `enable_topk_dynamic_filter_pushdown` & `enable_aggregate_dynamic_filter_pushdown`
/// So if you disable `enable_topk_dynamic_filter_pushdown`, then enable `enable_dynamic_filter_pushdown`, the `enable_topk_dynamic_filter_pushdown` will be overridden.
pub enable_dynamic_filter_pushdown: bool, default = true
/// When set to true, the optimizer will insert filters before a join between
/// a nullable and non-nullable column to filter out nulls on the nullable side. This
/// filter can add additional overhead when the file format does not fully support
/// predicate push down.
pub filter_null_join_keys: bool, default = false
/// Should DataFusion repartition data using the aggregate keys to execute aggregates
/// in parallel using the provided `target_partitions` level
pub repartition_aggregations: bool, default = true
/// Minimum total files size in bytes to perform file scan repartitioning.
pub repartition_file_min_size: usize, default = 10 * 1024 * 1024
/// Should DataFusion repartition data using the join keys to execute joins in parallel
/// using the provided `target_partitions` level
pub repartition_joins: bool, default = true
/// Should DataFusion allow symmetric hash joins for unbounded data sources even when
/// its inputs do not have any ordering or filtering If the flag is not enabled,
/// the SymmetricHashJoin operator will be unable to prune its internal buffers,
/// resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right,
/// RightAnti, and RightSemi - being produced only at the end of the execution.
/// This is not typical in stream processing. Additionally, without proper design for
/// long runner execution, all types of joins may encounter out-of-memory errors.
pub allow_symmetric_joins_without_pruning: bool, default = true
/// When set to `true`, datasource partitions will be repartitioned to achieve maximum parallelism.
/// This applies to both in-memory partitions and FileSource's file groups (1 group is 1 partition).
///
/// For FileSources, only Parquet and CSV formats are currently supported.
///
/// If set to `true` for a FileSource, all files will be repartitioned evenly (i.e., a single large file
/// might be partitioned into smaller chunks) for parallel scanning.
/// If set to `false` for a FileSource, different files will be read in parallel, but repartitioning won't
/// happen within a single file.
///
/// If set to `true` for an in-memory source, all memtable's partitions will have their batches
/// repartitioned evenly to the desired number of `target_partitions`. Repartitioning can change
/// the total number of partitions and batches per partition, but does not slice the initial
/// record tables provided to the MemTable on creation.
pub repartition_file_scans: bool, default = true
/// Minimum number of distinct partition values required to group files by their
/// Hive partition column values (enabling Hash partitioning declaration).
///
/// How the option is used:
/// - preserve_file_partitions=0: Disable it.
/// - preserve_file_partitions=1: Always enable it.
/// - preserve_file_partitions=N, actual file partitions=M: Only enable when M >= N.
/// This threshold preserves I/O parallelism when file partitioning is below it.
///
/// Note: This may reduce parallelism, rooting from the I/O level, if the number of distinct
/// partitions is less than the target_partitions.
pub preserve_file_partitions: usize, default = 0