From c9c1179e1f29c50c2f68365c88470c8e05f49a16 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 21 Jan 2023 06:21:16 -0500 Subject: [PATCH 01/10] Remove posgres container orchestration and move connection logic into postgres.rs --- .github/workflows/rust.yml | 20 +- datafusion/core/Cargo.toml | 1 - datafusion/core/tests/sqllogictests/README.md | 27 ++- .../postgres/postgres_create_table.sql | 21 -- .../src/engines/postgres/image.rs | 57 ------ .../sqllogictests/src/engines/postgres/mod.rs | 181 +++++++++++++----- .../core/tests/sqllogictests/src/main.rs | 62 ++---- .../core/tests/sqllogictests/src/setup.rs | 39 ---- .../sqllogictests/test_files/aggregate.slt | 22 +++ .../test_files/pg_compat_simple.slt | 61 ++++++ .../test_files/pg_compat_union.slt | 57 +++++- .../test_files/pg_compat_window.slt | 55 ++++++ 12 files changed, 391 insertions(+), 212 deletions(-) delete mode 100644 datafusion/core/tests/sqllogictests/postgres/postgres_create_table.sql delete mode 100644 datafusion/core/tests/sqllogictests/src/engines/postgres/image.rs diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 3f466ae0795c4..769c0bafdfc7b 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -232,9 +232,21 @@ jobs: name: "Run sqllogictest with Postgres runner" needs: [linux-build-lib] runs-on: ubuntu-latest + services: + postgres: + image: postgres:15 + env: + POSTGRES_PASSWORD: postgres + POSTGRES_DB: db_test + POSTGRES_INITDB_ARGS: --encoding=UTF-8 --lc-collate=C --lc-ctype=C + ports: + - 5432/tcp + options: >- + --health-cmd pg_isready + --health-interval 10s + --health-timeout 5s + --health-retries 5 steps: - - name: Check docker - run: docker ps - uses: actions/checkout@v3 with: submodules: true @@ -243,7 +255,9 @@ jobs: rustup toolchain install stable rustup default stable - name: Run sqllogictest - run: PG_COMPAT=true cargo test -p datafusion --test sqllogictests + run: PG_COMPAT=true PG_DSN="postgresql://postgres:posgres@localhost@$POSTGRES_PORT/db_test" cargo test -p datafusion --test sqllogictests + env: + POSTGRES_PORT: ${{ job.services.postgres.ports[5432] }} windows: name: cargo test (win64) diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 484f1538784de..becc733eb2eb5 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -117,7 +117,6 @@ rstest = "0.16.0" rust_decimal = { version = "1.27.0", features = ["tokio-pg"] } sqllogictest = "0.11.1" test-utils = { path = "../../test-utils" } -testcontainers = "0.14.0" thiserror = "1.0.37" tokio-postgres = "0.7.7" [target.'cfg(not(target_os = "windows"))'.dev-dependencies] diff --git a/datafusion/core/tests/sqllogictests/README.md b/datafusion/core/tests/sqllogictests/README.md index d83e0e4bba3f9..fa13b6e6e3769 100644 --- a/datafusion/core/tests/sqllogictests/README.md +++ b/datafusion/core/tests/sqllogictests/README.md @@ -46,16 +46,33 @@ cargo test -p datafusion --test sqllogictests -- information #### Running tests: Postgres compatibility -Test files that start with prefix `pg_compat_` verify compatibility with Postgres. -Datafusion runs these test files during normal sqllogictest runs. +Test files that start with prefix `pg_compat_` verify compatibility +with Postgres by running the same script files both with DataFusion and with Posgres -In order to run sqllogictests with Postgres execute: +In order to run the sqllogictests running against a previously running Postgres instance, do: ```shell -PG_COMPAT=true cargo test -p datafusion --test sqllogictests +PG_COMPAT=true PG_DSN="postgresql://postgres@127.0.0.1/postgres" cargo test -p datafusion --test sqllogictests +``` + +The environemnt variables: +1. `PG_COMPAT` instructs sqllogictest to run against Postgres (not DataFusion) +2. `PG_DSN` contains a `libpq` style connection string, whose format is described in + [the docs](https://docs.rs/tokio-postgres/latest/tokio_postgres/config/struct.Config.html#url) + +One way to create a suitable a posgres container in docker is to use +the [Official Image](https://hub.docker.com/_/postgres) with a command +such as the following. Note the collation **must** be set to `C` otherwise +`ORDER BY` will not match DataFusion and the tests will diff. + +```shell +docker run \ + -p5432:5432 \ + -e POSTGRES_INITDB_ARGS="--encoding=UTF-8 --lc-collate=C --lc-ctype=C" \ + -e POSTGRES_HOST_AUTH_METHOD=trust \ + postgres ``` -This command requires a docker binary. Check that docker is properly installed with `which docker`. #### Updating tests: Completion Mode diff --git a/datafusion/core/tests/sqllogictests/postgres/postgres_create_table.sql b/datafusion/core/tests/sqllogictests/postgres/postgres_create_table.sql deleted file mode 100644 index 7df5943ae2625..0000000000000 --- a/datafusion/core/tests/sqllogictests/postgres/postgres_create_table.sql +++ /dev/null @@ -1,21 +0,0 @@ -CREATE TABLE aggregate_test_100_by_sql -( - c1 character varying NOT NULL, - c2 smallint NOT NULL, - c3 smallint NOT NULL, - c4 smallint, - c5 integer, - c6 bigint NOT NULL, - c7 smallint NOT NULL, - c8 integer NOT NULL, - c9 bigint NOT NULL, - c10 character varying NOT NULL, - c11 real NOT NULL, - c12 double precision NOT NULL, - c13 character varying NOT NULL -); - -COPY aggregate_test_100_by_sql - FROM '/opt/data/csv/aggregate_test_100.csv' - DELIMITER ',' - CSV HEADER; diff --git a/datafusion/core/tests/sqllogictests/src/engines/postgres/image.rs b/datafusion/core/tests/sqllogictests/src/engines/postgres/image.rs deleted file mode 100644 index e19705eed6c86..0000000000000 --- a/datafusion/core/tests/sqllogictests/src/engines/postgres/image.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 testcontainers::core::WaitFor; -use testcontainers::images::generic::GenericImage; - -pub const PG_USER: &str = "postgres"; -pub const PG_PASSWORD: &str = "postgres"; -pub const PG_DB: &str = "test"; -pub const PG_PORT: u16 = 5432; - -pub fn postgres_docker_image() -> GenericImage { - let postgres_test_data = match datafusion::test_util::get_data_dir( - "POSTGRES_TEST_DATA", - "tests/sqllogictests/postgres", - ) { - Ok(pb) => pb.display().to_string(), - Err(err) => panic!("failed to get arrow data dir: {err}"), - }; - GenericImage::new("postgres", "15") - .with_wait_for(WaitFor::message_on_stderr( - "database system is ready to accept connections", - )) - .with_env_var("POSTGRES_DB", PG_DB) - .with_env_var("POSTGRES_USER", PG_USER) - .with_env_var("POSTGRES_PASSWORD", PG_PASSWORD) - .with_env_var( - "POSTGRES_INITDB_ARGS", - "--encoding=UTF-8 --lc-collate=C --lc-ctype=C", - ) - .with_exposed_port(PG_PORT) - .with_volume( - format!( - "{0}/csv/aggregate_test_100.csv", - datafusion::test_util::arrow_test_data() - ), - "/opt/data/csv/aggregate_test_100.csv", - ) - .with_volume( - format!("{0}/postgres_create_table.sql", postgres_test_data), - "/docker-entrypoint-initdb.d/0_create_table.sql", - ) -} diff --git a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs b/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs index fa5c2e3ed2b88..ab3e050f6bd43 100644 --- a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs +++ b/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. -use std::sync::Arc; -use std::time::Duration; +use std::{str::FromStr, time::Duration}; use async_trait::async_trait; +use bytes::Bytes; +use futures::{SinkExt, StreamExt}; use log::debug; use sqllogictest::{ColumnType, DBOutput}; use tokio::task::JoinHandle; @@ -29,69 +30,157 @@ use postgres_types::Type; use rust_decimal::Decimal; use tokio_postgres::{Column, Row}; -pub mod image; +// default connect string, can be overridden by environment PG_DSN +const PG_DSN: &str = "postgresql://postgres@127.0.0.1/test"; + +/// DataFusion sql-logicaltest error +#[derive(Debug, thiserror::Error)] +pub enum Error { + #[error("Postgres error: {0}")] + Postgres(#[from] tokio_postgres::error::Error), + #[error("Error handling copy command: {0}")] + Copy(String), +} + +pub type Result = std::result::Result; pub struct Postgres { - client: Arc, + client: tokio_postgres::Client, join_handle: JoinHandle<()>, + /// Filename, for display purposes file_name: String, } impl Postgres { - pub async fn connect_with_retry( - file_name: String, - host: &str, - port: u16, - db: &str, - user: &str, - pass: &str, - ) -> Result { + /// Creates a runner for executiong queries against an existing + /// posgres connection. `file_name` is used for display output + /// + /// The database connection details can be overridden by the + /// `PG_DSN` environment variable. + /// + /// This defaults to + /// + /// ```text + /// PG_DSN="postgresql://postgres@127.0.0.1/test" + /// ``` + /// + /// See https://docs.rs/tokio-postgres/latest/tokio_postgres/config/struct.Config.html#url for format + pub async fn connect(file_name: impl Into) -> Result { + let file_name = file_name.into(); + + let dsn = if let Ok(val) = std::env::var("PG_DSN") { + val + } else { + PG_DSN.to_string() + }; + + debug!("Using posgres dsn: {dsn}"); + + let config = tokio_postgres::Config::from_str(&dsn)?; + let mut retry = 0; + loop { - let connection_result = - Postgres::connect(file_name.clone(), host, port, db, user, pass).await; + let connection_result = config.connect(tokio_postgres::NoTls).await; + match connection_result { Err(e) if retry <= 3 => { debug!("Retrying connection error '{:?}'", e); retry += 1; tokio::time::sleep(Duration::from_secs(1)).await; } - result => break result, + Err(e) => return Err(Error::from(e)), + Ok((client, connection)) => { + let join_handle = tokio::spawn(async move { + if let Err(e) = connection.await { + log::error!("Postgres connection error: {:?}", e); + } + }); + + return Ok(Self { + client, + join_handle, + file_name, + }); + } } } } - async fn connect( - file_name: String, - host: &str, - port: u16, - db: &str, - user: &str, - pass: &str, - ) -> Result { - let (client, connection) = tokio_postgres::Config::new() - .host(host) - .port(port) - .dbname(db) - .user(user) - .password(pass) - .connect(tokio_postgres::NoTls) - .await?; - - let join_handle = tokio::spawn(async move { - if let Err(e) = connection.await { - log::error!("Postgres connection error: {:?}", e); + /// Special COPY command support. "COPY 'filename'" requires the + /// server to read the file which may not be possible (maybe it is + /// remote or running in some other docker container). + /// + /// Thus, we rewrite sql statements like + /// + /// ```sql + /// COPY ... FROM 'filename' ... + /// ``` + /// + /// Into + /// + /// ```sql + /// COPY ... FROM STDIN ... + /// ``` + /// + /// And read the file locally. + async fn run_copy_command(&mut self, sql: &str) -> Result { + let canonical_sql = sql.trim_start().to_ascii_lowercase(); + + debug!("Handling COPY command: {sql}"); + + // Hacky way to find the 'filename' in the statement + let mut tokens = canonical_sql.split_whitespace().peekable(); + let mut filename = None; + + // COPY FROM '/opt/data/csv/aggregate_test_100.csv' ... + // + // into + // + // COPY FROM STDIN ... + + let mut new_sql = vec![]; + while let Some(tok) = tokens.next() { + new_sql.push(tok); + // rewrite FROM to FROM STDIN + if tok == "from" { + filename = tokens.next(); + new_sql.push("STDIN"); } - }); + } + + let filename = filename.map(no_quotes).ok_or_else(|| { + Error::Copy(format!("Can not find filename in COPY: {sql}")) + })?; + + let new_sql = new_sql.join(" "); + debug!("Copying data from file {filename} using sql: {new_sql}"); + + // start the COPY command and get location to write data to + let tx = self.client.transaction().await?; + let sink = tx.copy_in(&new_sql).await?; + let mut sink = Box::pin(sink); + + // read the input file as a string ans feed it to the copy command + let data = std::fs::read_to_string(filename) + .map_err(|e| Error::Copy(format!("Error reading {}: {}", filename, e)))?; - Ok(Self { - client: Arc::new(client), - join_handle, - file_name, - }) + let mut data_stream = futures::stream::iter(vec![Ok(Bytes::from(data))]).boxed(); + + sink.send_all(&mut data_stream).await?; + sink.close().await?; + tx.commit().await?; + Ok(DBOutput::StatementComplete(0)) } } +/// remove single quotes from the start and end of the string +/// +/// 'filename' --> filename +fn no_quotes(t: &str) -> &str { + t.trim_start_matches('\'').trim_end_matches('\'') +} + impl Drop for Postgres { fn drop(&mut self) { self.join_handle.abort() @@ -142,13 +231,14 @@ fn cell_to_string(row: &Row, column: &Column, idx: usize) -> String { #[async_trait] impl sqllogictest::AsyncDB for Postgres { - type Error = tokio_postgres::error::Error; + type Error = Error; async fn run(&mut self, sql: &str) -> Result { println!("[{}] Running query: \"{}\"", self.file_name, sql); + let lower_sql = sql.trim_start().to_ascii_lowercase(); + let is_query_sql = { - let lower_sql = sql.trim_start().to_ascii_lowercase(); lower_sql.starts_with("select") || lower_sql.starts_with("values") || lower_sql.starts_with("show") @@ -159,6 +249,11 @@ impl sqllogictest::AsyncDB for Postgres { || lower_sql.starts_with("delete")) && lower_sql.contains("returning")) }; + + if lower_sql.starts_with("copy") { + return self.run_copy_command(sql).await; + } + if !is_query_sql { self.client.execute(sql, &[]).await?; return Ok(DBOutput::StatementComplete(0)); diff --git a/datafusion/core/tests/sqllogictests/src/main.rs b/datafusion/core/tests/sqllogictests/src/main.rs index 467ad5717c8f1..208cdb0f7a5f4 100644 --- a/datafusion/core/tests/sqllogictests/src/main.rs +++ b/datafusion/core/tests/sqllogictests/src/main.rs @@ -18,14 +18,11 @@ use std::error::Error; use std::path::{Path, PathBuf}; -use log::{debug, info}; -use testcontainers::clients::Cli as Docker; +use log::info; use datafusion::prelude::SessionContext; use crate::engines::datafusion::DataFusion; -use crate::engines::postgres; -use crate::engines::postgres::image::{PG_DB, PG_PASSWORD, PG_PORT, PG_USER}; use crate::engines::postgres::Postgres; mod engines; @@ -64,7 +61,7 @@ pub async fn main() -> Result<(), Box> { if is_pg_compatibility_test { run_test_file_with_postgres(&path, file_name).await?; } else { - debug!("Skipping test file {:?}", path); + println!("Skipping test file in postgres compatibility mode {path:?}",); } } else { run_test_file(&path, file_name, is_pg_compatibility_test).await?; @@ -80,7 +77,7 @@ async fn run_test_file( is_pg_compatibility_test: bool, ) -> Result<(), Box> { println!("Running with DataFusion runner: {}", path.display()); - let ctx = context_for_test_file(&file_name, is_pg_compatibility_test).await; + let ctx = context_for_test_file(&file_name).await; let mut runner = sqllogictest::Runner::new(DataFusion::new( ctx, file_name, @@ -96,21 +93,12 @@ async fn run_test_file_with_postgres( ) -> Result<(), Box> { info!("Running with Postgres runner: {}", path.display()); - let docker = Docker::default(); - let postgres_container = docker.run(postgres::image::postgres_docker_image()); + let postgres_client = Postgres::connect(file_name).await?; + + sqllogictest::Runner::new(postgres_client) + .run_file_async(path) + .await?; - let postgres_client = Postgres::connect_with_retry( - file_name, - "127.0.0.1", - postgres_container.get_host_port_ipv4(PG_PORT), - PG_DB, - PG_USER, - PG_PASSWORD, - ) - .await?; - let mut postgres_runner = sqllogictest::Runner::new(postgres_client); - - postgres_runner.run_file_async(path).await?; Ok(()) } @@ -123,7 +111,7 @@ async fn run_complete_file( info!("Using complete mode to complete: {}", path.display()); - let ctx = context_for_test_file(&file_name, is_pg_compatibility_test).await; + let ctx = context_for_test_file(&file_name).await; let mut runner = sqllogictest::Runner::new(DataFusion::new( ctx, file_name, @@ -149,27 +137,17 @@ fn read_test_files(options: &Options) -> Vec { } /// Create a SessionContext, configured for the specific test -async fn context_for_test_file( - file_name: &str, - is_pg_compatibility_test: bool, -) -> SessionContext { - if is_pg_compatibility_test { - info!("Registering pg compatibility tables"); - let ctx = SessionContext::new(); - setup::register_aggregate_csv_by_sql(&ctx).await; - ctx - } else { - match file_name { - "aggregate.slt" | "select.slt" => { - info!("Registering aggregate tables"); - let ctx = SessionContext::new(); - setup::register_aggregate_tables(&ctx).await; - ctx - } - _ => { - info!("Using default SessionContext"); - SessionContext::new() - } +async fn context_for_test_file(file_name: &str) -> SessionContext { + match file_name { + "aggregate.slt" | "select.slt" => { + info!("Registering aggregate tables"); + let ctx = SessionContext::new(); + setup::register_aggregate_tables(&ctx).await; + ctx + } + _ => { + info!("Using default SessionContext"); + SessionContext::new() } } } diff --git a/datafusion/core/tests/sqllogictests/src/setup.rs b/datafusion/core/tests/sqllogictests/src/setup.rs index af62932577d05..b52e06f3f7d12 100644 --- a/datafusion/core/tests/sqllogictests/src/setup.rs +++ b/datafusion/core/tests/sqllogictests/src/setup.rs @@ -33,7 +33,6 @@ use std::sync::Arc; use crate::utils; pub async fn register_aggregate_tables(ctx: &SessionContext) { - register_aggregate_csv_by_sql(ctx).await; register_aggregate_test_100(ctx).await; register_decimal_table(ctx); register_median_test_tables(ctx); @@ -111,44 +110,6 @@ fn register_median_test_tables(ctx: &SessionContext) { } } -pub async fn register_aggregate_csv_by_sql(ctx: &SessionContext) { - let test_data = datafusion::test_util::arrow_test_data(); - - let df = ctx - .sql(&format!( - " - CREATE EXTERNAL TABLE aggregate_test_100_by_sql ( - c1 VARCHAR NOT NULL, - c2 TINYINT NOT NULL, - c3 SMALLINT NOT NULL, - c4 SMALLINT, - c5 INT, - c6 BIGINT NOT NULL, - c7 SMALLINT NOT NULL, - c8 INT NOT NULL, - c9 BIGINT UNSIGNED NOT NULL, - c10 VARCHAR NOT NULL, - c11 FLOAT NOT NULL, - c12 DOUBLE NOT NULL, - c13 VARCHAR NOT NULL - ) - STORED AS CSV - WITH HEADER ROW - LOCATION '{test_data}/csv/aggregate_test_100.csv' - " - )) - .await - .expect("Creating dataframe for CREATE EXTERNAL TABLE"); - - // Mimic the CLI and execute the resulting plan -- even though it - // is effectively a no-op (returns zero rows) - let results = df.collect().await.expect("Executing CREATE EXTERNAL TABLE"); - assert!( - results.is_empty(), - "Expected no rows from executing CREATE EXTERNAL TABLE" - ); -} - fn register_test_data(ctx: &SessionContext) { let schema = Arc::new(Schema::new(vec![ Field::new("c1", DataType::Int64, true), diff --git a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt index 5dccb2427f3e7..3a89046ec1f91 100644 --- a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt +++ b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt @@ -15,6 +15,28 @@ # specific language governing permissions and limitations # under the License. + +statement ok +CREATE EXTERNAL TABLE aggregate_test_100_by_sql ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 BIGINT UNSIGNED NOT NULL, + c10 VARCHAR NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL +) +STORED AS CSV +WITH HEADER ROW +LOCATION '../../testing/data/csv/aggregate_test_100.csv' + + # FIX: custom absolute values # csv_query_avg_multi_batch diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_simple.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat_simple.slt index 3281b781fbed0..17e274e0851fa 100644 --- a/datafusion/core/tests/sqllogictests/test_files/pg_compat_simple.slt +++ b/datafusion/core/tests/sqllogictests/test_files/pg_compat_simple.slt @@ -15,6 +15,67 @@ # specific language governing permissions and limitations # under the License. +### +## Setup test for postgres +### + +onlyif postgres +statement ok +DROP TABLE IF EXISTS aggregate_test_100_by_sql; + +onlyif postgres +statement ok +CREATE TABLE aggregate_test_100_by_sql +( + c1 character varying NOT NULL, + c2 smallint NOT NULL, + c3 smallint NOT NULL, + c4 smallint, + c5 integer, + c6 bigint NOT NULL, + c7 smallint NOT NULL, + c8 integer NOT NULL, + c9 bigint NOT NULL, + c10 character varying NOT NULL, + c11 real NOT NULL, + c12 double precision NOT NULL, + c13 character varying NOT NULL +); + +# Copy the data +onlyif postgres +statement ok +COPY aggregate_test_100_by_sql + FROM '../../testing/data/csv/aggregate_test_100.csv' + DELIMITER ',' + CSV HEADER; + +### +## Setup test for datafusion +### +onlyif DataFusion +statement ok +CREATE EXTERNAL TABLE aggregate_test_100_by_sql ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 BIGINT UNSIGNED NOT NULL, + c10 VARCHAR NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL +) +STORED AS CSV +WITH HEADER ROW +LOCATION '../../testing/data/csv/aggregate_test_100.csv' + + + query RRRRR SELECT diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_union.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat_union.slt index e007b2a252a9d..8ad875be57a55 100644 --- a/datafusion/core/tests/sqllogictests/test_files/pg_compat_union.slt +++ b/datafusion/core/tests/sqllogictests/test_files/pg_compat_union.slt @@ -15,6 +15,62 @@ # specific language governing permissions and limitations # under the License. + +onlyif postgres +statement ok +DROP TABLE IF EXISTS aggregate_test_100_by_sql; + +onlyif postgres +statement ok +CREATE TABLE aggregate_test_100_by_sql +( + c1 character varying NOT NULL, + c2 smallint NOT NULL, + c3 smallint NOT NULL, + c4 smallint, + c5 integer, + c6 bigint NOT NULL, + c7 smallint NOT NULL, + c8 integer NOT NULL, + c9 bigint NOT NULL, + c10 character varying NOT NULL, + c11 real NOT NULL, + c12 double precision NOT NULL, + c13 character varying NOT NULL +); + +# Copy the data +onlyif postgres +statement ok +COPY aggregate_test_100_by_sql + FROM '../../testing/data/csv/aggregate_test_100.csv' + DELIMITER ',' + CSV HEADER; + +### +## Setup test for datafusion +### +onlyif DataFusion +statement ok +CREATE EXTERNAL TABLE aggregate_test_100_by_sql ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 BIGINT UNSIGNED NOT NULL, + c10 VARCHAR NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL +) +STORED AS CSV +WITH HEADER ROW +LOCATION '../../testing/data/csv/aggregate_test_100.csv' + query I SELECT * FROM ( SELECT c2 @@ -176,4 +232,3 @@ SELECT 1 num UNION ALL SELECT 2 num ORDER BY num; ---- 1 2 - diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_window.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat_window.slt index 58b64a55265c6..adfb9ad6ee6b7 100644 --- a/datafusion/core/tests/sqllogictests/test_files/pg_compat_window.slt +++ b/datafusion/core/tests/sqllogictests/test_files/pg_compat_window.slt @@ -16,6 +16,61 @@ # under the License. +onlyif postgres +statement ok +DROP TABLE IF EXISTS aggregate_test_100_by_sql; + +onlyif postgres +statement ok +CREATE TABLE aggregate_test_100_by_sql +( + c1 character varying NOT NULL, + c2 smallint NOT NULL, + c3 smallint NOT NULL, + c4 smallint, + c5 integer, + c6 bigint NOT NULL, + c7 smallint NOT NULL, + c8 integer NOT NULL, + c9 bigint NOT NULL, + c10 character varying NOT NULL, + c11 real NOT NULL, + c12 double precision NOT NULL, + c13 character varying NOT NULL +); + +# Copy the data +onlyif postgres +statement ok +COPY aggregate_test_100_by_sql + FROM '../../testing/data/csv/aggregate_test_100.csv' + DELIMITER ',' + CSV HEADER; + +### +## Setup test for datafusion +### +onlyif DataFusion +statement ok +CREATE EXTERNAL TABLE aggregate_test_100_by_sql ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 BIGINT UNSIGNED NOT NULL, + c10 VARCHAR NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL +) +STORED AS CSV +WITH HEADER ROW +LOCATION '../../testing/data/csv/aggregate_test_100.csv' + query IIIIIIIIII SELECT c9, From e86339c2fe832e35a92133f8274bef8a999dfb26 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 21 Jan 2023 09:38:11 -0500 Subject: [PATCH 02/10] prettier --- datafusion/core/tests/sqllogictests/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/sqllogictests/README.md b/datafusion/core/tests/sqllogictests/README.md index fa13b6e6e3769..c66bf06966d0e 100644 --- a/datafusion/core/tests/sqllogictests/README.md +++ b/datafusion/core/tests/sqllogictests/README.md @@ -56,6 +56,7 @@ PG_COMPAT=true PG_DSN="postgresql://postgres@127.0.0.1/postgres" cargo test -p d ``` The environemnt variables: + 1. `PG_COMPAT` instructs sqllogictest to run against Postgres (not DataFusion) 2. `PG_DSN` contains a `libpq` style connection string, whose format is described in [the docs](https://docs.rs/tokio-postgres/latest/tokio_postgres/config/struct.Config.html#url) @@ -73,7 +74,6 @@ docker run \ postgres ``` - #### Updating tests: Completion Mode In test script completion mode, `sqllogictests` reads a prototype script and runs the statements and queries against the database engine. The output is is a full script that is a copy of the prototype script with result inserted. From 35315b46558e1361e005ff2021d4864028cf8369 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 21 Jan 2023 11:10:04 -0500 Subject: [PATCH 03/10] fix up ci --- .github/workflows/rust.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 769c0bafdfc7b..aa693348ed17c 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -255,7 +255,7 @@ jobs: rustup toolchain install stable rustup default stable - name: Run sqllogictest - run: PG_COMPAT=true PG_DSN="postgresql://postgres:posgres@localhost@$POSTGRES_PORT/db_test" cargo test -p datafusion --test sqllogictests + run: PG_COMPAT=true PG_DSN="postgresql://postgres:postgres@localhost:$POSTGRES_PORT/db_test" cargo test -p datafusion --test sqllogictests env: POSTGRES_PORT: ${{ job.services.postgres.ports[5432] }} From 5569beee0bfa1ddf635722bf8cf9ca6c2b56ea0f Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 22 Jan 2023 07:03:15 -0500 Subject: [PATCH 04/10] Apply suggestions from code review Co-authored-by: Yevhenii Melnyk --- .../core/tests/sqllogictests/src/engines/postgres/mod.rs | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs b/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs index ab3e050f6bd43..898e04a109f8e 100644 --- a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs +++ b/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs @@ -52,7 +52,7 @@ pub struct Postgres { } impl Postgres { - /// Creates a runner for executiong queries against an existing + /// Creates a runner for executing queries against an existing /// posgres connection. `file_name` is used for display output /// /// The database connection details can be overridden by the @@ -68,11 +68,8 @@ impl Postgres { pub async fn connect(file_name: impl Into) -> Result { let file_name = file_name.into(); - let dsn = if let Ok(val) = std::env::var("PG_DSN") { - val - } else { - PG_DSN.to_string() - }; + let dsn = + std::env::var("PG_DSN").map_or(PG_DSN.to_string(), std::convert::identity); debug!("Using posgres dsn: {dsn}"); From c41a0a8dfbc0f8bc9f5067cb6d9c7955f49eb8b3 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 22 Jan 2023 12:12:14 -0500 Subject: [PATCH 05/10] Remove retries --- .../sqllogictests/src/engines/postgres/mod.rs | 48 ++++++++----------- 1 file changed, 20 insertions(+), 28 deletions(-) diff --git a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs b/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs index 898e04a109f8e..a53a00c93e46d 100644 --- a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs +++ b/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use std::{str::FromStr, time::Duration}; +use std::str::FromStr; use async_trait::async_trait; use bytes::Bytes; @@ -68,40 +68,32 @@ impl Postgres { pub async fn connect(file_name: impl Into) -> Result { let file_name = file_name.into(); - let dsn = + let dsn = std::env::var("PG_DSN").map_or(PG_DSN.to_string(), std::convert::identity); debug!("Using posgres dsn: {dsn}"); let config = tokio_postgres::Config::from_str(&dsn)?; - let mut retry = 0; - - loop { - let connection_result = config.connect(tokio_postgres::NoTls).await; - - match connection_result { - Err(e) if retry <= 3 => { - debug!("Retrying connection error '{:?}'", e); - retry += 1; - tokio::time::sleep(Duration::from_secs(1)).await; - } - Err(e) => return Err(Error::from(e)), - Ok((client, connection)) => { - let join_handle = tokio::spawn(async move { - if let Err(e) = connection.await { - log::error!("Postgres connection error: {:?}", e); - } - }); - - return Ok(Self { - client, - join_handle, - file_name, - }); - } + // hint to user what the connection string was + let res = config.connect(tokio_postgres::NoTls).await; + if let Err(_) = &res { + eprintln!("Error connecting to posgres using PG_DSN={dsn}"); + }; + + let (client, connection) = res?; + + let join_handle = tokio::spawn(async move { + if let Err(e) = connection.await { + log::error!("Postgres connection error: {:?}", e); } - } + }); + + Ok(Self { + client, + join_handle, + file_name, + }) } /// Special COPY command support. "COPY 'filename'" requires the From 99a2efdb3df097c9e1caa11657eaf7b3784a5920 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 22 Jan 2023 12:20:22 -0500 Subject: [PATCH 06/10] rename PG_DSN --> PG_URI --- .github/workflows/rust.yml | 2 +- datafusion/core/tests/sqllogictests/README.md | 4 ++-- .../sqllogictests/src/engines/postgres/mod.rs | 18 +++++++++--------- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index aa693348ed17c..cbc5418867dd0 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -255,7 +255,7 @@ jobs: rustup toolchain install stable rustup default stable - name: Run sqllogictest - run: PG_COMPAT=true PG_DSN="postgresql://postgres:postgres@localhost:$POSTGRES_PORT/db_test" cargo test -p datafusion --test sqllogictests + run: PG_COMPAT=true PG_URI="postgresql://postgres:postgres@localhost:$POSTGRES_PORT/db_test" cargo test -p datafusion --test sqllogictests env: POSTGRES_PORT: ${{ job.services.postgres.ports[5432] }} diff --git a/datafusion/core/tests/sqllogictests/README.md b/datafusion/core/tests/sqllogictests/README.md index c66bf06966d0e..e3cfbde2e615a 100644 --- a/datafusion/core/tests/sqllogictests/README.md +++ b/datafusion/core/tests/sqllogictests/README.md @@ -52,13 +52,13 @@ with Postgres by running the same script files both with DataFusion and with Pos In order to run the sqllogictests running against a previously running Postgres instance, do: ```shell -PG_COMPAT=true PG_DSN="postgresql://postgres@127.0.0.1/postgres" cargo test -p datafusion --test sqllogictests +PG_COMPAT=true PG_URI="postgresql://postgres@127.0.0.1/postgres" cargo test -p datafusion --test sqllogictests ``` The environemnt variables: 1. `PG_COMPAT` instructs sqllogictest to run against Postgres (not DataFusion) -2. `PG_DSN` contains a `libpq` style connection string, whose format is described in +2. `PG_URI` contains a `libpq` style connection string, whose format is described in [the docs](https://docs.rs/tokio-postgres/latest/tokio_postgres/config/struct.Config.html#url) One way to create a suitable a posgres container in docker is to use diff --git a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs b/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs index a53a00c93e46d..1c045c55c7a04 100644 --- a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs +++ b/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs @@ -30,8 +30,8 @@ use postgres_types::Type; use rust_decimal::Decimal; use tokio_postgres::{Column, Row}; -// default connect string, can be overridden by environment PG_DSN -const PG_DSN: &str = "postgresql://postgres@127.0.0.1/test"; +// default connect string, can be overridden by the `PG_URL` environment variable +const PG_URI: &str = "postgresql://postgres@127.0.0.1/test"; /// DataFusion sql-logicaltest error #[derive(Debug, thiserror::Error)] @@ -56,29 +56,29 @@ impl Postgres { /// posgres connection. `file_name` is used for display output /// /// The database connection details can be overridden by the - /// `PG_DSN` environment variable. + /// `PG_URI` environment variable. /// /// This defaults to /// /// ```text - /// PG_DSN="postgresql://postgres@127.0.0.1/test" + /// PG_URI="postgresql://postgres@127.0.0.1/test" /// ``` /// /// See https://docs.rs/tokio-postgres/latest/tokio_postgres/config/struct.Config.html#url for format pub async fn connect(file_name: impl Into) -> Result { let file_name = file_name.into(); - let dsn = - std::env::var("PG_DSN").map_or(PG_DSN.to_string(), std::convert::identity); + let uri = + std::env::var("PG_URI").map_or(PG_URI.to_string(), std::convert::identity); - debug!("Using posgres dsn: {dsn}"); + debug!("Using posgres connection string: {uri}"); - let config = tokio_postgres::Config::from_str(&dsn)?; + let config = tokio_postgres::Config::from_str(&uri)?; // hint to user what the connection string was let res = config.connect(tokio_postgres::NoTls).await; if let Err(_) = &res { - eprintln!("Error connecting to posgres using PG_DSN={dsn}"); + eprintln!("Error connecting to posgres using PG_URI={dsn}"); }; let (client, connection) = res?; From 649727bcf242a05804cd909c1d10e907498e947a Mon Sep 17 00:00:00 2001 From: Yevhenii Melnyk Date: Sat, 21 Jan 2023 21:55:13 +0100 Subject: [PATCH 07/10] Schema per sqllogictest file --- .../sqllogictests/src/engines/postgres/mod.rs | 28 ++++++++++++++++++- .../test_files/pg_compat_simple.slt | 4 --- .../test_files/pg_compat_union.slt | 4 --- .../test_files/pg_compat_window.slt | 4 --- 4 files changed, 27 insertions(+), 13 deletions(-) diff --git a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs b/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs index 1c045c55c7a04..3db3b80f9d86d 100644 --- a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs +++ b/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs @@ -78,7 +78,7 @@ impl Postgres { // hint to user what the connection string was let res = config.connect(tokio_postgres::NoTls).await; if let Err(_) = &res { - eprintln!("Error connecting to posgres using PG_URI={dsn}"); + eprintln!("Error connecting to posgres using PG_URI={uri}"); }; let (client, connection) = res?; @@ -89,6 +89,22 @@ impl Postgres { } }); + let schema = schema_name(&file_name); + + // create a new clean schema for running the test + debug!("Creating new empty schema '{schema}'"); + client + .execute(&format!("DROP SCHEMA IF EXISTS {} CASCADE", schema), &[]) + .await?; + + client + .execute(&format!("CREATE SCHEMA {}", schema), &[]) + .await?; + + client + .execute(&format!("SET search_path TO {}", schema), &[]) + .await?; + Ok(Self { client, join_handle, @@ -170,6 +186,16 @@ fn no_quotes(t: &str) -> &str { t.trim_start_matches('\'').trim_end_matches('\'') } +/// 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_") +} + impl Drop for Postgres { fn drop(&mut self) { self.join_handle.abort() diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_simple.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat_simple.slt index 17e274e0851fa..a35c559dc8393 100644 --- a/datafusion/core/tests/sqllogictests/test_files/pg_compat_simple.slt +++ b/datafusion/core/tests/sqllogictests/test_files/pg_compat_simple.slt @@ -19,10 +19,6 @@ ## Setup test for postgres ### -onlyif postgres -statement ok -DROP TABLE IF EXISTS aggregate_test_100_by_sql; - onlyif postgres statement ok CREATE TABLE aggregate_test_100_by_sql diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_union.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat_union.slt index 8ad875be57a55..978336a99f2cc 100644 --- a/datafusion/core/tests/sqllogictests/test_files/pg_compat_union.slt +++ b/datafusion/core/tests/sqllogictests/test_files/pg_compat_union.slt @@ -16,10 +16,6 @@ # under the License. -onlyif postgres -statement ok -DROP TABLE IF EXISTS aggregate_test_100_by_sql; - onlyif postgres statement ok CREATE TABLE aggregate_test_100_by_sql diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_window.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat_window.slt index adfb9ad6ee6b7..89136764abf0b 100644 --- a/datafusion/core/tests/sqllogictests/test_files/pg_compat_window.slt +++ b/datafusion/core/tests/sqllogictests/test_files/pg_compat_window.slt @@ -16,10 +16,6 @@ # under the License. -onlyif postgres -statement ok -DROP TABLE IF EXISTS aggregate_test_100_by_sql; - onlyif postgres statement ok CREATE TABLE aggregate_test_100_by_sql From 437df59b62c2c43719a0438a8f3d9035f24f6abb Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 22 Jan 2023 16:05:26 -0500 Subject: [PATCH 08/10] clean up --- .../tests/sqllogictests/test_files/pg_compat_simple.slt | 6 ++++++ .../tests/sqllogictests/test_files/pg_compat_types.slt | 7 +++++++ .../tests/sqllogictests/test_files/pg_compat_union.slt | 6 ++++++ .../tests/sqllogictests/test_files/pg_compat_window.slt | 6 ++++++ 4 files changed, 25 insertions(+) diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_simple.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat_simple.slt index a35c559dc8393..68194fcec73ce 100644 --- a/datafusion/core/tests/sqllogictests/test_files/pg_compat_simple.slt +++ b/datafusion/core/tests/sqllogictests/test_files/pg_compat_simple.slt @@ -571,3 +571,9 @@ e 4 96 -30336 427197269 7506304308750926996 95 48483 3521368277 5437030162957481 e 4 97 -13181 2047637360 6176835796788944083 158 53000 2042457019 9726016502640071617 0.7085086 0.123575399884 oHJMNvWuunsIMIWFnYG31RCfkOo2V7 e 5 -86 32514 -467659022 -8012578250188146150 254 2684 2861911482 2126626171973341689 0.12559289 0.014793053078 gxfHWUF8XgY2KdFxigxvNEXe2V2XMl e 5 64 -26526 1689098844 8950618259486183091 224 45253 662099130 16127995415060805595 0.2897315 0.575945048386 56MZa5O1hVtX4c5sbnCfxuX5kDChqI + +######## +# Clean up after the test +######## +statement ok +DROP TABLE aggregate_test_100_by_sql diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_types.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat_types.slt index ebe4df92ac73e..53857905580eb 100644 --- a/datafusion/core/tests/sqllogictests/test_files/pg_compat_types.slt +++ b/datafusion/core/tests/sqllogictests/test_files/pg_compat_types.slt @@ -120,3 +120,10 @@ query I select true::BOOLEAN, 'false'::BOOLEAN ---- true false + + +######## +# Clean up after the test +######## +statement ok +DROP TABLE aggregate_test_100_by_sql diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_union.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat_union.slt index 978336a99f2cc..31d2c3a0cfa59 100644 --- a/datafusion/core/tests/sqllogictests/test_files/pg_compat_union.slt +++ b/datafusion/core/tests/sqllogictests/test_files/pg_compat_union.slt @@ -228,3 +228,9 @@ SELECT 1 num UNION ALL SELECT 2 num ORDER BY num; ---- 1 2 + +######## +# Clean up after the test +######## +statement ok +DROP TABLE aggregate_test_100_by_sql diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_window.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat_window.slt index 89136764abf0b..36b0fac14ffe0 100644 --- a/datafusion/core/tests/sqllogictests/test_files/pg_compat_window.slt +++ b/datafusion/core/tests/sqllogictests/test_files/pg_compat_window.slt @@ -1482,3 +1482,9 @@ NULL 7 279 282 NULL 5 4 86 90 NULL 1 5 245 246 NULL 4 6 18 21 NULL + +######## +# Clean up after the test +######## +statement ok +DROP TABLE aggregate_test_100_by_sql From 2662509b3f0c13aeb8a3c7fb24d3523cc1b06385 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 23 Jan 2023 06:06:01 -0500 Subject: [PATCH 09/10] clippy --- .../core/tests/sqllogictests/src/engines/postgres/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs b/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs index 3db3b80f9d86d..dab5b4d38598d 100644 --- a/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs +++ b/datafusion/core/tests/sqllogictests/src/engines/postgres/mod.rs @@ -77,7 +77,7 @@ impl Postgres { // hint to user what the connection string was let res = config.connect(tokio_postgres::NoTls).await; - if let Err(_) = &res { + if res.is_err() { eprintln!("Error connecting to posgres using PG_URI={uri}"); }; @@ -190,7 +190,7 @@ fn no_quotes(t: &str) -> &str { /// return a schema name fn schema_name(file_name: &str) -> &str { file_name - .split(".") + .split('.') .next() .unwrap_or("default_schema") .trim_start_matches("pg_") From 797f6059fc51ad417c77e810c96c3382c609ff70 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 23 Jan 2023 06:38:53 -0500 Subject: [PATCH 10/10] remove overly aggressive cleanup --- .../tests/sqllogictests/test_files/pg_compat_types.slt | 7 ------- 1 file changed, 7 deletions(-) diff --git a/datafusion/core/tests/sqllogictests/test_files/pg_compat_types.slt b/datafusion/core/tests/sqllogictests/test_files/pg_compat_types.slt index 53857905580eb..ebe4df92ac73e 100644 --- a/datafusion/core/tests/sqllogictests/test_files/pg_compat_types.slt +++ b/datafusion/core/tests/sqllogictests/test_files/pg_compat_types.slt @@ -120,10 +120,3 @@ query I select true::BOOLEAN, 'false'::BOOLEAN ---- true false - - -######## -# Clean up after the test -######## -statement ok -DROP TABLE aggregate_test_100_by_sql