1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
//   http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied.  See the License for the
// specific language governing permissions and limitations
// under the License.

//! [`CsvFormat`], Comma Separated Value (CSV) [`FileFormat`] abstractions

use std::any::Any;
use std::collections::HashSet;
use std::fmt::{self, Debug};
use std::sync::Arc;

use super::write::orchestration::stateless_multipart_put;
use super::FileFormat;
use crate::datasource::file_format::file_compression_type::FileCompressionType;
use crate::datasource::file_format::write::BatchSerializer;
use crate::datasource::physical_plan::{
    CsvExec, FileGroupDisplay, FileScanConfig, FileSinkConfig,
};
use crate::error::Result;
use crate::execution::context::SessionState;
use crate::physical_plan::insert::{DataSink, DataSinkExec};
use crate::physical_plan::{
    DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, Statistics,
};

use arrow::array::RecordBatch;
use arrow::csv::WriterBuilder;
use arrow::datatypes::SchemaRef;
use arrow::datatypes::{DataType, Field, Fields, Schema};
use datafusion_common::config::CsvOptions;
use datafusion_common::file_options::csv_writer::CsvWriterOptions;
use datafusion_common::{exec_err, not_impl_err, DataFusionError};
use datafusion_execution::TaskContext;
use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement};
use datafusion_physical_plan::metrics::MetricsSet;

use async_trait::async_trait;
use bytes::{Buf, Bytes};
use futures::stream::BoxStream;
use futures::{pin_mut, Stream, StreamExt, TryStreamExt};
use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore};

/// Character Separated Value `FileFormat` implementation.
#[derive(Debug, Default)]
pub struct CsvFormat {
    options: CsvOptions,
}

impl CsvFormat {
    /// Return a newline delimited stream from the specified file on
    /// Stream, decompressing if necessary
    /// Each returned `Bytes` has a whole number of newline delimited rows
    async fn read_to_delimited_chunks(
        &self,
        store: &Arc<dyn ObjectStore>,
        object: &ObjectMeta,
    ) -> BoxStream<'static, Result<Bytes>> {
        // stream to only read as many rows as needed into memory
        let stream = store
            .get(&object.location)
            .await
            .map_err(DataFusionError::ObjectStore);
        let stream = match stream {
            Ok(stream) => self
                .read_to_delimited_chunks_from_stream(
                    stream
                        .into_stream()
                        .map_err(DataFusionError::ObjectStore)
                        .boxed(),
                )
                .await
                .map_err(DataFusionError::from)
                .left_stream(),
            Err(e) => {
                futures::stream::once(futures::future::ready(Err(e))).right_stream()
            }
        };
        stream.boxed()
    }

    async fn read_to_delimited_chunks_from_stream(
        &self,
        stream: BoxStream<'static, Result<Bytes>>,
    ) -> BoxStream<'static, Result<Bytes>> {
        let file_compression_type: FileCompressionType = self.options.compression.into();
        let decoder = file_compression_type.convert_stream(stream);
        let steam = match decoder {
            Ok(decoded_stream) => {
                newline_delimited_stream(decoded_stream.map_err(|e| match e {
                    DataFusionError::ObjectStore(e) => e,
                    err => object_store::Error::Generic {
                        store: "read to delimited chunks failed",
                        source: Box::new(err),
                    },
                }))
                .map_err(DataFusionError::from)
                .left_stream()
            }
            Err(e) => {
                futures::stream::once(futures::future::ready(Err(e))).right_stream()
            }
        };
        steam.boxed()
    }

    /// Set the csv options
    pub fn with_options(mut self, options: CsvOptions) -> Self {
        self.options = options;
        self
    }

    /// Retrieve the csv options
    pub fn options(&self) -> &CsvOptions {
        &self.options
    }

    /// Set a limit in terms of records to scan to infer the schema
    /// - default to `DEFAULT_SCHEMA_INFER_MAX_RECORD`
    pub fn with_schema_infer_max_rec(mut self, max_rec: usize) -> Self {
        self.options.schema_infer_max_rec = max_rec;
        self
    }

    /// Set true to indicate that the first line is a header.
    /// - default to true
    pub fn with_has_header(mut self, has_header: bool) -> Self {
        self.options.has_header = Some(has_header);
        self
    }

    /// Returns `Some(true)` if the first line is a header, `Some(false)` if
    /// it is not, and `None` if it is not specified.
    pub fn has_header(&self) -> Option<bool> {
        self.options.has_header
    }

    /// The character separating values within a row.
    /// - default to ','
    pub fn with_delimiter(mut self, delimiter: u8) -> Self {
        self.options.delimiter = delimiter;
        self
    }

    /// The quote character in a row.
    /// - default to '"'
    pub fn with_quote(mut self, quote: u8) -> Self {
        self.options.quote = quote;
        self
    }

    /// The escape character in a row.
    /// - default is None
    pub fn with_escape(mut self, escape: Option<u8>) -> Self {
        self.options.escape = escape;
        self
    }

    /// Set a `FileCompressionType` of CSV
    /// - defaults to `FileCompressionType::UNCOMPRESSED`
    pub fn with_file_compression_type(
        mut self,
        file_compression_type: FileCompressionType,
    ) -> Self {
        self.options.compression = file_compression_type.into();
        self
    }

    /// The delimiter character.
    pub fn delimiter(&self) -> u8 {
        self.options.delimiter
    }

    /// The quote character.
    pub fn quote(&self) -> u8 {
        self.options.quote
    }

    /// The escape character.
    pub fn escape(&self) -> Option<u8> {
        self.options.escape
    }
}

