diff --git a/datafusion/core/tests/sql/information_schema.rs b/datafusion/core/tests/sql/information_schema.rs deleted file mode 100644 index 1cb518099174..000000000000 --- a/datafusion/core/tests/sql/information_schema.rs +++ /dev/null @@ -1,220 +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 async_trait::async_trait; -use datafusion::execution::context::SessionState; -use datafusion::{ - catalog::{ - schema::{MemorySchemaProvider, SchemaProvider}, - CatalogProvider, MemoryCatalogProvider, - }, - datasource::{TableProvider, TableType}, -}; -use datafusion_expr::Expr; - -use super::*; - -#[tokio::test] -async fn information_schema_tables_tables_with_multiple_catalogs() { - let ctx = - SessionContext::with_config(SessionConfig::new().with_information_schema(true)); - let catalog = MemoryCatalogProvider::new(); - let schema = MemorySchemaProvider::new(); - schema - .register_table("t1".to_owned(), table_with_sequence(1, 1).unwrap()) - .unwrap(); - schema - .register_table("t2".to_owned(), table_with_sequence(1, 1).unwrap()) - .unwrap(); - catalog - .register_schema("my_schema", Arc::new(schema)) - .unwrap(); - ctx.register_catalog("my_catalog", Arc::new(catalog)); - - let catalog = MemoryCatalogProvider::new(); - let schema = MemorySchemaProvider::new(); - schema - .register_table("t3".to_owned(), table_with_sequence(1, 1).unwrap()) - .unwrap(); - catalog - .register_schema("my_other_schema", Arc::new(schema)) - .unwrap(); - ctx.register_catalog("my_other_catalog", Arc::new(catalog)); - - let result = plan_and_collect(&ctx, "SELECT * from information_schema.tables") - .await - .unwrap(); - - let expected = vec![ - "+------------------+--------------------+-------------+------------+", - "| table_catalog | table_schema | table_name | table_type |", - "+------------------+--------------------+-------------+------------+", - "| datafusion | information_schema | columns | VIEW |", - "| datafusion | information_schema | df_settings | VIEW |", - "| datafusion | information_schema | tables | VIEW |", - "| datafusion | information_schema | views | VIEW |", - "| my_catalog | information_schema | columns | VIEW |", - "| my_catalog | information_schema | df_settings | VIEW |", - "| my_catalog | information_schema | tables | VIEW |", - "| my_catalog | information_schema | views | VIEW |", - "| my_catalog | my_schema | t1 | BASE TABLE |", - "| my_catalog | my_schema | t2 | BASE TABLE |", - "| my_other_catalog | information_schema | columns | VIEW |", - "| my_other_catalog | information_schema | df_settings | VIEW |", - "| my_other_catalog | information_schema | tables | VIEW |", - "| my_other_catalog | information_schema | views | VIEW |", - "| my_other_catalog | my_other_schema | t3 | BASE TABLE |", - "+------------------+--------------------+-------------+------------+", - ]; - assert_batches_sorted_eq!(expected, &result); -} - -#[tokio::test] -async fn information_schema_tables_table_types() { - struct TestTable(TableType); - - #[async_trait] - impl TableProvider for TestTable { - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn table_type(&self) -> TableType { - self.0 - } - - fn schema(&self) -> SchemaRef { - unimplemented!() - } - - async fn scan( - &self, - _state: &SessionState, - _: Option<&Vec>, - _: &[Expr], - _: Option, - ) -> Result> { - unimplemented!() - } - } - - let ctx = - SessionContext::with_config(SessionConfig::new().with_information_schema(true)); - - ctx.register_table("physical", Arc::new(TestTable(TableType::Base))) - .unwrap(); - ctx.register_table("query", Arc::new(TestTable(TableType::View))) - .unwrap(); - ctx.register_table("temp", Arc::new(TestTable(TableType::Temporary))) - .unwrap(); - - let result = plan_and_collect(&ctx, "SELECT * from information_schema.tables") - .await - .unwrap(); - - let expected = vec![ - "+---------------+--------------------+-------------+-----------------+", - "| table_catalog | table_schema | table_name | table_type |", - "+---------------+--------------------+-------------+-----------------+", - "| datafusion | information_schema | columns | VIEW |", - "| datafusion | information_schema | df_settings | VIEW |", - "| datafusion | information_schema | tables | VIEW |", - "| datafusion | information_schema | views | VIEW |", - "| datafusion | public | physical | BASE TABLE |", - "| datafusion | public | query | VIEW |", - "| datafusion | public | temp | LOCAL TEMPORARY |", - "+---------------+--------------------+-------------+-----------------+", - ]; - assert_batches_sorted_eq!(expected, &result); -} - -fn table_with_many_types() -> Arc { - let schema = Schema::new(vec![ - Field::new("int32_col", DataType::Int32, false), - Field::new("float64_col", DataType::Float64, true), - Field::new("utf8_col", DataType::Utf8, true), - Field::new("large_utf8_col", DataType::LargeUtf8, false), - Field::new("binary_col", DataType::Binary, false), - Field::new("large_binary_col", DataType::LargeBinary, false), - Field::new( - "timestamp_nanos", - DataType::Timestamp(TimeUnit::Nanosecond, None), - false, - ), - ]); - - let batch = RecordBatch::try_new( - Arc::new(schema.clone()), - vec![ - Arc::new(Int32Array::from(vec![1])), - Arc::new(Float64Array::from(vec![1.0])), - Arc::new(StringArray::from(vec![Some("foo")])), - Arc::new(LargeStringArray::from(vec![Some("bar")])), - Arc::new(BinaryArray::from(vec![b"foo" as &[u8]])), - Arc::new(LargeBinaryArray::from(vec![b"foo" as &[u8]])), - Arc::new(TimestampNanosecondArray::from(vec![Some(123)])), - ], - ) - .unwrap(); - let provider = MemTable::try_new(Arc::new(schema), vec![vec![batch]]).unwrap(); - Arc::new(provider) -} - -#[tokio::test] -async fn information_schema_columns() { - let ctx = - SessionContext::with_config(SessionConfig::new().with_information_schema(true)); - let catalog = MemoryCatalogProvider::new(); - let schema = MemorySchemaProvider::new(); - - schema - .register_table("t1".to_owned(), table_with_sequence(1, 1).unwrap()) - .unwrap(); - - schema - .register_table("t2".to_owned(), table_with_many_types()) - .unwrap(); - catalog - .register_schema("my_schema", Arc::new(schema)) - .unwrap(); - ctx.register_catalog("my_catalog", Arc::new(catalog)); - - let result = plan_and_collect(&ctx, "SELECT * from information_schema.columns") - .await - .unwrap(); - - let expected = vec![ - "+---------------+--------------+------------+------------------+------------------+----------------+-------------+-----------------------------+--------------------------+------------------------+-------------------+-------------------------+---------------+--------------------+---------------+", - "| table_catalog | table_schema | table_name | column_name | ordinal_position | column_default | is_nullable | data_type | character_maximum_length | character_octet_length | numeric_precision | numeric_precision_radix | numeric_scale | datetime_precision | interval_type |", - "+---------------+--------------+------------+------------------+------------------+----------------+-------------+-----------------------------+--------------------------+------------------------+-------------------+-------------------------+---------------+--------------------+---------------+", - "| my_catalog | my_schema | t1 | i | 0 | | YES | Int32 | | | 32 | 2 | | | |", - "| my_catalog | my_schema | t2 | binary_col | 4 | | NO | Binary | | 2147483647 | | | | | |", - "| my_catalog | my_schema | t2 | float64_col | 1 | | YES | Float64 | | | 24 | 2 | | | |", - "| my_catalog | my_schema | t2 | int32_col | 0 | | NO | Int32 | | | 32 | 2 | | | |", - "| my_catalog | my_schema | t2 | large_binary_col | 5 | | NO | LargeBinary | | 9223372036854775807 | | | | | |", - "| my_catalog | my_schema | t2 | large_utf8_col | 3 | | NO | LargeUtf8 | | 9223372036854775807 | | | | | |", - "| my_catalog | my_schema | t2 | timestamp_nanos | 6 | | NO | Timestamp(Nanosecond, None) | | | | | | | |", - "| my_catalog | my_schema | t2 | utf8_col | 2 | | YES | Utf8 | | 2147483647 | | | | | |", - "+---------------+--------------+------------+------------------+------------------+----------------+-------------+-----------------------------+--------------------------+------------------------+-------------------+-------------------------+---------------+--------------------+---------------+", - ]; - assert_batches_sorted_eq!(expected, &result); -} - -/// Execute SQL and return results -async fn plan_and_collect(ctx: &SessionContext, sql: &str) -> Result> { - ctx.sql(sql).await?.collect().await -} diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index f1ab0ccea9e8..85a806428548 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -84,7 +84,6 @@ pub mod csv_files; pub mod explain_analyze; pub mod expr; pub mod group_by; -pub mod information_schema; pub mod joins; pub mod limit; pub mod order; diff --git a/datafusion/core/tests/sqllogictests/src/main.rs b/datafusion/core/tests/sqllogictests/src/main.rs index 58089be24509..c74d1cb11a47 100644 --- a/datafusion/core/tests/sqllogictests/src/main.rs +++ b/datafusion/core/tests/sqllogictests/src/main.rs @@ -271,6 +271,14 @@ async fn context_for_test_file(relative_path: &Path) -> Option { info!("Registering scalar tables"); setup::register_scalar_tables(test_ctx.session_ctx()).await; } + "information_schema_table_types.slt" => { + info!("Registering local temporary table"); + setup::register_temp_table(test_ctx.session_ctx()).await; + } + "information_schema_columns.slt" => { + info!("Registering table with many types"); + setup::register_table_with_many_types(test_ctx.session_ctx()).await; + } "avro.slt" => { #[cfg(feature = "avro")] { diff --git a/datafusion/core/tests/sqllogictests/src/setup.rs b/datafusion/core/tests/sqllogictests/src/setup.rs index 34365f509a53..32569c7575ce 100644 --- a/datafusion/core/tests/sqllogictests/src/setup.rs +++ b/datafusion/core/tests/sqllogictests/src/setup.rs @@ -15,14 +15,25 @@ // specific language governing permissions and limitations // under the License. +use arrow_schema::TimeUnit; +use async_trait::async_trait; +use datafusion::execution::context::SessionState; +use datafusion::physical_plan::ExecutionPlan; use datafusion::{ arrow::{ - array::Float64Array, - datatypes::{DataType, Field, Schema}, + array::{ + BinaryArray, Float64Array, Int32Array, LargeBinaryArray, LargeStringArray, + StringArray, TimestampNanosecondArray, + }, + datatypes::{DataType, Field, Schema, SchemaRef}, record_batch::RecordBatch, }, + catalog::{schema::MemorySchemaProvider, CatalogProvider, MemoryCatalogProvider}, + datasource::{MemTable, TableProvider, TableType}, prelude::{CsvReadOptions, SessionContext}, }; +use datafusion_common::DataFusionError; +use datafusion_expr::Expr; use std::fs::File; use std::io::Write; use std::sync::Arc; @@ -116,3 +127,84 @@ pub async fn register_partition_table(test_ctx: &mut TestContext) { .await .unwrap(); } + +// registers a LOCAL TEMPORARY table. +pub async fn register_temp_table(ctx: &SessionContext) { + struct TestTable(TableType); + + #[async_trait] + impl TableProvider for TestTable { + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn table_type(&self) -> TableType { + self.0 + } + + fn schema(&self) -> SchemaRef { + unimplemented!() + } + + async fn scan( + &self, + _state: &SessionState, + _: Option<&Vec>, + _: &[Expr], + _: Option, + ) -> Result, DataFusionError> { + unimplemented!() + } + } + + ctx.register_table( + "datafusion.public.temp", + Arc::new(TestTable(TableType::Temporary)), + ) + .unwrap(); +} + +pub async fn register_table_with_many_types(ctx: &SessionContext) { + let catalog = MemoryCatalogProvider::new(); + let schema = MemorySchemaProvider::new(); + + catalog + .register_schema("my_schema", Arc::new(schema)) + .unwrap(); + ctx.register_catalog("my_catalog", Arc::new(catalog)); + + ctx.register_table("my_catalog.my_schema.t2", table_with_many_types()) + .unwrap(); +} + +fn table_with_many_types() -> Arc { + let schema = Schema::new(vec![ + Field::new("int32_col", DataType::Int32, false), + Field::new("float64_col", DataType::Float64, true), + Field::new("utf8_col", DataType::Utf8, true), + Field::new("large_utf8_col", DataType::LargeUtf8, false), + Field::new("binary_col", DataType::Binary, false), + Field::new("large_binary_col", DataType::LargeBinary, false), + Field::new( + "timestamp_nanos", + DataType::Timestamp(TimeUnit::Nanosecond, None), + false, + ), + ]); + + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![ + Arc::new(Int32Array::from(vec![1])), + Arc::new(Float64Array::from(vec![1.0])), + Arc::new(StringArray::from(vec![Some("foo")])), + Arc::new(LargeStringArray::from(vec![Some("bar")])), + Arc::new(BinaryArray::from(vec![b"foo" as &[u8]])), + Arc::new(LargeBinaryArray::from(vec![b"foo" as &[u8]])), + Arc::new(TimestampNanosecondArray::from(vec![Some(123)])), + ], + ) + .unwrap(); + let provider = MemTable::try_new(Arc::new(schema), vec![vec![batch]]).unwrap(); + Arc::new(provider) +} diff --git a/datafusion/core/tests/sqllogictests/test_files/information_schema_columns.slt b/datafusion/core/tests/sqllogictests/test_files/information_schema_columns.slt new file mode 100644 index 000000000000..fcb653cedd16 --- /dev/null +++ b/datafusion/core/tests/sqllogictests/test_files/information_schema_columns.slt @@ -0,0 +1,53 @@ +# 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. + +statement ok +set datafusion.catalog.information_schema = true; + +statement ok +set datafusion.catalog.default_catalog = my_catalog; + +statement ok +set datafusion.catalog.default_schema = my_schema; + +########### +# Information schema columns +########### + +statement ok +CREATE TABLE t1 (i int) as values(1); + +# table t2 is created using rust code because it is not possible to set nullable columns with `arrow_cast` syntax + +query TTTTITTTIIIIIIT rowsort +SELECT * from information_schema.columns; +---- +my_catalog my_schema t1 i 0 NULL YES Int32 NULL NULL 32 2 NULL NULL NULL +my_catalog my_schema t2 binary_col 4 NULL NO Binary NULL 2147483647 NULL NULL NULL NULL NULL +my_catalog my_schema t2 float64_col 1 NULL YES Float64 NULL NULL 24 2 NULL NULL NULL +my_catalog my_schema t2 int32_col 0 NULL NO Int32 NULL NULL 32 2 NULL NULL NULL +my_catalog my_schema t2 large_binary_col 5 NULL NO LargeBinary NULL 9223372036854775807 NULL NULL NULL NULL NULL +my_catalog my_schema t2 large_utf8_col 3 NULL NO LargeUtf8 NULL 9223372036854775807 NULL NULL NULL NULL NULL +my_catalog my_schema t2 timestamp_nanos 6 NULL NO Timestamp(Nanosecond, None) NULL NULL NULL NULL NULL NULL NULL +my_catalog my_schema t2 utf8_col 2 NULL YES Utf8 NULL 2147483647 NULL NULL NULL NULL NULL + +# Cleanup +statement ok +drop table t1 + +statement ok +drop table t2 \ No newline at end of file diff --git a/datafusion/core/tests/sqllogictests/test_files/information_schema_multiple_catalogs.slt b/datafusion/core/tests/sqllogictests/test_files/information_schema_multiple_catalogs.slt new file mode 100644 index 000000000000..c7f4dcfd54d8 --- /dev/null +++ b/datafusion/core/tests/sqllogictests/test_files/information_schema_multiple_catalogs.slt @@ -0,0 +1,111 @@ +# 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. + + +# Verify the information schema does not exit by default +statement error DataFusion error: Error during planning: table 'datafusion.information_schema.tables' not found +SELECT * from information_schema.tables + +statement error DataFusion error: Error during planning: SHOW \[VARIABLE\] is not supported unless information_schema is enabled +show all + +# Turn it on + +# expect that the queries now work +statement ok +set datafusion.catalog.information_schema = true; + +# Verify the information schema now does exist and is empty +query TTTT rowsort +SELECT * from information_schema.tables; +---- +datafusion information_schema columns VIEW +datafusion information_schema df_settings VIEW +datafusion information_schema tables VIEW +datafusion information_schema views VIEW + +############ +# Create multiple catalogs +########### +statement ok +create database my_catalog; + +statement ok +create schema my_catalog.my_schema; + +statement ok +set datafusion.catalog.default_catalog = my_catalog; + +statement ok +set datafusion.catalog.default_schema = my_schema; + +statement ok +create table t1 as values(1); + +statement ok +create table t2 as values(1); + +statement ok +create database my_other_catalog; + +statement ok +create schema my_other_catalog.my_other_schema; + +statement ok +set datafusion.catalog.default_catalog = my_other_catalog; + +statement ok +set datafusion.catalog.default_schema = my_other_schema; + +statement ok +create table t3 as values(1); + +query TTTT rowsort +SELECT * from information_schema.tables; +---- +datafusion information_schema columns VIEW +datafusion information_schema df_settings VIEW +datafusion information_schema tables VIEW +datafusion information_schema views VIEW +my_catalog information_schema columns VIEW +my_catalog information_schema df_settings VIEW +my_catalog information_schema tables VIEW +my_catalog information_schema views VIEW +my_catalog my_schema t1 BASE TABLE +my_catalog my_schema t2 BASE TABLE +my_other_catalog information_schema columns VIEW +my_other_catalog information_schema df_settings VIEW +my_other_catalog information_schema tables VIEW +my_other_catalog information_schema views VIEW +my_other_catalog my_other_schema t3 BASE TABLE + +# Cleanup + +statement ok +drop table t3 + +statement ok +set datafusion.catalog.default_catalog = my_catalog; + +statement ok +set datafusion.catalog.default_schema = my_schema; + +statement ok +drop table t1 + +statement ok +drop table t2 diff --git a/datafusion/core/tests/sqllogictests/test_files/information_schema_table_types.slt b/datafusion/core/tests/sqllogictests/test_files/information_schema_table_types.slt new file mode 100644 index 000000000000..eb72f3399fe7 --- /dev/null +++ b/datafusion/core/tests/sqllogictests/test_files/information_schema_table_types.slt @@ -0,0 +1,51 @@ +# 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. + +# Turn it on +statement ok +set datafusion.catalog.information_schema = true; + +############ +# Table with many types +############ + +statement ok +create table physical as values(1); + +statement ok +create view query as values(1); + +# Temporary tables cannot be created using SQL syntax so it is done using Rust code. + +query TTTT rowsort +SELECT * from information_schema.tables; +---- +datafusion information_schema columns VIEW +datafusion information_schema df_settings VIEW +datafusion information_schema tables VIEW +datafusion information_schema views VIEW +datafusion public physical BASE TABLE +datafusion public query VIEW +datafusion public temp LOCAL TEMPORARY + +# Cleanup + +statement ok +drop table physical + +statement ok +drop view query