From 93817050aef126e0b699cd9b5bf0c1a36a0e3962 Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Sat, 29 Jul 2023 14:56:43 -0700 Subject: [PATCH 1/9] feat: add sqllogictests crate --- Cargo.toml | 4 +- datafusion/sqllogictest/Cargo.toml | 47 ++++ .../sqllogictest/src/engines/conversion.rs | 125 +++++++++ .../src/engines/datafusion_engine/error.rs | 50 ++++ .../src/engines/datafusion_engine/mod.rs | 7 + .../engines/datafusion_engine/normalize.rs | 261 ++++++++++++++++++ .../src/engines/datafusion_engine/runner.rs | 77 ++++++ datafusion/sqllogictest/src/engines/mod.rs | 21 ++ datafusion/sqllogictest/src/engines/output.rs | 57 ++++ datafusion/sqllogictest/src/lib.rs | 20 ++ 10 files changed, 667 insertions(+), 2 deletions(-) create mode 100644 datafusion/sqllogictest/Cargo.toml create mode 100644 datafusion/sqllogictest/src/engines/conversion.rs create mode 100644 datafusion/sqllogictest/src/engines/datafusion_engine/error.rs create mode 100644 datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs create mode 100644 datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs create mode 100644 datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs create mode 100644 datafusion/sqllogictest/src/engines/mod.rs create mode 100644 datafusion/sqllogictest/src/engines/output.rs create mode 100644 datafusion/sqllogictest/src/lib.rs diff --git a/Cargo.toml b/Cargo.toml index f32478970d2ba..673625434a52f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -17,7 +17,7 @@ [workspace] exclude = ["datafusion-cli"] -members = ["datafusion/common", "datafusion/core", "datafusion/expr", "datafusion/execution", "datafusion/optimizer", "datafusion/physical-expr", "datafusion/proto", "datafusion/proto/gen", "datafusion/sql", "datafusion/substrait", "datafusion-examples", "test-utils", "benchmarks", +members = ["datafusion/common", "datafusion/core", "datafusion/expr", "datafusion/execution", "datafusion/optimizer", "datafusion/physical-expr", "datafusion/proto", "datafusion/proto/gen", "datafusion/sql", "datafusion/sqllogictest", "datafusion/substrait", "datafusion-examples", "test-utils", "benchmarks", ] resolver = "2" @@ -56,4 +56,4 @@ lto = false opt-level = 3 overflow-checks = false panic = 'unwind' -rpath = false \ No newline at end of file +rpath = false diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml new file mode 100644 index 0000000000000..24f293e8a8b42 --- /dev/null +++ b/datafusion/sqllogictest/Cargo.toml @@ -0,0 +1,47 @@ +# 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. + +[package] +authors.workspace = true +edition.workspace = true +homepage.workspace = true +license.workspace = true +name = "datafusion-sqllogictest" +readme.workspace = true +repository.workspace = true +rust-version.workspace = true +version.workspace = true + +[lib] +name = "datafusion_sqllogictest" +path = "src/lib.rs" + +[dependencies] +arrow = {workspace = true} +async-trait = "0.1.41" +bigdecimal = "0.4.1" +datafusion = {path = "../core", version = "28.0.0"} +datafusion-common = {path = "../common", version = "28.0.0"} +half = "2.2.1" +itertools = "0.11" +lazy_static = {version = "^1.4.0"} +object_store = "0.6.1" +rust_decimal = {version = "1.27.0"} +sqllogictest = "0.15.0" +sqlparser.workspace = true +thiserror = "1.0.44" +tokio = {version = "1.0"} diff --git a/datafusion/sqllogictest/src/engines/conversion.rs b/datafusion/sqllogictest/src/engines/conversion.rs new file mode 100644 index 0000000000000..757bbbcbb96e1 --- /dev/null +++ b/datafusion/sqllogictest/src/engines/conversion.rs @@ -0,0 +1,125 @@ +// 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 arrow::datatypes::{Decimal128Type, DecimalType}; +use bigdecimal::BigDecimal; +use half::f16; +use rust_decimal::prelude::*; + +/// Represents a constant for NULL string in your database. +pub const NULL_STR: &str = "NULL"; + +/// Converts a bool value into a string. +/// +/// # Arguments +/// +/// * `value` - The bool value to convert. +pub fn bool_to_str(value: bool) -> String { + if value { + "true".to_string() + } else { + "false".to_string() + } +} + +/// Converts a varchar into a string, trimming end line breaks. +/// Returns "(empty)" string for empty input. +/// +/// # Arguments +/// +/// * `value` - The varchar value to convert. +pub fn varchar_to_str(value: &str) -> String { + if value.is_empty() { + "(empty)".to_string() + } else { + value.trim_end_matches('\n').to_string() + } +} + +/// Converts a 16-bit floating-point number into a string. +/// +/// # Arguments +/// +/// * `value` - The 16-bit floating-point number to convert. +pub fn f16_to_str(value: f16) -> String { + if value.is_nan() { + "NaN".to_string() + } else if value == f16::INFINITY { + "Infinity".to_string() + } else if value == f16::NEG_INFINITY { + "-Infinity".to_string() + } else { + big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) + } +} + +/// Converts a 32-bit floating-point number into a string. +/// +/// # Arguments +/// +/// * `value` - The 32-bit floating-point number to convert. +pub fn f32_to_str(value: f32) -> String { + if value.is_nan() { + "NaN".to_string() + } else if value == f32::INFINITY { + "Infinity".to_string() + } else if value == f32::NEG_INFINITY { + "-Infinity".to_string() + } else { + big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) + } +} + +/// Converts a 64-bit floating-point number into a string. +/// +/// # Arguments +/// +/// * `value` - The 64-bit floating-point number to convert. +pub fn f64_to_str(value: f64) -> String { + if value.is_nan() { + "NaN".to_string() + } else if value == f64::INFINITY { + "Infinity".to_string() + } else if value == f64::NEG_INFINITY { + "-Infinity".to_string() + } else { + big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) + } +} + +/// Converts a 128-bit integer into a string using specified precision and scale. +/// +/// # Arguments +/// +/// * `value` - The 128-bit integer to convert. +/// * `precision` - The number of significant digits. +/// * `scale` - The number of digits to the right of the decimal point. +pub fn i128_to_str(value: i128, precision: &u8, scale: &i8) -> String { + big_decimal_to_str( + BigDecimal::from_str(&Decimal128Type::format_decimal(value, *precision, *scale)) + .unwrap(), + ) +} + +/// Converts a BigDecimal into a string, rounding the result to 12 decimal places. +/// +/// # Arguments +/// +/// * `value` - The BigDecimal value to convert. +pub fn big_decimal_to_str(value: BigDecimal) -> String { + value.round(12).normalized().to_string() +} diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/error.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/error.rs new file mode 100644 index 0000000000000..5bb40aca2ab8f --- /dev/null +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/error.rs @@ -0,0 +1,50 @@ +// 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 arrow::error::ArrowError; +use datafusion_common::DataFusionError; +use sqllogictest::TestError; +use sqlparser::parser::ParserError; +use thiserror::Error; + +pub type Result = std::result::Result; + +/// DataFusion sql-logicaltest error +#[derive(Debug, Error)] +pub enum DFSqlLogicTestError { + /// Error from sqllogictest-rs + #[error("SqlLogicTest error(from sqllogictest-rs crate): {0}")] + SqlLogicTest(#[from] TestError), + /// Error from datafusion + #[error("DataFusion error: {0}")] + DataFusion(#[from] DataFusionError), + /// Error returned when SQL is syntactically incorrect. + #[error("SQL Parser error: {0}")] + Sql(#[from] ParserError), + /// Error from arrow-rs + #[error("Arrow error: {0}")] + Arrow(#[from] ArrowError), + /// Generic error + #[error("Other Error: {0}")] + Other(String), +} + +impl From for DFSqlLogicTestError { + fn from(value: String) -> Self { + DFSqlLogicTestError::Other(value) + } +} diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs new file mode 100644 index 0000000000000..d2c96bfcb50fd --- /dev/null +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs @@ -0,0 +1,7 @@ +mod error; +mod normalize; +mod runner; + +pub use error::*; +pub use normalize::*; +pub use runner::*; diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs new file mode 100644 index 0000000000000..a7e4a78c876f8 --- /dev/null +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs @@ -0,0 +1,261 @@ +// 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 arrow::{array, array::ArrayRef, datatypes::DataType, record_batch::RecordBatch}; +use datafusion_common::DFField; +use datafusion_common::DataFusionError; +use lazy_static::lazy_static; +use std::path::PathBuf; + +use crate::engines::output::DFColumnType; + +use super::super::conversion::*; +use super::error::{DFSqlLogicTestError, Result}; + +/// Converts `batches` to a result as expected by sqllogicteset. +pub(crate) fn convert_batches(batches: Vec) -> Result>> { + if batches.is_empty() { + Ok(vec![]) + } else { + let schema = batches[0].schema(); + let mut rows = vec![]; + for batch in batches { + // Verify schema + if !schema.contains(&batch.schema()) { + return Err(DFSqlLogicTestError::DataFusion(DataFusionError::Internal( + format!( + "Schema mismatch. Previously had\n{:#?}\n\nGot:\n{:#?}", + &schema, + batch.schema() + ), + ))); + } + + let new_rows = convert_batch(batch)? + .into_iter() + .flat_map(expand_row) + .map(normalize_paths); + rows.extend(new_rows); + } + Ok(rows) + } +} + +/// special case rows that have newlines in them (like explain plans) +// +/// Transform inputs like: +/// ```text +/// [ +/// "logical_plan", +/// "Sort: d.b ASC NULLS LAST\n Projection: d.b, MAX(d.a) AS max_a", +/// ] +/// ``` +/// +/// Into one cell per line, adding lines if necessary +/// ```text +/// [ +/// "logical_plan", +/// ] +/// [ +/// "Sort: d.b ASC NULLS LAST", +/// ] +/// [ <--- newly added row +/// "|-- Projection: d.b, MAX(d.a) AS max_a", +/// ] +/// ``` +fn expand_row(mut row: Vec) -> impl Iterator> { + use itertools::Either; + use std::iter::once; + + // check last cell + if let Some(cell) = row.pop() { + let lines: Vec<_> = cell.split('\n').collect(); + + // no newlines in last cell + if lines.len() < 2 { + row.push(cell); + return Either::Left(once(row)); + } + + // form new rows with each additional line + let new_lines: Vec<_> = lines + .into_iter() + .map(|l| { + // replace any leading spaces with '-' as + // `sqllogictest` ignores whitespace differences + // + // See https://github.com/apache/arrow-datafusion/issues/6328 + let content = l.trim_start(); + let new_prefix = "-".repeat(l.len() - content.len()); + vec![format!("{new_prefix}{content}")] + }) + .collect(); + + Either::Right(once(row).chain(new_lines.into_iter())) + } else { + Either::Left(once(row)) + } +} + +/// normalize path references +/// +/// ``` +/// CsvExec: files={1 group: [[path/to/datafusion/testing/data/csv/aggregate_test_100.csv]]}, ... +/// ``` +/// +/// into: +/// +/// ``` +/// CsvExec: files={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, ... +/// ``` +fn normalize_paths(mut row: Vec) -> Vec { + row.iter_mut().for_each(|s| { + let workspace_root: &str = WORKSPACE_ROOT.as_ref(); + if s.contains(workspace_root) { + *s = s.replace(workspace_root, "WORKSPACE_ROOT"); + } + }); + row +} + +/// return the location of the datafusion checkout +fn workspace_root() -> object_store::path::Path { + // e.g. /Software/arrow-datafusion/datafusion/core + let dir = PathBuf::from(env!("CARGO_MANIFEST_DIR")); + + // e.g. /Software/arrow-datafusion/datafusion + let workspace_root = dir + .parent() + .expect("Can not find parent of datafusion/core") + // e.g. /Software/arrow-datafusion + .parent() + .expect("parent of datafusion") + .to_string_lossy(); + + let sanitized_workplace_root = if cfg!(windows) { + // Object store paths are delimited with `/`, e.g. `D:/a/arrow-datafusion/arrow-datafusion/testing/data/csv/aggregate_test_100.csv`. + // The default windows delimiter is `\`, so the workplace path is `D:\a\arrow-datafusion\arrow-datafusion`. + workspace_root.replace(std::path::MAIN_SEPARATOR, object_store::path::DELIMITER) + } else { + workspace_root.to_string() + }; + + object_store::path::Path::parse(sanitized_workplace_root).unwrap() +} + +// holds the root directory +lazy_static! { + static ref WORKSPACE_ROOT: object_store::path::Path = workspace_root(); +} + +/// Convert a single batch to a `Vec>` for comparison +fn convert_batch(batch: RecordBatch) -> Result>> { + (0..batch.num_rows()) + .map(|row| { + batch + .columns() + .iter() + .map(|col| cell_to_string(col, row)) + .collect::>>() + }) + .collect() +} + +macro_rules! get_row_value { + ($array_type:ty, $column: ident, $row: ident) => {{ + let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + + array.value($row) + }}; +} + +/// Normalizes the content of a single cell in RecordBatch prior to printing. +/// +/// This is to make the output comparable to the semi-standard .slt format +/// +/// Normalizations applied to [NULL Values and empty strings] +/// +/// [NULL Values and empty strings]: https://duckdb.org/dev/sqllogictest/result_verification#null-values-and-empty-strings +/// +/// Floating numbers are rounded to have a consistent representation with the Postgres runner. +/// +pub fn cell_to_string(col: &ArrayRef, row: usize) -> Result { + if !col.is_valid(row) { + // represent any null value with the string "NULL" + Ok(NULL_STR.to_string()) + } else { + match col.data_type() { + DataType::Boolean => { + Ok(bool_to_str(get_row_value!(array::BooleanArray, col, row))) + } + DataType::Float16 => { + Ok(f16_to_str(get_row_value!(array::Float16Array, col, row))) + } + DataType::Float32 => { + Ok(f32_to_str(get_row_value!(array::Float32Array, col, row))) + } + DataType::Float64 => { + Ok(f64_to_str(get_row_value!(array::Float64Array, col, row))) + } + DataType::Decimal128(precision, scale) => { + let value = get_row_value!(array::Decimal128Array, col, row); + Ok(i128_to_str(value, precision, scale)) + } + DataType::LargeUtf8 => Ok(varchar_to_str(get_row_value!( + array::LargeStringArray, + col, + row + ))), + DataType::Utf8 => { + Ok(varchar_to_str(get_row_value!(array::StringArray, col, row))) + } + _ => arrow::util::display::array_value_to_string(col, row), + } + .map_err(DFSqlLogicTestError::Arrow) + } +} + +/// Converts columns to a result as expected by sqllogicteset. +pub(crate) fn convert_schema_to_types(columns: &[DFField]) -> Vec { + columns + .iter() + .map(|f| f.data_type()) + .map(|data_type| match data_type { + DataType::Boolean => DFColumnType::Boolean, + DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::UInt8 + | DataType::UInt16 + | DataType::UInt32 + | DataType::UInt64 => DFColumnType::Integer, + DataType::Float16 + | DataType::Float32 + | DataType::Float64 + | DataType::Decimal128(_, _) + | DataType::Decimal256(_, _) => DFColumnType::Float, + DataType::Utf8 | DataType::LargeUtf8 => DFColumnType::Text, + DataType::Date32 + | DataType::Date64 + | DataType::Time32(_) + | DataType::Time64(_) => DFColumnType::DateTime, + DataType::Timestamp(_, _) => DFColumnType::Timestamp, + _ => DFColumnType::Another, + }) + .collect() +} diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs new file mode 100644 index 0000000000000..62ea82900d9e0 --- /dev/null +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs @@ -0,0 +1,77 @@ +// 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 + +use std::{sync::Arc, time::Duration}; + +use arrow::record_batch::RecordBatch; +use async_trait::async_trait; +use datafusion::prelude::SessionContext; +use sqllogictest::{DBOutput, TestError}; + +use crate::engines::output::{DFColumnType, DFOutput}; + +use super::{convert_batches, convert_schema_to_types}; + +pub struct DataFusionTestRunner { + context: Arc, +} + +impl DataFusionTestRunner { + pub fn new(context: Arc) -> Self { + Self { context } + } +} + +async fn run_query( + ctx: &SessionContext, + sql: impl Into, +) -> Result { + let df = ctx.sql(sql.into().as_str()).await.unwrap(); + + let types = convert_schema_to_types(df.schema().fields().as_slice()); + let results: Vec = df.collect().await.unwrap(); + let rows = convert_batches(results).unwrap(); + + if rows.is_empty() && types.is_empty() { + Ok(DBOutput::StatementComplete(0)) + } else { + Ok(DBOutput::Rows { types, rows }) + } +} + +#[async_trait] +impl sqllogictest::AsyncDB for DataFusionTestRunner { + type Error = TestError; + type ColumnType = DFColumnType; + + async fn run(&mut self, sql: &str) -> Result { + run_query(&self.context, sql).await + } + + /// Engine name of current database. + fn engine_name(&self) -> &str { + "DataFusionTestRunner" + } + + /// [`Runner`] calls this function to perform sleep. + /// + /// The default implementation is `std::thread::sleep`, which is universal 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; + } +} diff --git a/datafusion/sqllogictest/src/engines/mod.rs b/datafusion/sqllogictest/src/engines/mod.rs new file mode 100644 index 0000000000000..7604edf4b0432 --- /dev/null +++ b/datafusion/sqllogictest/src/engines/mod.rs @@ -0,0 +1,21 @@ +// 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 + +mod conversion; +mod datafusion_engine; +mod output; + +pub use datafusion_engine::DataFusionTestRunner; diff --git a/datafusion/sqllogictest/src/engines/output.rs b/datafusion/sqllogictest/src/engines/output.rs new file mode 100644 index 0000000000000..24299856e00d5 --- /dev/null +++ b/datafusion/sqllogictest/src/engines/output.rs @@ -0,0 +1,57 @@ +// 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 sqllogictest::{ColumnType, DBOutput}; + +#[derive(Debug, PartialEq, Eq, Clone)] +pub enum DFColumnType { + Boolean, + DateTime, + Integer, + Float, + Text, + Timestamp, + Another, +} + +impl ColumnType for DFColumnType { + fn from_char(value: char) -> Option { + match value { + 'B' => Some(Self::Boolean), + 'D' => Some(Self::DateTime), + 'I' => Some(Self::Integer), + 'P' => Some(Self::Timestamp), + 'R' => Some(Self::Float), + 'T' => Some(Self::Text), + _ => Some(Self::Another), + } + } + + fn to_char(&self) -> char { + match self { + Self::Boolean => 'B', + Self::DateTime => 'D', + Self::Integer => 'I', + Self::Timestamp => 'P', + Self::Float => 'R', + Self::Text => 'T', + Self::Another => '?', + } + } +} + +pub(crate) type DFOutput = DBOutput; diff --git a/datafusion/sqllogictest/src/lib.rs b/datafusion/sqllogictest/src/lib.rs new file mode 100644 index 0000000000000..c2eedbcb3ec9c --- /dev/null +++ b/datafusion/sqllogictest/src/lib.rs @@ -0,0 +1,20 @@ +// 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. + +mod engines; + +pub use engines::DataFusionTestRunner; From 4f1507d0ae909499903a1eb9cb1767aedb2776cc Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Mon, 31 Jul 2023 15:08:21 -0700 Subject: [PATCH 2/9] feat: cleanup the PR from feedback --- datafusion/core/Cargo.toml | 1 + .../sqllogictests/src/engines/conversion.rs | 91 ------ .../src/engines/datafusion/error.rs | 50 ---- .../src/engines/datafusion/mod.rs | 86 ------ .../src/engines/datafusion/normalize.rs | 261 ------------------ .../src/engines/datafusion/util.rs | 55 ---- .../tests/sqllogictests/src/engines/mod.rs | 21 -- .../tests/sqllogictests/src/engines/output.rs | 57 ---- .../core/tests/sqllogictests/src/main.rs | 11 +- .../test_files/information_schema_columns.slt | 2 +- datafusion/sqllogictest/Cargo.toml | 10 + .../sqllogictest/src/engines/conversion.rs | 17 ++ .../src/engines/datafusion_engine/runner.rs | 41 +-- datafusion/sqllogictest/src/engines/mod.rs | 8 +- .../src/engines/postgres_engine}/mod.rs | 8 +- .../src/engines/postgres_engine}/types.rs | 0 datafusion/sqllogictest/src/lib.rs | 5 +- 17 files changed, 73 insertions(+), 651 deletions(-) delete mode 100644 datafusion/core/tests/sqllogictests/src/engines/conversion.rs delete mode 100644 datafusion/core/tests/sqllogictests/src/engines/datafusion/error.rs delete mode 100644 datafusion/core/tests/sqllogictests/src/engines/datafusion/mod.rs delete mode 100644 datafusion/core/tests/sqllogictests/src/engines/datafusion/normalize.rs delete mode 100644 datafusion/core/tests/sqllogictests/src/engines/datafusion/util.rs delete mode 100644 datafusion/core/tests/sqllogictests/src/engines/mod.rs delete mode 100644 datafusion/core/tests/sqllogictests/src/engines/output.rs rename datafusion/{core/tests/sqllogictests/src/engines/postgres => sqllogictest/src/engines/postgres_engine}/mod.rs (98%) rename datafusion/{core/tests/sqllogictests/src/engines/postgres => sqllogictest/src/engines/postgres_engine}/types.rs (100%) diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index d182f00a4f139..446a4ebc14519 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -102,6 +102,7 @@ bigdecimal = "0.4.1" criterion = { version = "0.5", features = ["async_tokio"] } csv = "1.1.6" ctor = "0.2.0" +datafusion-sqllogictest = { path = "../sqllogictest", features = ["postgres"] } doc-comment = "0.3" env_logger = "0.10" half = "2.2.1" diff --git a/datafusion/core/tests/sqllogictests/src/engines/conversion.rs b/datafusion/core/tests/sqllogictests/src/engines/conversion.rs deleted file mode 100644 index c069c2d4a48df..0000000000000 --- a/datafusion/core/tests/sqllogictests/src/engines/conversion.rs +++ /dev/null @@ -1,91 +0,0 @@ -// 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 arrow::datatypes::{Decimal128Type, DecimalType}; -use bigdecimal::BigDecimal; -use half::f16; -use rust_decimal::prelude::*; -use rust_decimal::Decimal; - -pub const NULL_STR: &str = "NULL"; - -pub fn bool_to_str(value: bool) -> String { - if value { - "true".to_string() - } else { - "false".to_string() - } -} - -pub fn varchar_to_str(value: &str) -> String { - if value.is_empty() { - "(empty)".to_string() - } else { - value.trim_end_matches('\n').to_string() - } -} - -pub fn f16_to_str(value: f16) -> String { - if value.is_nan() { - "NaN".to_string() - } else if value == f16::INFINITY { - "Infinity".to_string() - } else if value == f16::NEG_INFINITY { - "-Infinity".to_string() - } else { - big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) - } -} - -pub fn f32_to_str(value: f32) -> String { - if value.is_nan() { - "NaN".to_string() - } else if value == f32::INFINITY { - "Infinity".to_string() - } else if value == f32::NEG_INFINITY { - "-Infinity".to_string() - } else { - big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) - } -} - -pub fn f64_to_str(value: f64) -> String { - if value.is_nan() { - "NaN".to_string() - } else if value == f64::INFINITY { - "Infinity".to_string() - } else if value == f64::NEG_INFINITY { - "-Infinity".to_string() - } else { - big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) - } -} - -pub fn i128_to_str(value: i128, precision: &u8, scale: &i8) -> String { - big_decimal_to_str( - BigDecimal::from_str(&Decimal128Type::format_decimal(value, *precision, *scale)) - .unwrap(), - ) -} - -pub fn decimal_to_str(value: Decimal) -> String { - big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) -} - -pub fn big_decimal_to_str(value: BigDecimal) -> String { - value.round(12).normalized().to_string() -} diff --git a/datafusion/core/tests/sqllogictests/src/engines/datafusion/error.rs b/datafusion/core/tests/sqllogictests/src/engines/datafusion/error.rs deleted file mode 100644 index 5bb40aca2ab8f..0000000000000 --- a/datafusion/core/tests/sqllogictests/src/engines/datafusion/error.rs +++ /dev/null @@ -1,50 +0,0 @@ -// 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 arrow::error::ArrowError; -use datafusion_common::DataFusionError; -use sqllogictest::TestError; -use sqlparser::parser::ParserError; -use thiserror::Error; - -pub type Result = std::result::Result; - -/// DataFusion sql-logicaltest error -#[derive(Debug, Error)] -pub enum DFSqlLogicTestError { - /// Error from sqllogictest-rs - #[error("SqlLogicTest error(from sqllogictest-rs crate): {0}")] - SqlLogicTest(#[from] TestError), - /// Error from datafusion - #[error("DataFusion error: {0}")] - DataFusion(#[from] DataFusionError), - /// Error returned when SQL is syntactically incorrect. - #[error("SQL Parser error: {0}")] - Sql(#[from] ParserError), - /// Error from arrow-rs - #[error("Arrow error: {0}")] - Arrow(#[from] ArrowError), - /// Generic error - #[error("Other Error: {0}")] - Other(String), -} - -impl From for DFSqlLogicTestError { - fn from(value: String) -> Self { - DFSqlLogicTestError::Other(value) - } -} diff --git a/datafusion/core/tests/sqllogictests/src/engines/datafusion/mod.rs b/datafusion/core/tests/sqllogictests/src/engines/datafusion/mod.rs deleted file mode 100644 index dd30ef494d497..0000000000000 --- a/datafusion/core/tests/sqllogictests/src/engines/datafusion/mod.rs +++ /dev/null @@ -1,86 +0,0 @@ -// 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::path::PathBuf; -use std::time::Duration; - -use crate::engines::output::{DFColumnType, DFOutput}; - -use self::error::{DFSqlLogicTestError, Result}; -use async_trait::async_trait; -use datafusion::arrow::record_batch::RecordBatch; -use datafusion::prelude::SessionContext; -use log::info; -use sqllogictest::DBOutput; - -mod error; -mod normalize; -mod util; - -pub struct DataFusion { - ctx: SessionContext, - relative_path: PathBuf, -} - -impl DataFusion { - pub fn new(ctx: SessionContext, relative_path: PathBuf) -> Self { - Self { ctx, relative_path } - } -} - -#[async_trait] -impl sqllogictest::AsyncDB for DataFusion { - type Error = DFSqlLogicTestError; - type ColumnType = DFColumnType; - - async fn run(&mut self, sql: &str) -> Result { - info!( - "[{}] Running query: \"{}\"", - self.relative_path.display(), - sql - ); - run_query(&self.ctx, sql).await - } - - /// 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 universal 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; - } -} - -async fn run_query(ctx: &SessionContext, sql: impl Into) -> Result { - let df = ctx.sql(sql.into().as_str()).await?; - - let types = normalize::convert_schema_to_types(df.schema().fields()); - let results: Vec = df.collect().await?; - let rows = normalize::convert_batches(results)?; - - if rows.is_empty() && types.is_empty() { - Ok(DBOutput::StatementComplete(0)) - } else { - Ok(DBOutput::Rows { types, rows }) - } -} diff --git a/datafusion/core/tests/sqllogictests/src/engines/datafusion/normalize.rs b/datafusion/core/tests/sqllogictests/src/engines/datafusion/normalize.rs deleted file mode 100644 index 6dd4e17d7dd70..0000000000000 --- a/datafusion/core/tests/sqllogictests/src/engines/datafusion/normalize.rs +++ /dev/null @@ -1,261 +0,0 @@ -// 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 arrow::{array, array::ArrayRef, datatypes::DataType, record_batch::RecordBatch}; -use datafusion_common::DFField; -use datafusion_common::DataFusionError; -use lazy_static::lazy_static; -use std::path::PathBuf; - -use crate::engines::output::DFColumnType; - -use super::super::conversion::*; -use super::error::{DFSqlLogicTestError, Result}; - -/// Converts `batches` to a result as expected by sqllogicteset. -pub fn convert_batches(batches: Vec) -> Result>> { - if batches.is_empty() { - Ok(vec![]) - } else { - let schema = batches[0].schema(); - let mut rows = vec![]; - for batch in batches { - // Verify schema - if !schema.contains(&batch.schema()) { - return Err(DFSqlLogicTestError::DataFusion(DataFusionError::Internal( - format!( - "Schema mismatch. Previously had\n{:#?}\n\nGot:\n{:#?}", - &schema, - batch.schema() - ), - ))); - } - - let new_rows = convert_batch(batch)? - .into_iter() - .flat_map(expand_row) - .map(normalize_paths); - rows.extend(new_rows); - } - Ok(rows) - } -} - -/// special case rows that have newlines in them (like explain plans) -// -/// Transform inputs like: -/// ```text -/// [ -/// "logical_plan", -/// "Sort: d.b ASC NULLS LAST\n Projection: d.b, MAX(d.a) AS max_a", -/// ] -/// ``` -/// -/// Into one cell per line, adding lines if necessary -/// ```text -/// [ -/// "logical_plan", -/// ] -/// [ -/// "Sort: d.b ASC NULLS LAST", -/// ] -/// [ <--- newly added row -/// "|-- Projection: d.b, MAX(d.a) AS max_a", -/// ] -/// ``` -fn expand_row(mut row: Vec) -> impl Iterator> { - use itertools::Either; - use std::iter::once; - - // check last cell - if let Some(cell) = row.pop() { - let lines: Vec<_> = cell.split('\n').collect(); - - // no newlines in last cell - if lines.len() < 2 { - row.push(cell); - return Either::Left(once(row)); - } - - // form new rows with each additional line - let new_lines: Vec<_> = lines - .into_iter() - .map(|l| { - // replace any leading spaces with '-' as - // `sqllogictest` ignores whitespace differences - // - // See https://github.com/apache/arrow-datafusion/issues/6328 - let content = l.trim_start(); - let new_prefix = "-".repeat(l.len() - content.len()); - vec![format!("{new_prefix}{content}")] - }) - .collect(); - - Either::Right(once(row).chain(new_lines.into_iter())) - } else { - Either::Left(once(row)) - } -} - -/// normalize path references -/// -/// ``` -/// CsvExec: files={1 group: [[path/to/datafusion/testing/data/csv/aggregate_test_100.csv]]}, ... -/// ``` -/// -/// into: -/// -/// ``` -/// CsvExec: files={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, ... -/// ``` -fn normalize_paths(mut row: Vec) -> Vec { - row.iter_mut().for_each(|s| { - let workspace_root: &str = WORKSPACE_ROOT.as_ref(); - if s.contains(workspace_root) { - *s = s.replace(workspace_root, "WORKSPACE_ROOT"); - } - }); - row -} - -/// return the location of the datafusion checkout -fn workspace_root() -> object_store::path::Path { - // e.g. /Software/arrow-datafusion/datafusion/core - let dir = PathBuf::from(env!("CARGO_MANIFEST_DIR")); - - // e.g. /Software/arrow-datafusion/datafusion - let workspace_root = dir - .parent() - .expect("Can not find parent of datafusion/core") - // e.g. /Software/arrow-datafusion - .parent() - .expect("parent of datafusion") - .to_string_lossy(); - - let sanitized_workplace_root = if cfg!(windows) { - // Object store paths are delimited with `/`, e.g. `D:/a/arrow-datafusion/arrow-datafusion/testing/data/csv/aggregate_test_100.csv`. - // The default windows delimiter is `\`, so the workplace path is `D:\a\arrow-datafusion\arrow-datafusion`. - workspace_root.replace(std::path::MAIN_SEPARATOR, object_store::path::DELIMITER) - } else { - workspace_root.to_string() - }; - - object_store::path::Path::parse(sanitized_workplace_root).unwrap() -} - -// holds the root directory -lazy_static! { - static ref WORKSPACE_ROOT: object_store::path::Path = workspace_root(); -} - -/// Convert a single batch to a `Vec>` for comparison -fn convert_batch(batch: RecordBatch) -> Result>> { - (0..batch.num_rows()) - .map(|row| { - batch - .columns() - .iter() - .map(|col| cell_to_string(col, row)) - .collect::>>() - }) - .collect() -} - -macro_rules! get_row_value { - ($array_type:ty, $column: ident, $row: ident) => {{ - let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); - - array.value($row) - }}; -} - -/// Normalizes the content of a single cell in RecordBatch prior to printing. -/// -/// This is to make the output comparable to the semi-standard .slt format -/// -/// Normalizations applied to [NULL Values and empty strings] -/// -/// [NULL Values and empty strings]: https://duckdb.org/dev/sqllogictest/result_verification#null-values-and-empty-strings -/// -/// Floating numbers are rounded to have a consistent representation with the Postgres runner. -/// -pub fn cell_to_string(col: &ArrayRef, row: usize) -> Result { - if !col.is_valid(row) { - // represent any null value with the string "NULL" - Ok(NULL_STR.to_string()) - } else { - match col.data_type() { - DataType::Boolean => { - Ok(bool_to_str(get_row_value!(array::BooleanArray, col, row))) - } - DataType::Float16 => { - Ok(f16_to_str(get_row_value!(array::Float16Array, col, row))) - } - DataType::Float32 => { - Ok(f32_to_str(get_row_value!(array::Float32Array, col, row))) - } - DataType::Float64 => { - Ok(f64_to_str(get_row_value!(array::Float64Array, col, row))) - } - DataType::Decimal128(precision, scale) => { - let value = get_row_value!(array::Decimal128Array, col, row); - Ok(i128_to_str(value, precision, scale)) - } - DataType::LargeUtf8 => Ok(varchar_to_str(get_row_value!( - array::LargeStringArray, - col, - row - ))), - DataType::Utf8 => { - Ok(varchar_to_str(get_row_value!(array::StringArray, col, row))) - } - _ => arrow::util::display::array_value_to_string(col, row), - } - .map_err(DFSqlLogicTestError::Arrow) - } -} - -/// Converts columns to a result as expected by sqllogicteset. -pub fn convert_schema_to_types(columns: &[DFField]) -> Vec { - columns - .iter() - .map(|f| f.data_type()) - .map(|data_type| match data_type { - DataType::Boolean => DFColumnType::Boolean, - DataType::Int8 - | DataType::Int16 - | DataType::Int32 - | DataType::Int64 - | DataType::UInt8 - | DataType::UInt16 - | DataType::UInt32 - | DataType::UInt64 => DFColumnType::Integer, - DataType::Float16 - | DataType::Float32 - | DataType::Float64 - | DataType::Decimal128(_, _) - | DataType::Decimal256(_, _) => DFColumnType::Float, - DataType::Utf8 | DataType::LargeUtf8 => DFColumnType::Text, - DataType::Date32 - | DataType::Date64 - | DataType::Time32(_) - | DataType::Time64(_) => DFColumnType::DateTime, - DataType::Timestamp(_, _) => DFColumnType::Timestamp, - _ => DFColumnType::Another, - }) - .collect() -} diff --git a/datafusion/core/tests/sqllogictests/src/engines/datafusion/util.rs b/datafusion/core/tests/sqllogictests/src/engines/datafusion/util.rs deleted file mode 100644 index 424a297f24ec2..0000000000000 --- a/datafusion/core/tests/sqllogictests/src/engines/datafusion/util.rs +++ /dev/null @@ -1,55 +0,0 @@ -// 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 arrow::datatypes::DataType; -use datafusion_common::config::ConfigOptions; -use datafusion_common::TableReference; -use datafusion_expr::{AggregateUDF, ScalarUDF, TableSource, WindowUDF}; -use datafusion_sql::planner::ContextProvider; -use std::sync::Arc; - -pub struct LogicTestContextProvider {} - -// Only a mock, don't need to implement -impl ContextProvider for LogicTestContextProvider { - fn get_table_provider( - &self, - _name: TableReference, - ) -> datafusion_common::Result> { - todo!() - } - - fn get_function_meta(&self, _name: &str) -> Option> { - todo!() - } - - fn get_aggregate_meta(&self, _name: &str) -> Option> { - todo!() - } - - fn get_variable_type(&self, _variable_names: &[String]) -> Option { - todo!() - } - - fn options(&self) -> &ConfigOptions { - todo!() - } - - fn get_window_meta(&self, _name: &str) -> Option> { - todo!() - } -} diff --git a/datafusion/core/tests/sqllogictests/src/engines/mod.rs b/datafusion/core/tests/sqllogictests/src/engines/mod.rs deleted file mode 100644 index a2657bb60017b..0000000000000 --- a/datafusion/core/tests/sqllogictests/src/engines/mod.rs +++ /dev/null @@ -1,21 +0,0 @@ -// 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. - -mod conversion; -pub mod datafusion; -mod output; -pub mod postgres; diff --git a/datafusion/core/tests/sqllogictests/src/engines/output.rs b/datafusion/core/tests/sqllogictests/src/engines/output.rs deleted file mode 100644 index 0682f5df97c19..0000000000000 --- a/datafusion/core/tests/sqllogictests/src/engines/output.rs +++ /dev/null @@ -1,57 +0,0 @@ -// 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 sqllogictest::{ColumnType, DBOutput}; - -#[derive(Debug, PartialEq, Eq, Clone)] -pub enum DFColumnType { - Boolean, - DateTime, - Integer, - Float, - Text, - Timestamp, - Another, -} - -impl ColumnType for DFColumnType { - fn from_char(value: char) -> Option { - match value { - 'B' => Some(Self::Boolean), - 'D' => Some(Self::DateTime), - 'I' => Some(Self::Integer), - 'P' => Some(Self::Timestamp), - 'R' => Some(Self::Float), - 'T' => Some(Self::Text), - _ => Some(Self::Another), - } - } - - fn to_char(&self) -> char { - match self { - Self::Boolean => 'B', - Self::DateTime => 'D', - Self::Integer => 'I', - Self::Timestamp => 'P', - Self::Float => 'R', - Self::Text => 'T', - Self::Another => '?', - } - } -} - -pub type DFOutput = DBOutput; diff --git a/datafusion/core/tests/sqllogictests/src/main.rs b/datafusion/core/tests/sqllogictests/src/main.rs index c74d1cb11a471..1bc41297b2977 100644 --- a/datafusion/core/tests/sqllogictests/src/main.rs +++ b/datafusion/core/tests/sqllogictests/src/main.rs @@ -20,6 +20,7 @@ use std::path::{Path, PathBuf}; #[cfg(target_family = "windows")] use std::thread; +use datafusion_sqllogictest::{DataFusionTestEngine, PostgresTestEngine}; use futures::stream::StreamExt; use log::info; use sqllogictest::strict_column_validator; @@ -28,10 +29,6 @@ use tempfile::TempDir; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::{DataFusionError, Result}; -use crate::engines::datafusion::DataFusion; -use crate::engines::postgres::Postgres; - -mod engines; mod setup; const TEST_DIRECTORY: &str = "tests/sqllogictests/test_files/"; @@ -129,7 +126,7 @@ async fn run_test_file(test_file: TestFile) -> Result<()> { return Ok(()); }; let mut runner = sqllogictest::Runner::new(|| async { - Ok(DataFusion::new( + Ok(DataFusionTestEngine::new( test_ctx.session_ctx().clone(), relative_path.clone(), )) @@ -148,7 +145,7 @@ async fn run_test_file_with_postgres(test_file: TestFile) -> Result<()> { } = test_file; info!("Running with Postgres runner: {}", path.display()); let mut runner = - sqllogictest::Runner::new(|| Postgres::connect(relative_path.clone())); + sqllogictest::Runner::new(|| PostgresTestEngine::connect(relative_path.clone())); runner.with_column_validator(strict_column_validator); runner .run_file_async(path) @@ -171,7 +168,7 @@ async fn run_complete_file(test_file: TestFile) -> Result<()> { return Ok(()); }; let mut runner = sqllogictest::Runner::new(|| async { - Ok(DataFusion::new( + Ok(DataFusionTestEngine::new( test_ctx.session_ctx().clone(), relative_path.clone(), )) diff --git a/datafusion/core/tests/sqllogictests/test_files/information_schema_columns.slt b/datafusion/core/tests/sqllogictests/test_files/information_schema_columns.slt index fcb653cedd167..7cf845c16d738 100644 --- a/datafusion/core/tests/sqllogictests/test_files/information_schema_columns.slt +++ b/datafusion/core/tests/sqllogictests/test_files/information_schema_columns.slt @@ -50,4 +50,4 @@ statement ok drop table t1 statement ok -drop table t2 \ No newline at end of file +drop table t2 diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index 24f293e8a8b42..66961abdfaca5 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -41,7 +41,17 @@ itertools = "0.11" lazy_static = {version = "^1.4.0"} object_store = "0.6.1" rust_decimal = {version = "1.27.0"} +log = "^0.4" sqllogictest = "0.15.0" sqlparser.workspace = true thiserror = "1.0.44" tokio = {version = "1.0"} +bytes = {version = "1.4.0", optional = true} +futures = {version = "0.3.28", optional = true} +chrono = {version = "0.4.26", optional = true} +tokio-postgres = {version = "0.7.7", optional = true} +postgres-types = {version = "0.2.4", optional = true} +postgres-protocol = {version = "0.6.4", optional = true} + +[features] +postgres = ["bytes", "futures", "chrono", "tokio-postgres", "postgres-types", "postgres-protocol"] diff --git a/datafusion/sqllogictest/src/engines/conversion.rs b/datafusion/sqllogictest/src/engines/conversion.rs index 757bbbcbb96e1..97c8d59c724ef 100644 --- a/datafusion/sqllogictest/src/engines/conversion.rs +++ b/datafusion/sqllogictest/src/engines/conversion.rs @@ -123,3 +123,20 @@ pub fn i128_to_str(value: i128, precision: &u8, scale: &i8) -> String { pub fn big_decimal_to_str(value: BigDecimal) -> String { value.round(12).normalized().to_string() } + +/// Converts a 128-bit decimal into a string using specified precision and scale. +/// +/// # Arguments +/// +/// * `value` - The 128-bit decimal to convert. +/// +/// # Panics +/// +/// Panics if the decimal is not valid. +/// +/// # Returns +/// +/// The string representation of the decimal. +pub fn decimal_to_str(value: Decimal) -> String { + big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) +} diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs index 62ea82900d9e0..c15ea4e6c64c8 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs @@ -14,36 +14,40 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations -use std::{sync::Arc, time::Duration}; +use std::{time::Duration, path::PathBuf}; use arrow::record_batch::RecordBatch; use async_trait::async_trait; use datafusion::prelude::SessionContext; -use sqllogictest::{DBOutput, TestError}; +use sqllogictest::DBOutput; +use log::info; + +use super::{error::Result, DFSqlLogicTestError}; use crate::engines::output::{DFColumnType, DFOutput}; use super::{convert_batches, convert_schema_to_types}; -pub struct DataFusionTestRunner { - context: Arc, +pub struct DataFusionTestEngine { + ctx: SessionContext, + relative_path: PathBuf, } -impl DataFusionTestRunner { - pub fn new(context: Arc) -> Self { - Self { context } +impl DataFusionTestEngine { + pub fn new(context: SessionContext, relative_path: PathBuf) -> Self { + Self { ctx: context, relative_path } } } async fn run_query( ctx: &SessionContext, sql: impl Into, -) -> Result { - let df = ctx.sql(sql.into().as_str()).await.unwrap(); +) -> Result { + let df = ctx.sql(sql.into().as_str()).await?; let types = convert_schema_to_types(df.schema().fields().as_slice()); - let results: Vec = df.collect().await.unwrap(); - let rows = convert_batches(results).unwrap(); + let results: Vec = df.collect().await?; + let rows = convert_batches(results)?; if rows.is_empty() && types.is_empty() { Ok(DBOutput::StatementComplete(0)) @@ -53,17 +57,22 @@ async fn run_query( } #[async_trait] -impl sqllogictest::AsyncDB for DataFusionTestRunner { - type Error = TestError; +impl sqllogictest::AsyncDB for DataFusionTestEngine { + type Error = DFSqlLogicTestError; type ColumnType = DFColumnType; - async fn run(&mut self, sql: &str) -> Result { - run_query(&self.context, sql).await + async fn run(&mut self, sql: &str) -> Result { + info!( + "[{}] Running query: \"{}\"", + self.relative_path.display(), + sql + ); + run_query(&self.ctx, sql).await } /// Engine name of current database. fn engine_name(&self) -> &str { - "DataFusionTestRunner" + "DataFusionTestEngine" } /// [`Runner`] calls this function to perform sleep. diff --git a/datafusion/sqllogictest/src/engines/mod.rs b/datafusion/sqllogictest/src/engines/mod.rs index 7604edf4b0432..a4bb8dd5decdc 100644 --- a/datafusion/sqllogictest/src/engines/mod.rs +++ b/datafusion/sqllogictest/src/engines/mod.rs @@ -18,4 +18,10 @@ mod conversion; mod datafusion_engine; mod output; -pub use datafusion_engine::DataFusionTestRunner; +pub use datafusion_engine::DataFusionTestEngine; + +#[cfg(feature = "postgres")] +mod postgres_engine; + +#[cfg(feature = "postgres")] +pub use postgres_engine::PostgresTestEngine; diff --git a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs b/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs similarity index 98% rename from datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs rename to datafusion/sqllogictest/src/engines/postgres_engine/mod.rs index 2c6287b97bfd1..535a7410f0f1a 100644 --- a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs +++ b/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs @@ -49,14 +49,14 @@ pub enum Error { pub type Result = std::result::Result; -pub struct Postgres { +pub struct PostgresTestEngine { client: tokio_postgres::Client, join_handle: JoinHandle<()>, /// Relative test file path relative_path: PathBuf, } -impl Postgres { +impl PostgresTestEngine { /// Creates a runner for executing queries against an existing postgres connection. /// `relative_path` is used for display output and to create a postgres schema. /// @@ -205,14 +205,14 @@ fn schema_name(relative_path: &Path) -> String { .unwrap_or_else(|| "default_schema".to_string()) } -impl Drop for Postgres { +impl Drop for PostgresTestEngine { fn drop(&mut self) { self.join_handle.abort() } } #[async_trait] -impl sqllogictest::AsyncDB for Postgres { +impl sqllogictest::AsyncDB for PostgresTestEngine { type Error = Error; type ColumnType = DFColumnType; diff --git a/datafusion/core/tests/sqllogictests/src/engines/postgres/types.rs b/datafusion/sqllogictest/src/engines/postgres_engine/types.rs similarity index 100% rename from datafusion/core/tests/sqllogictests/src/engines/postgres/types.rs rename to datafusion/sqllogictest/src/engines/postgres_engine/types.rs diff --git a/datafusion/sqllogictest/src/lib.rs b/datafusion/sqllogictest/src/lib.rs index c2eedbcb3ec9c..9ccee9a8f9b93 100644 --- a/datafusion/sqllogictest/src/lib.rs +++ b/datafusion/sqllogictest/src/lib.rs @@ -17,4 +17,7 @@ mod engines; -pub use engines::DataFusionTestRunner; +pub use engines::DataFusionTestEngine; + +#[cfg(feature = "postgres")] +pub use engines::PostgresTestEngine; From 651653f45dd27134fd012c4d2a8dd61f981f42b7 Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Tue, 1 Aug 2023 06:38:39 -0700 Subject: [PATCH 3/9] fix: fix tests due to name issue --- .../core/tests/sqllogictests/src/main.rs | 8 +-- .../sqllogictest/src/engines/conversion.rs | 71 +++---------------- .../src/engines/datafusion_engine/runner.rs | 52 +++++++------- datafusion/sqllogictest/src/engines/mod.rs | 4 +- .../src/engines/postgres_engine/mod.rs | 8 +-- datafusion/sqllogictest/src/lib.rs | 4 +- 6 files changed, 47 insertions(+), 100 deletions(-) diff --git a/datafusion/core/tests/sqllogictests/src/main.rs b/datafusion/core/tests/sqllogictests/src/main.rs index 1bc41297b2977..7d23971fbfca5 100644 --- a/datafusion/core/tests/sqllogictests/src/main.rs +++ b/datafusion/core/tests/sqllogictests/src/main.rs @@ -20,7 +20,7 @@ use std::path::{Path, PathBuf}; #[cfg(target_family = "windows")] use std::thread; -use datafusion_sqllogictest::{DataFusionTestEngine, PostgresTestEngine}; +use datafusion_sqllogictest::{DataFusion, Postgres}; use futures::stream::StreamExt; use log::info; use sqllogictest::strict_column_validator; @@ -126,7 +126,7 @@ async fn run_test_file(test_file: TestFile) -> Result<()> { return Ok(()); }; let mut runner = sqllogictest::Runner::new(|| async { - Ok(DataFusionTestEngine::new( + Ok(DataFusion::new( test_ctx.session_ctx().clone(), relative_path.clone(), )) @@ -145,7 +145,7 @@ async fn run_test_file_with_postgres(test_file: TestFile) -> Result<()> { } = test_file; info!("Running with Postgres runner: {}", path.display()); let mut runner = - sqllogictest::Runner::new(|| PostgresTestEngine::connect(relative_path.clone())); + sqllogictest::Runner::new(|| Postgres::connect(relative_path.clone())); runner.with_column_validator(strict_column_validator); runner .run_file_async(path) @@ -168,7 +168,7 @@ async fn run_complete_file(test_file: TestFile) -> Result<()> { return Ok(()); }; let mut runner = sqllogictest::Runner::new(|| async { - Ok(DataFusionTestEngine::new( + Ok(DataFusion::new( test_ctx.session_ctx().clone(), relative_path.clone(), )) diff --git a/datafusion/sqllogictest/src/engines/conversion.rs b/datafusion/sqllogictest/src/engines/conversion.rs index 97c8d59c724ef..a44783b098c9b 100644 --- a/datafusion/sqllogictest/src/engines/conversion.rs +++ b/datafusion/sqllogictest/src/engines/conversion.rs @@ -23,12 +23,7 @@ use rust_decimal::prelude::*; /// Represents a constant for NULL string in your database. pub const NULL_STR: &str = "NULL"; -/// Converts a bool value into a string. -/// -/// # Arguments -/// -/// * `value` - The bool value to convert. -pub fn bool_to_str(value: bool) -> String { +pub(crate) fn bool_to_str(value: bool) -> String { if value { "true".to_string() } else { @@ -36,13 +31,7 @@ pub fn bool_to_str(value: bool) -> String { } } -/// Converts a varchar into a string, trimming end line breaks. -/// Returns "(empty)" string for empty input. -/// -/// # Arguments -/// -/// * `value` - The varchar value to convert. -pub fn varchar_to_str(value: &str) -> String { +pub(crate) fn varchar_to_str(value: &str) -> String { if value.is_empty() { "(empty)".to_string() } else { @@ -50,12 +39,7 @@ pub fn varchar_to_str(value: &str) -> String { } } -/// Converts a 16-bit floating-point number into a string. -/// -/// # Arguments -/// -/// * `value` - The 16-bit floating-point number to convert. -pub fn f16_to_str(value: f16) -> String { +pub(crate) fn f16_to_str(value: f16) -> String { if value.is_nan() { "NaN".to_string() } else if value == f16::INFINITY { @@ -67,12 +51,7 @@ pub fn f16_to_str(value: f16) -> String { } } -/// Converts a 32-bit floating-point number into a string. -/// -/// # Arguments -/// -/// * `value` - The 32-bit floating-point number to convert. -pub fn f32_to_str(value: f32) -> String { +pub(crate) fn f32_to_str(value: f32) -> String { if value.is_nan() { "NaN".to_string() } else if value == f32::INFINITY { @@ -84,12 +63,7 @@ pub fn f32_to_str(value: f32) -> String { } } -/// Converts a 64-bit floating-point number into a string. -/// -/// # Arguments -/// -/// * `value` - The 64-bit floating-point number to convert. -pub fn f64_to_str(value: f64) -> String { +pub(crate) fn f64_to_str(value: f64) -> String { if value.is_nan() { "NaN".to_string() } else if value == f64::INFINITY { @@ -101,42 +75,17 @@ pub fn f64_to_str(value: f64) -> String { } } -/// Converts a 128-bit integer into a string using specified precision and scale. -/// -/// # Arguments -/// -/// * `value` - The 128-bit integer to convert. -/// * `precision` - The number of significant digits. -/// * `scale` - The number of digits to the right of the decimal point. -pub fn i128_to_str(value: i128, precision: &u8, scale: &i8) -> String { +pub(crate) fn i128_to_str(value: i128, precision: &u8, scale: &i8) -> String { big_decimal_to_str( BigDecimal::from_str(&Decimal128Type::format_decimal(value, *precision, *scale)) .unwrap(), ) } -/// Converts a BigDecimal into a string, rounding the result to 12 decimal places. -/// -/// # Arguments -/// -/// * `value` - The BigDecimal value to convert. -pub fn big_decimal_to_str(value: BigDecimal) -> String { - value.round(12).normalized().to_string() +pub(crate) fn decimal_to_str(value: Decimal) -> String { + big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) } -/// Converts a 128-bit decimal into a string using specified precision and scale. -/// -/// # Arguments -/// -/// * `value` - The 128-bit decimal to convert. -/// -/// # Panics -/// -/// Panics if the decimal is not valid. -/// -/// # Returns -/// -/// The string representation of the decimal. -pub fn decimal_to_str(value: Decimal) -> String { - big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) +pub(crate) fn big_decimal_to_str(value: BigDecimal) -> String { + value.round(12).normalized().to_string() } diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs index c15ea4e6c64c8..3ce701304e4b3 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs @@ -14,50 +14,34 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations -use std::{time::Duration, path::PathBuf}; +use std::{path::PathBuf, time::Duration}; use arrow::record_batch::RecordBatch; use async_trait::async_trait; use datafusion::prelude::SessionContext; -use sqllogictest::DBOutput; use log::info; +use sqllogictest::DBOutput; -use super::{error::Result, DFSqlLogicTestError}; +use super::{error::Result, normalize, DFSqlLogicTestError}; use crate::engines::output::{DFColumnType, DFOutput}; -use super::{convert_batches, convert_schema_to_types}; - -pub struct DataFusionTestEngine { +pub struct DataFusion { ctx: SessionContext, relative_path: PathBuf, } -impl DataFusionTestEngine { +impl DataFusion { pub fn new(context: SessionContext, relative_path: PathBuf) -> Self { - Self { ctx: context, relative_path } - } -} - -async fn run_query( - ctx: &SessionContext, - sql: impl Into, -) -> Result { - let df = ctx.sql(sql.into().as_str()).await?; - - let types = convert_schema_to_types(df.schema().fields().as_slice()); - let results: Vec = df.collect().await?; - let rows = convert_batches(results)?; - - if rows.is_empty() && types.is_empty() { - Ok(DBOutput::StatementComplete(0)) - } else { - Ok(DBOutput::Rows { types, rows }) + Self { + ctx: context, + relative_path, + } } } #[async_trait] -impl sqllogictest::AsyncDB for DataFusionTestEngine { +impl sqllogictest::AsyncDB for DataFusion { type Error = DFSqlLogicTestError; type ColumnType = DFColumnType; @@ -72,7 +56,7 @@ impl sqllogictest::AsyncDB for DataFusionTestEngine { /// Engine name of current database. fn engine_name(&self) -> &str { - "DataFusionTestEngine" + "DataFusion" } /// [`Runner`] calls this function to perform sleep. @@ -84,3 +68,17 @@ impl sqllogictest::AsyncDB for DataFusionTestEngine { tokio::time::sleep(dur).await; } } + +async fn run_query(ctx: &SessionContext, sql: impl Into) -> Result { + let df = ctx.sql(sql.into().as_str()).await?; + + let types = normalize::convert_schema_to_types(df.schema().fields()); + let results: Vec = df.collect().await?; + let rows = normalize::convert_batches(results)?; + + if rows.is_empty() && types.is_empty() { + Ok(DBOutput::StatementComplete(0)) + } else { + Ok(DBOutput::Rows { types, rows }) + } +} diff --git a/datafusion/sqllogictest/src/engines/mod.rs b/datafusion/sqllogictest/src/engines/mod.rs index a4bb8dd5decdc..4e6297105f1bf 100644 --- a/datafusion/sqllogictest/src/engines/mod.rs +++ b/datafusion/sqllogictest/src/engines/mod.rs @@ -18,10 +18,10 @@ mod conversion; mod datafusion_engine; mod output; -pub use datafusion_engine::DataFusionTestEngine; +pub use datafusion_engine::DataFusion; #[cfg(feature = "postgres")] mod postgres_engine; #[cfg(feature = "postgres")] -pub use postgres_engine::PostgresTestEngine; +pub use postgres_engine::Postgres; diff --git a/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs b/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs index 535a7410f0f1a..2c6287b97bfd1 100644 --- a/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs +++ b/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs @@ -49,14 +49,14 @@ pub enum Error { pub type Result = std::result::Result; -pub struct PostgresTestEngine { +pub struct Postgres { client: tokio_postgres::Client, join_handle: JoinHandle<()>, /// Relative test file path relative_path: PathBuf, } -impl PostgresTestEngine { +impl Postgres { /// Creates a runner for executing queries against an existing postgres connection. /// `relative_path` is used for display output and to create a postgres schema. /// @@ -205,14 +205,14 @@ fn schema_name(relative_path: &Path) -> String { .unwrap_or_else(|| "default_schema".to_string()) } -impl Drop for PostgresTestEngine { +impl Drop for Postgres { fn drop(&mut self) { self.join_handle.abort() } } #[async_trait] -impl sqllogictest::AsyncDB for PostgresTestEngine { +impl sqllogictest::AsyncDB for Postgres { type Error = Error; type ColumnType = DFColumnType; diff --git a/datafusion/sqllogictest/src/lib.rs b/datafusion/sqllogictest/src/lib.rs index 9ccee9a8f9b93..b739d75777de0 100644 --- a/datafusion/sqllogictest/src/lib.rs +++ b/datafusion/sqllogictest/src/lib.rs @@ -17,7 +17,7 @@ mod engines; -pub use engines::DataFusionTestEngine; +pub use engines::DataFusion; #[cfg(feature = "postgres")] -pub use engines::PostgresTestEngine; +pub use engines::Postgres; From 9b56992ea08808a36c234ecca736602f27cdefdb Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Tue, 1 Aug 2023 06:49:18 -0700 Subject: [PATCH 4/9] refactor: cleanup --- datafusion/core/Cargo.toml | 2 +- .../sqllogictest/src/engines/datafusion_engine/runner.rs | 7 ++----- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 446a4ebc14519..e8e2cc8e19239 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -102,7 +102,7 @@ bigdecimal = "0.4.1" criterion = { version = "0.5", features = ["async_tokio"] } csv = "1.1.6" ctor = "0.2.0" -datafusion-sqllogictest = { path = "../sqllogictest", features = ["postgres"] } +datafusion-sqllogictest = { path = "../sqllogictest", version = "28.0.0", features = ["postgres"] } doc-comment = "0.3" env_logger = "0.10" half = "2.2.1" diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs index 3ce701304e4b3..ff952ade86c15 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs @@ -32,11 +32,8 @@ pub struct DataFusion { } impl DataFusion { - pub fn new(context: SessionContext, relative_path: PathBuf) -> Self { - Self { - ctx: context, - relative_path, - } + pub fn new(ctx: SessionContext, relative_path: PathBuf) -> Self { + Self { ctx, relative_path } } } From fe1ec4254b2102d1ee69bc78897a69d8c72a3f43 Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Tue, 1 Aug 2023 06:56:45 -0700 Subject: [PATCH 5/9] refactor: chopped off license --- datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs index ff952ade86c15..cc1eb8c4a4edb 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs @@ -13,6 +13,7 @@ // "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::{path::PathBuf, time::Duration}; From 0b9c4845306ca42aad02e018457766b15dc2f0c7 Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Tue, 1 Aug 2023 14:17:06 -0700 Subject: [PATCH 6/9] docs: license and module docstrings --- .../src/engines/datafusion_engine/mod.rs | 19 +++++++++++++++++++ datafusion/sqllogictest/src/engines/mod.rs | 4 ++++ .../src/engines/postgres_engine/mod.rs | 2 ++ 3 files changed, 25 insertions(+) diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs index d2c96bfcb50fd..b71af1e9f7153 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs @@ -1,3 +1,22 @@ +// 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. + +/// DataFusion engine implementation for sqllogictest. + mod error; mod normalize; mod runner; diff --git a/datafusion/sqllogictest/src/engines/mod.rs b/datafusion/sqllogictest/src/engines/mod.rs index 4e6297105f1bf..a24055adee1dd 100644 --- a/datafusion/sqllogictest/src/engines/mod.rs +++ b/datafusion/sqllogictest/src/engines/mod.rs @@ -13,6 +13,10 @@ // "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. + +/// Implementation of sqllogictest for datafusion. Optionally uses postgres for +/// comparison, if the `postgres` feature is enabled. mod conversion; mod datafusion_engine; diff --git a/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs b/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs index 2c6287b97bfd1..feadb80ec3a37 100644 --- a/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs +++ b/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +/// Postgres engine implementation for sqllogictest. + use std::path::{Path, PathBuf}; use std::str::FromStr; From b088240963e026d0cc41973574244ec764dbae63 Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Tue, 1 Aug 2023 14:18:23 -0700 Subject: [PATCH 7/9] docs: module docstrings --- datafusion/sqllogictest/src/engines/mod.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/sqllogictest/src/engines/mod.rs b/datafusion/sqllogictest/src/engines/mod.rs index a24055adee1dd..3fd676a4b6cd5 100644 --- a/datafusion/sqllogictest/src/engines/mod.rs +++ b/datafusion/sqllogictest/src/engines/mod.rs @@ -15,8 +15,7 @@ // specific language governing permissions and limitations // under the License. -/// Implementation of sqllogictest for datafusion. Optionally uses postgres for -/// comparison, if the `postgres` feature is enabled. +/// Implementation of sqllogictest for datafusion. mod conversion; mod datafusion_engine; From c462466cb356d3f176843b2934176777ae264343 Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Tue, 1 Aug 2023 19:30:04 -0700 Subject: [PATCH 8/9] docs: update docs for cargo checks --- datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs | 1 - datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs | 2 +- datafusion/sqllogictest/src/engines/mod.rs | 1 - datafusion/sqllogictest/src/engines/postgres_engine/mod.rs | 1 - 4 files changed, 1 insertion(+), 4 deletions(-) diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs index b71af1e9f7153..663bbdd5a3c7c 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs @@ -16,7 +16,6 @@ // under the License. /// DataFusion engine implementation for sqllogictest. - mod error; mod normalize; mod runner; diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs index cc1eb8c4a4edb..afd0a241ca5ef 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs @@ -57,7 +57,7 @@ impl sqllogictest::AsyncDB for DataFusion { "DataFusion" } - /// [`Runner`] calls this function to perform sleep. + /// [`DataFusion`] calls this function to perform sleep. /// /// The default implementation is `std::thread::sleep`, which is universal to any async runtime /// but would block the current thread. If you are running in tokio runtime, you should override diff --git a/datafusion/sqllogictest/src/engines/mod.rs b/datafusion/sqllogictest/src/engines/mod.rs index 3fd676a4b6cd5..a6a0886332ed7 100644 --- a/datafusion/sqllogictest/src/engines/mod.rs +++ b/datafusion/sqllogictest/src/engines/mod.rs @@ -16,7 +16,6 @@ // under the License. /// Implementation of sqllogictest for datafusion. - mod conversion; mod datafusion_engine; mod output; diff --git a/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs b/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs index feadb80ec3a37..fe2785603e76d 100644 --- a/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs +++ b/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs @@ -16,7 +16,6 @@ // under the License. /// Postgres engine implementation for sqllogictest. - use std::path::{Path, PathBuf}; use std::str::FromStr; From f86ffffebd0a6a4722611758566ed82ae47aa3bd Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 2 Aug 2023 09:05:01 -0400 Subject: [PATCH 9/9] Fix doc error --- .../sqllogictest/src/engines/datafusion_engine/normalize.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs index a7e4a78c876f8..954926ae3310e 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs @@ -113,13 +113,13 @@ fn expand_row(mut row: Vec) -> impl Iterator> { /// normalize path references /// -/// ``` +/// ```text /// CsvExec: files={1 group: [[path/to/datafusion/testing/data/csv/aggregate_test_100.csv]]}, ... /// ``` /// /// into: /// -/// ``` +/// ```text /// CsvExec: files={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, ... /// ``` fn normalize_paths(mut row: Vec) -> Vec {