Skip to content

Commit 58cc4e1

Browse files
berkaysynnadaalambozankabak
authored
Make CREATE EXTERNAL TABLE format options consistent, remove special syntax for HEADER ROW, DELIMITER and COMPRESSION (#10404)
* Simplify format options * Keep PG copy from tests same * Update datafusion/common/src/config.rs Co-authored-by: Andrew Lamb <[email protected]> * Update datafusion/core/src/datasource/file_format/csv.rs Co-authored-by: Andrew Lamb <[email protected]> * Remove WITH HEADER ROW * Review Part 1 * . * Fix failing tests * Revert "Fix failing tests" This reverts commit 9d81601. * Final commit * Minor * Review * Update avro.slt * Apply suggestions * Fix imports --------- Co-authored-by: Andrew Lamb <[email protected]> Co-authored-by: Mehmet Ozan Kabak <[email protected]>
1 parent 8cc92a9 commit 58cc4e1

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

72 files changed

+563
-734
lines changed

datafusion-cli/src/helper.rs

Lines changed: 49 additions & 37 deletions
Original file line numberDiff line numberDiff line change
@@ -20,25 +20,20 @@
2020
2121
use std::borrow::Cow;
2222

23+
use crate::highlighter::{NoSyntaxHighlighter, SyntaxHighlighter};
24+
2325
use datafusion::common::sql_datafusion_err;
2426
use datafusion::error::DataFusionError;
2527
use datafusion::sql::parser::{DFParser, Statement};
2628
use datafusion::sql::sqlparser::dialect::dialect_from_str;
2729
use datafusion::sql::sqlparser::parser::ParserError;
28-
use rustyline::completion::Completer;
29-
use rustyline::completion::FilenameCompleter;
30-
use rustyline::completion::Pair;
30+
31+
use rustyline::completion::{Completer, FilenameCompleter, Pair};
3132
use rustyline::error::ReadlineError;
3233
use rustyline::highlight::Highlighter;
3334
use rustyline::hint::Hinter;
34-
use rustyline::validate::ValidationContext;
35-
use rustyline::validate::ValidationResult;
36-
use rustyline::validate::Validator;
37-
use rustyline::Context;
38-
use rustyline::Helper;
39-
use rustyline::Result;
40-
41-
use crate::highlighter::{NoSyntaxHighlighter, SyntaxHighlighter};
35+
use rustyline::validate::{ValidationContext, ValidationResult, Validator};
36+
use rustyline::{Context, Helper, Result};
4237

4338
pub struct CliHelper {
4439
completer: FilenameCompleter,
@@ -259,52 +254,69 @@ mod tests {
259254

260255
// shoule be valid
261256
let result = readline_direct(
262-
Cursor::new(r"create external table test stored as csv location 'data.csv' delimiter ',';".as_bytes()),
263-
&validator,
264-
)?;
257+
Cursor::new(
258+
r"create external table test stored as csv location 'data.csv' options ('format.delimiter' ',');"
259+
.as_bytes(),
260+
),
261+
&validator,
262+
)?;
265263
assert!(matches!(result, ValidationResult::Valid(None)));
266264

267265
let result = readline_direct(
268-
Cursor::new(r"create external table test stored as csv location 'data.csv' delimiter '\0';".as_bytes()),
269-
&validator,
270-
)?;
266+
Cursor::new(
267+
r"create external table test stored as csv location 'data.csv' options ('format.delimiter' '\0');"
268+
.as_bytes()),
269+
&validator,
270+
)?;
271271
assert!(matches!(result, ValidationResult::Valid(None)));
272272

273273
let result = readline_direct(
274-
Cursor::new(r"create external table test stored as csv location 'data.csv' delimiter '\n';".as_bytes()),
275-
&validator,
276-
)?;
274+
Cursor::new(
275+
r"create external table test stored as csv location 'data.csv' options ('format.delimiter' '\n');"
276+
.as_bytes()),
277+
&validator,
278+
)?;
277279
assert!(matches!(result, ValidationResult::Valid(None)));
278280

279281
let result = readline_direct(
280-
Cursor::new(r"create external table test stored as csv location 'data.csv' delimiter '\r';".as_bytes()),
281-
&validator,
282-
)?;
282+
Cursor::new(
283+
r"create external table test stored as csv location 'data.csv' options ('format.delimiter' '\r');"
284+
.as_bytes()),
285+
&validator,
286+
)?;
283287
assert!(matches!(result, ValidationResult::Valid(None)));
284288