#[async_trait]
impl FileFormat for CsvFormat {
    fn as_any(&self) -> &dyn Any {
        self
    }

    async fn infer_schema(
        &self,
        state: &SessionState,
        store: &Arc<dyn ObjectStore>,
        objects: &[ObjectMeta],
    ) -> Result<SchemaRef> {
        let mut schemas = vec![];

        let mut records_to_read = self.options.schema_infer_max_rec;

        for object in objects {
            let stream = self.read_to_delimited_chunks(store, object).await;
            let (schema, records_read) = self
                .infer_schema_from_stream(state, records_to_read, stream)
                .await?;
            records_to_read -= records_read;
            schemas.push(schema);
            if records_to_read == 0 {
                break;
            }
        }

        let merged_schema = Schema::try_merge(schemas)?;
        Ok(Arc::new(merged_schema))
    }

    async fn infer_stats(
        &self,
        _state: &SessionState,
        _store: &Arc<dyn ObjectStore>,
        table_schema: SchemaRef,
        _object: &ObjectMeta,
    ) -> Result<Statistics> {
        Ok(Statistics::new_unknown(&table_schema))
    }

    async fn create_physical_plan(
        &self,
        state: &SessionState,
        conf: FileScanConfig,
        _filters: Option<&Arc<dyn PhysicalExpr>>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        let exec = CsvExec::new(
            conf,
            // If format options does not specify whether there is a header,
            // we consult configuration options.
            self.options
                .has_header
                .unwrap_or(state.config_options().catalog.has_header),
            self.options.delimiter,
            self.options.quote,
            self.options.escape,
            self.options.compression.into(),
        );
        Ok(Arc::new(exec))
    }

    async fn create_writer_physical_plan(
        &self,
        input: Arc<dyn ExecutionPlan>,
        _state: &SessionState,
        conf: FileSinkConfig,
        order_requirements: Option<Vec<PhysicalSortRequirement>>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        if conf.overwrite {
            return not_impl_err!("Overwrites are not implemented yet for CSV");
        }

        let writer_options = CsvWriterOptions::try_from(&self.options)?;

        let sink_schema = conf.output_schema().clone();
        let sink = Arc::new(CsvSink::new(conf, writer_options));

        Ok(Arc::new(DataSinkExec::new(
            input,
            sink,
            sink_schema,
            order_requirements,
        )) as _)
    }
}

impl CsvFormat {
    /// Return the inferred schema reading up to records_to_read from a
    /// stream of delimited chunks returning the inferred schema and the
    /// number of lines that were read
    async fn infer_schema_from_stream(
        &self,
        state: &SessionState,
        mut records_to_read: usize,
        stream: impl Stream<Item = Result<Bytes>>,
    ) -> Result<(Schema, usize)> {
        let mut total_records_read = 0;
        let mut column_names = vec![];
        let mut column_type_possibilities = vec![];
        let mut first_chunk = true;

        pin_mut!(stream);

        while let Some(chunk) = stream.next().await.transpose()? {
            let format = arrow::csv::reader::Format::default()
                .with_header(
                    first_chunk
                        && self
                            .options
                            .has_header
                            .unwrap_or(state.config_options().catalog.has_header),
                )
                .with_delimiter(self.options.delimiter);

            let (Schema { fields, .. }, records_read) =
                format.infer_schema(chunk.reader(), Some(records_to_read))?;

            records_to_read -= records_read;
            total_records_read += records_read;

            if first_chunk {
                // set up initial structures for recording inferred schema across chunks
                (column_names, column_type_possibilities) = fields
                    .into_iter()
                    .map(|field| {
                        let mut possibilities = HashSet::new();
                        if records_read > 0 {
                            // at least 1 data row read, record the inferred datatype
                            possibilities.insert(field.data_type().clone());
                        }
                        (field.name().clone(), possibilities)
                    })
                    .unzip();
                first_chunk = false;
            } else {
                if fields.len() != column_type_possibilities.len() {
                    return exec_err!(
                            "Encountered unequal lengths between records on CSV file whilst inferring schema. \
                             Expected {} records, found {} records",
                            column_type_possibilities.len(),
                            fields.len()
                        );
                }

                column_type_possibilities.iter_mut().zip(&fields).for_each(
                    |(possibilities, field)| {
                        possibilities.insert(field.data_type().clone());
                    },
                );
            }

            if records_to_read == 0 {
                break;
            }
        }

        let schema = build_schema_helper(column_names, &column_type_possibilities);
        Ok((schema, total_records_read))
    }
}

