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


##########
datafusion/core/tests/sqllogictests/postgres/test_files/values_list.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.
+
+# values without casting
+# postgres
+#   1 2.0 -3 2
+#   10 20.0 -30 4
+# datafusion:
+#   1 2 -3 2
+#   10 20 -30 4
+
+query IRII
+SELECT * FROM
+(VALUES (1,2.0::real,-3,1+1),(10,20.0::real,-30,2+2))

Review Comment:
   The original test doesn't have an explicit type cast. But Postgres treats 
`2.0` as a `numeric` type and prints it as `2.0`. Whereas in Datafusion it is 
`Float64` and the printed representation is `2`.
   
   In the python comparison it didn't matter, because the values were compared. 
In sqllogictest text-based representations are compared.



##########
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:
   Currently one needs to start a Postgres container before tests 
https://github.com/apache/arrow-datafusion/blob/master/.github/workflows/rust.yml#L167-L229.
 
   I introduced `testcontainers` to have an instance per `.slt` file the same 
way Datafusion does now.



##########
datafusion/core/tests/sqllogictests/src/engines/datafusion/mod.rs:
##########
@@ -0,0 +1,86 @@
+// 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::time::Duration;
+
+use sqllogictest::DBOutput;
+
+use self::error::{DFSqlLogicTestError, Result};
+use async_trait::async_trait;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::prelude::SessionContext;
+use datafusion_sql::parser::{DFParser, Statement};
+use insert::insert;
+use sqlparser::ast::Statement as SQLStatement;
+
+mod error;
+mod insert;
+mod normalize;
+
+pub struct DataFusion {

Review Comment:
   This is an existing implementation. I just moved it into a dedicated package.



##########
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 {
+    client: Arc<tokio_postgres::Client>,
+    join_handle: JoinHandle<()>,
+}
+
+pub const PG_USER: &str = "postgres";
+pub const PG_PASSWORD: &str = "postgres";
+pub const PG_DB: &str = "test";
+pub const PG_PORT: u16 = 5432;
+
+impl Postgres {
+    pub fn postgres_docker_image() -> GenericImage {
+        let postgres_test_data = match datafusion::test_util::get_data_dir(
+            "POSTGRES_TEST_DATA",
+            "tests/sqllogictests/postgres",
+        ) {
+            Ok(pb) => pb.display().to_string(),
+            Err(err) => panic!("failed to get arrow data dir: {err}"),
+        };
+        GenericImage::new("postgres", "15")
+            .with_wait_for(WaitFor::message_on_stderr(
+                "database system is ready to accept connections",
+            ))
+            .with_env_var("POSTGRES_DB", PG_DB)
+            .with_env_var("POSTGRES_USER", PG_USER)
+            .with_env_var("POSTGRES_PASSWORD", PG_PASSWORD)
+            .with_env_var(
+                "POSTGRES_INITDB_ARGS",
+                "--encoding=UTF-8 --lc-collate=C --lc-ctype=C",
+            )
+            .with_exposed_port(PG_PORT)
+            .with_volume(
+                format!(
+                    "{0}/csv/aggregate_test_100.csv",
+                    datafusion::test_util::arrow_test_data()
+                ),
+                "/opt/data/csv/aggregate_test_100.csv",
+            )
+            .with_volume(
+                format!("{0}/postgres_create_table.sql", postgres_test_data),
+                "/docker-entrypoint-initdb.d/0_create_table.sql",
+            )
+    }
+
+    pub async fn connect_with_retry(

Review Comment:
   In my implementation a docker container starts before each test. And 
sometimes first connections fail. I didn't find a solution to deterministically 
define when a container is fully ready, so I introduced a connection retry.



##########
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:
   I added a `round` here because Postgres produces a text average 
`7.8100000000000000`, but Datafusion gives `7.81`. 
   Treating such collisions explicitly is a drawback of using sqllogictest 
text-based results comparison.



##########
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:
   This implementation is a full copy of 
https://github.com/risinglightdb/sqllogictest-rs/blob/main/sqllogictest-bin/src/engines/postgres.rs
 from the `sqllogictest-bin`. It could make sense to make the 
`sqllogictest-bin` postgres client a public module 
https://github.com/risinglightdb/sqllogictest-rs/blob/main/sqllogictest-bin/src/engines/mod.rs#L1
 and use the cargo dependency.



##########
integration-tests/create_test_table_postgres.sql:
##########
@@ -2,8 +2,8 @@ CREATE TABLE IF NOT EXISTS test (
 c1 character varying NOT NULL,
 c2 integer NOT NULL,
 c3 smallint NOT NULL,
-c4 smallint NOT NULL,
-c5 integer NOT NULL,
+c4 smallint,

Review Comment:
   Making nullability the same as in the Datafusion version of this table.



##########
datafusion/core/tests/sqllogictests/src/setup.rs:
##########
@@ -111,23 +111,23 @@ fn register_median_test_tables(ctx: &SessionContext) {
     }
 }
 
-async fn register_aggregate_csv_by_sql(ctx: &SessionContext) {
+pub async fn register_aggregate_csv_by_sql(ctx: &SessionContext) {

Review Comment:
   I re-used the existing table creation function and updated the data types 
according to the sql from `integration-tests` 
https://github.com/apache/arrow-datafusion/blob/master/integration-tests/create_test_table.sql.
   
   I can, of course, introduce another function.



##########
datafusion/core/tests/sqllogictests/postgres/test_files/simple_window_partition_order_aggregation.slt:
##########
@@ -0,0 +1,37 @@
+# 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.
+
+#  before round
+#  4268716378 14 14 -13.8571428571428571 -194 118 -101         ---- postgres
+#  4268716378 14 14 -13.857142857142858  -194 118 -101         ---- datafusion
+
+#   before abs
+#   2818832252 16 16  0 -3 102 -111    ---- postgres
+#   2818832252 16 16 -0 -3 102 -111    ---- datafusion
+
+query IIIRIII
+SELECT
+  c9,
+  row_number() OVER (PARTITION BY c2 ORDER BY c9) AS row_number,
+  count(c3) OVER (PARTITION BY c2 ORDER BY c9) AS count_c3,
+  abs(round(avg(c3) OVER (PARTITION BY c2 ORDER BY c9), 0)) AS avg_c3_by_c2,

Review Comment:
   `avg` produces slightly different results. `-13.8571428571428571` for 
Postgres, and `-13.857142857142858`.
   
   Adding `round` produces an entry where Postgres has the result `0`, but 
Datafusion has `-0`.
   
   `-0` seems slightly odd to me, but it is a matter of interpreting 
compatibility with Postgres.
   Generally, adding explicit `round` is a limitation to handle float numbers 
in sqllogictest text comparison.



##########
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:
   This function is the only new code here.
   It is quite simple:
   - copies `.slt` file to a temp location
   - updates the temporary `.slt` file with Postgres results
   - runs the temporary `.slt` with predefined results file with Datafusion
   
   This way expected results were produced by Postgres. Thus running a test 
with Datafusion compares results of two engines.



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