Configuration Settings¶
The following configuration options can be passed to SessionConfig
to control various aspects of query execution.
For applications which do not expose SessionConfig
, like datafusion-cli
, these options may also be set via environment variables.
To construct a session with options from the environment, use SessionConfig::from_env
.
The name of the environment variable is the option’s key, transformed to uppercase and with periods replaced with underscores.
For example, to configure datafusion.execution.batch_size
you would set the DATAFUSION_EXECUTION_BATCH_SIZE
environment variable.
Values are parsed according to the same rules used in casts from Utf8.
If the value in the environment variable cannot be cast to the type of the configuration option, the default value will be used instead and a warning emitted.
Environment variables are read during SessionConfig
initialisation so they must be set beforehand and will not affect running sessions.
key |
default |
description |
---|---|---|
datafusion.catalog.create_default_catalog_and_schema |
true |
Whether the default catalog and schema should be created automatically. |
datafusion.catalog.default_catalog |
datafusion |
The default catalog name - this impacts what SQL queries use if not specified |
datafusion.catalog.default_schema |
public |
The default schema name - this impacts what SQL queries use if not specified |
datafusion.catalog.information_schema |
false |
Should DataFusion provide access to |
datafusion.catalog.location |
NULL |
Location scanned to load tables for |
datafusion.catalog.format |
NULL |
Type of |
datafusion.catalog.has_header |
true |
Default value for |
datafusion.catalog.newlines_in_values |
false |
Specifies whether newlines in (quoted) CSV values are supported. This is the default value for |
datafusion.execution.batch_size |
8192 |
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 |
datafusion.execution.coalesce_batches |
true |
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 |
datafusion.execution.collect_statistics |
false |
Should DataFusion collect statistics after listing files |
datafusion.execution.target_partitions |
0 |
Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system |
datafusion.execution.time_zone |
+00:00 |
The default time zone Some functions, e.g. |
datafusion.execution.parquet.enable_page_index |
true |
(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. |
datafusion.execution.parquet.pruning |
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 |
datafusion.execution.parquet.skip_metadata |
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 |
datafusion.execution.parquet.metadata_size_hint |
NULL |
(reading) If specified, the parquet reader will try and fetch the last |
datafusion.execution.parquet.pushdown_filters |
false |
(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”. |
datafusion.execution.parquet.reorder_filters |
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 |
datafusion.execution.parquet.schema_force_view_types |
false |
(reading) If true, parquet reader will read columns of |
datafusion.execution.parquet.data_pagesize_limit |
1048576 |
(writing) Sets best effort maximum size of data page in bytes |
datafusion.execution.parquet.write_batch_size |
1024 |
(writing) Sets write_batch_size in bytes |
datafusion.execution.parquet.writer_version |
1.0 |
(writing) Sets parquet writer version valid values are “1.0” and “2.0” |
datafusion.execution.parquet.compression |
zstd(3) |
(writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, 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. |
datafusion.execution.parquet.dictionary_enabled |
true |
(writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting |
datafusion.execution.parquet.dictionary_page_size_limit |
1048576 |
(writing) Sets best effort maximum dictionary page size, in bytes |
datafusion.execution.parquet.statistics_enabled |
page |
(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 |
datafusion.execution.parquet.max_statistics_size |
4096 |
(writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting |
datafusion.execution.parquet.max_row_group_size |
1048576 |
(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. |
datafusion.execution.parquet.created_by |
datafusion version 42.0.0 |
(writing) Sets “created by” property |
datafusion.execution.parquet.column_index_truncate_length |
64 |
(writing) Sets column index truncate length |
datafusion.execution.parquet.data_page_row_count_limit |
20000 |
(writing) Sets best effort maximum number of rows in data page |
datafusion.execution.parquet.encoding |
NULL |
(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 |
datafusion.execution.parquet.bloom_filter_on_read |
true |
(writing) Use any available bloom filters when reading parquet files |
datafusion.execution.parquet.bloom_filter_on_write |
false |
(writing) Write bloom filters for all columns when creating parquet files |
datafusion.execution.parquet.bloom_filter_fpp |
NULL |
(writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting |
datafusion.execution.parquet.bloom_filter_ndv |
NULL |
(writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting |
datafusion.execution.parquet.allow_single_file_parallelism |
true |
(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_filesn_row_groupsn_columns. |
datafusion.execution.parquet.maximum_parallel_row_group_writers |
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. |
datafusion.execution.parquet.maximum_buffered_record_batches_per_stream |
2 |
(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. |
datafusion.execution.planning_concurrency |
0 |
Fan-out during initial physical planning. This is mostly use to plan |
datafusion.execution.sort_spill_reservation_bytes |
10485760 |
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 |
datafusion.execution.sort_in_place_threshold_bytes |
1048576 |
When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. |
datafusion.execution.meta_fetch_concurrency |
32 |
Number of files to read in parallel when inferring schema and statistics |
datafusion.execution.minimum_parallel_output_files |
4 |
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. |
datafusion.execution.soft_max_rows_per_output_file |
50000000 |
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 |
datafusion.execution.max_buffered_batches_per_output_file |
2 |
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 |
datafusion.execution.listing_table_ignore_subdirectory |
true |
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. |
datafusion.execution.enable_recursive_ctes |
true |
Should DataFusion support recursive CTEs |
datafusion.execution.split_file_groups_by_statistics |
false |
Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental |
datafusion.execution.keep_partition_by_columns |
false |
Should DataFusion keep the columns used for partition_by in the output RecordBatches |
datafusion.execution.skip_partial_aggregation_probe_ratio_threshold |
0.8 |
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 |
datafusion.execution.skip_partial_aggregation_probe_rows_threshold |
100000 |
Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode |
datafusion.execution.use_row_number_estimates_to_optimize_partitioning |
false |
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 |
datafusion.optimizer.enable_distinct_aggregation_soft_limit |
true |
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. |
datafusion.optimizer.enable_round_robin_repartition |
true |
When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores |
datafusion.optimizer.enable_topk_aggregation |
true |
When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible |
datafusion.optimizer.filter_null_join_keys |
false |
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. |
datafusion.optimizer.repartition_aggregations |
true |
Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided |
datafusion.optimizer.repartition_file_min_size |
10485760 |
Minimum total files size in bytes to perform file scan repartitioning. |
datafusion.optimizer.repartition_joins |
true |
Should DataFusion repartition data using the join keys to execute joins in parallel using the provided |
datafusion.optimizer.allow_symmetric_joins_without_pruning |
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. |
datafusion.optimizer.repartition_file_scans |
true |
When set to |
datafusion.optimizer.repartition_windows |
true |
Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided |
datafusion.optimizer.repartition_sorts |
true |
Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below |
datafusion.optimizer.prefer_existing_sort |
false |
When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting |
datafusion.optimizer.skip_failed_rules |
false |
When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail |
datafusion.optimizer.max_passes |
3 |
Number of times that the optimizer will attempt to optimize the plan |
datafusion.optimizer.top_down_join_key_reordering |
true |
When set to true, the physical plan optimizer will run a top down process to reorder the join keys |
datafusion.optimizer.prefer_hash_join |
true |
When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory |
datafusion.optimizer.hash_join_single_partition_threshold |
1048576 |
The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition |
datafusion.optimizer.hash_join_single_partition_threshold_rows |
131072 |
The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition |
datafusion.optimizer.default_filter_selectivity |
20 |
The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). |
datafusion.optimizer.prefer_existing_union |
false |
When set to true, the optimizer will not attempt to convert Union to Interleave |
datafusion.optimizer.expand_views_at_output |
false |
When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces |
datafusion.explain.logical_plan_only |
false |
When set to true, the explain statement will only print logical plans |
datafusion.explain.physical_plan_only |
false |
When set to true, the explain statement will only print physical plans |
datafusion.explain.show_statistics |
false |
When set to true, the explain statement will print operator statistics for physical plans |
datafusion.explain.show_sizes |
true |
When set to true, the explain statement will print the partition sizes |
datafusion.explain.show_schema |
false |
When set to true, the explain statement will print schema information |
datafusion.sql_parser.parse_float_as_decimal |
false |
When set to true, SQL parser will parse float as decimal type |
datafusion.sql_parser.enable_ident_normalization |
true |
When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) |
datafusion.sql_parser.enable_options_value_normalization |
true |
When set to true, SQL parser will normalize options value (convert value to lowercase) |
datafusion.sql_parser.dialect |
generic |
Configure the SQL dialect used by DataFusion’s parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, and Ansi. |
datafusion.sql_parser.support_varchar_with_length |
true |
If true, permit lengths for |