fn build_schema_helper(names: Vec<String>, types: &[HashSet<DataType>]) -> Schema {
    let fields = names
        .into_iter()
        .zip(types)
        .map(|(field_name, data_type_possibilities)| {
            // ripped from arrow::csv::reader::infer_reader_schema_with_csv_options
            // determine data type based on possible types
            // if there are incompatible types, use DataType::Utf8
            match data_type_possibilities.len() {
                1 => Field::new(
                    field_name,
                    data_type_possibilities.iter().next().unwrap().clone(),
                    true,
                ),
                2 => {
                    if data_type_possibilities.contains(&DataType::Int64)
                        && data_type_possibilities.contains(&DataType::Float64)
                    {
                        // we have an integer and double, fall down to double
                        Field::new(field_name, DataType::Float64, true)
                    } else {
                        // default to Utf8 for conflicting datatypes (e.g bool and int)
                        Field::new(field_name, DataType::Utf8, true)
                    }
                }
                _ => Field::new(field_name, DataType::Utf8, true),
            }
        })
        .collect::<Fields>();
    Schema::new(fields)
}

impl Default for CsvSerializer {
    fn default() -> Self {
        Self::new()
    }
}

/// Define a struct for serializing CSV records to a stream
pub struct CsvSerializer {
    // CSV writer builder
    builder: WriterBuilder,
    // Flag to indicate whether there will be a header
    header: bool,
}

impl CsvSerializer {
    /// Constructor for the CsvSerializer object
    pub fn new() -> Self {
        Self {
            builder: WriterBuilder::new(),
            header: true,
        }
    }

    /// Method for setting the CSV writer builder
    pub fn with_builder(mut self, builder: WriterBuilder) -> Self {
        self.builder = builder;
        self
    }

    /// Method for setting the CSV writer header status
    pub fn with_header(mut self, header: bool) -> Self {
        self.header = header;
        self
    }
}

impl BatchSerializer for CsvSerializer {
    fn serialize(&self, batch: RecordBatch, initial: bool) -> Result<Bytes> {
        let mut buffer = Vec::with_capacity(4096);
        let builder = self.builder.clone();
        let header = self.header && initial;
        let mut writer = builder.with_header(header).build(&mut buffer);
        writer.write(&batch)?;
        drop(writer);
        Ok(Bytes::from(buffer))
    }
}

/// Implements [`DataSink`] for writing to a CSV file.
pub struct CsvSink {
    /// Config options for writing data
    config: FileSinkConfig,
    writer_options: CsvWriterOptions,
}

impl Debug for CsvSink {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        f.debug_struct("CsvSink").finish()
    }
}

impl DisplayAs for CsvSink {
    fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        match t {
            DisplayFormatType::Default | DisplayFormatType::Verbose => {
                write!(f, "CsvSink(file_groups=",)?;
                FileGroupDisplay(&self.config.file_groups).fmt_as(t, f)?;
                write!(f, ")")
            }
        }
    }
}

impl CsvSink {
    /// Create from config.
    pub fn new(config: FileSinkConfig, writer_options: CsvWriterOptions) -> Self {
        Self {
            config,
            writer_options,
        }
    }

    /// Retrieve the inner [`FileSinkConfig`].
    pub fn config(&self) -> &FileSinkConfig {
        &self.config
    }

    async fn multipartput_all(
        &self,
        data: SendableRecordBatchStream,
        context: &Arc<TaskContext>,
    ) -> Result<u64> {
        let builder = &self.writer_options.writer_options;

        let builder_clone = builder.clone();
        let options_clone = self.writer_options.clone();
        let get_serializer = move || {
            Arc::new(
                CsvSerializer::new()
                    .with_builder(builder_clone.clone())
                    .with_header(options_clone.writer_options.header()),
            ) as _
        };

        stateless_multipart_put(
            data,
            context,
            "csv".into(),
            Box::new(get_serializer),
            &self.config,
            self.writer_options.compression.into(),
        )
        .await
    }

    /// Retrieve the writer options
    pub fn writer_options(&self) -> &CsvWriterOptions {
        &self.writer_options
    }
}

#[async_trait]
impl DataSink for CsvSink {
    fn as_any(&self) -> &dyn Any {
        self
    }

