From 4498533f6e8864007a864e5e44faf1a5c4e8440f Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Thu, 29 Dec 2022 12:16:22 +0000 Subject: [PATCH 01/12] Structify ConfigOptions (#4517) --- benchmarks/src/bin/h2o.rs | 3 +- datafusion/core/src/bin/print_config_docs.rs | 4 +- .../core/src/catalog/information_schema.rs | 14 +- datafusion/core/src/config.rs | 967 ++++++++---------- .../src/datasource/file_format/parquet.rs | 12 +- datafusion/core/src/execution/context.rs | 296 ++---- .../src/physical_optimizer/enforcement.rs | 13 +- .../src/physical_optimizer/join_selection.rs | 9 +- .../src/physical_optimizer/repartition.rs | 6 +- .../src/physical_plan/coalesce_batches.rs | 23 +- .../src/physical_plan/file_format/parquet.rs | 16 +- datafusion/core/src/physical_plan/planner.rs | 22 +- datafusion/core/tests/config_from_env.rs | 15 +- datafusion/core/tests/parquet/mod.rs | 6 +- datafusion/core/tests/sql/explain_analyze.rs | 27 +- datafusion/core/tests/sql/mod.rs | 30 +- datafusion/core/tests/sql/set_variable.rs | 2 +- parquet-test-utils/src/lib.rs | 14 +- 18 files changed, 611 insertions(+), 868 deletions(-) diff --git a/benchmarks/src/bin/h2o.rs b/benchmarks/src/bin/h2o.rs index 88f4084e87a5..caa7d39950f4 100644 --- a/benchmarks/src/bin/h2o.rs +++ b/benchmarks/src/bin/h2o.rs @@ -63,7 +63,8 @@ async fn main() -> Result<()> { async fn group_by(opt: &GroupBy) -> Result<()> { let path = opt.path.to_str().unwrap(); - let config = SessionConfig::from_env().with_batch_size(65535); + let mut config = SessionConfig::from_env()?; + config.config_options_mut().built_in.execution.batch_size = 65535; let ctx = SessionContext::with_config(config); diff --git a/datafusion/core/src/bin/print_config_docs.rs b/datafusion/core/src/bin/print_config_docs.rs index 6e66fea5fe51..ce801dd95d51 100644 --- a/datafusion/core/src/bin/print_config_docs.rs +++ b/datafusion/core/src/bin/print_config_docs.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use datafusion::config::BuiltInConfigs; +use datafusion::config::ConfigOptions; fn main() { - let docs = BuiltInConfigs::generate_config_markdown(); + let docs = ConfigOptions::generate_config_markdown(); println!("{}", docs); } diff --git a/datafusion/core/src/catalog/information_schema.rs b/datafusion/core/src/catalog/information_schema.rs index 96e5585e0fb4..df98afe716d2 100644 --- a/datafusion/core/src/catalog/information_schema.rs +++ b/datafusion/core/src/catalog/information_schema.rs @@ -27,7 +27,7 @@ use arrow::{ record_batch::RecordBatch, }; -use crate::config::ConfigOptions; +use crate::config::{ConfigEntry, ConfigOptions}; use crate::datasource::streaming::{PartitionStream, StreamingTable}; use crate::datasource::TableProvider; use crate::execution::context::TaskContext; @@ -162,8 +162,8 @@ impl InformationSchemaConfig { config_options: &ConfigOptions, builder: &mut InformationSchemaDfSettingsBuilder, ) { - for (name, setting) in config_options.options() { - builder.add_setting(name, setting.to_string()); + for entry in config_options.entries() { + builder.add_setting(entry); } } } @@ -611,7 +611,7 @@ impl InformationSchemaDfSettings { fn new(config: InformationSchemaConfig) -> Self { let schema = Arc::new(Schema::new(vec![ Field::new("name", DataType::Utf8, false), - Field::new("setting", DataType::Utf8, false), + Field::new("setting", DataType::Utf8, true), ])); Self { schema, config } @@ -656,9 +656,9 @@ struct InformationSchemaDfSettingsBuilder { } impl InformationSchemaDfSettingsBuilder { - fn add_setting(&mut self, name: impl AsRef, setting: impl AsRef) { - self.names.append_value(name.as_ref()); - self.settings.append_value(setting.as_ref()); + fn add_setting(&mut self, entry: ConfigEntry) { + self.names.append_value(entry.key); + self.settings.append_option(entry.value); } fn finish(&mut self) -> RecordBatch { diff --git a/datafusion/core/src/config.rs b/datafusion/core/src/config.rs index 976b8c07376a..fc18087472ad 100644 --- a/datafusion/core/src/config.rs +++ b/datafusion/core/src/config.rs @@ -17,616 +17,517 @@ //! DataFusion Configuration Options -use arrow::datatypes::DataType; -use datafusion_common::ScalarValue; -use itertools::Itertools; -use log::warn; -use std::collections::{BTreeMap, HashMap}; -use std::env; -use std::fmt::{Debug, Formatter}; - -/*-************************************ -* Catalog related -**************************************/ -/// Configuration option "datafusion.catalog.create_default_catalog_and_schema" -pub const OPT_CREATE_DEFAULT_CATALOG_AND_SCHEMA: &str = - "datafusion.catalog.create_default_catalog_and_schema"; +use datafusion_common::{DataFusionError, Result}; +use std::any::Any; +use std::collections::BTreeMap; + +macro_rules! config_namespace { + ( + $(#[$meta:meta])* + $vis:vis struct $struct_name:ident { + $( + $(#[doc = $d:tt])* + $field_vis:vis $field_name:ident : $field_type:ty, default = $default:expr + )*$(,)* + } + ) => { + + #[derive(Debug, Clone, PartialEq)] + #[non_exhaustive] + $vis struct $struct_name{ + $( + $(#[doc = $d])* + $field_vis $field_name : $field_type, + )* + } + + impl ConfigField for $struct_name { + fn set(&mut self, key: &str, value: C) -> Result<()> { + let (key, rem) = key.split_once('.').unwrap_or((key, "")); + match key { + $( + stringify!($field_name) => self.$field_name.set(rem, value), + )* + _ => Err(DataFusionError::Internal( + format!(concat!("Config value \"{}\" not found on ", stringify!($struct_name)), key) + )) + } + } + + fn visit(&self, v: &mut V, key_prefix: &str, _description: &'static str) { + $( + let key = format!("{}.{}", key_prefix, stringify!($field_name)); + let desc = concat!($($d),*).trim(); + self.$field_name.visit(v, key.as_str(), desc); + )* + } + } + + impl Default for $struct_name { + fn default() -> Self { + Self { + $($field_name: $default),* + } + } + } + } +} + +config_namespace! { + pub struct CatalogOptions { + /// Number of partitions for query execution. Increasing partitions can increase + /// concurrency. Defaults to the number of cpu cores on the system. + pub create_default_catalog_and_schema: bool, default = true -/// Configuration option "datafusion.catalog.information_schema" -pub const OPT_INFORMATION_SCHEMA: &str = "datafusion.catalog.information_schema"; + /// 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 const OPT_CATALOG_LOCATION: &str = "datafusion.catalog.location"; - -/// Type of `TableProvider` to use when loading `default` schema -pub const OPT_CATALOG_TYPE: &str = "datafusion.catalog.type"; - -/*-************************************ -* Execution related -**************************************/ -/// Configuration option "datafusion.execution.batch_size" -pub const OPT_BATCH_SIZE: &str = "datafusion.execution.batch_size"; - -/// Configuration option "datafusion.execution.coalesce_batches" -pub const OPT_COALESCE_BATCHES: &str = "datafusion.execution.coalesce_batches"; - -/// Configuration option "datafusion.execution.coalesce_target_batch_size" -pub const OPT_COALESCE_TARGET_BATCH_SIZE: &str = - "datafusion.execution.coalesce_target_batch_size"; - -/// Configuration option "datafusion.execution.collect_statistics" -pub const OPT_COLLECT_STATISTICS: &str = "datafusion.execution.collect_statistics"; - -/// Configuration option "datafusion.execution.target_partitions" -pub const OPT_TARGET_PARTITIONS: &str = "datafusion.execution.target_partitions"; - -/// Configuration option "datafusion.execution.time_zone" -pub const OPT_TIME_ZONE: &str = "datafusion.execution.time_zone"; - -/*-************************************ -* Execution parquet related -**************************************/ -/// Configuration option "datafusion.execution.parquet.enable_page_index" -pub const OPT_PARQUET_ENABLE_PAGE_INDEX: &str = - "datafusion.execution.parquet.enable_page_index"; - -/// Configuration option "datafusion.execution.parquet.pruning" -pub const OPT_PARQUET_ENABLE_PRUNING: &str = "datafusion.execution.parquet.pruning"; - -/// Configuration option "datafusion.execution.parquet.pushdown_filters" -pub const OPT_PARQUET_PUSHDOWN_FILTERS: &str = - "datafusion.execution.parquet.pushdown_filters"; - -/// Configuration option "datafusion.execution.parquet.reorder_filters" -pub const OPT_PARQUET_REORDER_FILTERS: &str = - "datafusion.execution.parquet.reorder_filters"; - -/// Configuration option "datafusion.execution.parquet.skip_metadata" -pub const OPT_PARQUET_SKIP_METADATA: &str = "datafusion.execution.parquet.skip_metadata"; - -/// Configuration option "datafusion.execution.parquet.metadata_size_hint" -pub const OPT_PARQUET_METADATA_SIZE_HINT: &str = - "datafusion.execution.parquet.metadata_size_hint"; - -/*-************************************ -* Explain related -**************************************/ -/// Configuration option "datafusion.explain.logical_plan_only" -pub const OPT_EXPLAIN_LOGICAL_PLAN_ONLY: &str = "datafusion.explain.logical_plan_only"; - -/// Configuration option "datafusion.explain.physical_plan_only" -pub const OPT_EXPLAIN_PHYSICAL_PLAN_ONLY: &str = "datafusion.explain.physical_plan_only"; - -/*-************************************ -* Optimizer related -**************************************/ -/// Configuration option "datafusion.optimizer.filter_null_join_keys" -pub const OPT_FILTER_NULL_JOIN_KEYS: &str = "datafusion.optimizer.filter_null_join_keys"; - -/// Configuration option "datafusion.optimizer.repartition_aggregations" -pub const OPT_REPARTITION_AGGREGATIONS: &str = - "datafusion.optimizer.repartition_aggregations"; - -/// Configuration option "datafusion.optimizer.repartition_joins" -pub const OPT_REPARTITION_JOINS: &str = "datafusion.optimizer.repartition_joins"; - -/// Configuration option "datafusion.optimizer.repartition_windows" -pub const OPT_REPARTITION_WINDOWS: &str = "datafusion.optimizer.repartition_windows"; - -/// Configuration option "datafusion.optimizer.skip_failed_rules" -pub const OPT_OPTIMIZER_SKIP_FAILED_RULES: &str = - "datafusion.optimizer.skip_failed_rules"; - -/// Configuration option "datafusion.optimizer.max_passes" -pub const OPT_OPTIMIZER_MAX_PASSES: &str = "datafusion.optimizer.max_passes"; - -/// Configuration option "datafusion.optimizer.top_down_join_key_reordering" -pub const OPT_TOP_DOWN_JOIN_KEY_REORDERING: &str = - "datafusion.optimizer.top_down_join_key_reordering"; - -/// Configuration option "datafusion.optimizer.prefer_hash_join" -pub const OPT_PREFER_HASH_JOIN: &str = "datafusion.optimizer.prefer_hash_join"; - -/// Configuration option "atafusion.optimizer.hash_join_single_partition_threshold" -pub const OPT_HASH_JOIN_SINGLE_PARTITION_THRESHOLD: &str = - "datafusion.optimizer.hash_join_single_partition_threshold"; + /// Location scanned to load tables for `default` schema + pub location: Option, default = None -/// Configuration option "datafusion.execution.round_robin_repartition" -pub const OPT_ENABLE_ROUND_ROBIN_REPARTITION: &str = - "datafusion.optimizer.enable_round_robin_repartition"; + /// Type of `TableProvider` to use when loading `default` schema + pub format: Option, default = None -/// Definition of a configuration option -pub struct ConfigDefinition { - /// key used to identifier this configuration option - key: String, - /// Description to be used in generated documentation - description: String, - /// Data type of this option - data_type: DataType, - /// Default value - default_value: ScalarValue, + /// If the file has a header + pub has_header: bool, default = false + } } -macro_rules! get_conf_value { - ($SELF: expr, $TPE: ident, $KEY: expr, $TPE_NAME: expr) => { - match $SELF.get($KEY) { - Some(ScalarValue::$TPE(v)) => v, - Some(v) => { - warn!( - "Config type mismatch for {}. Expected: {}, got: {:?}", - $KEY, $TPE_NAME, &v - ); - None - } - None => None, - } - }; +config_namespace! { + pub struct ExecutionOptions { + /// Default batch size while creating new batches, it's especially useful for + /// buffer-in-memory batches since creating tiny batches would results in too much + /// metadata memory consumption + pub batch_size: usize, default = 8192 + + /// 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 + + /// Target batch size when coalescing batches. Used in conjunction with `coalesce_batches` + pub coalesce_target_batch_size: usize, default = 4096 + + /// Should DataFusion collect statistics after listing files + pub collect_statistics: bool, default = false + + /// 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, default = num_cpus::get() + + /// The default time zone + /// + /// Some functions, e.g. EXTRACT(HOUR from SOME_TIME), shift the underlying datetime + /// according to this time zone, and then extract the hour + pub time_zone: Option, default = Some("+00:00".into()) + + /// When set to true, the physical plan optimizer will try to add round robin + /// repartition to increase parallelism to leverage more CPU cores + pub round_robin_repartition: bool, default = true + + /// Parquet options + pub parquet: ParquetOptions, default = Default::default() + } } -impl ConfigDefinition { - /// Create a configuration option definition - pub fn new( - name: impl Into, - description: impl Into, - data_type: DataType, - default_value: ScalarValue, - ) -> Self { - Self { - key: name.into(), - description: description.into(), - data_type, - default_value, - } +config_namespace! { + pub struct ParquetOptions { + /// If true, uses parquet data page level metadata (Page Index) statistics + /// to reduce the number of rows decoded. + pub enable_page_index: bool, default = false + + /// 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 enable_pruning: bool, default = true + + /// 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 + + /// If specified, the parquet reader will try and fetch the last `size_hint` + /// bytes of the parquet file optimistically. If not specified, two read are required: + /// One read to fetch the 8-byte parquet footer and + /// another to fetch the metadata length encoded in the footer + pub metadata_size_hint: Option, default = None + + /// If true, filter expressions are be applied during the parquet decoding operation to + /// reduce the number of rows decoded + pub pushdown_filters: bool, default = false + + /// 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 } +} - /// Create a configuration option definition with a boolean value - pub fn new_bool( - key: impl Into, - description: impl Into, - default_value: bool, - ) -> Self { - Self::new( - key, - description, - DataType::Boolean, - ScalarValue::Boolean(Some(default_value)), - ) - } - - /// Create a configuration option definition with a u64 value - pub fn new_u64( - key: impl Into, - description: impl Into, - default_value: u64, - ) -> Self { - Self::new( - key, - description, - DataType::UInt64, - ScalarValue::UInt64(Some(default_value)), - ) - } - - /// Create a configuration option definition with a string value - pub fn new_string( - key: impl Into, - description: impl Into, - default_value: Option, - ) -> Self { - Self::new( - key, - description, - DataType::Utf8, - ScalarValue::Utf8(default_value), - ) +config_namespace! { + pub struct OptimizerOptions { + /// 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 + + /// 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 repartition data using the partitions keys to execute window + /// functions in parallel using the provided `target_partitions` level" + pub repartition_windows: bool, default = true + + /// 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 + pub skip_failed_rules: bool, default = true + + /// Number of times that the optimizer will attempt to optimize the plan + pub max_passes: usize, default = 3 + + /// When set to true, the physical plan optimizer will run a top down + /// process to reorder the join keys + pub top_down_join_key_reordering: bool, default = 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 + pub prefer_hash_join: bool, default = true + + /// The maximum estimated size in bytes for one input side of a HashJoin + /// will be collected into a single partition + pub hash_join_single_partition_threshold: usize, default = 1024 * 1024 } } -/// Contains definitions for all built-in configuration options -pub struct BuiltInConfigs { - /// Configuration option definitions - config_definitions: Vec, +config_namespace! { + pub struct ExplainOptions { + /// When set to true, the explain statement will only print logical plans + pub logical_plan_only: bool, default = false + + /// When set to true, the explain statement will only print physical plans + pub physical_plan_only: bool, default = false + } } -impl Default for BuiltInConfigs { - fn default() -> Self { - Self::new() +config_namespace! { + pub struct DataFusionOptions { + /// Catalog options + pub catalog: CatalogOptions, default = Default::default() + + /// Execution options + pub execution: ExecutionOptions, default = Default::default() + + /// Explain options + pub optimizer: OptimizerOptions, default = Default::default() + + /// Explain options + pub explain: ExplainOptions, default = Default::default() } } -impl BuiltInConfigs { - /// Create a new BuiltInConfigs struct containing definitions for all built-in - /// configuration options +/// A key value pair, with a corresponding description +#[derive(Debug)] +pub struct ConfigEntry { + /// A unique string to identify this config value + pub key: String, + + /// The value if any + pub value: Option, + + /// A description of this configuration entry + pub description: &'static str, +} + +/// Configuration options struct, able to store both built-in configuration and custom options +#[derive(Debug, Default, Clone)] +pub struct ConfigOptions { + /// Built-in DataFusion configuration + pub built_in: DataFusionOptions, + + /// Optional extensions registered using [`Extensions::insert`] + pub extensions: Extensions, +} + +impl ConfigOptions { + /// Creates a new [`ConfigOptions`] with default values pub fn new() -> Self { - Self { - config_definitions: vec![ConfigDefinition::new_u64( - OPT_TARGET_PARTITIONS, - "Number of partitions for query execution. Increasing partitions can increase \ - concurrency. Defaults to the number of cpu cores on the system.", - num_cpus::get() as u64, - ), - - ConfigDefinition::new_bool( - OPT_CREATE_DEFAULT_CATALOG_AND_SCHEMA, - "Whether the default catalog and schema should be created automatically.", - true - ), - - ConfigDefinition::new_bool( - OPT_INFORMATION_SCHEMA, - "Should DataFusion provide access to `information_schema` \ - virtual tables for displaying schema information", - false - ), - - ConfigDefinition::new_bool( - OPT_REPARTITION_JOINS, - "Should DataFusion repartition data using the join keys to execute joins in parallel \ - using the provided `target_partitions` level", - true - ), - - ConfigDefinition::new_bool( - OPT_REPARTITION_AGGREGATIONS, - "Should DataFusion repartition data using the aggregate keys to execute aggregates \ - in parallel using the provided `target_partitions` level", - true - ), - - ConfigDefinition::new_bool( - OPT_REPARTITION_WINDOWS, - "Should DataFusion repartition data using the partitions keys to execute window \ - functions in parallel using the provided `target_partitions` level", - true - ), - - ConfigDefinition::new_bool( - OPT_COLLECT_STATISTICS, - "Should DataFusion collect statistics after listing files", - false - ), - - ConfigDefinition::new_bool( - OPT_FILTER_NULL_JOIN_KEYS, - "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.", - false, - ), - ConfigDefinition::new_bool( - OPT_EXPLAIN_LOGICAL_PLAN_ONLY, - "When set to true, the explain statement will only print logical plans.", - false, - ), - ConfigDefinition::new_bool( - OPT_EXPLAIN_PHYSICAL_PLAN_ONLY, - "When set to true, the explain statement will only print physical plans.", - false, - ), - ConfigDefinition::new_u64( - OPT_BATCH_SIZE, - "Default batch size while creating new batches, it's especially useful for \ - buffer-in-memory batches since creating tiny batches would results in too much metadata \ - memory consumption.", - 8192, - ), - ConfigDefinition::new_bool( - OPT_COALESCE_BATCHES, - format!("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 \ - '{}'.", OPT_COALESCE_TARGET_BATCH_SIZE), - true, - ), - ConfigDefinition::new_u64( - OPT_COALESCE_TARGET_BATCH_SIZE, - format!("Target batch size when coalescing batches. Uses in conjunction with the \ - configuration setting '{}'.", OPT_COALESCE_BATCHES), - 4096, - ), - ConfigDefinition::new_string( - OPT_TIME_ZONE, - "The session time zone which some function require \ - e.g. EXTRACT(HOUR from SOME_TIME) shift the underline datetime according to the time zone, - then extract the hour.", - Some("+00:00".into()), - ), - ConfigDefinition::new_bool( - OPT_PARQUET_PUSHDOWN_FILTERS, - "If true, filter expressions are be applied during the parquet decoding operation to \ - reduce the number of rows decoded.", - false, - ), - ConfigDefinition::new_bool( - OPT_PARQUET_REORDER_FILTERS, - "If true, filter expressions evaluated during the parquet decoding opearation \ - 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.", - false, - ), - ConfigDefinition::new_bool( - OPT_PARQUET_ENABLE_PAGE_INDEX, - "If true, uses parquet data page level metadata (Page Index) statistics \ - to reduce the number of rows decoded.", - false, - ), - ConfigDefinition::new_bool( - OPT_PARQUET_ENABLE_PRUNING, - "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.", - true, - ), - ConfigDefinition::new_bool( - OPT_PARQUET_SKIP_METADATA, - "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.", - true, - ), - ConfigDefinition::new( - OPT_PARQUET_METADATA_SIZE_HINT, - "If specified, the parquet reader will try and fetch the last `size_hint` \ - bytes of the parquet file optimistically. If not specified, two read are required: \ - One read to fetch the 8-byte parquet footer and \ - another to fetch the metadata length encoded in the footer.", - DataType::UInt64, - ScalarValue::UInt64(None), - ), - ConfigDefinition::new_bool( - OPT_OPTIMIZER_SKIP_FAILED_RULES, - "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.", - true - ), - ConfigDefinition::new_u64( - OPT_OPTIMIZER_MAX_PASSES, - "Number of times that the optimizer will attempt to optimize the plan", - 3 - ), - ConfigDefinition::new_string( - OPT_CATALOG_LOCATION, - "Location scanned to load tables for `default` schema, defaults to None", - None, - ), - ConfigDefinition::new_string( - OPT_CATALOG_TYPE, - "Type of `TableProvider` to use when loading `default` schema. Defaults to None", - None, - ), - ConfigDefinition::new_bool( - OPT_TOP_DOWN_JOIN_KEY_REORDERING, - "When set to true, the physical plan optimizer will run a top down process to reorder the join keys. Defaults to true", - true, - ), - ConfigDefinition::new_bool( - OPT_PREFER_HASH_JOIN, - "When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently\ - than SortMergeJoin but consumes more memory. Defaults to true", - true, - ), - ConfigDefinition::new_u64( - OPT_HASH_JOIN_SINGLE_PARTITION_THRESHOLD, - "The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition", - 1024 * 1024, - ), - ConfigDefinition::new_bool( - OPT_ENABLE_ROUND_ROBIN_REPARTITION, - "When set to true, the physical plan optimizer will try to add round robin repartition to increase parallelism to leverage more CPU cores", - true, - ), - ] + Self::default() + } + + /// Create new ConfigOptions struct, taking values from + /// environment variables where possible. + /// + /// For example, setting `DATAFUSION_EXECUTION_BATCH_SIZE` will + /// control `datafusion.execution.batch_size`. + pub fn from_env() -> Result { + let mut ret = Self::default(); + for (k, v) in std::env::vars_os() { + let k = k.to_string_lossy().to_ascii_lowercase().replace('_', "."); + let v = v.to_string_lossy(); + + if let Some((prefix, key)) = k.split_once('.') { + if prefix == "datafusion" { + ret.built_in.set(key, v.as_ref())? + } else if let Some(e) = ret.extensions.0.get_mut(prefix) { + e.0.set(key, v.as_ref())? + } + } + } + Ok(ret) + } + + /// Set a configuration option + pub fn set(&mut self, key: &str, value: &str) -> Result<()> { + let (prefix, key) = key.split_once('.').ok_or_else(|| { + DataFusionError::Internal(format!( + "could not find config namespace for key \"{}\"", + key + )) + })?; + + if prefix == "datafusion" { + return self.built_in.set(key, value); + } + + let e = self.extensions.0.get_mut(prefix); + let e = e.ok_or_else(|| { + DataFusionError::Internal(format!( + "Could not find config namespace \"{}\"", + prefix + )) + })?; + e.0.set(key, value) + } + + /// Returns the [`ConfigEntry`] stored within this [`ConfigOptions`] + pub fn entries(&self) -> Vec { + struct Visitor(Vec); + + impl FieldVisitor for Visitor { + fn visit_some( + &mut self, + key: &str, + value: C, + description: &'static str, + ) { + self.0.push(ConfigEntry { + key: key.to_string(), + value: Some(value.to_string()), + description, + }) + } + + fn visit_none(&mut self, key: &str, description: &'static str) { + self.0.push(ConfigEntry { + key: key.to_string(), + value: None, + description, + }) + } } + + let mut v = Visitor(vec![]); + self.built_in.visit(&mut v, "datafusion", ""); + + v.0.extend(self.extensions.0.values().flat_map(|e| e.0.entries())); + v.0 } /// Generate documentation that can be included in the user guide pub fn generate_config_markdown() -> String { use std::fmt::Write as _; - let configs = Self::new(); - let mut docs = "| key | type | default | description |\n".to_string(); - docs += "|-----|------|---------|-------------|\n"; - let config_definitions: Vec<_> = configs - .config_definitions - .into_iter() - .map(normalize_for_display) - .collect(); + let mut s = Self::default(); + s.built_in.execution.target_partitions = 0; // Normalize for display - for config in config_definitions.iter().sorted_by_key(|c| c.key.as_str()) { + let mut docs = "| key | default | description |\n".to_string(); + docs += "|-----|---------|-------------|\n"; + for entry in s.entries() { let _ = writeln!( &mut docs, - "| {} | {} | {} | {} |", - config.key, config.data_type, config.default_value, config.description + "| {} | {} | {} |", + entry.key, + entry.value.as_deref().unwrap_or(""), + entry.description ); } docs } } -/// Normalizes a config definition prior to markdown display -fn normalize_for_display(mut v: ConfigDefinition) -> ConfigDefinition { - // Since the default value of target_partitions depends on the number of cores, - // set the default value to 0 in the docs. - if v.key == OPT_TARGET_PARTITIONS { - v.default_value = ScalarValue::UInt64(Some(0)) - } - v +/// [`ConfigExtension`] provides a mechanism to store third-party configuration within DataFusion +/// +/// Unfortunately associated constants are not currently object-safe, and so this +/// extends the object-safe [`ExtensionOptions`] +pub trait ConfigExtension: ExtensionOptions { + const PREFIX: &'static str; } -/// Configuration options struct. This can contain values for built-in and custom options -#[derive(Clone)] -pub struct ConfigOptions { - options: HashMap, +/// An object-safe API for storing arbitrary configuration +pub trait ExtensionOptions: Send + Sync + std::fmt::Debug + 'static { + fn as_any(&self) -> &dyn Any; + + fn as_any_mut(&mut self) -> &mut dyn Any; + + fn cloned(&self) -> Box; + + fn set(&mut self, key: &str, value: &str) -> Result<()>; + + fn entries(&self) -> Vec; } -/// Print the configurations in an ordered way so that we can directly compare the equality of two ConfigOptions by their debug strings -impl Debug for ConfigOptions { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - f.debug_struct("ConfigOptions") - .field( - "options", - &format!("{:?}", BTreeMap::from_iter(self.options.iter())), - ) - .finish() +/// A type-safe container for [`ConfigExtension`] +#[derive(Debug, Default, Clone)] +pub struct Extensions(BTreeMap<&'static str, ExtensionBox>); + +impl Extensions { + /// Registers a [`ConfigExtension`] with this [`ConfigOptions`] + pub fn insert(&mut self, extension: T) { + assert_ne!(T::PREFIX, "datafusion"); + let e = ExtensionBox(Box::new(extension)); + self.0.insert(T::PREFIX, e); } -} -impl Default for ConfigOptions { - fn default() -> Self { - Self::new() + /// Retrieves the extension of the given type if any + pub fn get(&self) -> Option<&T> { + self.0.get(T::PREFIX)?.0.as_any().downcast_ref() } -} -impl ConfigOptions { - /// Create new ConfigOptions struct - pub fn new() -> Self { - let built_in = BuiltInConfigs::new(); - let mut options = HashMap::with_capacity(built_in.config_definitions.len()); - for config_def in &built_in.config_definitions { - options.insert(config_def.key.clone(), config_def.default_value.clone()); - } - Self { options } + /// Retrieves the extension of the given type if any + pub fn get_mut(&mut self) -> Option<&mut T> { + let e = self.0.get_mut(T::PREFIX)?; + e.0.as_any_mut().downcast_mut() } +} - /// Create new ConfigOptions struct, taking values from - /// environment variables where possible. - /// - /// For example, setting `DATAFUSION_EXECUTION_BATCH_SIZE` will - /// control `datafusion.execution.batch_size`. - pub fn from_env() -> Self { - let built_in = BuiltInConfigs::new(); - let mut options = HashMap::with_capacity(built_in.config_definitions.len()); - for config_def in &built_in.config_definitions { - let config_value = { - let mut env_key = config_def.key.replace('.', "_"); - env_key.make_ascii_uppercase(); - match env::var(&env_key) { - Ok(value) => match ScalarValue::try_from_string( - value.clone(), - &config_def.data_type, - ) { - Ok(parsed) => parsed, - Err(_) => { - warn!("Warning: could not parse environment variable {}={} to type {}.", env_key, value, config_def.data_type); - config_def.default_value.clone() - } - }, - Err(_) => config_def.default_value.clone(), - } - }; - options.insert(config_def.key.clone(), config_value); - } - Self { options } +#[derive(Debug)] +struct ExtensionBox(Box); + +impl Clone for ExtensionBox { + fn clone(&self) -> Self { + Self(self.0.cloned()) } +} - /// set a configuration option - pub fn set(&mut self, key: &str, value: ScalarValue) { - self.options.insert(key.to_string(), value); +/// A trait that performs fallible coercion into a [`ConfigField`] +trait ConfigValue: std::fmt::Display { + fn parse_usize(&self) -> Result; + + fn parse_bool(&self) -> Result; +} + +impl ConfigValue for bool { + fn parse_usize(&self) -> Result { + Ok(*self as _) } - /// set a boolean configuration option - pub fn set_bool(&mut self, key: &str, value: bool) { - self.set(key, ScalarValue::Boolean(Some(value))) + fn parse_bool(&self) -> Result { + Ok(*self) } +} - /// set a `u64` configuration option - pub fn set_u64(&mut self, key: &str, value: u64) { - self.set(key, ScalarValue::UInt64(Some(value))) +impl ConfigValue for usize { + fn parse_usize(&self) -> Result { + Ok(*self as _) } - /// set a `usize` configuration option - pub fn set_usize(&mut self, key: &str, value: usize) { - let value: u64 = value.try_into().expect("convert u64 to usize"); - self.set(key, ScalarValue::UInt64(Some(value))) + fn parse_bool(&self) -> Result { + Ok(*self != 0) } +} - /// set a `String` configuration option - pub fn set_string(&mut self, key: &str, value: impl Into) { - self.set(key, ScalarValue::Utf8(Some(value.into()))) +impl ConfigValue for &str { + fn parse_usize(&self) -> Result { + self.parse() + .map_err(|e| DataFusionError::External(Box::new(e))) } - /// get a configuration option - pub fn get(&self, key: &str) -> Option { - self.options.get(key).cloned() + fn parse_bool(&self) -> Result { + self.parse() + .map_err(|e| DataFusionError::External(Box::new(e))) } +} - /// get a boolean configuration option - pub fn get_bool(&self, key: &str) -> Option { - get_conf_value!(self, Boolean, key, "bool") +impl ConfigValue for String { + fn parse_usize(&self) -> Result { + self.as_str().parse_usize() } - /// get a u64 configuration option - pub fn get_u64(&self, key: &str) -> Option { - get_conf_value!(self, UInt64, key, "u64") + fn parse_bool(&self) -> Result { + self.as_str().parse_bool() } +} + +/// A trait implemented by `config_namespace` and for field types that provides +/// the ability to walk and mutate the configuration tree +trait ConfigField { + fn visit(&self, v: &mut V, key: &str, description: &'static str); + + fn set(&mut self, key: &str, value: C) -> Result<()>; +} - /// get a u64 configuration option as a usize - pub fn get_usize(&self, key: &str) -> Option { - let v = get_conf_value!(self, UInt64, key, "usize"); - v.and_then(|v| v.try_into().ok()) +impl ConfigField for Option { + fn visit(&self, v: &mut V, key: &str, description: &'static str) { + match self { + Some(s) => s.visit(v, key, description), + None => v.visit_none(key, description), + } } - /// get a string configuration option - pub fn get_string(&self, key: &str) -> Option { - get_conf_value!(self, Utf8, key, "string") + fn set(&mut self, key: &str, value: C) -> Result<()> { + self.get_or_insert_with(Default::default).set(key, value) } +} - /// Access the underlying hashmap - pub fn options(&self) -> &HashMap { - &self.options +impl ConfigField for String { + fn visit(&self, v: &mut V, key: &str, description: &'static str) { + v.visit_some(key, self.as_str(), description) } - /// Tests if the key exists in the configuration - pub fn exists(&self, key: &str) -> bool { - self.options().contains_key(key) + fn set(&mut self, _key: &str, value: C) -> Result<()> { + *self = value.to_string(); + Ok(()) } } -#[cfg(test)] -mod test { - use crate::config::{BuiltInConfigs, ConfigOptions}; - - #[test] - fn docs() { - let docs = BuiltInConfigs::generate_config_markdown(); - let mut lines = docs.lines(); - assert_eq!( - lines.next().unwrap(), - "| key | type | default | description |" - ); - let configs = BuiltInConfigs::default(); - for config in configs.config_definitions { - assert!(docs.contains(&config.key)); - } +impl ConfigField for bool { + fn visit(&self, v: &mut V, key: &str, description: &'static str) { + v.visit_some(key, *self, description) } - #[test] - fn get_then_set() { - let mut config = ConfigOptions::new(); - let config_key = "datafusion.optimizer.filter_null_join_keys"; - assert!(!config.get_bool(config_key).unwrap_or_default()); - config.set_bool(config_key, true); - assert!(config.get_bool(config_key).unwrap_or_default()); + fn set(&mut self, _key: &str, value: C) -> Result<()> { + *self = value.parse_bool()?; + Ok(()) } +} - #[test] - fn get_invalid_config() { - let config = ConfigOptions::new(); - let invalid_key = "not.valid"; - assert!(!config.exists(invalid_key)); - assert!(!config.get_bool(invalid_key).unwrap_or_default()); +impl ConfigField for usize { + fn visit(&self, v: &mut V, key: &str, description: &'static str) { + v.visit_some(key, *self, description) } - #[test] - fn get_config_in_invalid_format() { - let config = ConfigOptions::new(); - let key = "datafusion.execution.batch_size"; - - assert!(config.exists(key)); - assert_eq!(None, config.get_string(key)); - assert!(!config.get_bool(key).unwrap_or_default()); + fn set(&mut self, _key: &str, value: C) -> Result<()> { + *self = value.parse_usize()?; + Ok(()) } } + +/// An implementation trait used to recursively walk configuration +trait FieldVisitor { + fn visit_some( + &mut self, + key: &str, + value: C, + description: &'static str, + ); + + fn visit_none(&mut self, key: &str, description: &'static str); +} diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 9f238a43cab8..4908e3972beb 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -40,9 +40,7 @@ use crate::arrow::array::{ }; use crate::arrow::datatypes::{DataType, Field}; use crate::config::ConfigOptions; -use crate::config::OPT_PARQUET_ENABLE_PRUNING; -use crate::config::OPT_PARQUET_METADATA_SIZE_HINT; -use crate::config::OPT_PARQUET_SKIP_METADATA; + use crate::datasource::{create_max_min_accs, get_col_stats}; use crate::error::Result; use crate::execution::context::SessionState; @@ -87,8 +85,7 @@ impl ParquetFormat { /// Return true if pruning is enabled pub fn enable_pruning(&self, config_options: &ConfigOptions) -> bool { self.enable_pruning - .or_else(|| config_options.get_bool(OPT_PARQUET_ENABLE_PRUNING)) - .unwrap_or(true) + .unwrap_or_else(|| config_options.built_in.execution.parquet.enable_pruning) } /// Provide a hint to the size of the file metadata. If a hint is provided @@ -105,7 +102,7 @@ impl ParquetFormat { /// Return the metadata size hint if set pub fn metadata_size_hint(&self, config_options: &ConfigOptions) -> Option { self.metadata_size_hint - .or_else(|| config_options.get_usize(OPT_PARQUET_METADATA_SIZE_HINT)) + .or_else(|| config_options.built_in.execution.parquet.metadata_size_hint) } /// Tell the parquet reader to skip any metadata that may be in @@ -122,8 +119,7 @@ impl ParquetFormat { /// schema merging. pub fn skip_metadata(&self, config_options: &ConfigOptions) -> bool { self.skip_metadata - .or_else(|| config_options.get_bool(OPT_PARQUET_SKIP_METADATA)) - .unwrap_or(true) + .unwrap_or_else(|| config_options.built_in.execution.parquet.skip_metadata) } } diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 323e50ebd8cd..6fef57466268 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -18,11 +18,6 @@ //! SessionContext contains methods for registering data sources and executing queries use crate::{ catalog::catalog::{CatalogList, MemoryCatalogList}, - config::{ - OPT_COLLECT_STATISTICS, OPT_CREATE_DEFAULT_CATALOG_AND_SCHEMA, - OPT_INFORMATION_SCHEMA, OPT_PARQUET_ENABLE_PRUNING, OPT_REPARTITION_AGGREGATIONS, - OPT_REPARTITION_JOINS, OPT_REPARTITION_WINDOWS, OPT_TARGET_PARTITIONS, - }, datasource::listing::{ListingOptions, ListingTable}, datasource::{MemTable, ViewTable}, logical_expr::{PlanType, ToStringifiedPlan}, @@ -35,7 +30,6 @@ use crate::{ pub use datafusion_physical_expr::execution_props::ExecutionProps; use datafusion_physical_expr::var_provider::is_system_variables; use parking_lot::RwLock; -use std::str::FromStr; use std::sync::Arc; use std::{ any::{Any, TypeId}, @@ -71,11 +65,7 @@ use datafusion_sql::{ResolvedTableReference, TableReference}; use crate::physical_optimizer::coalesce_batches::CoalesceBatches; use crate::physical_optimizer::repartition::Repartition; -use crate::config::{ - ConfigOptions, OPT_BATCH_SIZE, OPT_COALESCE_BATCHES, OPT_COALESCE_TARGET_BATCH_SIZE, - OPT_ENABLE_ROUND_ROBIN_REPARTITION, OPT_FILTER_NULL_JOIN_KEYS, - OPT_OPTIMIZER_MAX_PASSES, OPT_OPTIMIZER_SKIP_FAILED_RULES, -}; +use crate::config::ConfigOptions; use crate::execution::{runtime_env::RuntimeEnv, FunctionRegistry}; use crate::physical_optimizer::enforcement::BasicEnforcement; use crate::physical_plan::file_format::{plan_to_csv, plan_to_json, plan_to_parquet}; @@ -378,52 +368,8 @@ impl SessionContext { variable, value, .. }) => { let mut state = self.state.write(); - let config_options = &mut state.config.config_options; - - let old_value = config_options.get(&variable).ok_or_else(|| { - DataFusionError::Execution(format!( - "Can not SET variable: Unknown Variable {}", - variable - )) - })?; - - match old_value { - ScalarValue::Boolean(_) => { - let new_value = value.parse::().map_err(|_| { - DataFusionError::Execution(format!( - "Failed to parse {} as bool", - value, - )) - })?; - config_options.set_bool(&variable, new_value); - } - - ScalarValue::UInt64(_) => { - let new_value = value.parse::().map_err(|_| { - DataFusionError::Execution(format!( - "Failed to parse {} as u64", - value, - )) - })?; - config_options.set_u64(&variable, new_value); - } - - ScalarValue::Utf8(_) => { - let new_value = value.parse::().map_err(|_| { - DataFusionError::Execution(format!( - "Failed to parse {} as String", - value, - )) - })?; - config_options.set_string(&variable, new_value); - } - - _ => { - return Err(DataFusionError::Execution( - "Unsupported Scalar Value Type".to_string(), - )) - } - } + let config_options = &mut state.config.options; + config_options.set(&variable, &value)?; drop(state); self.return_empty_dataframe() @@ -1125,19 +1071,6 @@ impl QueryPlanner for DefaultQueryPlanner { } } -/// Session Configuration entry name for 'TARGET_PARTITIONS' -pub const TARGET_PARTITIONS: &str = "target_partitions"; -/// Session Configuration entry name for 'REPARTITION_JOINS' -pub const REPARTITION_JOINS: &str = "repartition_joins"; -/// Session Configuration entry name for 'REPARTITION_AGGREGATIONS' -pub const REPARTITION_AGGREGATIONS: &str = "repartition_aggregations"; -/// Session Configuration entry name for 'REPARTITION_WINDOWS' -pub const REPARTITION_WINDOWS: &str = "repartition_windows"; -/// Session Configuration entry name for 'PARQUET_PRUNING' -pub const PARQUET_PRUNING: &str = "parquet_pruning"; -/// Session Configuration entry name for 'COLLECT_STATISTICS' -pub const COLLECT_STATISTICS: &str = "collect_statistics"; - /// Map that holds opaque objects indexed by their type. /// /// Data is wrapped into an [`Arc`] to enable [`Clone`] while still being [object safe]. @@ -1178,7 +1111,7 @@ pub struct SessionConfig { /// due to `resolve_table_ref` which passes back references) default_schema: String, /// Configuration options - config_options: ConfigOptions, + options: ConfigOptions, /// Opaque extensions. extensions: AnyMap, } @@ -1188,7 +1121,7 @@ impl Default for SessionConfig { Self { default_catalog: DEFAULT_CATALOG.to_owned(), default_schema: DEFAULT_SCHEMA.to_owned(), - config_options: ConfigOptions::new(), + options: ConfigOptions::new(), // Assume no extensions by default. extensions: HashMap::with_capacity_and_hasher( 0, @@ -1205,16 +1138,16 @@ impl SessionConfig { } /// Create an execution config with config options read from the environment - pub fn from_env() -> Self { - Self { - config_options: ConfigOptions::from_env(), + pub fn from_env() -> Result { + Ok(Self { + options: ConfigOptions::from_env()?, ..Default::default() - } + }) } /// Set a configuration option pub fn set(mut self, key: &str, value: ScalarValue) -> Self { - self.config_options.set(key, value); + self.options.set(key, &value.to_string()).unwrap(); self } @@ -1240,66 +1173,57 @@ impl SessionConfig { } /// Customize batch size - pub fn with_batch_size(self, n: usize) -> Self { + pub fn with_batch_size(mut self, n: usize) -> Self { // batch size must be greater than zero assert!(n > 0); - self.set_u64(OPT_BATCH_SIZE, n.try_into().unwrap()) + self.options.built_in.execution.batch_size = n; + self } /// Customize [`OPT_TARGET_PARTITIONS`] - pub fn with_target_partitions(self, n: usize) -> Self { + pub fn with_target_partitions(mut self, n: usize) -> Self { // partition count must be greater than zero assert!(n > 0); - self.set_usize(OPT_TARGET_PARTITIONS, n) + self.options.built_in.execution.target_partitions = n; + self } /// get target_partitions pub fn target_partitions(&self) -> usize { - self.config_options - .get_usize(OPT_TARGET_PARTITIONS) - .expect("target partitions must be set") + self.options.built_in.execution.target_partitions } /// Is the information schema enabled? pub fn information_schema(&self) -> bool { - self.config_options - .get_bool(OPT_INFORMATION_SCHEMA) - .unwrap_or_default() + self.options.built_in.catalog.information_schema } /// Should the context create the default catalog and schema? pub fn create_default_catalog_and_schema(&self) -> bool { - self.config_options - .get_bool(OPT_CREATE_DEFAULT_CATALOG_AND_SCHEMA) - .unwrap_or_default() + self.options + .built_in + .catalog + .create_default_catalog_and_schema } /// Are joins repartitioned during execution? pub fn repartition_joins(&self) -> bool { - self.config_options - .get_bool(OPT_REPARTITION_JOINS) - .unwrap_or_default() + self.options.built_in.optimizer.repartition_joins } /// Are aggregates repartitioned during execution? pub fn repartition_aggregations(&self) -> bool { - self.config_options - .get_bool(OPT_REPARTITION_AGGREGATIONS) - .unwrap_or_default() + self.options.built_in.optimizer.repartition_aggregations } /// Are window functions repartitioned during execution? pub fn repartition_window_functions(&self) -> bool { - self.config_options - .get_bool(OPT_REPARTITION_WINDOWS) - .unwrap_or_default() + self.options.built_in.optimizer.repartition_windows } /// Are statistics collected during execution? pub fn collect_statistics(&self) -> bool { - self.config_options - .get_bool(OPT_COLLECT_STATISTICS) - .unwrap_or_default() + self.options.built_in.execution.collect_statistics } /// Selects a name for the default catalog and schema @@ -1315,66 +1239,57 @@ impl SessionConfig { /// Controls whether the default catalog and schema will be automatically created pub fn with_create_default_catalog_and_schema(mut self, create: bool) -> Self { - self.config_options - .set_bool(OPT_CREATE_DEFAULT_CATALOG_AND_SCHEMA, create); + self.options + .built_in + .catalog + .create_default_catalog_and_schema = create; self } /// Enables or disables the inclusion of `information_schema` virtual tables pub fn with_information_schema(mut self, enabled: bool) -> Self { - self.config_options - .set_bool(OPT_INFORMATION_SCHEMA, enabled); + self.options.built_in.catalog.information_schema = enabled; self } /// Enables or disables the use of repartitioning for joins to improve parallelism pub fn with_repartition_joins(mut self, enabled: bool) -> Self { - self.config_options.set_bool(OPT_REPARTITION_JOINS, enabled); + self.options.built_in.optimizer.repartition_joins = enabled; self } /// Enables or disables the use of repartitioning for aggregations to improve parallelism pub fn with_repartition_aggregations(mut self, enabled: bool) -> Self { - self.config_options - .set_bool(OPT_REPARTITION_AGGREGATIONS, enabled); + self.options.built_in.optimizer.repartition_aggregations = enabled; self } /// Enables or disables the use of repartitioning for window functions to improve parallelism pub fn with_repartition_windows(mut self, enabled: bool) -> Self { - self.config_options - .set_bool(OPT_REPARTITION_WINDOWS, enabled); + self.options.built_in.optimizer.repartition_windows = enabled; self } /// Enables or disables the use of pruning predicate for parquet readers to skip row groups pub fn with_parquet_pruning(mut self, enabled: bool) -> Self { - self.config_options - .set_bool(OPT_PARQUET_ENABLE_PRUNING, enabled); + self.options.built_in.execution.parquet.enable_pruning = enabled; self } /// Returns true if pruning predicate should be used to skip parquet row groups pub fn parquet_pruning(&self) -> bool { - self.config_options - .get_bool(OPT_PARQUET_ENABLE_PRUNING) - .unwrap_or(false) + self.options.built_in.execution.parquet.enable_pruning } /// Enables or disables the collection of statistics after listing files pub fn with_collect_statistics(mut self, enabled: bool) -> Self { - self.config_options - .set_bool(OPT_COLLECT_STATISTICS, enabled); + self.options.built_in.execution.collect_statistics = enabled; self } /// Get the currently configured batch size pub fn batch_size(&self) -> usize { - self.config_options - .get_u64(OPT_BATCH_SIZE) - .unwrap_or_default() - .try_into() - .unwrap() + self.options.built_in.execution.batch_size } /// Convert configuration options to name-value pairs with values @@ -1387,33 +1302,9 @@ impl SessionConfig { pub fn to_props(&self) -> HashMap { let mut map = HashMap::new(); // copy configs from config_options - for (k, v) in self.config_options.options() { - map.insert(k.to_string(), format!("{}", v)); + for entry in self.options.entries() { + map.insert(entry.key, entry.value.unwrap_or_else(Default::default)); } - map.insert( - TARGET_PARTITIONS.to_owned(), - format!("{}", self.target_partitions()), - ); - map.insert( - REPARTITION_JOINS.to_owned(), - format!("{}", self.repartition_joins()), - ); - map.insert( - REPARTITION_AGGREGATIONS.to_owned(), - format!("{}", self.repartition_aggregations()), - ); - map.insert( - REPARTITION_WINDOWS.to_owned(), - format!("{}", self.repartition_window_functions()), - ); - map.insert( - PARQUET_PRUNING.to_owned(), - format!("{}", self.parquet_pruning()), - ); - map.insert( - COLLECT_STATISTICS.to_owned(), - format!("{}", self.collect_statistics()), - ); map } @@ -1422,14 +1313,14 @@ impl SessionConfig { /// /// [`config_options`]: SessionContext::config_option pub fn config_options(&self) -> &ConfigOptions { - &self.config_options + &self.options } /// Return a mutable handle to the configuration options. /// /// [`config_options`]: SessionContext::config_option pub fn config_options_mut(&mut self) -> &mut ConfigOptions { - &mut self.config_options + &mut self.options } /// Add extensions. @@ -1575,11 +1466,7 @@ impl SessionState { // - it's conflicted with some parts of the BasicEnforcement, since it will // introduce additional repartitioning while the BasicEnforcement aims at // reducing unnecessary repartitioning. - if config - .config_options - .get_bool(OPT_ENABLE_ROUND_ROBIN_REPARTITION) - .unwrap_or_default() - { + if config.options.built_in.execution.round_robin_repartition { physical_optimizers.push(Arc::new(Repartition::new())); } //- Currently it will depend on the partition number to decide whether to change the @@ -1611,18 +1498,9 @@ impl SessionState { physical_optimizers.push(Arc::new(OptimizeSorts::new())); // It will not influence the distribution and ordering of the whole plan tree. // Therefore, to avoid influencing other rules, it should be run at last. - if config - .config_options - .get_bool(OPT_COALESCE_BATCHES) - .unwrap_or_default() - { + if config.options.built_in.execution.coalesce_batches { physical_optimizers.push(Arc::new(CoalesceBatches::new( - config - .config_options - .get_u64(OPT_COALESCE_TARGET_BATCH_SIZE) - .unwrap_or_default() - .try_into() - .unwrap(), + config.options.built_in.execution.coalesce_target_batch_size, ))); } // The PipelineChecker rule will reject non-runnable query plans that use @@ -1649,23 +1527,16 @@ impl SessionState { runtime: &Arc, default_catalog: &MemoryCatalogProvider, ) { - let url = config.config_options.get("datafusion.catalog.location"); - let format = config.config_options.get("datafusion.catalog.type"); + let url = config.options.built_in.catalog.location.as_ref(); + let format = config.options.built_in.catalog.format.as_ref(); let (url, format) = match (url, format) { (Some(url), Some(format)) => (url, format), _ => return, }; - if url.is_null() || format.is_null() { - return; - } let url = url.to_string(); let format = format.to_string(); - let has_header = config.config_options.get("datafusion.catalog.has_header"); - let has_header: bool = has_header - .map(|x| FromStr::from_str(&x.to_string()).unwrap_or_default()) - .unwrap_or_default(); - + let has_header = config.options.built_in.catalog.has_header; let url = Url::parse(url.as_str()).expect("Invalid default catalog location!"); let authority = match url.host_str() { Some(host) => format!("{}://{}", url.scheme(), host), @@ -1782,26 +1653,14 @@ impl SessionState { pub fn optimize(&self, plan: &LogicalPlan) -> Result { // TODO: Implement OptimizerContext directly on DataFrame (#4631) (#4626) let config = { - let config_options = self.config_options(); + let config = &self.config_options().built_in.optimizer; OptimizerContext::new() - .with_skip_failing_rules( - config_options - .get_bool(OPT_OPTIMIZER_SKIP_FAILED_RULES) - .unwrap_or_default(), - ) - .with_max_passes( - config_options - .get_u64(OPT_OPTIMIZER_MAX_PASSES) - .unwrap_or_default() as u8, - ) + .with_skip_failing_rules(config.skip_failed_rules) + .with_max_passes(config.max_passes as u8) .with_query_execution_start_time( self.execution_props.query_execution_start_time, ) - .filter_null_keys( - config_options - .get_bool(OPT_FILTER_NULL_JOIN_KEYS) - .unwrap_or_default(), - ) + .filter_null_keys(config.filter_null_join_keys) }; if let LogicalPlan::Explain(e) = plan { @@ -1919,7 +1778,18 @@ impl ContextProvider for SessionState { } fn get_config_option(&self, variable: &str) -> Option { - self.config_options().get(variable) + // TOOD: Move ConfigOptions into common crate + match variable { + "datafusion.execution.time_zone" => self + .config + .options + .built_in + .execution + .time_zone + .as_ref() + .map(|s| ScalarValue::Utf8(Some(s.clone()))), + _ => unimplemented!(), + } } } @@ -1977,38 +1847,10 @@ impl TaskContext { aggregate_functions: HashMap>, runtime: Arc, ) -> Self { - let session_config = if task_props.is_empty() { - SessionConfig::new() - } else { - SessionConfig::new() - .with_batch_size(task_props.get(OPT_BATCH_SIZE).unwrap().parse().unwrap()) - .with_target_partitions( - task_props.get(TARGET_PARTITIONS).unwrap().parse().unwrap(), - ) - .with_repartition_joins( - task_props.get(REPARTITION_JOINS).unwrap().parse().unwrap(), - ) - .with_repartition_aggregations( - task_props - .get(REPARTITION_AGGREGATIONS) - .unwrap() - .parse() - .unwrap(), - ) - .with_repartition_windows( - task_props - .get(REPARTITION_WINDOWS) - .unwrap() - .parse() - .unwrap(), - ) - .with_parquet_pruning( - task_props.get(PARQUET_PRUNING).unwrap().parse().unwrap(), - ) - .with_collect_statistics( - task_props.get(COLLECT_STATISTICS).unwrap().parse().unwrap(), - ) - }; + let mut session_config = SessionConfig::new(); + for (k, v) in task_props { + let _ = session_config.config_options_mut().set(&k, &v); + } Self { task_id: Some(task_id), @@ -2369,7 +2211,7 @@ mod tests { let runtime = Arc::new(RuntimeEnv::new(rt_cfg).unwrap()); let cfg = SessionConfig::new() .set_str("datafusion.catalog.location", url.as_str()) - .set_str("datafusion.catalog.type", "CSV") + .set_str("datafusion.catalog.format", "CSV") .set_str("datafusion.catalog.has_header", "true"); let session_state = SessionState::with_config_rt(cfg, runtime); let ctx = SessionContext::with_state(session_state); diff --git a/datafusion/core/src/physical_optimizer/enforcement.rs b/datafusion/core/src/physical_optimizer/enforcement.rs index f017d0e0aaf8..35228580ff1b 100644 --- a/datafusion/core/src/physical_optimizer/enforcement.rs +++ b/datafusion/core/src/physical_optimizer/enforcement.rs @@ -18,9 +18,7 @@ //! Enforcement optimizer rules are used to make sure the plan's Distribution and Ordering //! requirements are met by inserting necessary [[RepartitionExec]] and [[SortExec]]. //! -use crate::config::{ - ConfigOptions, OPT_TARGET_PARTITIONS, OPT_TOP_DOWN_JOIN_KEY_REORDERING, -}; +use crate::config::ConfigOptions; use crate::error::Result; use crate::physical_optimizer::utils::{add_sort_above_child, ordering_satisfy}; use crate::physical_optimizer::PhysicalOptimizerRule; @@ -72,10 +70,9 @@ impl PhysicalOptimizerRule for BasicEnforcement { plan: Arc, config: &ConfigOptions, ) -> Result> { - let target_partitions = config.get_usize(OPT_TARGET_PARTITIONS).unwrap(); - let top_down_join_key_reordering = config - .get_bool(OPT_TOP_DOWN_JOIN_KEY_REORDERING) - .unwrap_or_default(); + let config = &config.built_in; + let target_partitions = config.execution.target_partitions; + let top_down_join_key_reordering = config.optimizer.top_down_join_key_reordering; let new_plan = if top_down_join_key_reordering { // Run a top-down process to adjust input key ordering recursively let plan_requirements = PlanWithKeyRequirements::new(plan); @@ -1137,7 +1134,7 @@ mod tests { let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); let mut config = ConfigOptions::new(); - config.set_usize(OPT_TARGET_PARTITIONS, 10); + config.built_in.execution.target_partitions = 10; // run optimizer let optimizer = BasicEnforcement {}; diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index c479e2c9af69..bc6d95bdbe25 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use arrow::datatypes::Schema; -use crate::config::{ConfigOptions, OPT_HASH_JOIN_SINGLE_PARTITION_THRESHOLD}; +use crate::config::ConfigOptions; use crate::logical_expr::JoinType; use crate::physical_plan::expressions::Column; use crate::physical_plan::joins::{ @@ -214,11 +214,8 @@ impl PhysicalOptimizerRule for JoinSelection { plan: Arc, config: &ConfigOptions, ) -> Result> { - let collect_left_threshold: usize = config - .get_u64(OPT_HASH_JOIN_SINGLE_PARTITION_THRESHOLD) - .unwrap_or_default() - .try_into() - .unwrap(); + let config = &config.built_in.optimizer; + let collect_left_threshold = config.hash_join_single_partition_threshold; plan.transform_up(&|plan| { if let Some(hash_join) = plan.as_any().downcast_ref::() { match hash_join.partition_mode() { diff --git a/datafusion/core/src/physical_optimizer/repartition.rs b/datafusion/core/src/physical_optimizer/repartition.rs index 5bdbf59a606c..19512f9b3c13 100644 --- a/datafusion/core/src/physical_optimizer/repartition.rs +++ b/datafusion/core/src/physical_optimizer/repartition.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use super::optimizer::PhysicalOptimizerRule; -use crate::config::{ConfigOptions, OPT_TARGET_PARTITIONS}; +use crate::config::ConfigOptions; use crate::error::Result; use crate::physical_plan::Partitioning::*; use crate::physical_plan::{ @@ -210,7 +210,7 @@ impl PhysicalOptimizerRule for Repartition { plan: Arc, config: &ConfigOptions, ) -> Result> { - let target_partitions = config.get_usize(OPT_TARGET_PARTITIONS).unwrap(); + let target_partitions = config.built_in.execution.target_partitions; // Don't run optimizer if target_partitions == 1 if target_partitions == 1 { Ok(plan) @@ -364,7 +364,7 @@ mod tests { let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); let mut config = ConfigOptions::new(); - config.set_usize(OPT_TARGET_PARTITIONS, 10); + config.built_in.execution.target_partitions = 10; // run optimizer let optimizers: Vec> = vec![ diff --git a/datafusion/core/src/physical_plan/coalesce_batches.rs b/datafusion/core/src/physical_plan/coalesce_batches.rs index 582edc103358..fa8a18318db3 100644 --- a/datafusion/core/src/physical_plan/coalesce_batches.rs +++ b/datafusion/core/src/physical_plan/coalesce_batches.rs @@ -308,7 +308,6 @@ pub fn concat_batches( #[cfg(test)] mod tests { use super::*; - use crate::config::{OPT_COALESCE_BATCHES, OPT_COALESCE_TARGET_BATCH_SIZE}; use crate::datasource::MemTable; use crate::physical_plan::filter::FilterExec; use crate::physical_plan::projection::ProjectionExec; @@ -319,9 +318,14 @@ mod tests { #[tokio::test] async fn test_custom_batch_size() -> Result<()> { - let ctx = SessionContext::with_config( - SessionConfig::new().set_u64(OPT_COALESCE_TARGET_BATCH_SIZE, 1234), - ); + let mut config = SessionConfig::new(); + config + .config_options_mut() + .built_in + .execution + .coalesce_target_batch_size = 1234; + + let ctx = SessionContext::with_config(config); let plan = create_physical_plan(ctx).await?; let projection = plan.as_any().downcast_ref::().unwrap(); let coalesce = projection @@ -335,9 +339,14 @@ mod tests { #[tokio::test] async fn test_disable_coalesce() -> Result<()> { - let ctx = SessionContext::with_config( - SessionConfig::new().set_bool(OPT_COALESCE_BATCHES, false), - ); + let mut config = SessionConfig::new(); + config + .config_options_mut() + .built_in + .execution + .coalesce_batches = false; + + let ctx = SessionContext::with_config(config); let plan = create_physical_plan(ctx).await?; let projection = plan.as_any().downcast_ref::().unwrap(); // projection should directly wrap filter with no coalesce step diff --git a/datafusion/core/src/physical_plan/file_format/parquet.rs b/datafusion/core/src/physical_plan/file_format/parquet.rs index f6e7d2ded747..a4ac0788b0e1 100644 --- a/datafusion/core/src/physical_plan/file_format/parquet.rs +++ b/datafusion/core/src/physical_plan/file_format/parquet.rs @@ -25,9 +25,7 @@ use std::fs; use std::ops::Range; use std::sync::Arc; -use crate::config::OPT_PARQUET_PUSHDOWN_FILTERS; -use crate::config::OPT_PARQUET_REORDER_FILTERS; -use crate::config::{ConfigOptions, OPT_PARQUET_ENABLE_PAGE_INDEX}; +use crate::config::ConfigOptions; use crate::datasource::file_format::parquet::fetch_parquet_metadata; use crate::physical_plan::file_format::file_stream::{ FileOpenFuture, FileOpener, FileStream, @@ -202,9 +200,7 @@ impl ParquetExec { /// Return the value described in [`Self::with_pushdown_filters`] fn pushdown_filters(&self, config_options: &ConfigOptions) -> bool { self.pushdown_filters - .or_else(|| config_options.get_bool(OPT_PARQUET_PUSHDOWN_FILTERS)) - // default to false - .unwrap_or_default() + .unwrap_or_else(|| config_options.built_in.execution.parquet.pushdown_filters) } /// If true, the `RowFilter` made by `pushdown_filters` may try to @@ -219,9 +215,7 @@ impl ParquetExec { /// Return the value described in [`Self::with_reorder_filters`] fn reorder_filters(&self, config_options: &ConfigOptions) -> bool { self.reorder_filters - .or_else(|| config_options.get_bool(OPT_PARQUET_REORDER_FILTERS)) - // default to false - .unwrap_or_default() + .unwrap_or_else(|| config_options.built_in.execution.parquet.reorder_filters) } /// If enabled, the reader will read the page index @@ -236,9 +230,7 @@ impl ParquetExec { /// Return the value described in [`Self::with_enable_page_index`] fn enable_page_index(&self, config_options: &ConfigOptions) -> bool { self.enable_page_index - .or_else(|| config_options.get_bool(OPT_PARQUET_ENABLE_PAGE_INDEX)) - // default to false - .unwrap_or_default() + .unwrap_or_else(|| config_options.built_in.execution.parquet.enable_page_index) } } diff --git a/datafusion/core/src/physical_plan/planner.rs b/datafusion/core/src/physical_plan/planner.rs index 5b001f01678e..f5ea681eeda4 100644 --- a/datafusion/core/src/physical_plan/planner.rs +++ b/datafusion/core/src/physical_plan/planner.rs @@ -22,9 +22,6 @@ use super::{ aggregates, empty::EmptyExec, joins::PartitionMode, udaf, union::UnionExec, values::ValuesExec, windows, }; -use crate::config::{ - OPT_EXPLAIN_LOGICAL_PLAN_ONLY, OPT_EXPLAIN_PHYSICAL_PLAN_ONLY, OPT_PREFER_HASH_JOIN, -}; use crate::datasource::source_as_provider; use crate::execution::context::{ExecutionProps, SessionState}; use crate::logical_expr::utils::generate_sort_key; @@ -995,9 +992,7 @@ impl DefaultPhysicalPlanner { _ => None }; - let prefer_hash_join = session_state.config().config_options() - .get_bool(OPT_PREFER_HASH_JOIN) - .unwrap_or_default(); + let prefer_hash_join = session_state.config_options().built_in.optimizer.prefer_hash_join; if join_on.is_empty() { // there is no equal join condition, use the nested loop join // TODO optimize the plan, and use the config of `target_partitions` and `repartition_joins` @@ -1715,21 +1710,14 @@ impl DefaultPhysicalPlanner { use PlanType::*; let mut stringified_plans = vec![]; - if !session_state - .config_options() - .get_bool(OPT_EXPLAIN_PHYSICAL_PLAN_ONLY) - .unwrap_or_default() - { - stringified_plans = e.stringified_plans.clone(); + let config = &session_state.config_options().built_in.explain; + if !config.physical_plan_only { + stringified_plans = e.stringified_plans.clone(); stringified_plans.push(e.plan.to_stringified(FinalLogicalPlan)); } - if !session_state - .config_options() - .get_bool(OPT_EXPLAIN_LOGICAL_PLAN_ONLY) - .unwrap_or_default() - { + if !config.logical_plan_only { let input = self .create_initial_plan(e.plan.as_ref(), session_state) .await?; diff --git a/datafusion/core/tests/config_from_env.rs b/datafusion/core/tests/config_from_env.rs index 20d153d8f83b..84a03b2f8ec2 100644 --- a/datafusion/core/tests/config_from_env.rs +++ b/datafusion/core/tests/config_from_env.rs @@ -20,30 +20,27 @@ use std::env; #[test] fn get_config_bool_from_env() { - let config_key = "datafusion.optimizer.filter_null_join_keys"; let env_key = "DATAFUSION_OPTIMIZER_FILTER_NULL_JOIN_KEYS"; env::set_var(env_key, "true"); - let config = ConfigOptions::from_env(); + let config = ConfigOptions::from_env().unwrap(); env::remove_var(env_key); - assert!(config.get_bool(config_key).unwrap_or_default()); + assert!(config.built_in.optimizer.filter_null_join_keys); } #[test] fn get_config_int_from_env() { - let config_key = "datafusion.execution.batch_size"; let env_key = "DATAFUSION_EXECUTION_BATCH_SIZE"; env::set_var(env_key, "4096"); - let config = ConfigOptions::from_env(); + let config = ConfigOptions::from_env().unwrap(); env::remove_var(env_key); - assert_eq!(config.get_u64(config_key).unwrap_or_default(), 4096); + assert_eq!(config.built_in.execution.batch_size, 4096); } #[test] fn get_config_int_from_env_invalid() { - let config_key = "datafusion.execution.coalesce_target_batch_size"; let env_key = "DATAFUSION_EXECUTION_COALESCE_TARGET_BATCH_SIZE"; env::set_var(env_key, "abc"); - let config = ConfigOptions::from_env(); + let config = ConfigOptions::from_env().unwrap(); env::remove_var(env_key); - assert_eq!(config.get_u64(config_key).unwrap_or_default(), 4096); // set to its default value + assert_eq!(config.built_in.execution.coalesce_target_batch_size, 4096); // set to its default value } diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index d3bf4909c7f6..2c5cd1310a5c 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -28,7 +28,6 @@ use arrow::{ util::pretty::pretty_format_batches, }; use chrono::{Datelike, Duration}; -use datafusion::config::OPT_PARQUET_ENABLE_PAGE_INDEX; use datafusion::{ datasource::{provider_as_source, TableProvider}, physical_plan::{ @@ -142,7 +141,10 @@ impl ContextWithParquet { Unit::Page => { config .config_options_mut() - .set_bool(OPT_PARQUET_ENABLE_PAGE_INDEX, true); + .built_in + .execution + .parquet + .enable_page_index = true; make_test_file_page(scenario).await } }; diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index d3848c10aa7b..2a21a92bf75a 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -16,10 +16,7 @@ // under the License. use super::*; -use datafusion::{ - config::{OPT_EXPLAIN_LOGICAL_PLAN_ONLY, OPT_EXPLAIN_PHYSICAL_PLAN_ONLY}, - physical_plan::display::DisplayableExecutionPlan, -}; +use datafusion::physical_plan::display::DisplayableExecutionPlan; #[tokio::test] async fn explain_analyze_baseline_metrics() { @@ -864,7 +861,12 @@ async fn csv_explain_analyze_verbose() { #[tokio::test] async fn explain_logical_plan_only() { - let config = SessionConfig::new().set_bool(OPT_EXPLAIN_LOGICAL_PLAN_ONLY, true); + let mut config = SessionConfig::new(); + config + .config_options_mut() + .built_in + .explain + .logical_plan_only = true; let ctx = SessionContext::with_config(config); let sql = "EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c3)"; let actual = execute(&ctx, sql).await; @@ -883,7 +885,12 @@ async fn explain_logical_plan_only() { #[tokio::test] async fn explain_physical_plan_only() { - let config = SessionConfig::new().set_bool(OPT_EXPLAIN_PHYSICAL_PLAN_ONLY, true); + let mut config = SessionConfig::new(); + config + .config_options_mut() + .built_in + .explain + .physical_plan_only = true; let ctx = SessionContext::with_config(config); let sql = "EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c3)"; let actual = execute(&ctx, sql).await; @@ -902,8 +909,12 @@ async fn explain_physical_plan_only() { #[tokio::test] async fn explain_nested() { async fn test_nested_explain(explain_phy_plan_flag: bool) { - let config = SessionConfig::new() - .set_bool(OPT_EXPLAIN_PHYSICAL_PLAN_ONLY, explain_phy_plan_flag); + let mut config = SessionConfig::new(); + config + .config_options_mut() + .built_in + .explain + .physical_plan_only = explain_phy_plan_flag; let ctx = SessionContext::with_config(config); let sql = "EXPLAIN explain select 1"; let dataframe = ctx.sql(sql).await.unwrap(); diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index c7822e5b4a3d..7b0f8cc5aeba 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -25,7 +25,6 @@ use arrow::{ use chrono::prelude::*; use chrono::Duration; -use datafusion::config::OPT_PREFER_HASH_JOIN; use datafusion::datasource::TableProvider; use datafusion::from_slice::FromSlice; use datafusion::logical_expr::{Aggregate, LogicalPlan, Projection, TableScan}; @@ -567,9 +566,14 @@ fn create_sort_merge_join_context( column_left: &str, column_right: &str, ) -> Result { - let ctx = SessionContext::with_config( - SessionConfig::new().set_bool(OPT_PREFER_HASH_JOIN, false), - ); + let mut config = SessionConfig::new(); + config + .config_options_mut() + .built_in + .optimizer + .prefer_hash_join = false; + + let ctx = SessionContext::with_config(config); let t1_schema = Arc::new(Schema::new(vec![ Field::new(column_left, DataType::UInt32, true), @@ -615,11 +619,19 @@ fn create_sort_merge_join_context( } fn create_sort_merge_join_datatype_context() -> Result { - let ctx = SessionContext::with_config( - SessionConfig::new() - .set_bool(OPT_PREFER_HASH_JOIN, false) - .with_target_partitions(2), - ); + let mut config = SessionConfig::new(); + config + .config_options_mut() + .built_in + .optimizer + .prefer_hash_join = false; + config + .config_options_mut() + .built_in + .execution + .target_partitions = 2; + + let ctx = SessionContext::with_config(config); let t1_schema = Schema::new(vec![ Field::new("c1", DataType::Date32, true), diff --git a/datafusion/core/tests/sql/set_variable.rs b/datafusion/core/tests/sql/set_variable.rs index 085e7b1f4f55..5f18c8c76f8e 100644 --- a/datafusion/core/tests/sql/set_variable.rs +++ b/datafusion/core/tests/sql/set_variable.rs @@ -236,7 +236,7 @@ async fn set_u64_variable_bad_value() { assert_eq!( err.to_string(), - "Execution error: Failed to parse -1 as u64" + "External error: invalid digit found in string" ); let err = plan_and_collect(&ctx, "SET datafusion.execution.batch_size to abc") diff --git a/parquet-test-utils/src/lib.rs b/parquet-test-utils/src/lib.rs index 7a873b8c702a..3d296f0cc31e 100644 --- a/parquet-test-utils/src/lib.rs +++ b/parquet-test-utils/src/lib.rs @@ -23,10 +23,6 @@ use std::sync::Arc; use datafusion::arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; use datafusion::common::ToDFSchema; -use datafusion::config::{ - OPT_PARQUET_ENABLE_PAGE_INDEX, OPT_PARQUET_PUSHDOWN_FILTERS, - OPT_PARQUET_REORDER_FILTERS, -}; use datafusion::datasource::listing::{ListingTableUrl, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::error::Result; @@ -61,10 +57,12 @@ pub struct ParquetScanOptions { impl ParquetScanOptions { /// Returns a [`SessionConfig`] with the given options pub fn config(&self) -> SessionConfig { - SessionConfig::new() - .set_bool(OPT_PARQUET_PUSHDOWN_FILTERS, self.pushdown_filters) - .set_bool(OPT_PARQUET_REORDER_FILTERS, self.reorder_filters) - .set_bool(OPT_PARQUET_ENABLE_PAGE_INDEX, self.enable_page_index) + let mut config = SessionConfig::new(); + let mut parquet = &mut config.config_options_mut().built_in.execution.parquet; + parquet.pushdown_filters = self.pushdown_filters; + parquet.reorder_filters = self.reorder_filters; + parquet.enable_page_index = self.enable_page_index; + config } } From 54c2e957763b9bf5a76e2e858fcefb544968982b Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Thu, 29 Dec 2022 17:32:56 +0000 Subject: [PATCH 02/12] Simplify --- datafusion/core/src/config.rs | 141 ++++++------------ .../src/physical_plan/file_format/parquet.rs | 5 +- 2 files changed, 46 insertions(+), 100 deletions(-) diff --git a/datafusion/core/src/config.rs b/datafusion/core/src/config.rs index fc18087472ad..ce5d46f9ecef 100644 --- a/datafusion/core/src/config.rs +++ b/datafusion/core/src/config.rs @@ -20,6 +20,7 @@ use datafusion_common::{DataFusionError, Result}; use std::any::Any; use std::collections::BTreeMap; +use std::fmt::Display; macro_rules! config_namespace { ( @@ -42,7 +43,7 @@ macro_rules! config_namespace { } impl ConfigField for $struct_name { - fn set(&mut self, key: &str, value: C) -> Result<()> { + fn set(&mut self, key: &str, value: &str) -> Result<()> { let (key, rem) = key.split_once('.').unwrap_or((key, "")); match key { $( @@ -54,7 +55,7 @@ macro_rules! config_namespace { } } - fn visit(&self, v: &mut V, key_prefix: &str, _description: &'static str) { + fn visit(&self, v: &mut V, key_prefix: &str, _description: &'static str) { $( let key = format!("{}.{}", key_prefix, stringify!($field_name)); let desc = concat!($($d),*).trim(); @@ -311,11 +312,11 @@ impl ConfigOptions { pub fn entries(&self) -> Vec { struct Visitor(Vec); - impl FieldVisitor for Visitor { - fn visit_some( + impl Visit for Visitor { + fn some( &mut self, key: &str, - value: C, + value: V, description: &'static str, ) { self.0.push(ConfigEntry { @@ -325,7 +326,7 @@ impl ConfigOptions { }) } - fn visit_none(&mut self, key: &str, description: &'static str) { + fn none(&mut self, key: &str, description: &'static str) { self.0.push(ConfigEntry { key: key.to_string(), value: None, @@ -417,117 +418,61 @@ impl Clone for ExtensionBox { } } -/// A trait that performs fallible coercion into a [`ConfigField`] -trait ConfigValue: std::fmt::Display { - fn parse_usize(&self) -> Result; - - fn parse_bool(&self) -> Result; -} - -impl ConfigValue for bool { - fn parse_usize(&self) -> Result { - Ok(*self as _) - } - - fn parse_bool(&self) -> Result { - Ok(*self) - } -} - -impl ConfigValue for usize { - fn parse_usize(&self) -> Result { - Ok(*self as _) - } - - fn parse_bool(&self) -> Result { - Ok(*self != 0) - } -} - -impl ConfigValue for &str { - fn parse_usize(&self) -> Result { - self.parse() - .map_err(|e| DataFusionError::External(Box::new(e))) - } - - fn parse_bool(&self) -> Result { - self.parse() - .map_err(|e| DataFusionError::External(Box::new(e))) - } -} - -impl ConfigValue for String { - fn parse_usize(&self) -> Result { - self.as_str().parse_usize() - } - - fn parse_bool(&self) -> Result { - self.as_str().parse_bool() - } -} - /// A trait implemented by `config_namespace` and for field types that provides /// the ability to walk and mutate the configuration tree trait ConfigField { - fn visit(&self, v: &mut V, key: &str, description: &'static str); + fn visit(&self, v: &mut V, key: &str, description: &'static str); - fn set(&mut self, key: &str, value: C) -> Result<()>; + fn set(&mut self, key: &str, value: &str) -> Result<()>; } impl ConfigField for Option { - fn visit(&self, v: &mut V, key: &str, description: &'static str) { + fn visit(&self, v: &mut V, key: &str, description: &'static str) { match self { Some(s) => s.visit(v, key, description), - None => v.visit_none(key, description), + None => v.none(key, description), } } - fn set(&mut self, key: &str, value: C) -> Result<()> { + fn set(&mut self, key: &str, value: &str) -> Result<()> { self.get_or_insert_with(Default::default).set(key, value) } } -impl ConfigField for String { - fn visit(&self, v: &mut V, key: &str, description: &'static str) { - v.visit_some(key, self.as_str(), description) - } - - fn set(&mut self, _key: &str, value: C) -> Result<()> { - *self = value.to_string(); - Ok(()) - } -} - -impl ConfigField for bool { - fn visit(&self, v: &mut V, key: &str, description: &'static str) { - v.visit_some(key, *self, description) - } +macro_rules! config_field { + ($t:ty) => { + impl ConfigField for $t { + fn visit(&self, v: &mut V, key: &str, description: &'static str) { + v.some(key, self, description) + } - fn set(&mut self, _key: &str, value: C) -> Result<()> { - *self = value.parse_bool()?; - Ok(()) - } + fn set(&mut self, key: &str, value: &str) -> Result<()> { + *self = value.parse().map_err(|e| { + DataFusionError::Context( + format!( + concat!( + "Parsing {} as ", + stringify!($t), + " for config key {}" + ), + value, key + ), + Box::new(DataFusionError::External(Box::new(e))), + ) + })?; + Ok(()) + } + } + }; } -impl ConfigField for usize { - fn visit(&self, v: &mut V, key: &str, description: &'static str) { - v.visit_some(key, *self, description) - } - - fn set(&mut self, _key: &str, value: C) -> Result<()> { - *self = value.parse_usize()?; - Ok(()) - } -} +config_field!(String); +config_field!(bool); +config_field!(usize); /// An implementation trait used to recursively walk configuration -trait FieldVisitor { - fn visit_some( - &mut self, - key: &str, - value: C, - description: &'static str, - ); - - fn visit_none(&mut self, key: &str, description: &'static str); +trait Visit { + fn some(&mut self, key: &str, value: V, description: &'static str); + + fn none(&mut self, key: &str, description: &'static str); } diff --git a/datafusion/core/src/physical_plan/file_format/parquet.rs b/datafusion/core/src/physical_plan/file_format/parquet.rs index a4ac0788b0e1..a2228364a5ff 100644 --- a/datafusion/core/src/physical_plan/file_format/parquet.rs +++ b/datafusion/core/src/physical_plan/file_format/parquet.rs @@ -229,8 +229,9 @@ impl ParquetExec { /// Return the value described in [`Self::with_enable_page_index`] fn enable_page_index(&self, config_options: &ConfigOptions) -> bool { - self.enable_page_index - .unwrap_or_else(|| config_options.built_in.execution.parquet.enable_page_index) + self.enable_page_index.unwrap_or_else(|| { + config_options.built_in.execution.parquet.enable_page_index + }) } } From b5a2241fb856972871cc46534b2338796a07bfbb Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Thu, 29 Dec 2022 18:04:23 +0000 Subject: [PATCH 03/12] Fix environment variables --- datafusion/core/src/config.rs | 94 ++++++++++++++++-------- datafusion/core/tests/config_from_env.rs | 5 +- 2 files changed, 64 insertions(+), 35 deletions(-) diff --git a/datafusion/core/src/config.rs b/datafusion/core/src/config.rs index ce5d46f9ecef..00768fe872d6 100644 --- a/datafusion/core/src/config.rs +++ b/datafusion/core/src/config.rs @@ -24,7 +24,7 @@ use std::fmt::Display; macro_rules! config_namespace { ( - $(#[$meta:meta])* + $(#[doc = $struct_d:tt])* $vis:vis struct $struct_name:ident { $( $(#[doc = $d:tt])* @@ -33,6 +33,7 @@ macro_rules! config_namespace { } ) => { + $(#[doc = $struct_d])* #[derive(Debug, Clone, PartialEq)] #[non_exhaustive] $vis struct $struct_name{ @@ -75,6 +76,7 @@ macro_rules! config_namespace { } config_namespace! { + /// Options related to catalog and directory scanning pub struct CatalogOptions { /// Number of partitions for query execution. Increasing partitions can increase /// concurrency. Defaults to the number of cpu cores on the system. @@ -96,6 +98,7 @@ config_namespace! { } config_namespace! { + /// Options related to query execution pub struct ExecutionOptions { /// Default batch size while creating new batches, it's especially useful for /// buffer-in-memory batches since creating tiny batches would results in too much @@ -134,6 +137,7 @@ config_namespace! { } config_namespace! { + /// Options related to reading of parquet files pub struct ParquetOptions { /// If true, uses parquet data page level metadata (Page Index) statistics /// to reduce the number of rows decoded. @@ -167,6 +171,7 @@ config_namespace! { } config_namespace! { + /// Options related to query optimization pub struct OptimizerOptions { /// 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 @@ -209,6 +214,7 @@ config_namespace! { } config_namespace! { + /// Options controlling explain output pub struct ExplainOptions { /// When set to true, the explain statement will only print logical plans pub logical_plan_only: bool, default = false @@ -219,6 +225,7 @@ config_namespace! { } config_namespace! { + /// All built-in DataFusion options pub struct DataFusionOptions { /// Catalog options pub catalog: CatalogOptions, default = Default::default() @@ -263,28 +270,6 @@ impl ConfigOptions { Self::default() } - /// Create new ConfigOptions struct, taking values from - /// environment variables where possible. - /// - /// For example, setting `DATAFUSION_EXECUTION_BATCH_SIZE` will - /// control `datafusion.execution.batch_size`. - pub fn from_env() -> Result { - let mut ret = Self::default(); - for (k, v) in std::env::vars_os() { - let k = k.to_string_lossy().to_ascii_lowercase().replace('_', "."); - let v = v.to_string_lossy(); - - if let Some((prefix, key)) = k.split_once('.') { - if prefix == "datafusion" { - ret.built_in.set(key, v.as_ref())? - } else if let Some(e) = ret.extensions.0.get_mut(prefix) { - e.0.set(key, v.as_ref())? - } - } - } - Ok(ret) - } - /// Set a configuration option pub fn set(&mut self, key: &str, value: &str) -> Result<()> { let (prefix, key) = key.split_once('.').ok_or_else(|| { @@ -308,6 +293,43 @@ impl ConfigOptions { e.0.set(key, value) } + /// Create new ConfigOptions struct, taking values from + /// environment variables where possible. + /// + /// For example, setting `DATAFUSION_EXECUTION_BATCH_SIZE` will + /// control `datafusion.execution.batch_size`. + pub fn from_env() -> Result { + struct Visitor(Vec); + + impl Visit for Visitor { + fn some(&mut self, key: &str, _: V, _: &'static str) { + self.0.push(key.to_string()) + } + + fn none(&mut self, key: &str, _: &'static str) { + self.0.push(key.to_string()) + } + } + + // Extract the names of all fields and then look up the corresponding + // environment variables. This isn't hugely efficient but avoids + // ambiguity between `a.b` and `a_b` which would both correspond + // to an environment variable of `A_B` + + let mut keys = Visitor(vec![]); + let mut ret = Self::default(); + ret.built_in.visit(&mut keys, "datafusion", ""); + + for key in keys.0 { + let env = key.to_uppercase().replace('.', "_"); + if let Some(var) = std::env::var_os(env) { + ret.set(&key, var.to_string_lossy().as_ref())?; + } + } + + Ok(ret) + } + /// Returns the [`ConfigEntry`] stored within this [`ConfigOptions`] pub fn entries(&self) -> Vec { struct Visitor(Vec); @@ -369,19 +391,34 @@ impl ConfigOptions { /// Unfortunately associated constants are not currently object-safe, and so this /// extends the object-safe [`ExtensionOptions`] pub trait ConfigExtension: ExtensionOptions { + /// Configuration namespace prefix to use + /// + /// All values under this will be prefixed with `$PREFIX + "."` const PREFIX: &'static str; } /// An object-safe API for storing arbitrary configuration pub trait ExtensionOptions: Send + Sync + std::fmt::Debug + 'static { + /// Return `self` as [`Any`] + /// + /// This is needed until trait upcasting is stabilised fn as_any(&self) -> &dyn Any; + /// Return `self` as [`Any`] + /// + /// This is needed until trait upcasting is stabilised fn as_any_mut(&mut self) -> &mut dyn Any; + /// Return a deep clone of this [`ExtensionOptions`] + /// + /// It is important this does not share mutable state to avoid consistency issues + /// with configuration changing whilst queries are executing fn cloned(&self) -> Box; + /// Set the given `key`, `value` pair fn set(&mut self, key: &str, value: &str) -> Result<()>; + /// Returns the [`ConfigEntry`] stored in this [`ExtensionOptions`] fn entries(&self) -> Vec; } @@ -446,17 +483,10 @@ macro_rules! config_field { v.some(key, self, description) } - fn set(&mut self, key: &str, value: &str) -> Result<()> { + fn set(&mut self, _: &str, value: &str) -> Result<()> { *self = value.parse().map_err(|e| { DataFusionError::Context( - format!( - concat!( - "Parsing {} as ", - stringify!($t), - " for config key {}" - ), - value, key - ), + format!(concat!("Error parsing {} as ", stringify!($t),), value), Box::new(DataFusionError::External(Box::new(e))), ) })?; diff --git a/datafusion/core/tests/config_from_env.rs b/datafusion/core/tests/config_from_env.rs index 84a03b2f8ec2..49c92e30cfb2 100644 --- a/datafusion/core/tests/config_from_env.rs +++ b/datafusion/core/tests/config_from_env.rs @@ -40,7 +40,6 @@ fn get_config_int_from_env() { fn get_config_int_from_env_invalid() { let env_key = "DATAFUSION_EXECUTION_COALESCE_TARGET_BATCH_SIZE"; env::set_var(env_key, "abc"); - let config = ConfigOptions::from_env().unwrap(); - env::remove_var(env_key); - assert_eq!(config.built_in.execution.coalesce_target_batch_size, 4096); // set to its default value + let err = ConfigOptions::from_env().unwrap_err().to_string(); + assert_eq!(err, "Error parsing abc as usize for config key \ncaused by\nExternal error: invalid digit found in string") } From c5bae84cff98e8f1097725fb7d20cfd26c2fb51e Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Thu, 29 Dec 2022 18:16:00 +0000 Subject: [PATCH 04/12] Tweaks --- datafusion/core/src/config.rs | 16 +++++++--------- datafusion/core/tests/config_from_env.rs | 12 ++++-------- datafusion/core/tests/sql/set_variable.rs | 12 ++++++------ 3 files changed, 17 insertions(+), 23 deletions(-) diff --git a/datafusion/core/src/config.rs b/datafusion/core/src/config.rs index 00768fe872d6..1f5ce76f3af1 100644 --- a/datafusion/core/src/config.rs +++ b/datafusion/core/src/config.rs @@ -58,7 +58,7 @@ macro_rules! config_namespace { fn visit(&self, v: &mut V, key_prefix: &str, _description: &'static str) { $( - let key = format!("{}.{}", key_prefix, stringify!($field_name)); + let key = format!(concat!("{}.", stringify!($field_name)), key_prefix); let desc = concat!($($d),*).trim(); self.$field_name.visit(v, key.as_str(), desc); )* @@ -273,10 +273,9 @@ impl ConfigOptions { /// Set a configuration option pub fn set(&mut self, key: &str, value: &str) -> Result<()> { let (prefix, key) = key.split_once('.').ok_or_else(|| { - DataFusionError::Internal(format!( - "could not find config namespace for key \"{}\"", - key - )) + DataFusionError::External(format!( + "could not find config namespace for key \"{key}\"", + ).into()) })?; if prefix == "datafusion" { @@ -285,10 +284,9 @@ impl ConfigOptions { let e = self.extensions.0.get_mut(prefix); let e = e.ok_or_else(|| { - DataFusionError::Internal(format!( - "Could not find config namespace \"{}\"", - prefix - )) + DataFusionError::External(format!( + "Could not find config namespace \"{prefix}\"", + ).into()) })?; e.0.set(key, value) } diff --git a/datafusion/core/tests/config_from_env.rs b/datafusion/core/tests/config_from_env.rs index 49c92e30cfb2..cce05b02c753 100644 --- a/datafusion/core/tests/config_from_env.rs +++ b/datafusion/core/tests/config_from_env.rs @@ -18,28 +18,24 @@ use datafusion::config::ConfigOptions; use std::env; + #[test] -fn get_config_bool_from_env() { +fn from_env() { + // Note: these must be a single test to avoid interference from concurrent execution let env_key = "DATAFUSION_OPTIMIZER_FILTER_NULL_JOIN_KEYS"; env::set_var(env_key, "true"); let config = ConfigOptions::from_env().unwrap(); env::remove_var(env_key); assert!(config.built_in.optimizer.filter_null_join_keys); -} -#[test] -fn get_config_int_from_env() { let env_key = "DATAFUSION_EXECUTION_BATCH_SIZE"; env::set_var(env_key, "4096"); let config = ConfigOptions::from_env().unwrap(); env::remove_var(env_key); assert_eq!(config.built_in.execution.batch_size, 4096); -} -#[test] -fn get_config_int_from_env_invalid() { let env_key = "DATAFUSION_EXECUTION_COALESCE_TARGET_BATCH_SIZE"; env::set_var(env_key, "abc"); let err = ConfigOptions::from_env().unwrap_err().to_string(); - assert_eq!(err, "Error parsing abc as usize for config key \ncaused by\nExternal error: invalid digit found in string") + assert_eq!(err, "Error parsing abc as usize\ncaused by\nExternal error: invalid digit found in string") } diff --git a/datafusion/core/tests/sql/set_variable.rs b/datafusion/core/tests/sql/set_variable.rs index 5f18c8c76f8e..6a85f7df166d 100644 --- a/datafusion/core/tests/sql/set_variable.rs +++ b/datafusion/core/tests/sql/set_variable.rs @@ -110,7 +110,7 @@ async fn set_variable_unknown_variable() { .unwrap_err(); assert_eq!( err.to_string(), - "Execution error: Can not SET variable: Unknown Variable aabbcc" + "External error: could not find config namespace for key \"aabbcc\"" ); } @@ -161,7 +161,7 @@ async fn set_bool_variable_bad_value() { assert_eq!( err.to_string(), - "Execution error: Failed to parse 1 as bool" + "Error parsing 1 as bool\ncaused by\nExternal error: provided string was not `true` or `false`" ); let err = plan_and_collect(&ctx, "SET datafusion.execution.coalesce_batches to abc") @@ -170,7 +170,7 @@ async fn set_bool_variable_bad_value() { assert_eq!( err.to_string(), - "Execution error: Failed to parse abc as bool" + "Error parsing abc as bool\ncaused by\nExternal error: provided string was not `true` or `false`" ); } @@ -236,7 +236,7 @@ async fn set_u64_variable_bad_value() { assert_eq!( err.to_string(), - "External error: invalid digit found in string" + "Error parsing -1 as usize\ncaused by\nExternal error: invalid digit found in string" ); let err = plan_and_collect(&ctx, "SET datafusion.execution.batch_size to abc") @@ -245,7 +245,7 @@ async fn set_u64_variable_bad_value() { assert_eq!( err.to_string(), - "Execution error: Failed to parse abc as u64" + "Error parsing abc as usize\ncaused by\nExternal error: invalid digit found in string" ); let err = plan_and_collect(&ctx, "SET datafusion.execution.batch_size to 0.1") @@ -254,7 +254,7 @@ async fn set_u64_variable_bad_value() { assert_eq!( err.to_string(), - "Execution error: Failed to parse 0.1 as u64" + "Error parsing 0.1 as usize\ncaused by\nExternal error: invalid digit found in string" ); } From af2a338b7f5cb17053b9a935884729646b8bbbd4 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Thu, 29 Dec 2022 18:31:46 +0000 Subject: [PATCH 05/12] Update datafusion-cli --- datafusion-cli/src/main.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion-cli/src/main.rs b/datafusion-cli/src/main.rs index c4e198ff009c..b9ada1a1f0fa 100644 --- a/datafusion-cli/src/main.rs +++ b/datafusion-cli/src/main.rs @@ -96,7 +96,7 @@ pub async fn main() -> Result<()> { env::set_current_dir(p).unwrap(); }; - let mut session_config = SessionConfig::from_env().with_information_schema(true); + let mut session_config = SessionConfig::from_env()?.with_information_schema(true); if let Some(batch_size) = args.batch_size { session_config = session_config.with_batch_size(batch_size); From e826dfa581b0208a3f6924626e19071774715f5a Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Thu, 29 Dec 2022 18:41:45 +0000 Subject: [PATCH 06/12] Format --- datafusion/core/src/config.rs | 12 ++++++------ datafusion/core/tests/config_from_env.rs | 1 - 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/config.rs b/datafusion/core/src/config.rs index 1f5ce76f3af1..21ff904588fe 100644 --- a/datafusion/core/src/config.rs +++ b/datafusion/core/src/config.rs @@ -273,9 +273,9 @@ impl ConfigOptions { /// Set a configuration option pub fn set(&mut self, key: &str, value: &str) -> Result<()> { let (prefix, key) = key.split_once('.').ok_or_else(|| { - DataFusionError::External(format!( - "could not find config namespace for key \"{key}\"", - ).into()) + DataFusionError::External( + format!("could not find config namespace for key \"{key}\"",).into(), + ) })?; if prefix == "datafusion" { @@ -284,9 +284,9 @@ impl ConfigOptions { let e = self.extensions.0.get_mut(prefix); let e = e.ok_or_else(|| { - DataFusionError::External(format!( - "Could not find config namespace \"{prefix}\"", - ).into()) + DataFusionError::External( + format!("Could not find config namespace \"{prefix}\"",).into(), + ) })?; e.0.set(key, value) } diff --git a/datafusion/core/tests/config_from_env.rs b/datafusion/core/tests/config_from_env.rs index cce05b02c753..1c73d432e175 100644 --- a/datafusion/core/tests/config_from_env.rs +++ b/datafusion/core/tests/config_from_env.rs @@ -18,7 +18,6 @@ use datafusion::config::ConfigOptions; use std::env; - #[test] fn from_env() { // Note: these must be a single test to avoid interference from concurrent execution From f6ef94329a4f24a476f7c0c463e42b0f1ad51ba3 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Thu, 29 Dec 2022 18:49:44 +0000 Subject: [PATCH 07/12] Misc cleanup --- benchmarks/src/bin/h2o.rs | 7 +++-- .../src/datasource/file_format/parquet.rs | 8 ++--- datafusion/core/src/execution/context.rs | 22 ++++++++----- .../src/physical_plan/coalesce_batches.rs | 27 ++++++---------- .../src/physical_plan/file_format/parquet.rs | 9 +++--- datafusion/core/tests/parquet/mod.rs | 8 ++--- datafusion/core/tests/sql/explain_analyze.rs | 31 ++++++------------- datafusion/core/tests/sql/mod.rs | 29 ++++++----------- parquet-test-utils/src/lib.rs | 12 +++---- 9 files changed, 63 insertions(+), 90 deletions(-) diff --git a/benchmarks/src/bin/h2o.rs b/benchmarks/src/bin/h2o.rs index 6f031e387fc5..940ccd1cd514 100644 --- a/benchmarks/src/bin/h2o.rs +++ b/benchmarks/src/bin/h2o.rs @@ -18,6 +18,7 @@ //! DataFusion h2o benchmarks use datafusion::arrow::datatypes::{DataType, Field, Schema}; +use datafusion::config::ConfigOptions; use datafusion::datasource::file_format::csv::CsvFormat; use datafusion::datasource::listing::{ ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, @@ -63,10 +64,10 @@ async fn main() -> Result<()> { async fn group_by(opt: &GroupBy) -> Result<()> { let path = opt.path.to_str().unwrap(); - let mut config = SessionConfig::from_env()?; - config.config_options_mut().built_in.execution.batch_size = 65535; + let mut config = ConfigOptions::from_env()?; + config.built_in.execution.batch_size = 65535; - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::with_config(config.into()); let schema = Schema::new(vec![ Field::new("id1", DataType::Utf8, false), diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 52536f333036..e116ffa6338d 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -85,7 +85,7 @@ impl ParquetFormat { /// Return true if pruning is enabled pub fn enable_pruning(&self, config_options: &ConfigOptions) -> bool { self.enable_pruning - .unwrap_or_else(|| config_options.built_in.execution.parquet.enable_pruning) + .unwrap_or(config_options.built_in.execution.parquet.enable_pruning) } /// Provide a hint to the size of the file metadata. If a hint is provided @@ -101,8 +101,8 @@ impl ParquetFormat { /// Return the metadata size hint if set pub fn metadata_size_hint(&self, config_options: &ConfigOptions) -> Option { - self.metadata_size_hint - .or_else(|| config_options.built_in.execution.parquet.metadata_size_hint) + let hint = config_options.built_in.execution.parquet.metadata_size_hint; + self.metadata_size_hint.or(hint) } /// Tell the parquet reader to skip any metadata that may be in @@ -119,7 +119,7 @@ impl ParquetFormat { /// schema merging. pub fn skip_metadata(&self, config_options: &ConfigOptions) -> bool { self.skip_metadata - .unwrap_or_else(|| config_options.built_in.execution.parquet.skip_metadata) + .unwrap_or(config_options.built_in.execution.parquet.skip_metadata) } } diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 9c1e7e098909..54767b69bb9d 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -1131,10 +1131,7 @@ impl SessionConfig { /// Create an execution config with config options read from the environment pub fn from_env() -> Result { - Ok(Self { - options: ConfigOptions::from_env()?, - ..Default::default() - }) + Ok(ConfigOptions::from_env()?.into()) } /// Set a configuration option @@ -1295,7 +1292,7 @@ impl SessionConfig { let mut map = HashMap::new(); // copy configs from config_options for entry in self.options.entries() { - map.insert(entry.key, entry.value.unwrap_or_else(Default::default)); + map.insert(entry.key, entry.value.unwrap_or_default()); } map @@ -1384,6 +1381,15 @@ impl SessionConfig { } } +impl From for SessionConfig { + fn from(options: ConfigOptions) -> Self { + Self { + options, + ..Default::default() + } + } +} + /// Execution context for registering data sources and executing queries #[derive(Clone)] pub struct SessionState { @@ -1837,15 +1843,15 @@ impl TaskContext { aggregate_functions: HashMap>, runtime: Arc, ) -> Self { - let mut session_config = SessionConfig::new(); + let mut config = ConfigOptions::new(); for (k, v) in task_props { - let _ = session_config.config_options_mut().set(&k, &v); + let _ = config.set(&k, &v); } Self { task_id: Some(task_id), session_id, - session_config, + session_config: config.into(), scalar_functions, aggregate_functions, runtime, diff --git a/datafusion/core/src/physical_plan/coalesce_batches.rs b/datafusion/core/src/physical_plan/coalesce_batches.rs index fa8a18318db3..94a8fbd2e36a 100644 --- a/datafusion/core/src/physical_plan/coalesce_batches.rs +++ b/datafusion/core/src/physical_plan/coalesce_batches.rs @@ -308,24 +308,21 @@ pub fn concat_batches( #[cfg(test)] mod tests { use super::*; + use crate::config::ConfigOptions; use crate::datasource::MemTable; use crate::physical_plan::filter::FilterExec; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::{memory::MemoryExec, repartition::RepartitionExec}; - use crate::prelude::{SessionConfig, SessionContext}; + use crate::prelude::SessionContext; use crate::test::create_vec_batches; use arrow::datatypes::{DataType, Field, Schema}; #[tokio::test] async fn test_custom_batch_size() -> Result<()> { - let mut config = SessionConfig::new(); - config - .config_options_mut() - .built_in - .execution - .coalesce_target_batch_size = 1234; - - let ctx = SessionContext::with_config(config); + let mut config = ConfigOptions::new(); + config.built_in.execution.coalesce_target_batch_size = 1234; + + let ctx = SessionContext::with_config(config.into()); let plan = create_physical_plan(ctx).await?; let projection = plan.as_any().downcast_ref::().unwrap(); let coalesce = projection @@ -339,14 +336,10 @@ mod tests { #[tokio::test] async fn test_disable_coalesce() -> Result<()> { - let mut config = SessionConfig::new(); - config - .config_options_mut() - .built_in - .execution - .coalesce_batches = false; - - let ctx = SessionContext::with_config(config); + let mut config = ConfigOptions::new(); + config.built_in.execution.coalesce_batches = false; + + let ctx = SessionContext::with_config(config.into()); let plan = create_physical_plan(ctx).await?; let projection = plan.as_any().downcast_ref::().unwrap(); // projection should directly wrap filter with no coalesce step diff --git a/datafusion/core/src/physical_plan/file_format/parquet.rs b/datafusion/core/src/physical_plan/file_format/parquet.rs index 4f231d611a0f..af49f27ccd3e 100644 --- a/datafusion/core/src/physical_plan/file_format/parquet.rs +++ b/datafusion/core/src/physical_plan/file_format/parquet.rs @@ -200,7 +200,7 @@ impl ParquetExec { /// Return the value described in [`Self::with_pushdown_filters`] fn pushdown_filters(&self, config_options: &ConfigOptions) -> bool { self.pushdown_filters - .unwrap_or_else(|| config_options.built_in.execution.parquet.pushdown_filters) + .unwrap_or(config_options.built_in.execution.parquet.pushdown_filters) } /// If true, the `RowFilter` made by `pushdown_filters` may try to @@ -215,7 +215,7 @@ impl ParquetExec { /// Return the value described in [`Self::with_reorder_filters`] fn reorder_filters(&self, config_options: &ConfigOptions) -> bool { self.reorder_filters - .unwrap_or_else(|| config_options.built_in.execution.parquet.reorder_filters) + .unwrap_or(config_options.built_in.execution.parquet.reorder_filters) } /// If enabled, the reader will read the page index @@ -229,9 +229,8 @@ impl ParquetExec { /// Return the value described in [`Self::with_enable_page_index`] fn enable_page_index(&self, config_options: &ConfigOptions) -> bool { - self.enable_page_index.unwrap_or_else(|| { - config_options.built_in.execution.parquet.enable_page_index - }) + self.enable_page_index + .unwrap_or(config_options.built_in.execution.parquet.enable_page_index) } } diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index 8c23c9e9d6ae..7796023ad8ad 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -139,12 +139,8 @@ impl ContextWithParquet { let file = match unit { Unit::RowGroup => make_test_file_rg(scenario).await, Unit::Page => { - config - .config_options_mut() - .built_in - .execution - .parquet - .enable_page_index = true; + let config = config.config_options_mut(); + config.built_in.execution.parquet.enable_page_index = true; make_test_file_page(scenario).await } }; diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 1fbc7504db74..dd64110c7e59 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -16,6 +16,7 @@ // under the License. use super::*; +use datafusion::config::ConfigOptions; use datafusion::physical_plan::display::DisplayableExecutionPlan; #[tokio::test] @@ -847,13 +848,9 @@ async fn csv_explain_analyze_verbose() { #[tokio::test] async fn explain_logical_plan_only() { - let mut config = SessionConfig::new(); - config - .config_options_mut() - .built_in - .explain - .logical_plan_only = true; - let ctx = SessionContext::with_config(config); + let mut config = ConfigOptions::new(); + config.built_in.explain.logical_plan_only = true; + let ctx = SessionContext::with_config(config.into()); let sql = "EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c3)"; let actual = execute(&ctx, sql).await; let actual = normalize_vec_for_explain(actual); @@ -871,13 +868,9 @@ async fn explain_logical_plan_only() { #[tokio::test] async fn explain_physical_plan_only() { - let mut config = SessionConfig::new(); - config - .config_options_mut() - .built_in - .explain - .physical_plan_only = true; - let ctx = SessionContext::with_config(config); + let mut config = ConfigOptions::new(); + config.built_in.explain.physical_plan_only = true; + let ctx = SessionContext::with_config(config.into()); let sql = "EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c3)"; let actual = execute(&ctx, sql).await; let actual = normalize_vec_for_explain(actual); @@ -895,13 +888,9 @@ async fn explain_physical_plan_only() { #[tokio::test] async fn explain_nested() { async fn test_nested_explain(explain_phy_plan_flag: bool) { - let mut config = SessionConfig::new(); - config - .config_options_mut() - .built_in - .explain - .physical_plan_only = explain_phy_plan_flag; - let ctx = SessionContext::with_config(config); + let mut config = ConfigOptions::new(); + config.built_in.explain.physical_plan_only = explain_phy_plan_flag; + let ctx = SessionContext::with_config(config.into()); let sql = "EXPLAIN explain select 1"; let dataframe = ctx.sql(sql).await.unwrap(); let err = dataframe.create_physical_plan().await.unwrap_err(); diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index b95be06103c4..adb1fbe86f7c 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -25,6 +25,7 @@ use arrow::{ use chrono::prelude::*; use chrono::Duration; +use datafusion::config::ConfigOptions; use datafusion::datasource::TableProvider; use datafusion::from_slice::FromSlice; use datafusion::logical_expr::{Aggregate, LogicalPlan, Projection, TableScan}; @@ -566,14 +567,10 @@ fn create_sort_merge_join_context( column_left: &str, column_right: &str, ) -> Result { - let mut config = SessionConfig::new(); - config - .config_options_mut() - .built_in - .optimizer - .prefer_hash_join = false; + let mut config = ConfigOptions::new(); + config.built_in.optimizer.prefer_hash_join = false; - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::with_config(config.into()); let t1_schema = Arc::new(Schema::new(vec![ Field::new(column_left, DataType::UInt32, true), @@ -619,19 +616,11 @@ fn create_sort_merge_join_context( } fn create_sort_merge_join_datatype_context() -> Result { - let mut config = SessionConfig::new(); - config - .config_options_mut() - .built_in - .optimizer - .prefer_hash_join = false; - config - .config_options_mut() - .built_in - .execution - .target_partitions = 2; - - let ctx = SessionContext::with_config(config); + let mut config = ConfigOptions::new(); + config.built_in.optimizer.prefer_hash_join = false; + config.built_in.execution.target_partitions = 2; + + let ctx = SessionContext::with_config(config.into()); let t1_schema = Schema::new(vec![ Field::new("c1", DataType::Date32, true), diff --git a/parquet-test-utils/src/lib.rs b/parquet-test-utils/src/lib.rs index a931502e1990..f11d1cda6f09 100644 --- a/parquet-test-utils/src/lib.rs +++ b/parquet-test-utils/src/lib.rs @@ -23,6 +23,7 @@ use std::sync::Arc; use datafusion::arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; use datafusion::common::ToDFSchema; +use datafusion::config::ConfigOptions; use datafusion::datasource::listing::{ListingTableUrl, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::error::Result; @@ -57,12 +58,11 @@ pub struct ParquetScanOptions { impl ParquetScanOptions { /// Returns a [`SessionConfig`] with the given options pub fn config(&self) -> SessionConfig { - let mut config = SessionConfig::new(); - let mut parquet = &mut config.config_options_mut().built_in.execution.parquet; - parquet.pushdown_filters = self.pushdown_filters; - parquet.reorder_filters = self.reorder_filters; - parquet.enable_page_index = self.enable_page_index; - config + let mut config = ConfigOptions::new(); + config.built_in.execution.parquet.pushdown_filters = self.pushdown_filters; + config.built_in.execution.parquet.reorder_filters = self.reorder_filters; + config.built_in.execution.parquet.enable_page_index = self.enable_page_index; + config.into() } } From e63237de68b87a7997550a7455d37336261a75ee Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Thu, 29 Dec 2022 20:00:02 +0000 Subject: [PATCH 08/12] Further fixes --- benchmarks/src/bin/h2o.rs | 2 +- datafusion/core/src/config.rs | 10 +++++----- datafusion/core/src/datasource/file_format/parquet.rs | 2 +- datafusion/core/src/execution/context.rs | 6 +++--- .../sqllogictests/test_files/information_schema.slt | 3 ++- 5 files changed, 12 insertions(+), 11 deletions(-) diff --git a/benchmarks/src/bin/h2o.rs b/benchmarks/src/bin/h2o.rs index 940ccd1cd514..77023bb142fa 100644 --- a/benchmarks/src/bin/h2o.rs +++ b/benchmarks/src/bin/h2o.rs @@ -24,7 +24,7 @@ use datafusion::datasource::listing::{ ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, }; use datafusion::datasource::MemTable; -use datafusion::prelude::{CsvReadOptions, SessionConfig}; +use datafusion::prelude::CsvReadOptions; use datafusion::{arrow::util::pretty, error::Result, prelude::SessionContext}; use std::path::PathBuf; use std::sync::Arc; diff --git a/datafusion/core/src/config.rs b/datafusion/core/src/config.rs index 21ff904588fe..c769b9606692 100644 --- a/datafusion/core/src/config.rs +++ b/datafusion/core/src/config.rs @@ -127,10 +127,6 @@ config_namespace! { /// according to this time zone, and then extract the hour pub time_zone: Option, default = Some("+00:00".into()) - /// When set to true, the physical plan optimizer will try to add round robin - /// repartition to increase parallelism to leverage more CPU cores - pub round_robin_repartition: bool, default = true - /// Parquet options pub parquet: ParquetOptions, default = Default::default() } @@ -146,7 +142,7 @@ config_namespace! { /// 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 enable_pruning: bool, default = true + pub pruning: bool, default = true /// 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 @@ -173,6 +169,10 @@ config_namespace! { config_namespace! { /// Options related to query optimization pub struct OptimizerOptions { + /// When set to true, the physical plan optimizer will try to add round robin + /// repartition to increase parallelism to leverage more CPU cores + pub enable_round_robin_repartition: 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 diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index e116ffa6338d..1f3bb379c037 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -85,7 +85,7 @@ impl ParquetFormat { /// Return true if pruning is enabled pub fn enable_pruning(&self, config_options: &ConfigOptions) -> bool { self.enable_pruning - .unwrap_or(config_options.built_in.execution.parquet.enable_pruning) + .unwrap_or(config_options.built_in.execution.parquet.pruning) } /// Provide a hint to the size of the file metadata. If a hint is provided diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 54767b69bb9d..de5f9b753031 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -1261,13 +1261,13 @@ impl SessionConfig { /// Enables or disables the use of pruning predicate for parquet readers to skip row groups pub fn with_parquet_pruning(mut self, enabled: bool) -> Self { - self.options.built_in.execution.parquet.enable_pruning = enabled; + self.options.built_in.execution.parquet.pruning = enabled; self } /// Returns true if pruning predicate should be used to skip parquet row groups pub fn parquet_pruning(&self) -> bool { - self.options.built_in.execution.parquet.enable_pruning + self.options.built_in.execution.parquet.pruning } /// Enables or disables the collection of statistics after listing files @@ -1464,7 +1464,7 @@ impl SessionState { // - it's conflicted with some parts of the BasicEnforcement, since it will // introduce additional repartitioning while the BasicEnforcement aims at // reducing unnecessary repartitioning. - if config.options.built_in.execution.round_robin_repartition { + if config.options.built_in.optimizer.enable_round_robin_repartition { physical_optimizers.push(Arc::new(Repartition::new())); } //- Currently it will depend on the partition number to decide whether to change the diff --git a/datafusion/core/tests/sqllogictests/test_files/information_schema.slt b/datafusion/core/tests/sqllogictests/test_files/information_schema.slt index da2db8de64de..54e9471ad1ac 100644 --- a/datafusion/core/tests/sqllogictests/test_files/information_schema.slt +++ b/datafusion/core/tests/sqllogictests/test_files/information_schema.slt @@ -103,9 +103,10 @@ query R SHOW ALL ---- datafusion.catalog.create_default_catalog_and_schema true +datafusion.catalog.format NULL +datafusion.catalog.has_header false datafusion.catalog.information_schema true datafusion.catalog.location NULL -datafusion.catalog.type NULL datafusion.execution.batch_size 8192 datafusion.execution.coalesce_batches true datafusion.execution.coalesce_target_batch_size 4096 From bafec8b28325adbe9644c8490be1ee28c8d7e743 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Thu, 29 Dec 2022 20:04:01 +0000 Subject: [PATCH 09/12] Format --- datafusion/core/src/execution/context.rs | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index de5f9b753031..eb7f853a1feb 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -1456,6 +1456,9 @@ impl SessionState { // We need to take care of the rule ordering. They may influence each other. let mut physical_optimizers: Vec> = vec![Arc::new(AggregateStatistics::new())]; + + let built_in = &config.options.built_in; + // - In order to increase the parallelism, it will change the output partitioning // of some operators in the plan tree, which will influence other rules. // Therefore, it should be run as soon as possible. @@ -1464,7 +1467,7 @@ impl SessionState { // - it's conflicted with some parts of the BasicEnforcement, since it will // introduce additional repartitioning while the BasicEnforcement aims at // reducing unnecessary repartitioning. - if config.options.built_in.optimizer.enable_round_robin_repartition { + if built_in.optimizer.enable_round_robin_repartition { physical_optimizers.push(Arc::new(Repartition::new())); } //- Currently it will depend on the partition number to decide whether to change the @@ -1496,9 +1499,9 @@ impl SessionState { physical_optimizers.push(Arc::new(OptimizeSorts::new())); // It will not influence the distribution and ordering of the whole plan tree. // Therefore, to avoid influencing other rules, it should be run at last. - if config.options.built_in.execution.coalesce_batches { + if built_in.execution.coalesce_batches { physical_optimizers.push(Arc::new(CoalesceBatches::new( - config.options.built_in.execution.coalesce_target_batch_size, + built_in.execution.coalesce_target_batch_size, ))); } // The PipelineChecker rule will reject non-runnable query plans that use From f316e0f4594a3634cfe399159085f38eaca589b4 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Thu, 29 Dec 2022 20:50:17 +0000 Subject: [PATCH 10/12] Update config.md --- datafusion/core/src/config.rs | 5 ++- docs/source/user-guide/configs.md | 62 +++++++++++++++---------------- 2 files changed, 35 insertions(+), 32 deletions(-) diff --git a/datafusion/core/src/config.rs b/datafusion/core/src/config.rs index c769b9606692..8a21a15e5e58 100644 --- a/datafusion/core/src/config.rs +++ b/datafusion/core/src/config.rs @@ -371,12 +371,15 @@ impl ConfigOptions { let mut docs = "| key | default | description |\n".to_string(); docs += "|-----|---------|-------------|\n"; + let mut entries = s.entries(); + entries.sort_unstable_by(|a, b| a.key.cmp(&b.key)); + for entry in s.entries() { let _ = writeln!( &mut docs, "| {} | {} | {} |", entry.key, - entry.value.as_deref().unwrap_or(""), + entry.value.as_deref().unwrap_or("NULL"), entry.description ); } diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 1c5f08656388..2eaf8cffd233 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,34 +35,34 @@ Values are parsed according to the [same rules used in casts from Utf8](https:// 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 | type | default | description | -| --------------------------------------------------------- | ------- | ------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| datafusion.catalog.create_default_catalog_and_schema | Boolean | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.information_schema | Boolean | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | Utf8 | NULL | Location scanned to load tables for `default` schema, defaults to None | -| datafusion.catalog.type | Utf8 | NULL | Type of `TableProvider` to use when loading `default` schema. Defaults to None | -| datafusion.execution.batch_size | UInt64 | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would results in too much metadata memory consumption. | -| datafusion.execution.coalesce_batches | Boolean | 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.coalesce_target_batch_size'. | -| datafusion.execution.coalesce_target_batch_size | UInt64 | 4096 | Target batch size when coalescing batches. Uses in conjunction with the configuration setting 'datafusion.execution.coalesce_batches'. | -| datafusion.execution.collect_statistics | Boolean | false | Should DataFusion collect statistics after listing files | -| datafusion.execution.parquet.enable_page_index | Boolean | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | -| datafusion.execution.parquet.metadata_size_hint | UInt64 | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two read are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer. | -| datafusion.execution.parquet.pruning | Boolean | true | 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.pushdown_filters | Boolean | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. | -| datafusion.execution.parquet.reorder_filters | Boolean | false | If true, filter expressions evaluated during the parquet decoding opearation 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.skip_metadata | Boolean | true | 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.target_partitions | UInt64 | 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 | Utf8 | +00:00 | The session time zone which some function require e.g. EXTRACT(HOUR from SOME_TIME) shift the underline datetime according to the time zone, | -| then extract the hour. | -| datafusion.explain.logical_plan_only | Boolean | false | When set to true, the explain statement will only print logical plans. | -| datafusion.explain.physical_plan_only | Boolean | false | When set to true, the explain statement will only print physical plans. | -| datafusion.optimizer.enable_round_robin_repartition | Boolean | true | When set to true, the physical plan optimizer will try to add round robin repartition to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.filter_null_join_keys | Boolean | 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.hash_join_single_partition_threshold | UInt64 | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.max_passes | UInt64 | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.prefer_hash_join | Boolean | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficientlythan SortMergeJoin but consumes more memory. Defaults to true | -| datafusion.optimizer.repartition_aggregations | Boolean | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_joins | Boolean | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_windows | Boolean | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | -| datafusion.optimizer.skip_failed_rules | Boolean | true | 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.top_down_join_key_reordering | Boolean | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys. Defaults to true | +| key | default | description | +| --------------------------------------------------------- | ------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| datafusion.catalog.create_default_catalog_and_schema | true | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of cpu cores on the system. | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | false | If the file has a header | +| 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 results 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.coalesce_target_batch_size | 4096 | Target batch size when coalescing batches. Used in conjunction with `coalesce_batches` | +| 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. EXTRACT(HOUR from SOME_TIME), shift the underlying datetime according to this time zone, and then extract the hour | +| datafusion.execution.parquet.enable_page_index | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | +| datafusion.execution.parquet.pruning | true | 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 | 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 | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two read are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | +| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | +| datafusion.execution.parquet.reorder_filters | false | 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.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartition to increase parallelism to leverage more CPU cores | +| 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 `target_partitions` level" | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level" | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level" | +| datafusion.optimizer.skip_failed_rules | true | 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.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 | From ee2255fd3e221973310d341f5e53a65262ff2cae Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Fri, 30 Dec 2022 18:13:02 +0000 Subject: [PATCH 11/12] Flatten configuration --- benchmarks/src/bin/h2o.rs | 2 +- datafusion/core/src/config.rs | 65 +++++++++++-------- .../src/datasource/file_format/parquet.rs | 6 +- datafusion/core/src/execution/context.rs | 59 +++++++---------- .../src/physical_optimizer/enforcement.rs | 3 +- .../src/physical_optimizer/join_selection.rs | 2 +- .../src/physical_optimizer/repartition.rs | 4 +- .../src/physical_plan/coalesce_batches.rs | 4 +- .../src/physical_plan/file_format/parquet.rs | 6 +- datafusion/core/src/physical_plan/planner.rs | 4 +- datafusion/core/tests/config_from_env.rs | 6 +- datafusion/core/tests/parquet/mod.rs | 2 +- datafusion/core/tests/sql/explain_analyze.rs | 6 +- datafusion/core/tests/sql/mod.rs | 8 +-- parquet-test-utils/src/lib.rs | 6 +- 15 files changed, 93 insertions(+), 90 deletions(-) diff --git a/benchmarks/src/bin/h2o.rs b/benchmarks/src/bin/h2o.rs index 77023bb142fa..c747cb0a96ca 100644 --- a/benchmarks/src/bin/h2o.rs +++ b/benchmarks/src/bin/h2o.rs @@ -65,7 +65,7 @@ async fn main() -> Result<()> { async fn group_by(opt: &GroupBy) -> Result<()> { let path = opt.path.to_str().unwrap(); let mut config = ConfigOptions::from_env()?; - config.built_in.execution.batch_size = 65535; + config.execution.batch_size = 65535; let ctx = SessionContext::with_config(config.into()); diff --git a/datafusion/core/src/config.rs b/datafusion/core/src/config.rs index 4f3d3bb54471..091cd1f34946 100644 --- a/datafusion/core/src/config.rs +++ b/datafusion/core/src/config.rs @@ -34,7 +34,7 @@ macro_rules! config_namespace { ) => { $(#[doc = $struct_d])* - #[derive(Debug, Clone, PartialEq)] + #[derive(Debug, Clone)] #[non_exhaustive] $vis struct $struct_name{ $( @@ -221,23 +221,6 @@ config_namespace! { } } -config_namespace! { - /// All built-in DataFusion options - pub struct DataFusionOptions { - /// Catalog options - pub catalog: CatalogOptions, default = Default::default() - - /// Execution options - pub execution: ExecutionOptions, default = Default::default() - - /// Explain options - pub optimizer: OptimizerOptions, default = Default::default() - - /// Explain options - pub explain: ExplainOptions, default = Default::default() - } -} - /// A key value pair, with a corresponding description #[derive(Debug)] pub struct ConfigEntry { @@ -252,15 +235,45 @@ pub struct ConfigEntry { } /// Configuration options struct, able to store both built-in configuration and custom options -#[derive(Debug, Default, Clone)] +#[derive(Debug, Clone, Default)] +#[non_exhaustive] pub struct ConfigOptions { - /// Built-in DataFusion configuration - pub built_in: DataFusionOptions, - + /// Catalog options + pub catalog: CatalogOptions, + /// Execution options + pub execution: ExecutionOptions, + /// Explain options + pub optimizer: OptimizerOptions, + /// Explain options + pub explain: ExplainOptions, /// Optional extensions registered using [`Extensions::insert`] pub extensions: Extensions, } +impl ConfigField for ConfigOptions { + fn set(&mut self, key: &str, value: &str) -> Result<()> { + // Extensions are handled in the public `ConfigOptions::set` + let (key, rem) = key.split_once('.').unwrap_or((key, "")); + match key { + "catalog" => self.catalog.set(rem, value), + "execution" => self.execution.set(rem, value), + "optimizer" => self.optimizer.set(rem, value), + "explain" => self.explain.set(rem, value), + _ => Err(DataFusionError::Internal(format!( + "Config value \"{}\" not found on ConfigOptions", + key + ))), + } + } + + fn visit(&self, v: &mut V, _key_prefix: &str, _description: &'static str) { + self.catalog.visit(v, "datafusion.catalog", ""); + self.execution.visit(v, "datafusion.execution", ""); + self.optimizer.visit(v, "datafusion.optimizer", ""); + self.explain.visit(v, "datafusion.explain", ""); + } +} + impl ConfigOptions { /// Creates a new [`ConfigOptions`] with default values pub fn new() -> Self { @@ -276,7 +289,7 @@ impl ConfigOptions { })?; if prefix == "datafusion" { - return self.built_in.set(key, value); + return ConfigField::set(self, key, value); } let e = self.extensions.0.get_mut(prefix); @@ -313,7 +326,7 @@ impl ConfigOptions { let mut keys = Visitor(vec![]); let mut ret = Self::default(); - ret.built_in.visit(&mut keys, "datafusion", ""); + ret.visit(&mut keys, "datafusion", ""); for key in keys.0 { let env = key.to_uppercase().replace('.', "_"); @@ -353,7 +366,7 @@ impl ConfigOptions { } let mut v = Visitor(vec![]); - self.built_in.visit(&mut v, "datafusion", ""); + self.visit(&mut v, "datafusion", ""); v.0.extend(self.extensions.0.values().flat_map(|e| e.0.entries())); v.0 @@ -364,7 +377,7 @@ impl ConfigOptions { use std::fmt::Write as _; let mut s = Self::default(); - s.built_in.execution.target_partitions = 0; // Normalize for display + s.execution.target_partitions = 0; // Normalize for display let mut docs = "| key | default | description |\n".to_string(); docs += "|-----|---------|-------------|\n"; diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 1f3bb379c037..0d577cecfe76 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -85,7 +85,7 @@ impl ParquetFormat { /// Return true if pruning is enabled pub fn enable_pruning(&self, config_options: &ConfigOptions) -> bool { self.enable_pruning - .unwrap_or(config_options.built_in.execution.parquet.pruning) + .unwrap_or(config_options.execution.parquet.pruning) } /// Provide a hint to the size of the file metadata. If a hint is provided @@ -101,7 +101,7 @@ impl ParquetFormat { /// Return the metadata size hint if set pub fn metadata_size_hint(&self, config_options: &ConfigOptions) -> Option { - let hint = config_options.built_in.execution.parquet.metadata_size_hint; + let hint = config_options.execution.parquet.metadata_size_hint; self.metadata_size_hint.or(hint) } @@ -119,7 +119,7 @@ impl ParquetFormat { /// schema merging. pub fn skip_metadata(&self, config_options: &ConfigOptions) -> bool { self.skip_metadata - .unwrap_or(config_options.built_in.execution.parquet.skip_metadata) + .unwrap_or(config_options.execution.parquet.skip_metadata) } } diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 614a85c8110f..1e52c84ac103 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -1165,7 +1165,7 @@ impl SessionConfig { pub fn with_batch_size(mut self, n: usize) -> Self { // batch size must be greater than zero assert!(n > 0); - self.options.built_in.execution.batch_size = n; + self.options.execution.batch_size = n; self } @@ -1173,46 +1173,43 @@ impl SessionConfig { pub fn with_target_partitions(mut self, n: usize) -> Self { // partition count must be greater than zero assert!(n > 0); - self.options.built_in.execution.target_partitions = n; + self.options.execution.target_partitions = n; self } /// get target_partitions pub fn target_partitions(&self) -> usize { - self.options.built_in.execution.target_partitions + self.options.execution.target_partitions } /// Is the information schema enabled? pub fn information_schema(&self) -> bool { - self.options.built_in.catalog.information_schema + self.options.catalog.information_schema } /// Should the context create the default catalog and schema? pub fn create_default_catalog_and_schema(&self) -> bool { - self.options - .built_in - .catalog - .create_default_catalog_and_schema + self.options.catalog.create_default_catalog_and_schema } /// Are joins repartitioned during execution? pub fn repartition_joins(&self) -> bool { - self.options.built_in.optimizer.repartition_joins + self.options.optimizer.repartition_joins } /// Are aggregates repartitioned during execution? pub fn repartition_aggregations(&self) -> bool { - self.options.built_in.optimizer.repartition_aggregations + self.options.optimizer.repartition_aggregations } /// Are window functions repartitioned during execution? pub fn repartition_window_functions(&self) -> bool { - self.options.built_in.optimizer.repartition_windows + self.options.optimizer.repartition_windows } /// Are statistics collected during execution? pub fn collect_statistics(&self) -> bool { - self.options.built_in.execution.collect_statistics + self.options.execution.collect_statistics } /// Selects a name for the default catalog and schema @@ -1228,57 +1225,54 @@ impl SessionConfig { /// Controls whether the default catalog and schema will be automatically created pub fn with_create_default_catalog_and_schema(mut self, create: bool) -> Self { - self.options - .built_in - .catalog - .create_default_catalog_and_schema = create; + self.options.catalog.create_default_catalog_and_schema = create; self } /// Enables or disables the inclusion of `information_schema` virtual tables pub fn with_information_schema(mut self, enabled: bool) -> Self { - self.options.built_in.catalog.information_schema = enabled; + self.options.catalog.information_schema = enabled; self } /// Enables or disables the use of repartitioning for joins to improve parallelism pub fn with_repartition_joins(mut self, enabled: bool) -> Self { - self.options.built_in.optimizer.repartition_joins = enabled; + self.options.optimizer.repartition_joins = enabled; self } /// Enables or disables the use of repartitioning for aggregations to improve parallelism pub fn with_repartition_aggregations(mut self, enabled: bool) -> Self { - self.options.built_in.optimizer.repartition_aggregations = enabled; + self.options.optimizer.repartition_aggregations = enabled; self } /// Enables or disables the use of repartitioning for window functions to improve parallelism pub fn with_repartition_windows(mut self, enabled: bool) -> Self { - self.options.built_in.optimizer.repartition_windows = enabled; + self.options.optimizer.repartition_windows = enabled; self } /// Enables or disables the use of pruning predicate for parquet readers to skip row groups pub fn with_parquet_pruning(mut self, enabled: bool) -> Self { - self.options.built_in.execution.parquet.pruning = enabled; + self.options.execution.parquet.pruning = enabled; self } /// Returns true if pruning predicate should be used to skip parquet row groups pub fn parquet_pruning(&self) -> bool { - self.options.built_in.execution.parquet.pruning + self.options.execution.parquet.pruning } /// Enables or disables the collection of statistics after listing files pub fn with_collect_statistics(mut self, enabled: bool) -> Self { - self.options.built_in.execution.collect_statistics = enabled; + self.options.execution.collect_statistics = enabled; self } /// Get the currently configured batch size pub fn batch_size(&self) -> usize { - self.options.built_in.execution.batch_size + self.options.execution.batch_size } /// Convert configuration options to name-value pairs with values @@ -1457,8 +1451,6 @@ impl SessionState { let mut physical_optimizers: Vec> = vec![Arc::new(AggregateStatistics::new())]; - let built_in = &config.options.built_in; - // - In order to increase the parallelism, it will change the output partitioning // of some operators in the plan tree, which will influence other rules. // Therefore, it should be run as soon as possible. @@ -1467,7 +1459,7 @@ impl SessionState { // - it's conflicted with some parts of the BasicEnforcement, since it will // introduce additional repartitioning while the BasicEnforcement aims at // reducing unnecessary repartitioning. - if built_in.optimizer.enable_round_robin_repartition { + if config.options.optimizer.enable_round_robin_repartition { physical_optimizers.push(Arc::new(Repartition::new())); } //- Currently it will depend on the partition number to decide whether to change the @@ -1499,9 +1491,9 @@ impl SessionState { physical_optimizers.push(Arc::new(OptimizeSorts::new())); // It will not influence the distribution and ordering of the whole plan tree. // Therefore, to avoid influencing other rules, it should be run at last. - if built_in.execution.coalesce_batches { + if config.options.execution.coalesce_batches { physical_optimizers.push(Arc::new(CoalesceBatches::new( - built_in.execution.batch_size, + config.options.execution.batch_size, ))); } // The PipelineChecker rule will reject non-runnable query plans that use @@ -1528,8 +1520,8 @@ impl SessionState { runtime: &Arc, default_catalog: &MemoryCatalogProvider, ) { - let url = config.options.built_in.catalog.location.as_ref(); - let format = config.options.built_in.catalog.format.as_ref(); + let url = config.options.catalog.location.as_ref(); + let format = config.options.catalog.format.as_ref(); let (url, format) = match (url, format) { (Some(url), Some(format)) => (url, format), _ => return, @@ -1537,7 +1529,7 @@ impl SessionState { let url = url.to_string(); let format = format.to_string(); - let has_header = config.options.built_in.catalog.has_header; + let has_header = config.options.catalog.has_header; let url = Url::parse(url.as_str()).expect("Invalid default catalog location!"); let authority = match url.host_str() { Some(host) => format!("{}://{}", url.scheme(), host), @@ -1654,7 +1646,7 @@ impl SessionState { pub fn optimize(&self, plan: &LogicalPlan) -> Result { // TODO: Implement OptimizerContext directly on DataFrame (#4631) (#4626) let config = { - let config = &self.config_options().built_in.optimizer; + let config = &self.config_options().optimizer; OptimizerContext::new() .with_skip_failing_rules(config.skip_failed_rules) .with_max_passes(config.max_passes as u8) @@ -1784,7 +1776,6 @@ impl ContextProvider for SessionState { "datafusion.execution.time_zone" => self .config .options - .built_in .execution .time_zone .as_ref() diff --git a/datafusion/core/src/physical_optimizer/enforcement.rs b/datafusion/core/src/physical_optimizer/enforcement.rs index 22b74d948c75..0b8206a78956 100644 --- a/datafusion/core/src/physical_optimizer/enforcement.rs +++ b/datafusion/core/src/physical_optimizer/enforcement.rs @@ -70,7 +70,6 @@ impl PhysicalOptimizerRule for BasicEnforcement { plan: Arc, config: &ConfigOptions, ) -> Result> { - let config = &config.built_in; let target_partitions = config.execution.target_partitions; let top_down_join_key_reordering = config.optimizer.top_down_join_key_reordering; let new_plan = if top_down_join_key_reordering { @@ -1134,7 +1133,7 @@ mod tests { let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); let mut config = ConfigOptions::new(); - config.built_in.execution.target_partitions = 10; + config.execution.target_partitions = 10; // run optimizer let optimizer = BasicEnforcement {}; diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 16626a37b3e3..2589fa625e31 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -214,7 +214,7 @@ impl PhysicalOptimizerRule for JoinSelection { plan: Arc, config: &ConfigOptions, ) -> Result> { - let config = &config.built_in.optimizer; + let config = &config.optimizer; let collect_left_threshold = config.hash_join_single_partition_threshold; plan.transform_up(&|plan| { if let Some(hash_join) = plan.as_any().downcast_ref::() { diff --git a/datafusion/core/src/physical_optimizer/repartition.rs b/datafusion/core/src/physical_optimizer/repartition.rs index 19512f9b3c13..9320de41e032 100644 --- a/datafusion/core/src/physical_optimizer/repartition.rs +++ b/datafusion/core/src/physical_optimizer/repartition.rs @@ -210,7 +210,7 @@ impl PhysicalOptimizerRule for Repartition { plan: Arc, config: &ConfigOptions, ) -> Result> { - let target_partitions = config.built_in.execution.target_partitions; + let target_partitions = config.execution.target_partitions; // Don't run optimizer if target_partitions == 1 if target_partitions == 1 { Ok(plan) @@ -364,7 +364,7 @@ mod tests { let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); let mut config = ConfigOptions::new(); - config.built_in.execution.target_partitions = 10; + config.execution.target_partitions = 10; // run optimizer let optimizers: Vec> = vec![ diff --git a/datafusion/core/src/physical_plan/coalesce_batches.rs b/datafusion/core/src/physical_plan/coalesce_batches.rs index c2e7e3e79062..b50ac08ba5ad 100644 --- a/datafusion/core/src/physical_plan/coalesce_batches.rs +++ b/datafusion/core/src/physical_plan/coalesce_batches.rs @@ -320,7 +320,7 @@ mod tests { #[tokio::test] async fn test_custom_batch_size() -> Result<()> { let mut config = ConfigOptions::new(); - config.built_in.execution.batch_size = 1234; + config.execution.batch_size = 1234; let ctx = SessionContext::with_config(config.into()); let plan = create_physical_plan(ctx).await?; @@ -337,7 +337,7 @@ mod tests { #[tokio::test] async fn test_disable_coalesce() -> Result<()> { let mut config = ConfigOptions::new(); - config.built_in.execution.coalesce_batches = false; + config.execution.coalesce_batches = false; let ctx = SessionContext::with_config(config.into()); let plan = create_physical_plan(ctx).await?; diff --git a/datafusion/core/src/physical_plan/file_format/parquet.rs b/datafusion/core/src/physical_plan/file_format/parquet.rs index af49f27ccd3e..a7c62d418550 100644 --- a/datafusion/core/src/physical_plan/file_format/parquet.rs +++ b/datafusion/core/src/physical_plan/file_format/parquet.rs @@ -200,7 +200,7 @@ impl ParquetExec { /// Return the value described in [`Self::with_pushdown_filters`] fn pushdown_filters(&self, config_options: &ConfigOptions) -> bool { self.pushdown_filters - .unwrap_or(config_options.built_in.execution.parquet.pushdown_filters) + .unwrap_or(config_options.execution.parquet.pushdown_filters) } /// If true, the `RowFilter` made by `pushdown_filters` may try to @@ -215,7 +215,7 @@ impl ParquetExec { /// Return the value described in [`Self::with_reorder_filters`] fn reorder_filters(&self, config_options: &ConfigOptions) -> bool { self.reorder_filters - .unwrap_or(config_options.built_in.execution.parquet.reorder_filters) + .unwrap_or(config_options.execution.parquet.reorder_filters) } /// If enabled, the reader will read the page index @@ -230,7 +230,7 @@ impl ParquetExec { /// Return the value described in [`Self::with_enable_page_index`] fn enable_page_index(&self, config_options: &ConfigOptions) -> bool { self.enable_page_index - .unwrap_or(config_options.built_in.execution.parquet.enable_page_index) + .unwrap_or(config_options.execution.parquet.enable_page_index) } } diff --git a/datafusion/core/src/physical_plan/planner.rs b/datafusion/core/src/physical_plan/planner.rs index b68b1d8683a1..507b7a671220 100644 --- a/datafusion/core/src/physical_plan/planner.rs +++ b/datafusion/core/src/physical_plan/planner.rs @@ -992,7 +992,7 @@ impl DefaultPhysicalPlanner { _ => None }; - let prefer_hash_join = session_state.config_options().built_in.optimizer.prefer_hash_join; + let prefer_hash_join = session_state.config_options().optimizer.prefer_hash_join; if join_on.is_empty() { // there is no equal join condition, use the nested loop join // TODO optimize the plan, and use the config of `target_partitions` and `repartition_joins` @@ -1708,7 +1708,7 @@ impl DefaultPhysicalPlanner { use PlanType::*; let mut stringified_plans = vec![]; - let config = &session_state.config_options().built_in.explain; + let config = &session_state.config_options().explain; if !config.physical_plan_only { stringified_plans = e.stringified_plans.clone(); diff --git a/datafusion/core/tests/config_from_env.rs b/datafusion/core/tests/config_from_env.rs index 58937feece0a..a420f5c9f5a3 100644 --- a/datafusion/core/tests/config_from_env.rs +++ b/datafusion/core/tests/config_from_env.rs @@ -25,14 +25,14 @@ fn from_env() { env::set_var(env_key, "true"); let config = ConfigOptions::from_env().unwrap(); env::remove_var(env_key); - assert!(config.built_in.optimizer.filter_null_join_keys); + assert!(config.optimizer.filter_null_join_keys); let env_key = "DATAFUSION_EXECUTION_BATCH_SIZE"; // for valid testing env::set_var(env_key, "4096"); let config = ConfigOptions::from_env().unwrap(); - assert_eq!(config.built_in.execution.batch_size, 4096); + assert_eq!(config.execution.batch_size, 4096); // for invalid testing env::set_var(env_key, "abc"); @@ -41,5 +41,5 @@ fn from_env() { env::remove_var(env_key); let config = ConfigOptions::from_env().unwrap(); - assert_eq!(config.built_in.execution.batch_size, 8192); // set to its default value + assert_eq!(config.execution.batch_size, 8192); // set to its default value } diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index 7796023ad8ad..123095186ed2 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -140,7 +140,7 @@ impl ContextWithParquet { Unit::RowGroup => make_test_file_rg(scenario).await, Unit::Page => { let config = config.config_options_mut(); - config.built_in.execution.parquet.enable_page_index = true; + config.execution.parquet.enable_page_index = true; make_test_file_page(scenario).await } }; diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 04c743b291fd..80666dedce5f 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -855,7 +855,7 @@ async fn csv_explain_analyze_verbose() { #[tokio::test] async fn explain_logical_plan_only() { let mut config = ConfigOptions::new(); - config.built_in.explain.logical_plan_only = true; + config.explain.logical_plan_only = true; let ctx = SessionContext::with_config(config.into()); let sql = "EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c3)"; let actual = execute(&ctx, sql).await; @@ -875,7 +875,7 @@ async fn explain_logical_plan_only() { #[tokio::test] async fn explain_physical_plan_only() { let mut config = ConfigOptions::new(); - config.built_in.explain.physical_plan_only = true; + config.explain.physical_plan_only = true; let ctx = SessionContext::with_config(config.into()); let sql = "EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c3)"; let actual = execute(&ctx, sql).await; @@ -895,7 +895,7 @@ async fn explain_physical_plan_only() { async fn explain_nested() { async fn test_nested_explain(explain_phy_plan_flag: bool) { let mut config = ConfigOptions::new(); - config.built_in.explain.physical_plan_only = explain_phy_plan_flag; + config.explain.physical_plan_only = explain_phy_plan_flag; let ctx = SessionContext::with_config(config.into()); let sql = "EXPLAIN explain select 1"; let dataframe = ctx.sql(sql).await.unwrap(); diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 821e00accec5..445950e1d085 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -571,7 +571,7 @@ fn create_sort_merge_join_context( column_right: &str, ) -> Result { let mut config = ConfigOptions::new(); - config.built_in.optimizer.prefer_hash_join = false; + config.optimizer.prefer_hash_join = false; let ctx = SessionContext::with_config(config.into()); @@ -620,9 +620,9 @@ fn create_sort_merge_join_context( fn create_sort_merge_join_datatype_context() -> Result { let mut config = ConfigOptions::new(); - config.built_in.optimizer.prefer_hash_join = false; - config.built_in.execution.target_partitions = 2; - config.built_in.execution.batch_size = 4096; + config.optimizer.prefer_hash_join = false; + config.execution.target_partitions = 2; + config.execution.batch_size = 4096; let ctx = SessionContext::with_config(config.into()); diff --git a/parquet-test-utils/src/lib.rs b/parquet-test-utils/src/lib.rs index f11d1cda6f09..e1b3c5c18a77 100644 --- a/parquet-test-utils/src/lib.rs +++ b/parquet-test-utils/src/lib.rs @@ -59,9 +59,9 @@ impl ParquetScanOptions { /// Returns a [`SessionConfig`] with the given options pub fn config(&self) -> SessionConfig { let mut config = ConfigOptions::new(); - config.built_in.execution.parquet.pushdown_filters = self.pushdown_filters; - config.built_in.execution.parquet.reorder_filters = self.reorder_filters; - config.built_in.execution.parquet.enable_page_index = self.enable_page_index; + config.execution.parquet.pushdown_filters = self.pushdown_filters; + config.execution.parquet.reorder_filters = self.reorder_filters; + config.execution.parquet.enable_page_index = self.enable_page_index; config.into() } } From bd68058175e21b2963b79fb71c24d3c9eebb7a9d Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies Date: Fri, 30 Dec 2022 18:26:02 +0000 Subject: [PATCH 12/12] Document macro --- datafusion/core/src/config.rs | 76 +++++++++++++++++++++++++++++++++++ 1 file changed, 76 insertions(+) diff --git a/datafusion/core/src/config.rs b/datafusion/core/src/config.rs index 091cd1f34946..4e6cddcd5ae0 100644 --- a/datafusion/core/src/config.rs +++ b/datafusion/core/src/config.rs @@ -22,6 +22,82 @@ use std::any::Any; use std::collections::BTreeMap; use std::fmt::Display; +/// 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 +/// +/// For example, +/// +/// ```ignore +/// config_namespace! { +/// /// Amazing config +/// pub struct MyConfig { +/// /// Field 1 doc +/// field1: String, default = "".to_string() +/// +/// /// Field 2 doc +/// field2: usize, default = 232 +/// +/// /// Field 3 doc +/// field3: Option, 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, +/// } +/// 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" => self.field1.set(rem, value), +/// "field2" => self.field2.set(rem, value), +/// "field3" => self.field3.set(rem, value), +/// _ => Err(DataFusionError::Internal(format!( +/// "Config value \"{}\" not found on MyConfig", +/// key +/// ))), +/// } +/// } +/// +/// fn 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_rules! config_namespace { ( $(#[doc = $struct_d:tt])*