You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/11/29 17:52:48 UTC

[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #4395: Add sqllogictests (v0)

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


##########
Cargo.toml:
##########
@@ -31,6 +31,7 @@ members = [
     "test-utils",
     "parquet-test-utils",
     "benchmarks",
+    "tests/sqllogictests",

Review Comment:
   I recommend moving this test into `datafusion/core/tests` so that it would then be run via
   
   ```shell
   cargo test -p datafusion --test sqllogictests
   ```
   
   I don't see any reason to put it into its own top level crate (though if others feel differently perhaps we could move the code into `datafusion/sqllogictest` to match the structure of the other crates in this repo.



##########
tests/sqllogictests/src/main.rs:
##########
@@ -0,0 +1,121 @@
+// 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 async_trait::async_trait;
+use datafusion::arrow::csv::WriterBuilder;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::prelude::SessionContext;
+use std::path::PathBuf;
+
+use sqllogictest::TestError;
+pub type Result<T> = std::result::Result<T, TestError>;
+
+mod setup;
+mod utils;
+
+const TEST_DIRECTORY: &str = "tests/sqllogictests/test_files";
+const TEST_CATEGORIES: [TestCategory; 2] =
+    [TestCategory::Aggregate, TestCategory::ArrowTypeOf];
+
+pub enum TestCategory {
+    Aggregate,
+    ArrowTypeOf,
+}
+
+impl TestCategory {
+    fn as_str(&self) -> &'static str {
+        match self {
+            TestCategory::Aggregate => "Aggregate",
+            TestCategory::ArrowTypeOf => "ArrowTypeOf",
+        }
+    }
+
+    fn test_filename(&self) -> &'static str {
+        match self {
+            TestCategory::Aggregate => "aggregate.slt",
+            TestCategory::ArrowTypeOf => "arrow_typeof.slt",
+        }
+    }
+
+    async fn register_test_tables(&self, ctx: &SessionContext) {
+        println!("[{}] Registering tables", self.as_str());
+        match self {
+            TestCategory::Aggregate => setup::register_aggregate_tables(ctx).await,
+            TestCategory::ArrowTypeOf => (),
+        }
+    }
+}
+
+pub struct DataFusion {
+    ctx: SessionContext,
+    test_category: TestCategory,
+}
+
+#[async_trait]
+impl sqllogictest::AsyncDB for DataFusion {
+    type Error = TestError;
+
+    async fn run(&mut self, sql: &str) -> Result<String> {
+        println!(
+            "[{}] Running query: \"{}\"",
+            self.test_category.as_str(),
+            sql
+        );
+        let result = run_query(&self.ctx, sql).await?;
+        Ok(result)
+    }
+}
+
+#[tokio::main]
+pub async fn main() -> Result<()> {
+    for test_category in TEST_CATEGORIES {
+        let filename = PathBuf::from(format!(
+            "{}/{}",
+            TEST_DIRECTORY,
+            test_category.test_filename()
+        ));
+        let ctx = SessionContext::new();
+        test_category.register_test_tables(&ctx).await;
+
+        let mut tester = sqllogictest::Runner::new(DataFusion { ctx, test_category });
+        // TODO: use tester.run_parallel_async()
+        tester.run_file_async(filename).await.unwrap();
+    }
+
+    Ok(())
+}
+
+fn format_batches(batches: &[RecordBatch]) -> Result<String> {
+    let mut bytes = vec![];
+    {
+        let builder = WriterBuilder::new().has_headers(false).with_delimiter(b',');

Review Comment:
   is the reason to write out CSV output so that we can reuse existing slt files?



##########
tests/sqllogictests/src/main.rs:
##########
@@ -0,0 +1,121 @@
+// 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 async_trait::async_trait;
+use datafusion::arrow::csv::WriterBuilder;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::prelude::SessionContext;
+use std::path::PathBuf;
+
+use sqllogictest::TestError;
+pub type Result<T> = std::result::Result<T, TestError>;
+
+mod setup;
+mod utils;
+
+const TEST_DIRECTORY: &str = "tests/sqllogictests/test_files";
+const TEST_CATEGORIES: [TestCategory; 2] =
+    [TestCategory::Aggregate, TestCategory::ArrowTypeOf];
+
+pub enum TestCategory {
+    Aggregate,
+    ArrowTypeOf,
+}
+
+impl TestCategory {
+    fn as_str(&self) -> &'static str {
+        match self {
+            TestCategory::Aggregate => "Aggregate",
+            TestCategory::ArrowTypeOf => "ArrowTypeOf",
+        }
+    }
+
+    fn test_filename(&self) -> &'static str {
+        match self {
+            TestCategory::Aggregate => "aggregate.slt",
+            TestCategory::ArrowTypeOf => "arrow_typeof.slt",
+        }
+    }
+
+    async fn register_test_tables(&self, ctx: &SessionContext) {
+        println!("[{}] Registering tables", self.as_str());
+        match self {
+            TestCategory::Aggregate => setup::register_aggregate_tables(ctx).await,
+            TestCategory::ArrowTypeOf => (),
+        }
+    }
+}
+
+pub struct DataFusion {
+    ctx: SessionContext,
+    test_category: TestCategory,
+}
+
+#[async_trait]
+impl sqllogictest::AsyncDB for DataFusion {
+    type Error = TestError;
+
+    async fn run(&mut self, sql: &str) -> Result<String> {
+        println!(
+            "[{}] Running query: \"{}\"",
+            self.test_category.as_str(),
+            sql
+        );
+        let result = run_query(&self.ctx, sql).await?;
+        Ok(result)
+    }
+}

