Skip to content

Commit

Permalink
Drive sqllogictest runner on directory contents rather than hard code…
Browse files Browse the repository at this point in the history
…d list (#4472)

* Update sqllogictest runner based on files

* port some tests

* fmt

* improve comments

* fix compilation on windows
  • Loading branch information
alamb authored Dec 2, 2022
1 parent 1aa645f commit b229e0f
Show file tree
Hide file tree
Showing 3 changed files with 123 additions and 148 deletions.
95 changes: 0 additions & 95 deletions datafusion/core/tests/sql/information_schema.rs
Original file line number Diff line number Diff line change
Expand Up @@ -669,98 +669,3 @@ async fn show_external_create_table() {
async fn plan_and_collect(ctx: &SessionContext, sql: &str) -> Result<Vec<RecordBatch>> {
ctx.sql(sql).await?.collect().await
}

#[tokio::test]
async fn show_variable_in_config_options() {
let ctx =
SessionContext::with_config(SessionConfig::new().with_information_schema(true));
let sql = "SHOW datafusion.execution.batch_size";
let results = plan_and_collect(&ctx, sql).await.unwrap();

let expected = vec![
"+---------------------------------+---------+",
"| name | setting |",
"+---------------------------------+---------+",
"| datafusion.execution.batch_size | 8192 |",
"+---------------------------------+---------+",
];

assert_batches_eq!(expected, &results);
}

#[tokio::test]
async fn show_all() {
let ctx =
SessionContext::with_config(SessionConfig::new().with_information_schema(true));
let sql = "SHOW ALL";

let results = plan_and_collect(&ctx, sql).await.unwrap();

// Has all the default values, should be in order by name
let expected = vec![
"+-----------------------------------------------------------+---------+",
"| name | setting |",
"+-----------------------------------------------------------+---------+",
"| datafusion.catalog.location | NULL |",
"| datafusion.catalog.type | NULL |",
"| datafusion.execution.batch_size | 8192 |",
"| datafusion.execution.coalesce_batches | true |",
"| datafusion.execution.coalesce_target_batch_size | 4096 |",
"| datafusion.execution.parquet.enable_page_index | false |",
"| datafusion.execution.parquet.metadata_size_hint | NULL |",
"| datafusion.execution.parquet.pruning | true |",
"| datafusion.execution.parquet.pushdown_filters | false |",
"| datafusion.execution.parquet.reorder_filters | false |",
"| datafusion.execution.parquet.skip_metadata | true |",
"| datafusion.execution.time_zone | +00:00 |",
"| datafusion.explain.logical_plan_only | false |",
"| datafusion.explain.physical_plan_only | false |",
"| datafusion.optimizer.filter_null_join_keys | false |",
"| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 |",
"| datafusion.optimizer.max_passes | 3 |",
"| datafusion.optimizer.prefer_hash_join | true |",
"| datafusion.optimizer.skip_failed_rules | true |",
"| datafusion.optimizer.top_down_join_key_reordering | true |",
"+-----------------------------------------------------------+---------+",
];

assert_batches_eq!(expected, &results);
}

#[tokio::test]
async fn show_time_zone_default_utc() {
// https://github.com/apache/arrow-datafusion/issues/3255
let ctx =
SessionContext::with_config(SessionConfig::new().with_information_schema(true));
let sql = "SHOW TIME ZONE";
let results = plan_and_collect(&ctx, sql).await.unwrap();

let expected = vec![
"+--------------------------------+---------+",
"| name | setting |",
"+--------------------------------+---------+",
"| datafusion.execution.time_zone | +00:00 |",
"+--------------------------------+---------+",
];

assert_batches_eq!(expected, &results);
}

#[tokio::test]
async fn show_timezone_default_utc() {
// https://github.com/apache/arrow-datafusion/issues/3255
let ctx =
SessionContext::with_config(SessionConfig::new().with_information_schema(true));
let sql = "SHOW TIMEZONE";
let results = plan_and_collect(&ctx, sql).await.unwrap();

let expected = vec![
"+--------------------------------+---------+",
"| name | setting |",
"+--------------------------------+---------+",
"| datafusion.execution.time_zone | +00:00 |",
"+--------------------------------+---------+",
];

assert_batches_eq!(expected, &results);
}
115 changes: 62 additions & 53 deletions datafusion/core/tests/sqllogictests/src/main.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,8 @@
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 datafusion::prelude::{SessionConfig, SessionContext};
use std::path::Path;
use std::time::Duration;

use sqllogictest::TestError;
Expand All @@ -29,53 +29,18 @@ 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,
file_name: String,
}

#[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
);
println!("[{}] Running query: \"{}\"", self.file_name, sql);
let result = run_query(&self.ctx, sql).await?;
Ok(result)
}
Expand All @@ -96,26 +61,70 @@ impl sqllogictest::AsyncDB for DataFusion {
}