285289
let result = readline_direct(
286-
Cursor::new(r"create external table test stored as csv location 'data.csv' delimiter '\t';".as_bytes()),
287-
&validator,
288-
)?;
290+
Cursor::new(
291+
r"create external table test stored as csv location 'data.csv' options ('format.delimiter' '\t');"
292+
.as_bytes()),
293+
&validator,
294+
)?;
289295
assert!(matches!(result, ValidationResult::Valid(None)));
290296

291297
let result = readline_direct(
292-
Cursor::new(r"create external table test stored as csv location 'data.csv' delimiter '\\';".as_bytes()),
293-
&validator,
294-
)?;
298+
Cursor::new(
299+
r"create external table test stored as csv location 'data.csv' options ('format.delimiter' '\\');"
300+
.as_bytes()),
301+
&validator,
302+
)?;
295303
assert!(matches!(result, ValidationResult::Valid(None)));
296304

297-
// should be invalid
298305
let result = readline_direct(
299-
Cursor::new(r"create external table test stored as csv location 'data.csv' delimiter ',,';".as_bytes()),
300-
&validator,
301-
)?;
302-
assert!(matches!(result, ValidationResult::Invalid(Some(_))));
306+
Cursor::new(
307+
r"create external table test stored as csv location 'data.csv' options ('format.delimiter' ',,');"
308+
.as_bytes()),
309+
&validator,
310+
)?;
311+
assert!(matches!(result, ValidationResult::Valid(None)));
303312

313+
// should be invalid
304314
let result = readline_direct(
305-
Cursor::new(r"create external table test stored as csv location 'data.csv' delimiter '\u{07}';".as_bytes()),
306-
&validator,
307-
)?;
315+
Cursor::new(
316+
r"create external table test stored as csv location 'data.csv' options ('format.delimiter' '\u{07}');"
317+
.as_bytes()),
318+
&validator,
319+
)?;
308320
assert!(matches!(result, ValidationResult::Invalid(Some(_))));
309321

310322
Ok(())

datafusion/common/src/config.rs

