use std::sync::Arc;
use arrow::datatypes::{Schema, SchemaRef};
use crate::datasource::file_format::json::DEFAULT_JSON_EXTENSION;
use crate::datasource::{
file_format::{avro::AvroFormat, csv::CsvFormat},
listing::ListingOptions,
};
#[derive(Copy, Clone)]
pub struct CsvReadOptions<'a> {
pub has_header: bool,
pub delimiter: u8,
pub schema: Option<&'a Schema>,
pub schema_infer_max_records: usize,
pub file_extension: &'a str,
}
impl<'a> Default for CsvReadOptions<'a> {
fn default() -> Self {
Self::new()
}
}
impl<'a> CsvReadOptions<'a> {
pub fn new() -> Self {
Self {
has_header: true,
schema: None,
schema_infer_max_records: 1000,
delimiter: b',',
file_extension: ".csv",
}
}
pub fn has_header(mut self, has_header: bool) -> Self {
self.has_header = has_header;
self
}
pub fn delimiter(mut self, delimiter: u8) -> Self {
self.delimiter = delimiter;
self
}
pub fn file_extension(mut self, file_extension: &'a str) -> Self {
self.file_extension = file_extension;
self
}
pub fn delimiter_option(mut self, delimiter: Option<u8>) -> Self {
if let Some(d) = delimiter {
self.delimiter = d;
}
self
}
pub fn schema(mut self, schema: &'a Schema) -> Self {
self.schema = Some(schema);
self
}
pub fn schema_infer_max_records(mut self, max_records: usize) -> Self {
self.schema_infer_max_records = max_records;
self
}
pub fn to_listing_options(&self, target_partitions: usize) -> ListingOptions {
let file_format = CsvFormat::default()
.with_has_header(self.has_header)
.with_delimiter(self.delimiter)
.with_schema_infer_max_rec(Some(self.schema_infer_max_records));
ListingOptions {
format: Arc::new(file_format),
collect_stat: false,
file_extension: self.file_extension.to_owned(),
target_partitions,
table_partition_cols: vec![],
}
}
}
#[derive(Clone)]
pub struct AvroReadOptions<'a> {
pub schema: Option<SchemaRef>,
pub file_extension: &'a str,
}
impl<'a> Default for AvroReadOptions<'a> {
fn default() -> Self {
Self {
schema: None,
file_extension: ".avro",
}
}
}
impl<'a> AvroReadOptions<'a> {
pub fn to_listing_options(&self, target_partitions: usize) -> ListingOptions {
let file_format = AvroFormat::default();
ListingOptions {
format: Arc::new(file_format),
collect_stat: false,
file_extension: self.file_extension.to_owned(),
target_partitions,
table_partition_cols: vec![],
}
}
}
#[derive(Clone)]
pub struct NdJsonReadOptions<'a> {
pub schema: Option<SchemaRef>,
pub schema_infer_max_records: usize,
pub file_extension: &'a str,
}
impl<'a> Default for NdJsonReadOptions<'a> {
fn default() -> Self {
Self {
schema: None,
schema_infer_max_records: 1000,
file_extension: DEFAULT_JSON_EXTENSION,
}
}
}