    fn metrics(&self) -> Option<MetricsSet> {
        None
    }

    async fn write_all(
        &self,
        data: SendableRecordBatchStream,
        context: &Arc<TaskContext>,
    ) -> Result<u64> {
        let total_count = self.multipartput_all(data, context).await?;
        Ok(total_count)
    }
}

#[cfg(test)]
mod tests {
    use super::super::test_util::scan_format;
    use super::*;
    use crate::arrow::util::pretty;
    use crate::assert_batches_eq;
    use crate::datasource::file_format::file_compression_type::FileCompressionType;
    use crate::datasource::file_format::test_util::VariableStream;
    use crate::datasource::listing::ListingOptions;
    use crate::physical_plan::collect;
    use crate::prelude::{CsvReadOptions, SessionConfig, SessionContext};
    use crate::test_util::arrow_test_data;

    use arrow::compute::concat_batches;
    use datafusion_common::cast::as_string_array;
    use datafusion_common::internal_err;
    use datafusion_common::stats::Precision;
    use datafusion_common::{FileType, GetExt};
    use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv};
    use datafusion_expr::{col, lit};

    use chrono::DateTime;
    use object_store::local::LocalFileSystem;
    use object_store::path::Path;
    use regex::Regex;
    use rstest::*;

    #[tokio::test]
    async fn read_small_batches() -> Result<()> {
        let config = SessionConfig::new().with_batch_size(2);
        let session_ctx = SessionContext::new_with_config(config);
        let state = session_ctx.state();
        let task_ctx = state.task_ctx();
        // skip column 9 that overflows the automaticly discovered column type of i64 (u64 would work)
        let projection = Some(vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 10, 11, 12]);
        let exec =
            get_exec(&state, "aggregate_test_100.csv", projection, None, true).await?;
        let stream = exec.execute(0, task_ctx)?;

        let tt_batches: i32 = stream
            .map(|batch| {
                let batch = batch.unwrap();
                assert_eq!(12, batch.num_columns());
                assert_eq!(2, batch.num_rows());
            })
            .fold(0, |acc, _| async move { acc + 1i32 })
            .await;

        assert_eq!(tt_batches, 50 /* 100/2 */);

        // test metadata
        assert_eq!(exec.statistics()?.num_rows, Precision::Absent);
        assert_eq!(exec.statistics()?.total_byte_size, Precision::Absent);

        Ok(())
    }

    #[tokio::test]
    async fn read_limit() -> Result<()> {
        let session_ctx = SessionContext::new();
        let state = session_ctx.state();
        let task_ctx = session_ctx.task_ctx();
        let projection = Some(vec![0, 1, 2, 3]);
        let exec =
            get_exec(&state, "aggregate_test_100.csv", projection, Some(1), true).await?;
        let batches = collect(exec, task_ctx).await?;
        assert_eq!(1, batches.len());
        assert_eq!(4, batches[0].num_columns());
        assert_eq!(1, batches[0].num_rows());

        Ok(())
    }

    #[tokio::test]
    async fn infer_schema() -> Result<()> {
        let session_ctx = SessionContext::new();
        let state = session_ctx.state();

        let projection = None;
        let exec =
            get_exec(&state, "aggregate_test_100.csv", projection, None, true).await?;

        let x: Vec<String> = exec
            .schema()
            .fields()
            .iter()
            .map(|f| format!("{}: {:?}", f.name(), f.data_type()))
            .collect();
        assert_eq!(
            vec![
                "c1: Utf8",
                "c2: Int64",
                "c3: Int64",
                "c4: Int64",
                "c5: Int64",
                "c6: Int64",
                "c7: Int64",
                "c8: Int64",
                "c9: Int64",
                "c10: Int64",
                "c11: Float64",
                "c12: Float64",
                "c13: Utf8"
            ],
            x
        );

        Ok(())
    }

    #[tokio::test]
    async fn read_char_column() -> Result<()> {
        let session_ctx = SessionContext::new();
        let state = session_ctx.state();
        let task_ctx = session_ctx.task_ctx();
        let projection = Some(vec![0]);
        let exec =
            get_exec(&state, "aggregate_test_100.csv", projection, None, true).await?;

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

        assert_eq!(1, batches.len());
        assert_eq!(1, batches[0].num_columns());
        assert_eq!(100, batches[0].num_rows());

        let array = as_string_array(batches[0].column(0))?;
        let mut values: Vec<&str> = vec![];
        for i in 0..5 {
            values.push(array.value(i));
        }

        assert_eq!(vec!["c", "d", "b", "a", "b"], values);

        Ok(())
    }

    #[tokio::test]
    async fn test_infer_schema_stream() -> Result<()> {
        let session_ctx = SessionContext::new();
        let state = session_ctx.state();
        let variable_object_store =
            Arc::new(VariableStream::new(Bytes::from("1,2,3,4,5\n"), 200));
        let object_meta = ObjectMeta {
            location: Path::parse("/")?,
            last_modified: DateTime::default(),
            size: usize::MAX,
            e_tag: None,
            version: None,
        };

        let num_rows_to_read = 100;
        let csv_format = CsvFormat::default()
            .with_has_header(false)
            .with_schema_infer_max_rec(num_rows_to_read);
        let inferred_schema = csv_format
            .infer_schema(
                &state,
                &(variable_object_store.clone() as Arc<dyn ObjectStore>),
                &[object_meta],
            )
            .await?;

        let actual_fields: Vec<_> = inferred_schema
            .fields()
            .iter()
            .map(|f| format!("{}: {:?}", f.name(), f.data_type()))
            .collect();
        assert_eq!(
            vec![
                "column_1: Int64",
                "column_2: Int64",
                "column_3: Int64",
                "column_4: Int64",
                "column_5: Int64"
            ],
            actual_fields
        );
        // ensuring on csv infer that it won't try to read entire file
        // should only read as many rows as was configured in the CsvFormat
        assert_eq!(
            num_rows_to_read,
            variable_object_store.get_iterations_detected()
        );

        Ok(())
    }

    #[rstest(
        file_compression_type,
        case(FileCompressionType::UNCOMPRESSED),
        case(FileCompressionType::GZIP),
        case(FileCompressionType::BZIP2),
        case(FileCompressionType::XZ),
        case(FileCompressionType::ZSTD)
    )]
    #[cfg(feature = "compression")]
    #[tokio::test]
    async fn query_compress_data(
        file_compression_type: FileCompressionType,
    ) -> Result<()> {
        let runtime = Arc::new(RuntimeEnv::new(RuntimeConfig::new()).unwrap());
        let mut cfg = SessionConfig::new();
        cfg.options_mut().catalog.has_header = true;
        let session_state = SessionState::new_with_config_rt(cfg, runtime);
        let integration = LocalFileSystem::new_with_prefix(arrow_test_data()).unwrap();
        let path = Path::from("csv/aggregate_test_100.csv");
        let csv = CsvFormat::default().with_has_header(true);
        let records_to_read = csv.options().schema_infer_max_rec;
        let store = Arc::new(integration) as Arc<dyn ObjectStore>;
        let original_stream = store.get(&path).await?;

        //convert original_stream to compressed_stream for next step
        let compressed_stream =
            file_compression_type.to_owned().convert_to_compress_stream(
                original_stream
                    .into_stream()
                    .map_err(DataFusionError::from)
                    .boxed(),
            );

        //prepare expected schema for assert_eq
        let expected = Schema::new(vec![
            Field::new("c1", DataType::Utf8, true),
            Field::new("c2", DataType::Int64, true),
            Field::new("c3", DataType::Int64, true),
            Field::new("c4", DataType::Int64, true),
            Field::new("c5", DataType::Int64, true),
            Field::new("c6", DataType::Int64, true),
            Field::new("c7", DataType::Int64, true),
            Field::new("c8", DataType::Int64, true),
            Field::new("c9", DataType::Int64, true),
            Field::new("c10", DataType::Int64, true),
            Field::new("c11", DataType::Float64, true),
            Field::new("c12", DataType::Float64, true),
            Field::new("c13", DataType::Utf8, true),
        ]);

        let compressed_csv = csv.with_file_compression_type(file_compression_type);

        //convert compressed_stream to decoded_stream
        let decoded_stream = compressed_csv
            .read_to_delimited_chunks_from_stream(compressed_stream.unwrap())
            .await;
        let (schema, records_read) = compressed_csv
            .infer_schema_from_stream(&session_state, records_to_read, decoded_stream)
            .await?;

        assert_eq!(expected, schema);
        assert_eq!(100, records_read);
        Ok(())
    }

    #[cfg(feature = "compression")]
    #[tokio::test]
    async fn query_compress_csv() -> Result<()> {
        let ctx = SessionContext::new();

        let csv_options = CsvReadOptions::default()
            .has_header(true)
            .file_compression_type(FileCompressionType::GZIP)
            .file_extension("csv.gz");
        let df = ctx
            .read_csv(
                &format!("{}/csv/aggregate_test_100.csv.gz", arrow_test_data()),
                csv_options,
            )
            .await?;

        let record_batch = df
            .filter(col("c1").eq(lit("a")).and(col("c2").gt(lit("4"))))?
            .select_columns(&["c2", "c3"])?
            .collect()
            .await?;
        #[rustfmt::skip]
            let expected = ["+----+------+",
            "| c2 | c3   |",
            "+----+------+",
            "| 5  | 36   |",
            "| 5  | -31  |",
            "| 5  | -101 |",
            "+----+------+"];
        assert_batches_eq!(expected, &record_batch);
        Ok(())
    }

    async fn get_exec(
        state: &SessionState,
        file_name: &str,
        projection: Option<Vec<usize>>,
        limit: Option<usize>,
        has_header: bool,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        let root = format!("{}/csv", crate::test_util::arrow_test_data());
        let format = CsvFormat::default().with_has_header(has_header);
        scan_format(state, &format, &root, file_name, projection, limit).await
    }

    #[tokio::test]
    async fn test_csv_serializer() -> Result<()> {
        let ctx = SessionContext::new();
        let df = ctx
            .read_csv(
                &format!("{}/csv/aggregate_test_100.csv", arrow_test_data()),
                CsvReadOptions::default().has_header(true),
            )
            .await?;
        let batches = df
            .select_columns(&["c2", "c3"])?
            .limit(0, Some(10))?
            .collect()
            .await?;
        let batch = concat_batches(&batches[0].schema(), &batches)?;
        let serializer = CsvSerializer::new();
        let bytes = serializer.serialize(batch, true)?;
        assert_eq!(
            "c2,c3\n2,1\n5,-40\n1,29\n1,-85\n5,-82\n4,-111\n3,104\n3,13\n1,38\n4,-38\n",
            String::from_utf8(bytes.into()).unwrap()
        );
        Ok(())
    }

    #[tokio::test]
    async fn test_csv_serializer_no_header() -> Result<()> {
        let ctx = SessionContext::new();
        let df = ctx
            .read_csv(
                &format!("{}/csv/aggregate_test_100.csv", arrow_test_data()),
                CsvReadOptions::default().has_header(true),
            )
            .await?;
        let batches = df
            .select_columns(&["c2", "c3"])?
            .limit(0, Some(10))?
            .collect()
            .await?;
        let batch = concat_batches(&batches[0].schema(), &batches)?;
        let serializer = CsvSerializer::new().with_header(false);
        let bytes = serializer.serialize(batch, true)?;
        assert_eq!(
            "2,1\n5,-40\n1,29\n1,-85\n5,-82\n4,-111\n3,104\n3,13\n1,38\n4,-38\n",
            String::from_utf8(bytes.into()).unwrap()
        );
        Ok(())
    }

    /// Explain the `sql` query under `ctx` to make sure the underlying csv scan is parallelized
    /// e.g. "CsvExec: file_groups={2 groups:" in plan means 2 CsvExec runs concurrently
    async fn count_query_csv_partitions(
        ctx: &SessionContext,
        sql: &str,
    ) -> Result<usize> {
        let df = ctx.sql(&format!("EXPLAIN {sql}")).await?;
        let result = df.collect().await?;
        let plan = format!("{}", &pretty::pretty_format_batches(&result)?);

        let re = Regex::new(r"CsvExec: file_groups=\{(\d+) group").unwrap();

        if let Some(captures) = re.captures(&plan) {
            if let Some(match_) = captures.get(1) {
                let n_partitions = match_.as_str().parse::<usize>().unwrap();
                return Ok(n_partitions);
            }
        }

        internal_err!("query contains no CsvExec")
    }

    #[rstest(n_partitions, case(1), case(2), case(3), case(4))]
    #[tokio::test]
    async fn test_csv_parallel_basic(n_partitions: usize) -> Result<()> {
        let config = SessionConfig::new()
            .with_repartition_file_scans(true)
            .with_repartition_file_min_size(0)
            .with_target_partitions(n_partitions);
        let ctx = SessionContext::new_with_config(config);
        let testdata = arrow_test_data();
        ctx.register_csv(
            "aggr",
            &format!("{testdata}/csv/aggregate_test_100.csv"),
            CsvReadOptions::new().has_header(true),
        )
        .await?;

        let query = "select sum(c2) from aggr;";
        let query_result = ctx.sql(query).await?.collect().await?;
        let actual_partitions = count_query_csv_partitions(&ctx, query).await?;

        #[rustfmt::skip]
        let expected = ["+--------------+",
            "| SUM(aggr.c2) |",
            "+--------------+",
            "| 285          |",
            "+--------------+"];
        assert_batches_eq!(expected, &query_result);
        assert_eq!(n_partitions, actual_partitions);

        Ok(())
    }

    #[rstest(n_partitions, case(1), case(2), case(3), case(4))]
    #[cfg(feature = "compression")]
    #[tokio::test]
    async fn test_csv_parallel_compressed(n_partitions: usize) -> Result<()> {
        let config = SessionConfig::new()
            .with_repartition_file_scans(true)
            .with_repartition_file_min_size(0)
            .with_target_partitions(n_partitions);
        let csv_options = CsvReadOptions::default()
            .has_header(true)
            .file_compression_type(FileCompressionType::GZIP)
            .file_extension("csv.gz");
        let ctx = SessionContext::new_with_config(config);
        let testdata = arrow_test_data();
        ctx.register_csv(
            "aggr",
            &format!("{testdata}/csv/aggregate_test_100.csv.gz"),
            csv_options,
        )
        .await?;

        let query = "select sum(c3) from aggr;";
        let query_result = ctx.sql(query).await?.collect().await?;
        let actual_partitions = count_query_csv_partitions(&ctx, query).await?;

        #[rustfmt::skip]
        let expected = ["+--------------+",
            "| SUM(aggr.c3) |",
            "+--------------+",
            "| 781          |",
            "+--------------+"];
        assert_batches_eq!(expected, &query_result);
        assert_eq!(1, actual_partitions); // Compressed csv won't be scanned in parallel

        Ok(())
    }

    /// Read a single empty csv file in parallel
    ///
    /// empty_0_byte.csv:
    /// (file is empty)
    #[rstest(n_partitions, case(1), case(2), case(3), case(4))]
    #[tokio::test]
    async fn test_csv_parallel_empty_file(n_partitions: usize) -> Result<()> {
        let config = SessionConfig::new()
            .with_repartition_file_scans(true)
            .with_repartition_file_min_size(0)
            .with_target_partitions(n_partitions);
        let ctx = SessionContext::new_with_config(config);
        ctx.register_csv(
            "empty",
            "tests/data/empty_0_byte.csv",
            CsvReadOptions::new().has_header(false),
        )
        .await?;

        // Require a predicate to enable repartition for the optimizer
        let query = "select * from empty where random() > 0.5;";
        let query_result = ctx.sql(query).await?.collect().await?;
        let actual_partitions = count_query_csv_partitions(&ctx, query).await?;

        #[rustfmt::skip]
        let expected = ["++",
            "++"];
        assert_batches_eq!(expected, &query_result);
        assert_eq!(1, actual_partitions); // Won't get partitioned if all files are empty

        Ok(())
    }

    /// Read a single empty csv file with header in parallel
    ///
    /// empty.csv:
    /// c1,c2,c3
    #[rstest(n_partitions, case(1), case(2), case(3))]
    #[tokio::test]
    async fn test_csv_parallel_empty_with_header(n_partitions: usize) -> Result<()> {
        let config = SessionConfig::new()
            .with_repartition_file_scans(true)
            .with_repartition_file_min_size(0)
            .with_target_partitions(n_partitions);
        let ctx = SessionContext::new_with_config(config);
        ctx.register_csv(
            "empty",
            "tests/data/empty.csv",
            CsvReadOptions::new().has_header(true),
        )
        .await?;

        // Require a predicate to enable repartition for the optimizer
        let query = "select * from empty where random() > 0.5;";
        let query_result = ctx.sql(query).await?.collect().await?;
        let actual_partitions = count_query_csv_partitions(&ctx, query).await?;

        #[rustfmt::skip]
        let expected = ["++",
            "++"];
        assert_batches_eq!(expected, &query_result);
        assert_eq!(n_partitions, actual_partitions);

        Ok(())
    }

    /// Read multiple empty csv files in parallel
    ///
    /// all_empty
    /// ├── empty0.csv
    /// ├── empty1.csv
    /// └── empty2.csv
    ///
    /// empty0.csv/empty1.csv/empty2.csv:
    /// (file is empty)
    #[rstest(n_partitions, case(1), case(2), case(3), case(4))]
    #[tokio::test]
    async fn test_csv_parallel_multiple_empty_files(n_partitions: usize) -> Result<()> {
        let config = SessionConfig::new()
            .with_repartition_file_scans(true)
            .with_repartition_file_min_size(0)
            .with_target_partitions(n_partitions);
        let ctx = SessionContext::new_with_config(config);
        let file_format = CsvFormat::default().with_has_header(false);
        let listing_options = ListingOptions::new(Arc::new(file_format))
            .with_file_extension(FileType::CSV.get_ext());
        ctx.register_listing_table(
            "empty",
            "tests/data/empty_files/all_empty/",
            listing_options,
            None,
            None,
        )
        .await
        .unwrap();

        // Require a predicate to enable repartition for the optimizer
        let query = "select * from empty where random() > 0.5;";
        let query_result = ctx.sql(query).await?.collect().await?;
        let actual_partitions = count_query_csv_partitions(&ctx, query).await?;

        #[rustfmt::skip]
        let expected = ["++",
            "++"];
        assert_batches_eq!(expected, &query_result);
        assert_eq!(1, actual_partitions); // Won't get partitioned if all files are empty

        Ok(())
    }

    /// Read multiple csv files (some are empty) in parallel
    ///
    /// some_empty
    /// ├── a_empty.csv
    /// ├── b.csv
    /// ├── c_empty.csv
    /// ├── d.csv
    /// └── e_empty.csv
    ///
    /// a_empty.csv/c_empty.csv/e_empty.csv:
    /// (file is empty)
    ///
    /// b.csv/d.csv:
    /// 1\n
    /// 1\n
    /// 1\n
    /// 1\n
    /// 1\n
    #[rstest(n_partitions, case(1), case(2), case(3), case(4))]
    #[tokio::test]
    async fn test_csv_parallel_some_file_empty(n_partitions: usize) -> Result<()> {
        let config = SessionConfig::new()
            .with_repartition_file_scans(true)
            .with_repartition_file_min_size(0)
            .with_target_partitions(n_partitions);
        let ctx = SessionContext::new_with_config(config);
        let file_format = CsvFormat::default().with_has_header(false);
        let listing_options = ListingOptions::new(Arc::new(file_format))
            .with_file_extension(FileType::CSV.get_ext());
        ctx.register_listing_table(
            "empty",
            "tests/data/empty_files/some_empty",
            listing_options,
            None,
            None,
        )
        .await
        .unwrap();

        // Require a predicate to enable repartition for the optimizer
        let query = "select sum(column_1) from empty where column_1 > 0;";
        let query_result = ctx.sql(query).await?.collect().await?;
        let actual_partitions = count_query_csv_partitions(&ctx, query).await?;

        #[rustfmt::skip]
        let expected = ["+---------------------+",
            "| SUM(empty.column_1) |",
            "+---------------------+",
            "| 10                  |",
            "+---------------------+"];
        assert_batches_eq!(expected, &query_result);
        assert_eq!(n_partitions, actual_partitions); // Won't get partitioned if all files are empty

        Ok(())
    }

    /// Parallel scan on a csv file with only 1 byte in each line
    /// Testing partition byte range land on line boundaries
    ///
    /// one_col.csv:
    /// 5\n
    /// 5\n
    /// (...10 rows total)
    #[rstest(n_partitions, case(1), case(2), case(3), case(5), case(10), case(32))]
    #[tokio::test]
    async fn test_csv_parallel_one_col(n_partitions: usize) -> Result<()> {
        let config = SessionConfig::new()
            .with_repartition_file_scans(true)
            .with_repartition_file_min_size(0)
            .with_target_partitions(n_partitions);
        let ctx = SessionContext::new_with_config(config);

        ctx.register_csv(
            "one_col",
            "tests/data/one_col.csv",
            CsvReadOptions::new().has_header(false),
        )
        .await?;

        let query = "select sum(column_1) from one_col where column_1 > 0;";
        let query_result = ctx.sql(query).await?.collect().await?;
        let actual_partitions = count_query_csv_partitions(&ctx, query).await?;

        #[rustfmt::skip]
        let expected = ["+-----------------------+",
            "| SUM(one_col.column_1) |",
            "+-----------------------+",
            "| 50                    |",
            "+-----------------------+"];
        let file_size = if cfg!(target_os = "windows") {
            30 // new line on Win is '\r\n'
        } else {
            20
        };
        // A 20-Byte file at most get partitioned into 20 chunks
        let expected_partitions = if n_partitions <= file_size {
            n_partitions
        } else {
            file_size
        };
        assert_batches_eq!(expected, &query_result);
        assert_eq!(expected_partitions, actual_partitions);

        Ok(())
    }

    /// Parallel scan on a csv file with 2 wide rows
    /// The byte range of a partition might be within some line
    ///
    /// wode_rows.csv:
    /// 1, 1, ..., 1\n (100 columns total)
    /// 2, 2, ..., 2\n
    #[rstest(n_partitions, case(1), case(2), case(10), case(16))]
    #[tokio::test]
    async fn test_csv_parallel_wide_rows(n_partitions: usize) -> Result<()> {
        let config = SessionConfig::new()
            .with_repartition_file_scans(true)
            .with_repartition_file_min_size(0)
            .with_target_partitions(n_partitions);
        let ctx = SessionContext::new_with_config(config);
        ctx.register_csv(
            "wide_rows",
            "tests/data/wide_rows.csv",
            CsvReadOptions::new().has_header(false),
        )
        .await?;

        let query = "select sum(column_1) + sum(column_33) + sum(column_50) + sum(column_77) + sum(column_100) as sum_of_5_cols from wide_rows where column_1 > 0;";
        let query_result = ctx.sql(query).await?.collect().await?;
        let actual_partitions = count_query_csv_partitions(&ctx, query).await?;

        #[rustfmt::skip]
        let expected = ["+---------------+",
            "| sum_of_5_cols |",
            "+---------------+",
            "| 15            |",
            "+---------------+"];
        assert_batches_eq!(expected, &query_result);
        assert_eq!(n_partitions, actual_partitions);

        Ok(())
    }
}