Lines changed: 15 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -1564,18 +1564,21 @@ config_namespace_with_hashmap! {
15641564
config_namespace! {
15651565
/// Options controlling CSV format
15661566
pub struct CsvOptions {
1567-
pub has_header: bool, default = true
1567+
/// Specifies whether there is a CSV header (i.e. the first line
1568+
/// consists of is column names). The value `None` indicates that
1569+
/// the configuration should be consulted.
1570+
pub has_header: Option<bool>, default = None
15681571
pub delimiter: u8, default = b','
15691572
pub quote: u8, default = b'"'
15701573
pub escape: Option<u8>, default = None
15711574
pub compression: CompressionTypeVariant, default = CompressionTypeVariant::UNCOMPRESSED
15721575
pub schema_infer_max_rec: usize, default = 100
1573-
pub date_format: Option<String>, default = None
1574-
pub datetime_format: Option<String>, default = None
1575-
pub timestamp_format: Option<String>, default = None
1576-
pub timestamp_tz_format: Option<String>, default = None
1577-
pub time_format: Option<String>, default = None
1578-
pub null_value: Option<String>, default = None
1576+
pub date_format: Option<String>, default = None
1577+
pub datetime_format: Option<String>, default = None
1578+
pub timestamp_format: Option<String>, default = None
1579+
pub timestamp_tz_format: Option<String>, default = None
1580+
pub time_format: Option<String>, default = None
1581+
pub null_value: Option<String>, default = None
15791582
}
15801583
}
15811584

@@ -1600,12 +1603,14 @@ impl CsvOptions {
16001603
/// Set true to indicate that the first line is a header.
16011604
/// - default to true
16021605
pub fn with_has_header(mut self, has_header: bool) -> Self {
1603-
self.has_header = has_header;
1606+
self.has_header = Some(has_header);
16041607
self
16051608
}
16061609

1607-
/// True if the first line is a header.
1608-
pub fn has_header(&self) -> bool {
1610+
/// Returns true if the first line is a header. If format options does not
1611+
/// specify whether there is a header, returns `None` (indicating that the
1612+
/// configuration should be consulted).
1613+
pub fn has_header(&self) -> Option<bool> {
16091614
self.has_header
16101615
}
16111616

datafusion/common/src/file_options/csv_writer.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -50,7 +50,7 @@ impl TryFrom<&CsvOptions> for CsvWriterOptions {
5050

5151
fn try_from(value: &CsvOptions) -> Result<Self> {
5252
let mut builder = WriterBuilder::default()
53-
.with_header(value.has_header)
53+
.with_header(value.has_header.unwrap_or(false))
5454
.with_delimiter(value.delimiter);
5555

5656
if let Some(v) = &value.date_format {

datafusion/core/src/catalog/listing_schema.rs

Lines changed: 0 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@ use crate::datasource::provider::TableProviderFactory;
2727
use crate::datasource::TableProvider;
2828
use crate::execution::context::SessionState;
2929

30-
use datafusion_common::parsers::CompressionTypeVariant;
3130
use datafusion_common::{Constraints, DFSchema, DataFusionError, TableReference};
3231
use datafusion_expr::CreateExternalTable;
3332

@@ -58,7 +57,6 @@ pub struct ListingSchemaProvider {
5857
store: Arc<dyn ObjectStore>,
5958
tables: Arc<Mutex<HashMap<String, Arc<dyn TableProvider>>>>,
6059
format: String,
61-
has_header: bool,
6260
}
6361

6462
impl ListingSchemaProvider {
@@ -77,7 +75,6 @@ impl ListingSchemaProvider {
7775
factory: Arc<dyn TableProviderFactory>,
7876
store: Arc<dyn ObjectStore>,
7977
format: String,
80-
has_header: bool,
8178
) -> Self {
8279
Self {
8380
authority,
@@ -86,7 +83,6 @@ impl ListingSchemaProvider {
8683
store,
8784
tables: Arc::new(Mutex::new(HashMap::new())),
8885
format,
89-
has_header,
9086
}
9187
}
9288

@@ -139,12 +135,9 @@ impl ListingSchemaProvider {
139135
name,
140136
location: table_url,
141137
file_type: self.format.clone(),
142-
has_header: self.has_header,
143-
delimiter: ',',
144138
table_partition_cols: vec![],
145139
if_not_exists: false,
146140
definition: None,
147-
file_compression_type: CompressionTypeVariant::UNCOMPRESSED,
148141
order_exprs: vec![],
149142
unbounded: false,
150143
options: Default::default(),

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

Lines changed: 39 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -32,8 +32,9 @@ use crate::datasource::physical_plan::{
3232
use crate::error::Result;
3333
use crate::execution::context::SessionState;
3434
use crate::physical_plan::insert::{DataSink, DataSinkExec};
35-
use crate::physical_plan::{DisplayAs, DisplayFormatType, Statistics};
36-
use crate::physical_plan::{ExecutionPlan, SendableRecordBatchStream};
35+
use crate::physical_plan::{
36+
DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, Statistics,
37+
};
3738

3839
use arrow::array::RecordBatch;
3940
use arrow::csv::WriterBuilder;
@@ -136,12 +137,13 @@ impl CsvFormat {
136137
/// Set true to indicate that the first line is a header.
137138
/// - default to true
138139
pub fn with_has_header(mut self, has_header: bool) -> Self {
139-
self.options.has_header = has_header;
140+
self.options.has_header = Some(has_header);
140141
self
141142
}
142143

143-
/// True if the first line is a header.
144-
pub fn has_header(&self) -> bool {
144+
/// Returns `Some(true)` if the first line is a header, `Some(false)` if
145+
/// it is not, and `None` if it is not specified.
146+
pub fn has_header(&self) -> Option<bool> {
145147
self.options.has_header
146148
}
147149

@@ -200,7 +202,7 @@ impl FileFormat for CsvFormat {
200202

201203
async fn infer_schema(
202204
&self,
203-
_state: &SessionState,
205+
state: &SessionState,
204206
store: &Arc<dyn ObjectStore>,
205207
objects: &[ObjectMeta],
206208
) -> Result<SchemaRef> {
@@ -211,7 +213,7 @@ impl FileFormat for CsvFormat {
211213
for object in objects {
212214
let stream = self.read_to_delimited_chunks(store, object).await;
213215
let (schema, records_read) = self
214-
.infer_schema_from_stream(records_to_read, stream)
216+
.infer_schema_from_stream(state, records_to_read, stream)
215217
.await?;
216218
records_to_read -= records_read;
217219
schemas.push(schema);
@@ -236,13 +238,17 @@ impl FileFormat for CsvFormat {
236238

237239
async fn create_physical_plan(
238240
&self,
239-
_state: &SessionState,
241+
state: &SessionState,
240242
conf: FileScanConfig,
241243
_filters: Option<&Arc<dyn PhysicalExpr>>,
242244
) -> Result<Arc<dyn ExecutionPlan>> {
243245
let exec = CsvExec::new(
244246
conf,
245-
self.options.has_header,
247+
// If format options does not specify whether there is a header,
248+
// we consult configuration options.
249+
self.options
250+
.has_header
251+
.unwrap_or(state.config_options().catalog.has_header),
246252
self.options.delimiter,
247253
self.options.quote,
248254
self.options.escape,
@@ -286,6 +292,7 @@ impl CsvFormat {
286292
/// number of lines that were read
287293
async fn infer_schema_from_stream(
288294
&self,
295+
state: &SessionState,
289296
mut records_to_read: usize,
290297
stream: impl Stream<Item = Result<Bytes>>,
291298
) -> Result<(Schema, usize)> {
@@ -298,7 +305,13 @@ impl CsvFormat {
298305

299306
while let Some(chunk) = stream.next().await.transpose()? {
300307
let format = arrow::csv::reader::Format::default()
301-
.with_header(self.options.has_header && first_chunk)
308+
.with_header(
309+
first_chunk
310+
&& self
311+
.options
312+
.has_header
313+
.unwrap_or(state.config_options().catalog.has_header),
314+
)
302315
.with_delimiter(self.options.delimiter);
303316

304317
let (Schema { fields, .. }, records_read) =
@@ -538,6 +551,7 @@ mod tests {
538551
use datafusion_common::cast::as_string_array;
539552
use datafusion_common::stats::Precision;
540553
use datafusion_common::{internal_err, GetExt};
554+
use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv};
541555
use datafusion_expr::{col, lit};
542556

543557
use chrono::DateTime;
@@ -554,7 +568,8 @@ mod tests {
554568
let task_ctx = state.task_ctx();
555569
// skip column 9 that overflows the automaticly discovered column type of i64 (u64 would work)
556570
let projection = Some(vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 10, 11, 12]);
557-
let exec = get_exec(&state, "aggregate_test_100.csv", projection, None).await?;
571+
let exec =
572+
get_exec(&state, "aggregate_test_100.csv", projection, None, true).await?;
558573
let stream = exec.execute(0, task_ctx)?;
559574

560575
let tt_batches: i32 = stream
@@ -582,7 +597,7 @@ mod tests {
582597
let task_ctx = session_ctx.task_ctx();
583598
let projection = Some(vec![0, 1, 2, 3]);
584599
let exec =
585-
get_exec(&state, "aggregate_test_100.csv", projection, Some(1)).await?;
600+
get_exec(&state, "aggregate_test_100.csv", projection, Some(1), true).await?;
586601
let batches = collect(exec, task_ctx).await?;
587602
assert_eq!(1, batches.len());
588603
assert_eq!(4, batches[0].num_columns());
@@ -597,7 +612,8 @@ mod tests {
597612
let state = session_ctx.state();
598613

599614
let projection = None;
600-
let exec = get_exec(&state, "aggregate_test_100.csv", projection, None).await?;
615+
let exec =
616+
get_exec(&state, "aggregate_test_100.csv", projection, None, true).await?;
601617

602618
let x: Vec<String> = exec
603619
.schema()
@@ -633,7 +649,8 @@ mod tests {
633649
let state = session_ctx.state();
634650
let task_ctx = session_ctx.task_ctx();
635651
let projection = Some(vec![0]);
636-
let exec = get_exec(&state, "aggregate_test_100.csv", projection, None).await?;
652+
let exec =
653+
get_exec(&state, "aggregate_test_100.csv", projection, None, true).await?;
637654

638655
let batches = collect(exec, task_ctx).await.expect("Collect batches");
639656

@@ -716,6 +733,11 @@ mod tests {
716733
async fn query_compress_data(
717734
file_compression_type: FileCompressionType,
718735
) -> Result<()> {
736+
let runtime = Arc::new(RuntimeEnv::new(RuntimeConfig::new()).unwrap());
737+
let mut cfg = SessionConfig::new();
738+
cfg.options_mut().catalog.has_header = true;
739+
let session_state = SessionState::new_with_config_rt(cfg, runtime);
740+
719741
let integration = LocalFileSystem::new_with_prefix(arrow_test_data()).unwrap();
720742

721743
let path = Path::from("csv/aggregate_test_100.csv");
@@ -757,7 +779,7 @@ mod tests {
757779
.read_to_delimited_chunks_from_stream(compressed_stream.unwrap())
758780
.await;
759781
let (schema, records_read) = compressed_csv
760-
.infer_schema_from_stream(records_to_read, decoded_stream)
782+
.infer_schema_from_stream(&session_state, records_to_read, decoded_stream)
761783
.await?;
762784

763785
assert_eq!(expected, schema);
@@ -803,9 +825,10 @@ mod tests {
803825
file_name: &str,
804826
projection: Option<Vec<usize>>,
805827
limit: Option<usize>,
828+
has_header: bool,
806829
) -> Result<Arc<dyn ExecutionPlan>> {
807830
let root = format!("{}/csv", crate::test_util::arrow_test_data());
808-
let format = CsvFormat::default();
831+
let format = CsvFormat::default().with_has_header(has_header);
809832
scan_format(state, &format, &root, file_name, projection, limit).await
810833
}
811834

0 commit comments

Comments
 (0)