You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by al...@apache.org on 2023/01/25 11:16:47 UTC

[arrow-datafusion] branch master updated: [sqllogictest] Read subdirectories in `test_files` (#5033)

This is an automated email from the ASF dual-hosted git repository.

alamb pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow-datafusion.git


The following commit(s) were added to refs/heads/master by this push:
     new 2aa14900e [sqllogictest] Read subdirectories in `test_files` (#5033)
2aa14900e is described below

commit 2aa14900e23d2eb97e950f1758cdbe0f8a27e55b
Author: Yevhenii Melnyk <me...@gmail.com>
AuthorDate: Wed Jan 25 12:16:41 2023 +0100

    [sqllogictest] Read subdirectories in `test_files` (#5033)
    
    * [sqllogictest] Read subdirectories in `test_files`
    
    * Update the relative path type
    
    * expect instead of unwrap
    
    * Keep only ascii characters in Postgres schema name
    
    * clippy
---
 .../sqllogictests/src/engines/datafusion/mod.rs    | 13 ++-
 .../sqllogictests/src/engines/postgres/mod.rs      | 41 ++++++----
 datafusion/core/tests/sqllogictests/src/main.rs    | 92 +++++++++++++---------
 .../{ => pg_compat}/pg_compat_simple.slt           |  0
 .../test_files/{ => pg_compat}/pg_compat_types.slt |  0
 .../test_files/{ => pg_compat}/pg_compat_union.slt |  0
 .../{ => pg_compat}/pg_compat_window.slt           |  0
 7 files changed, 90 insertions(+), 56 deletions(-)

diff --git a/datafusion/core/tests/sqllogictests/src/engines/datafusion/mod.rs b/datafusion/core/tests/sqllogictests/src/engines/datafusion/mod.rs
index cb4990a42..7da453d77 100644
--- a/datafusion/core/tests/sqllogictests/src/engines/datafusion/mod.rs
+++ b/datafusion/core/tests/sqllogictests/src/engines/datafusion/mod.rs
@@ -15,6 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
+use std::path::PathBuf;
 use std::time::Duration;
 
 use sqllogictest::DBOutput;
@@ -36,12 +37,12 @@ mod util;
 
 pub struct DataFusion {
     ctx: SessionContext,
-    file_name: String,
+    relative_path: PathBuf,
 }
 
 impl DataFusion {
-    pub fn new(ctx: SessionContext, file_name: String) -> Self {
-        Self { ctx, file_name }
+    pub fn new(ctx: SessionContext, relative_path: PathBuf) -> Self {
+        Self { ctx, relative_path }
     }
 }
 
@@ -50,7 +51,11 @@ impl sqllogictest::AsyncDB for DataFusion {
     type Error = DFSqlLogicTestError;
 
     async fn run(&mut self, sql: &str) -> Result<DBOutput> {
-        println!("[{}] Running query: \"{}\"", self.file_name, sql);
+        println!(
+            "[{}] Running query: \"{}\"",
+            self.relative_path.display(),
+            sql
+        );
         let result = run_query(&self.ctx, sql).await?;
         Ok(result)
     }
diff --git a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs b/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs
index dab5b4d38..cfefecf69 100644
--- a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs
+++ b/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs
@@ -15,6 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
+use std::path::{Path, PathBuf};
 use std::str::FromStr;
 
 use async_trait::async_trait;
@@ -47,13 +48,13 @@ pub type Result<T, E = Error> = std::result::Result<T, E>;
 pub struct Postgres {
     client: tokio_postgres::Client,
     join_handle: JoinHandle<()>,
-    /// Filename, for display purposes
-    file_name: String,
+    /// Relative test file path
+    relative_path: PathBuf,
 }
 
 impl Postgres {
-    /// Creates a runner for executing queries against an existing
-    /// posgres connection. `file_name` is used for display output
+    /// Creates a runner for executing queries against an existing postgres connection.
+    /// `relative_path` is used for display output and to create a postgres schema.
     ///
     /// The database connection details can be overridden by the
     /// `PG_URI` environment variable.
@@ -65,9 +66,7 @@ impl Postgres {
     /// ```
     ///
     /// See https://docs.rs/tokio-postgres/latest/tokio_postgres/config/struct.Config.html#url for format
-    pub async fn connect(file_name: impl Into<String>) -> Result<Self> {
-        let file_name = file_name.into();
-
+    pub async fn connect(relative_path: PathBuf) -> Result<Self> {
         let uri =
             std::env::var("PG_URI").map_or(PG_URI.to_string(), std::convert::identity);
 
@@ -89,7 +88,7 @@ impl Postgres {
             }
         });
 
-        let schema = schema_name(&file_name);
+        let schema = schema_name(&relative_path);
 
         // create a new clean schema for running the test
         debug!("Creating new empty schema '{schema}'");
@@ -108,7 +107,7 @@ impl Postgres {
         Ok(Self {
             client,
             join_handle,
-            file_name,
+            relative_path,
         })
     }
 
@@ -188,12 +187,18 @@ fn no_quotes(t: &str) -> &str {
 
 /// Given a file name like pg_compat_foo.slt
 /// return a schema name
-fn schema_name(file_name: &str) -> &str {
-    file_name
-        .split('.')
-        .next()
-        .unwrap_or("default_schema")
-        .trim_start_matches("pg_")
+fn schema_name(relative_path: &Path) -> String {
+    relative_path
+        .file_name()
+        .map(|name| {
+            name.to_string_lossy()
+                .chars()
+                .filter(|ch| ch.is_ascii_alphabetic())
+                .collect::<String>()
+                .trim_start_matches("pg_")
+                .to_string()
+        })
+        .unwrap_or_else(|| "default_schema".to_string())
 }
 
 impl Drop for Postgres {
@@ -249,7 +254,11 @@ impl sqllogictest::AsyncDB for Postgres {
     type Error = Error;
 
     async fn run(&mut self, sql: &str) -> Result<DBOutput, Self::Error> {
-        println!("[{}] Running query: \"{}\"", self.file_name, sql);
+        println!(
+            "[{}] Running query: \"{}\"",
+            self.relative_path.display(),
+            sql
+        );
 
         let lower_sql = sql.trim_start().to_ascii_lowercase();
 
diff --git a/datafusion/core/tests/sqllogictests/src/main.rs b/datafusion/core/tests/sqllogictests/src/main.rs
index 2f3febb25..4fb6cc26a 100644
--- a/datafusion/core/tests/sqllogictests/src/main.rs
+++ b/datafusion/core/tests/sqllogictests/src/main.rs
@@ -29,7 +29,7 @@ mod engines;
 mod setup;
 mod utils;
 
-const TEST_DIRECTORY: &str = "tests/sqllogictests/test_files";
+const TEST_DIRECTORY: &str = "tests/sqllogictests/test_files/";
 const PG_COMPAT_FILE_PREFIX: &str = "pg_compat_";
 
 #[tokio::main]
@@ -47,40 +47,37 @@ pub async fn main() -> Result<(), Box<dyn Error>> {
 
     let options = Options::new();
 
-    let files: Vec<_> = read_test_files(&options);
-
-    info!("Running test files {:?}", files);
-
-    for path in files {
-        let file_name = path.file_name().unwrap().to_str().unwrap().to_string();
-
+    for (path, relative_path) in read_test_files(&options) {
         if options.complete_mode {
-            run_complete_file(&path, file_name).await?;
+            run_complete_file(&path, relative_path).await?;
         } else if options.postgres_runner {
-            run_test_file_with_postgres(&path, file_name).await?;
+            run_test_file_with_postgres(&path, relative_path).await?;
         } else {
-            run_test_file(&path, file_name).await?;
+            run_test_file(&path, relative_path).await?;
         }
     }
 
     Ok(())
 }
 
-async fn run_test_file(path: &PathBuf, file_name: String) -> Result<(), Box<dyn Error>> {
-    println!("Running with DataFusion runner: {}", path.display());
-    let ctx = context_for_test_file(&file_name).await;
-    let mut runner = sqllogictest::Runner::new(DataFusion::new(ctx, file_name));
+async fn run_test_file(
+    path: &Path,
+    relative_path: PathBuf,
+) -> Result<(), Box<dyn Error>> {
+    info!("Running with DataFusion runner: {}", path.display());
+    let ctx = context_for_test_file(&relative_path).await;
+    let mut runner = sqllogictest::Runner::new(DataFusion::new(ctx, relative_path));
     runner.run_file_async(path).await?;
     Ok(())
 }
 
 async fn run_test_file_with_postgres(
-    path: &PathBuf,
-    file_name: String,
+    path: &Path,
+    relative_path: PathBuf,
 ) -> Result<(), Box<dyn Error>> {
     info!("Running with Postgres runner: {}", path.display());
 
-    let postgres_client = Postgres::connect(file_name).await?;
+    let postgres_client = Postgres::connect(relative_path).await?;
 
     sqllogictest::Runner::new(postgres_client)
         .run_file_async(path)
@@ -90,17 +87,15 @@ async fn run_test_file_with_postgres(
 }
 
 async fn run_complete_file(
-    path: &PathBuf,
-    file_name: String,
+    path: &Path,
+    relative_path: PathBuf,
 ) -> Result<(), Box<dyn Error>> {
     use sqllogictest::{default_validator, update_test_file};
 
     info!("Using complete mode to complete: {}", path.display());
 
-    let ctx = context_for_test_file(&file_name).await;
-    let mut runner = sqllogictest::Runner::new(DataFusion::new(ctx, file_name));
-
-    info!("Using complete mode to complete {}", path.display());
+    let ctx = context_for_test_file(&relative_path).await;
+    let mut runner = sqllogictest::Runner::new(DataFusion::new(ctx, relative_path));
     let col_separator = " ";
     let validator = default_validator;
     update_test_file(path, &mut runner, col_separator, validator)
@@ -110,18 +105,42 @@ async fn run_complete_file(
     Ok(())
 }
 
-fn read_test_files(options: &Options) -> Vec<PathBuf> {
-    std::fs::read_dir(TEST_DIRECTORY)
-        .unwrap()
-        .map(|path| path.unwrap().path())
-        .filter(|path| options.check_test_file(path.as_path()))
-        .filter(|path| options.check_pg_compat_file(path.as_path()))
-        .collect()
+fn read_test_files<'a>(
+    options: &'a Options,
+) -> Box<dyn Iterator<Item = (PathBuf, PathBuf)> + 'a> {
+    Box::new(
+        read_dir_recursive(TEST_DIRECTORY)
+            .map(|path| {
+                (
+                    path.clone(),
+                    PathBuf::from(
+                        path.to_string_lossy().strip_prefix(TEST_DIRECTORY).unwrap(),
+                    ),
+                )
+            })
+            .filter(|(_, relative_path)| options.check_test_file(relative_path))
+            .filter(|(path, _)| options.check_pg_compat_file(path.as_path())),
+    )
+}
+
+fn read_dir_recursive<P: AsRef<Path>>(path: P) -> Box<dyn Iterator<Item = PathBuf>> {
+    Box::new(
+        std::fs::read_dir(path)
+            .expect("Readable directory")
+            .map(|path| path.expect("Readable entry").path())
+            .flat_map(|path| {
+                if path.is_dir() {
+                    read_dir_recursive(path)
+                } else {
+                    Box::new(std::iter::once(path))
+                }
+            }),
+    )
 }
 
 /// Create a SessionContext, configured for the specific test
-async fn context_for_test_file(file_name: &str) -> SessionContext {
-    match file_name {
+async fn context_for_test_file(relative_path: &Path) -> SessionContext {
+    match relative_path.file_name().unwrap().to_str().unwrap() {
         "aggregate.slt" | "select.slt" => {
             info!("Registering aggregate tables");
             let ctx = SessionContext::new();
@@ -185,14 +204,15 @@ impl Options {
     /// To be compatible with this, treat the command line arguments as a
     /// filter and that does a substring match on each input.  returns
     /// true f this path should be run
-    fn check_test_file(&self, path: &Path) -> bool {
+    fn check_test_file(&self, relative_path: &Path) -> bool {
         if self.filters.is_empty() {
             return true;
         }
 
         // otherwise check if any filter matches
-        let path_str = path.to_string_lossy();
-        self.filters.iter().any(|filter| path_str.contains(filter))
+        self.filters
+            .iter()
+            .any(|filter| relative_path.to_string_lossy().contains(filter))
     }
 
     /// Postgres runner executes only tests in files with specific names
diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_simple.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat/pg_compat_simple.slt
similarity index 100%
rename from datafusion/core/tests/sqllogictests/test_files/pg_compat_simple.slt
rename to datafusion/core/tests/sqllogictests/test_files/pg_compat/pg_compat_simple.slt
diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_types.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat/pg_compat_types.slt
similarity index 100%
rename from datafusion/core/tests/sqllogictests/test_files/pg_compat_types.slt
rename to datafusion/core/tests/sqllogictests/test_files/pg_compat/pg_compat_types.slt
diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_union.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat/pg_compat_union.slt
similarity index 100%
rename from datafusion/core/tests/sqllogictests/test_files/pg_compat_union.slt
rename to datafusion/core/tests/sqllogictests/test_files/pg_compat/pg_compat_union.slt
diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_window.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat/pg_compat_window.slt
similarity index 100%
rename from datafusion/core/tests/sqllogictests/test_files/pg_compat_window.slt
rename to datafusion/core/tests/sqllogictests/test_files/pg_compat/pg_compat_window.slt