alamb commented on code in PR #4834:
URL: https://github.com/apache/arrow-datafusion/pull/4834#discussion_r1064023857


##########
datafusion/core/tests/sqllogictests/postgres/test_files/self_join_with_alias.slt:
##########
@@ -0,0 +1,24 @@
+# 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.
+
+query I

Review Comment:
   I realize this has one test per query to mirror the approach in the python 
test, but I think it would be perfectly fine (and probably much nicer) to put 
all the queries into the same file



##########
datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs:
##########
@@ -0,0 +1,197 @@
+// 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 std::time::Duration;
+
+use async_trait::async_trait;
+use log::info;
+use sqllogictest::{ColumnType, DBOutput};
+use testcontainers::core::WaitFor;
+use testcontainers::images::generic::GenericImage;
+use tokio::task::JoinHandle;
+
+pub struct Postgres {

Review Comment:
   I think coping it from sqllogictest-rs would be ok for now; However,  
@xxchan  has been very interested in helping to make sqllogictest easier to use 
/ refactor -- if they were interested we could propose an upstream PR to put 
this logic somewhere (perhaps in the main sqllogictest-rs crate itself or a 
sqllogictest-postgres crate 🤔 )



##########
datafusion/core/tests/sqllogictests/postgres/test_files/simple_except.slt:
##########
@@ -0,0 +1,27 @@
+# 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.
+
+query I
+SELECT * FROM (
+    SELECT c2
+    FROM aggregate_test_100_by_sql t1
+    EXCEPT
+    SELECT c2
+    FROM aggregate_test_100_by_sql t2
+    WHERE c2 IN (3, 4)
+) s
+ORDER BY c2

Review Comment:
   I think you have to include expected output here to get sqllogictest to 
verify the output
   
   ```suggestion
   query I
   SELECT * FROM (
       SELECT c2
       FROM aggregate_test_100_by_sql t1
       EXCEPT
       SELECT c2
       FROM aggregate_test_100_by_sql t2
       WHERE c2 IN (3, 4)
   ) s
   ORDER BY c2
   ---
   foo bar
   ```



##########
datafusion/core/tests/sqllogictests/postgres/test_files/simple_aggregation.slt:
##########
@@ -0,0 +1,29 @@
+# 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.
+
+#   100 100 7.8100000000000000 781 125 -117     --- postgres
+#   100 100 7.81 781 125 -117                   --- datafusion
+
+query IIRIII
+SELECT
+  count(*) AS count_all,
+  count(c3) AS count_c3,
+  round(avg(c3), 0) AS avg,

Review Comment:
   See my comment on https://github.com/apache/arrow-datafusion/issues/4462 
about potentially normalizing prior to converting to string



##########
datafusion/core/tests/sqllogictests/src/main.rs:
##########
@@ -15,112 +15,135 @@
 // specific language governing permissions and limitations
 // under the License.
 
-use async_trait::async_trait;
-use datafusion::arrow::record_batch::RecordBatch;
-use datafusion::prelude::SessionContext;
-use datafusion_sql::parser::{DFParser, Statement};
-use log::info;
-use normalize::convert_batches;
-use sqllogictest::DBOutput;
-use sqlparser::ast::Statement as SQLStatement;
-use std::path::Path;
-use std::time::Duration;
-
-use crate::error::{DFSqlLogicTestError, Result};
-use crate::insert::insert;
-
-mod error;
-mod insert;
-mod normalize;
-mod setup;
-mod utils;
+use std::error::Error;
+use std::fs::copy;
+use std::path::{Path, PathBuf};
 
-const TEST_DIRECTORY: &str = "tests/sqllogictests/test_files";
-
-pub struct DataFusion {
-    ctx: SessionContext,
-    file_name: String,
-}
+use log::info;
+use tempfile::tempdir;
+use testcontainers::clients::Cli as Docker;
 
-#[async_trait]
-impl sqllogictest::AsyncDB for DataFusion {
-    type Error = DFSqlLogicTestError;
+use datafusion::prelude::SessionContext;
 
-    async fn run(&mut self, sql: &str) -> Result<DBOutput> {
-        println!("[{}] Running query: \"{}\"", self.file_name, sql);
-        let result = run_query(&self.ctx, sql).await?;
-        Ok(result)
-    }
+use crate::engines::datafusion::DataFusion;
+use crate::engines::postgres::{Postgres, PG_DB, PG_PASSWORD, PG_PORT, PG_USER};
 
-    /// Engine name of current database.
-    fn engine_name(&self) -> &str {
-        "DataFusion"
-    }
+mod engines;
+mod setup;
+mod utils;
 
-    /// [`Runner`] calls this function to perform sleep.
-    ///
-    /// The default implementation is `std::thread::sleep`, which is 
universial to any async runtime
-    /// but would block the current thread. If you are running in tokio 
runtime, you should override
-    /// this by `tokio::time::sleep`.
-    async fn sleep(dur: Duration) {
-        tokio::time::sleep(dur).await;
-    }
-}
+const TEST_DIRECTORY: &str = "tests/sqllogictests/test_files";
+const TEST_DIRECTORY_POSTGRES: &str = 
"tests/sqllogictests/postgres/test_files";
 
 #[tokio::main]
 #[cfg(target_family = "windows")]
-pub async fn main() -> Result<()> {
+pub async fn main() -> Result<(), Box<dyn Error>> {
     println!("Skipping test on windows");
     Ok(())
 }
 
 #[tokio::main]
 #[cfg(not(target_family = "windows"))]
-pub async fn main() -> Result<()> {
+pub async fn main() -> Result<(), Box<dyn Error>> {
     // Enable logging (e.g. set RUST_LOG=debug to see debug logs)
-
-    use sqllogictest::{default_validator, update_test_file};
     env_logger::init();
 
     let options = Options::new();
 
-    // default to all files in test directory filtering based on name
-    let files: Vec<_> = std::fs::read_dir(TEST_DIRECTORY)
-        .unwrap()
-        .map(|path| path.unwrap().path())
-        .filter(|path| options.check_test_file(path.as_path()))
-        .collect();
+    let files: Vec<_> = read_test_files(&options);
 
     info!("Running test files {:?}", files);
 
     for path in files {
-        println!("Running: {}", path.display());
-
         let file_name = 
path.file_name().unwrap().to_str().unwrap().to_string();
 
-        // Create the test runner
-        let ctx = context_for_test_file(&file_name).await;
-        let mut runner = sqllogictest::Runner::new(DataFusion { ctx, file_name 
});
-
-        // run each file using its own new DB
-        //
-        // We could run these tests in parallel eventually if we wanted.
         if options.complete_mode {
-            info!("Using complete mode to complete {}", path.display());
-            let col_separator = " ";
-            let validator = default_validator;
-            update_test_file(path, runner, col_separator, validator)
-                .await
-                .map_err(|e| e.to_string())?;
+            run_complete_file(&path, file_name).await?;
+        } else if options.postgres_mode {
+            run_postgres_test_file(&path, file_name).await?;
         } else {
-            // run the test normally:
-            runner.run_file_async(path).await?;
+            run_test_file(&path, file_name).await?;
         }
     }
 
     Ok(())
 }
 
+async fn run_test_file(path: &PathBuf, file_name: String) -> Result<(), 
Box<dyn Error>> {
+    println!("Running: {}", path.display());
+    let ctx = context_for_test_file(&file_name).await;
+    let mut runner = sqllogictest::Runner::new(DataFusion::new(ctx, 
file_name));
+    runner.run_file_async(path).await?;
+    Ok(())
+}
+
+async fn run_postgres_test_file(

Review Comment:
   I think I would make more sense if the expected results were  part of the 
.slt file, as they are in other slt based systems, rather than dynamically 
generating the content. 



##########
datafusion/core/Cargo.toml:
##########
@@ -112,8 +112,9 @@ parquet-test-utils = { path = "../../parquet-test-utils" }
 rstest = "0.16.0"
 sqllogictest = "0.10.0"
 test-utils = { path = "../../test-utils" }
+tokio-postgres = "0.7.7"
 thiserror = "1.0.37"
-
+testcontainers = "0.14.0"

Review Comment:
   An alternate way that I suggest would be to only run the postgres comparison 
tests if some environment variable was set (e.g. 
   
   `DF_TEST_POSGRES_DSN` or something
   
   ANd then we can have the runners create the container and set that 
environment variable. Developers who wanted to run the postgres integration 
test could do so by setting the environment variable as well



-- 
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]

Reply via email to