alamb commented on code in PR #11060:
URL: https://github.com/apache/datafusion/pull/11060#discussion_r1650104711


##########
datafusion/core/src/datasource/file_format/file_compression_type.rs:
##########
@@ -245,90 +250,16 @@ pub trait FileTypeExt {
     fn get_ext_with_compression(&self, c: FileCompressionType) -> 
Result<String>;
 }
 
-impl FileTypeExt for FileType {
-    fn get_ext_with_compression(&self, c: FileCompressionType) -> 
Result<String> {
-        let ext = self.get_ext();
-
-        match self {
-            FileType::JSON | FileType::CSV => Ok(format!("{}{}", ext, 
c.get_ext())),
-            FileType::AVRO | FileType::ARROW => match c.variant {
-                UNCOMPRESSED => Ok(ext),
-                _ => Err(DataFusionError::Internal(
-                    "FileCompressionType can be specified for CSV/JSON 
FileType.".into(),
-                )),
-            },
-            #[cfg(feature = "parquet")]
-            FileType::PARQUET => match c.variant {
-                UNCOMPRESSED => Ok(ext),
-                _ => Err(DataFusionError::Internal(
-                    "FileCompressionType can be specified for CSV/JSON 
FileType.".into(),
-                )),
-            },
-        }
-    }
-}
-
 #[cfg(test)]
 mod tests {
     use std::str::FromStr;
 
-    use crate::datasource::file_format::file_compression_type::{
-        FileCompressionType, FileTypeExt,
-    };
+    use 
crate::datasource::file_format::file_compression_type::FileCompressionType;
     use crate::error::DataFusionError;
 
-    use datafusion_common::file_options::file_type::FileType;
-
     use bytes::Bytes;
     use futures::StreamExt;
 
-    #[test]
-    fn get_ext_with_compression() {

Review Comment:
   Should we port these tests instead of just removing them?



##########
datafusion/common/src/config.rs:
##########
@@ -1682,28 +1690,15 @@ impl Display for FormatOptions {
     }
 }
 
-impl From<FileType> for FormatOptions {
-    fn from(value: FileType) -> Self {
-        match value {
-            FileType::ARROW => FormatOptions::ARROW,
-            FileType::AVRO => FormatOptions::AVRO,
-            #[cfg(feature = "parquet")]

Review Comment:
   it is certainly nice to avoid many of these `            #[cfg(feature = 
"parquet")]` that were sprinkled throughout the code



##########
datafusion/common/src/config.rs:
##########
@@ -1249,6 +1246,15 @@ impl TableOptions {
         clone
     }
 
+    /// Sets the file format for the table.
+    ///
+    /// # Parameters
+    ///
+    /// * `format`: The file format to use (e.g., CSV, Parquet).
+    pub fn set_file_format(&mut self, format: ConfigFileType) {

Review Comment:
   the parameter is called "format" but the struct is `ConfigFileType`
   
   Maybe we should call this `set_config_type`? Or maybe it could just pass in 
a `Arc<dyn FIleFormat>~?



##########
datafusion/common/src/config.rs:
##########
@@ -1116,6 +1116,16 @@ macro_rules! extensions_options {
     }
 }
 
+/// These file types have special built in behavior for configuration.
+/// Use TableOptions::Extensions for configuring other file types.
+#[derive(Debug, Clone)]
+pub enum ConfigFileType {

Review Comment:
   🤔  perhaps we should add some Trait to unify the handling of options for 
built in formats and custom formats 🤔 



##########
datafusion/proto/src/logical_plan/file_formats.rs:
##########
@@ -0,0 +1,399 @@
+// 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.
+
+use std::sync::Arc;
+
+use datafusion::{
+    datasource::file_format::{
+        arrow::ArrowFormatFactory, csv::CsvFormatFactory, 
json::JsonFormatFactory,
+        parquet::ParquetFormatFactory, FileFormatFactory,
+    },
+    prelude::SessionContext,
+};
+use datafusion_common::not_impl_err;
+
+use super::LogicalExtensionCodec;
+
+#[derive(Debug)]
+pub struct CsvLogicalExtensionCodec;
+
+// TODO! This is a placeholder for now and needs to be implemented for real.
+impl LogicalExtensionCodec for CsvLogicalExtensionCodec {
+    fn try_decode(
+        &self,
+        _buf: &[u8],
+        _inputs: &[datafusion_expr::LogicalPlan],
+        _ctx: &datafusion::prelude::SessionContext,
+    ) -> datafusion_common::Result<datafusion_expr::Extension> {
+        not_impl_err!("Method not implemented")
+    }
+
+    fn try_encode(
+        &self,
+        _node: &datafusion_expr::Extension,
+        _buf: &mut Vec<u8>,
+    ) -> datafusion_common::Result<()> {
+        not_impl_err!("Method not implemented")
+    }
+
+    fn try_decode_table_provider(
+        &self,
+        _buf: &[u8],
+        _schema: arrow::datatypes::SchemaRef,
+        _ctx: &datafusion::prelude::SessionContext,
+    ) -> datafusion_common::Result<
+        std::sync::Arc<dyn datafusion::datasource::TableProvider>,
+    > {
+        not_impl_err!("Method not implemented")
+    }
+
+    fn try_encode_table_provider(
+        &self,
+        _node: std::sync::Arc<dyn datafusion::datasource::TableProvider>,
+        _buf: &mut Vec<u8>,
+    ) -> datafusion_common::Result<()> {
+        not_impl_err!("Method not implemented")
+    }
+
+    fn try_decode_file_format(
+        &self,
+        __buf: &[u8],
+        __ctx: &SessionContext,
+    ) -> datafusion_common::Result<Arc<dyn FileFormatFactory>> {
+        Ok(Arc::new(CsvFormatFactory::new()))
+    }
+
+    fn try_encode_file_format(
+        &self,
+        __buf: &[u8],
+        __node: Arc<dyn FileFormatFactory>,
+    ) -> datafusion_common::Result<()> {
+        Ok(())
+    }
+
+    fn try_decode_udf(
+        &self,
+        name: &str,
+        __buf: &[u8],
+    ) -> datafusion_common::Result<Arc<datafusion_expr::ScalarUDF>> {
+        not_impl_err!("LogicalExtensionCodec is not provided for scalar 
function {name}")
+    }
+
+    fn try_encode_udf(
+        &self,
+        __node: &datafusion_expr::ScalarUDF,
+        __buf: &mut Vec<u8>,
+    ) -> datafusion_common::Result<()> {
+        Ok(())
+    }
+}
+
+#[derive(Debug)]
+pub struct JsonLogicalExtensionCodec;
+
+// TODO! This is a placeholder for now and needs to be implemented for real.

Review Comment:
   do we have to complete the TODO?
   



##########
datafusion/expr/src/logical_plan/dml.rs:
##########
@@ -35,8 +35,8 @@ pub struct CopyTo {
     pub output_url: String,
     /// Determines which, if any, columns should be used for hive-style 
partitioned writes
     pub partition_by: Vec<String>,
-    /// File format options.
-    pub format_options: FormatOptions,
+    /// File type trait
+    pub file_type: Arc<dyn FileType>,

Review Comment:
   What is the reason it can't be `FileFormatFactory` directly 🤔 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to