#[tokio::main]
#[cfg(target_family = "windows")]
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;

if !cfg!(target_os = "windows") {
let mut tester = sqllogictest::Runner::new(DataFusion { ctx, test_category });
// TODO: use tester.run_parallel_async()
tester.run_file_async(filename).await?;
}
println!("Skipping test on windows");
Ok(())
}

#[tokio::main]
#[cfg(not(target_family = "windows"))]
pub async fn main() -> Result<()> {
let paths = std::fs::read_dir(TEST_DIRECTORY).unwrap();

// run each file using its own new SessionContext
//
// Note: can't use tester.run_parallel_async()
// as that will reuse the same SessionContext
//
// We could run these tests in parallel eventually if we wanted.

for path in paths {
// TODO better error handling
let path = path.unwrap().path();

run_file(&path).await?;
}

Ok(())
}

/// Run the tests in the specified `.slt` file
async fn run_file(path: &Path) -> Result<()> {
println!("Running: {}", path.display());

let file_name = path.file_name().unwrap().to_str().unwrap().to_string();

let ctx = context_for_test_file(&file_name).await;

let mut tester = sqllogictest::Runner::new(DataFusion { ctx, file_name });
tester.run_file_async(path).await?;

Ok(())
}

/// Create a SessionContext, configured for the specific test
async fn context_for_test_file(file_name: &str) -> SessionContext {
match file_name {
"aggregate.slt" => {
println!("Registering aggregate tables");
let ctx = SessionContext::new();
setup::register_aggregate_tables(&ctx).await;
ctx
}
"information_schema.slt" => {
println!("Enabling information schema");
SessionContext::with_config(
SessionConfig::new().with_information_schema(true),
)
}
_ => {
println!("Using default SessionContex");
SessionContext::new()
}
}
}

fn format_batches(batches: &[RecordBatch]) -> Result<String> {
let mut bytes = vec![];
{
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
# 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.

# show all variables
query R
SHOW ALL
----
datafusion.catalog.location NULL
datafusion.catalog.type NULL
datafusion.execution.batch_size 8192
datafusion.execution.coalesce_batches true
datafusion.execution.coalesce_target_batch_size 4096
datafusion.execution.parquet.enable_page_index false
datafusion.execution.parquet.metadata_size_hint NULL
datafusion.execution.parquet.pruning true
datafusion.execution.parquet.pushdown_filters false
datafusion.execution.parquet.reorder_filters false
datafusion.execution.parquet.skip_metadata true
datafusion.execution.time_zone +00:00
datafusion.explain.logical_plan_only false
datafusion.explain.physical_plan_only false
datafusion.optimizer.filter_null_join_keys false
datafusion.optimizer.hash_join_single_partition_threshold 1048576
datafusion.optimizer.max_passes 3
datafusion.optimizer.prefer_hash_join true
datafusion.optimizer.skip_failed_rules true
datafusion.optimizer.top_down_join_key_reordering true

# show_variable_in_config_options
query R
SHOW datafusion.execution.batch_size
----
datafusion.execution.batch_size 8192

# show_time_zone_default_utc
# https://github.com/apache/arrow-datafusion/issues/3255
query R
SHOW TIME ZONE
----
datafusion.execution.time_zone +00:00

# show_timezone_default_utc
# https://github.com/apache/arrow-datafusion/issues/3255
query R
SHOW TIMEZONE
----
datafusion.execution.time_zone +00:00

0 comments on commit b229e0f

Please sign in to comment.