Review Comment:
   ```suggestion
   
       /// Engine name of current database.
       fn engine_name(&self) -> &str {
           "DataFusion"
       }
       
       /// [`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;
       }
       
   }
   ```



##########
tests/sqllogictests/src/main.rs:
##########
@@ -0,0 +1,121 @@
+// 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 async_trait::async_trait;
+use datafusion::arrow::csv::WriterBuilder;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::prelude::SessionContext;
+use std::path::PathBuf;
+
+use sqllogictest::TestError;
+pub type Result<T> = std::result::Result<T, TestError>;
+
+mod setup;
+mod utils;
+
+const TEST_DIRECTORY: &str = "tests/sqllogictests/test_files";
+const TEST_CATEGORIES: [TestCategory; 2] =
+    [TestCategory::Aggregate, TestCategory::ArrowTypeOf];
+
+pub enum TestCategory {
+    Aggregate,
+    ArrowTypeOf,
+}
+
+impl TestCategory {
+    fn as_str(&self) -> &'static str {
+        match self {
+            TestCategory::Aggregate => "Aggregate",
+            TestCategory::ArrowTypeOf => "ArrowTypeOf",
+        }
+    }
+
+    fn test_filename(&self) -> &'static str {
+        match self {
+            TestCategory::Aggregate => "aggregate.slt",
+            TestCategory::ArrowTypeOf => "arrow_typeof.slt",
+        }
+    }
+
+    async fn register_test_tables(&self, ctx: &SessionContext) {
+        println!("[{}] Registering tables", self.as_str());
+        match self {
+            TestCategory::Aggregate => setup::register_aggregate_tables(ctx).await,
+            TestCategory::ArrowTypeOf => (),
+        }
+    }
+}
+
+pub struct DataFusion {
+    ctx: SessionContext,
+    test_category: TestCategory,
+}
+
+#[async_trait]
+impl sqllogictest::AsyncDB for DataFusion {
+    type Error = TestError;
+
+    async fn run(&mut self, sql: &str) -> Result<String> {
+        println!(
+            "[{}] Running query: \"{}\"",
+            self.test_category.as_str(),
+            sql
+        );
+        let result = run_query(&self.ctx, sql).await?;
+        Ok(result)
+    }
+}
+
+#[tokio::main]
+pub async fn main() -> Result<()> {
+    for test_category in TEST_CATEGORIES {
+        let filename = PathBuf::from(format!(
+            "{}/{}",
+            TEST_DIRECTORY,
+            test_category.test_filename()
+        ));
+        let ctx = SessionContext::new();
+        test_category.register_test_tables(&ctx).await;
+
+        let mut tester = sqllogictest::Runner::new(DataFusion { ctx, test_category });
+        // TODO: use tester.run_parallel_async()
+        tester.run_file_async(filename).await.unwrap();

Review Comment:
   ```suggestion
           tester.run_file_async(filename).await?;
   ```
   
   



##########
tests/sqllogictests/README.md:
##########
@@ -0,0 +1,45 @@
+#### Overview
+
+This is the Datafusion implementation of [sqllogictest](https://www.sqlite.org/sqllogictest/doc/trunk/about.wiki). We use [sqllogictest-rs](https://github.com/risinglightdb/sqllogictest-rs) as a parser/runner of `.slt` files in `test_files`.
+
+#### Running tests
+
+`cargo run -p datafusion-sqllogictests`
+
+#### Setup
+
+
+#### sqllogictests
+
+> :warning: **Warning**:Datafusion's sqllogictest implementation and migration is still in progress. Definitions taken from https://www.sqlite.org/sqllogictest/doc/trunk/about.wiki
+
+sqllogictest is a program originally written for SQLite to verify the correctness of SQL queries against the SQLLite engine. The program is engine-agnostic and can parse sqllogictest files (`.slt`), runs queries against an SQL engine and compare the output to the expected output.

Review Comment:
   BTW this is an amazing writeup -- thank you -- I recommend we eventually move this content into the sqllogictest repo and link to that document here
   
   



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org