@@ -24,7 +24,7 @@ use std::str::FromStr;
2424
2525use crate :: error:: _config_err;
2626use crate :: parsers:: CompressionTypeVariant ;
27- use crate :: { DataFusionError , FileType , Result } ;
27+ use crate :: { DataFusionError , Result } ;
2828
2929/// A macro that wraps a configuration struct and automatically derives
3030/// [`Default`] and [`ConfigField`] for it, allowing it to be used
@@ -204,6 +204,11 @@ config_namespace! {
204204 /// MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, and Ansi.
205205 pub dialect: String , default = "generic" . to_string( )
206206
207+ /// If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but
208+ /// ignore the length. If false, error if a `VARCHAR` with a length is
209+ /// specified. The Arrow type system does not have a notion of maximum
210+ /// string length and thus DataFusion can not enforce such limits.
211+ pub support_varchar_with_length: bool , default = true
207212 }
208213}
209214
@@ -303,6 +308,7 @@ config_namespace! {
303308 /// statistics into the same file groups.
304309 /// Currently experimental
305310 pub split_file_groups_by_statistics: bool , default = false
311+
306312 }
307313}
308314
@@ -1116,6 +1122,16 @@ macro_rules! extensions_options {
11161122 }
11171123}
11181124
1125+ /// These file types have special built in behavior for configuration.
1126+ /// Use TableOptions::Extensions for configuring other file types.
1127+ #[ derive( Debug , Clone ) ]
1128+ pub enum ConfigFileType {
1129+ CSV ,
1130+ #[ cfg( feature = "parquet" ) ]
1131+ PARQUET ,
1132+ JSON ,
1133+ }
1134+
11191135/// Represents the configuration options available for handling different table formats within a data processing application.
11201136/// This struct encompasses options for various file formats including CSV, Parquet, and JSON, allowing for flexible configuration
11211137/// of parsing and writing behaviors specific to each format. Additionally, it supports extending functionality through custom extensions.
@@ -1134,7 +1150,7 @@ pub struct TableOptions {
11341150
11351151 /// The current file format that the table operations should assume. This option allows
11361152 /// for dynamic switching between the supported file types (e.g., CSV, Parquet, JSON).
1137- pub current_format : Option < FileType > ,
1153+ pub current_format : Option < ConfigFileType > ,
11381154
11391155 /// Optional extensions that can be used to extend or customize the behavior of the table
11401156 /// options. Extensions can be registered using `Extensions::insert` and might include
@@ -1152,10 +1168,9 @@ impl ConfigField for TableOptions {
11521168 if let Some ( file_type) = & self . current_format {
11531169 match file_type {
11541170 #[ cfg( feature = "parquet" ) ]
1155- FileType :: PARQUET => self . parquet . visit ( v, "format" , "" ) ,
1156- FileType :: CSV => self . csv . visit ( v, "format" , "" ) ,
1157- FileType :: JSON => self . json . visit ( v, "format" , "" ) ,
1158- _ => { }
1171+ ConfigFileType :: PARQUET => self . parquet . visit ( v, "format" , "" ) ,
1172+ ConfigFileType :: CSV => self . csv . visit ( v, "format" , "" ) ,
1173+ ConfigFileType :: JSON => self . json . visit ( v, "format" , "" ) ,
11591174 }
11601175 } else {
11611176 self . csv . visit ( v, "csv" , "" ) ;
@@ -1188,12 +1203,9 @@ impl ConfigField for TableOptions {
11881203 match key {
11891204 "format" => match format {
11901205 #[ cfg( feature = "parquet" ) ]
1191- FileType :: PARQUET => self . parquet . set ( rem, value) ,
1192- FileType :: CSV => self . csv . set ( rem, value) ,
1193- FileType :: JSON => self . json . set ( rem, value) ,
1194- _ => {
1195- _config_err ! ( "Config value \" {key}\" is not supported on {}" , format)
1196- }
1206+ ConfigFileType :: PARQUET => self . parquet . set ( rem, value) ,
1207+ ConfigFileType :: CSV => self . csv . set ( rem, value) ,
1208+ ConfigFileType :: JSON => self . json . set ( rem, value) ,
11971209 } ,
11981210 _ => _config_err ! ( "Config value \" {key}\" not found on TableOptions" ) ,
11991211 }
@@ -1210,15 +1222,6 @@ impl TableOptions {
12101222 Self :: default ( )
12111223 }
12121224
1213- /// Sets the file format for the table.
1214- ///
1215- /// # Parameters
1216- ///
1217- /// * `format`: The file format to use (e.g., CSV, Parquet).
1218- pub fn set_file_format ( & mut self , format : FileType ) {
1219- self . current_format = Some ( format) ;
1220- }
1221-
12221225 /// Creates a new `TableOptions` instance initialized with settings from a given session config.
12231226 ///
12241227 /// # Parameters
@@ -1249,6 +1252,15 @@ impl TableOptions {
12491252 clone
12501253 }
12511254
1255+ /// Sets the file format for the table.
1256+ ///
1257+ /// # Parameters
1258+ ///
1259+ /// * `format`: The file format to use (e.g., CSV, Parquet).
1260+ pub fn set_config_format ( & mut self , format : ConfigFileType ) {
1261+ self . current_format = Some ( format) ;
1262+ }
1263+
12521264 /// Sets the extensions for this `TableOptions` instance.
12531265 ///
12541266 /// # Parameters
@@ -1673,6 +1685,8 @@ config_namespace! {
16731685 }
16741686}
16751687
1688+ pub trait FormatOptionsExt : Display { }
1689+
16761690#[ derive( Debug , Clone , PartialEq ) ]
16771691#[ allow( clippy:: large_enum_variant) ]
16781692pub enum FormatOptions {
@@ -1698,28 +1712,15 @@ impl Display for FormatOptions {
16981712 }
16991713}
17001714
1701- impl From < FileType > for FormatOptions {
1702- fn from ( value : FileType ) -> Self {
1703- match value {
1704- FileType :: ARROW => FormatOptions :: ARROW ,
1705- FileType :: AVRO => FormatOptions :: AVRO ,
1706- #[ cfg( feature = "parquet" ) ]
1707- FileType :: PARQUET => FormatOptions :: PARQUET ( TableParquetOptions :: default ( ) ) ,
1708- FileType :: CSV => FormatOptions :: CSV ( CsvOptions :: default ( ) ) ,
1709- FileType :: JSON => FormatOptions :: JSON ( JsonOptions :: default ( ) ) ,
1710- }
1711- }
1712- }
1713-
17141715#[ cfg( test) ]
17151716mod tests {
17161717 use std:: any:: Any ;
17171718 use std:: collections:: HashMap ;
17181719
17191720 use crate :: config:: {
1720- ConfigEntry , ConfigExtension , ExtensionOptions , Extensions , TableOptions ,
1721+ ConfigEntry , ConfigExtension , ConfigFileType , ExtensionOptions , Extensions ,
1722+ TableOptions ,
17211723 } ;
1722- use crate :: FileType ;
17231724
17241725 #[ derive( Default , Debug , Clone ) ]
17251726 pub struct TestExtensionConfig {
@@ -1777,7 +1778,7 @@ mod tests {
17771778 let mut extension = Extensions :: new ( ) ;
17781779 extension. insert ( TestExtensionConfig :: default ( ) ) ;
17791780 let mut table_config = TableOptions :: new ( ) . with_extensions ( extension) ;
1780- table_config. set_file_format ( FileType :: CSV ) ;
1781+ table_config. set_config_format ( ConfigFileType :: CSV ) ;
17811782 table_config. set ( "format.delimiter" , ";" ) . unwrap ( ) ;
17821783 assert_eq ! ( table_config. csv. delimiter, b';' ) ;
17831784 table_config. set ( "test.bootstrap.servers" , "asd" ) . unwrap ( ) ;
@@ -1794,7 +1795,7 @@ mod tests {
17941795 #[ test]
17951796 fn csv_u8_table_options ( ) {
17961797 let mut table_config = TableOptions :: new ( ) ;
1797- table_config. set_file_format ( FileType :: CSV ) ;
1798+ table_config. set_config_format ( ConfigFileType :: CSV ) ;
17981799 table_config. set ( "format.delimiter" , ";" ) . unwrap ( ) ;
17991800 assert_eq ! ( table_config. csv. delimiter as char , ';' ) ;
18001801 table_config. set ( "format.escape" , "\" " ) . unwrap ( ) ;
@@ -1807,7 +1808,7 @@ mod tests {
18071808 #[ test]
18081809 fn parquet_table_options ( ) {
18091810 let mut table_config = TableOptions :: new ( ) ;
1810- table_config. set_file_format ( FileType :: PARQUET ) ;
1811+ table_config. set_config_format ( ConfigFileType :: PARQUET ) ;
18111812 table_config
18121813 . set ( "format.bloom_filter_enabled::col1" , "true" )
18131814 . unwrap ( ) ;
@@ -1821,7 +1822,7 @@ mod tests {
18211822 #[ test]
18221823 fn parquet_table_options_config_entry ( ) {
18231824 let mut table_config = TableOptions :: new ( ) ;
1824- table_config. set_file_format ( FileType :: PARQUET ) ;
1825+ table_config. set_config_format ( ConfigFileType :: PARQUET ) ;
18251826 table_config
18261827 . set ( "format.bloom_filter_enabled::col1" , "true" )
18271828 . unwrap ( ) ;
@@ -1835,7 +1836,7 @@ mod tests {
18351836 #[ test]
18361837 fn parquet_table_options_config_metadata_entry ( ) {
18371838 let mut table_config = TableOptions :: new ( ) ;
1838- table_config. set_file_format ( FileType :: PARQUET ) ;
1839+ table_config. set_config_format ( ConfigFileType :: PARQUET ) ;
18391840 table_config. set ( "format.metadata::key1" , "" ) . unwrap ( ) ;
18401841 table_config. set ( "format.metadata::key2" , "value2" ) . unwrap ( ) ;
18411842 table_config
0 commit comments