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

[arrow-datafusion] branch master updated: [Feature] support describe file (#4995)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new ab00bc118 [Feature] support describe file (#4995)
ab00bc118 is described below

commit ab00bc11835f98dd06fa1262d23db2ce1e53a154
Author: xyz <a9...@gmail.com>
AuthorDate: Tue Jan 24 20:21:18 2023 +0800

    [Feature] support describe file (#4995)
    
    To know the columns/types in a file, users has to create an external table, and describe the table.
    Sometimes the infer schema is wrong for creating table, to make it right, user need to drop the table, and recreate a table with the specify schema.
    To solve this problem, we add describe file interface in datafusion-clie, With the Describe File, user can know the infer schema is wrong before creating the table.
    
    Syntax:
    Describe file_path,
    
    Example:
    DESCRIBE 'tests/data/aggregate_simple_pipe.csv';
    
    Return:
    column_name data_type is_nullable
    c1 Float32 NO
    c2 Float64 NO
    c3 Boolean NO
    
    Signed-off-by: xyz <a9...@gmail.com>
    
    Signed-off-by: xyz <a9...@gmail.com>
---
 datafusion-cli/src/command.rs                      | 10 ++--
 datafusion/core/src/execution/context.rs           | 60 +++++++++++++++++++-
 datafusion/core/src/physical_plan/planner.rs       |  5 ++
 .../tests/sqllogictests/test_files/describe.slt    | 64 ++++++++++++++++++++++
 datafusion/expr/src/lib.rs                         | 11 ++--
 datafusion/expr/src/logical_plan/mod.rs            | 10 ++--
 datafusion/expr/src/logical_plan/plan.rs           | 25 ++++++++-
 datafusion/expr/src/utils.rs                       |  1 +
 .../optimizer/src/common_subexpr_eliminate.rs      |  1 +
 datafusion/optimizer/src/push_down_projection.rs   |  1 +
 datafusion/proto/src/logical_plan/mod.rs           |  5 +-
 datafusion/sql/src/parser.rs                       |  8 ++-
 datafusion/sql/src/statement.rs                    | 39 ++++++-------
 13 files changed, 193 insertions(+), 47 deletions(-)

diff --git a/datafusion-cli/src/command.rs b/datafusion-cli/src/command.rs
index 0cef24519..5563c31bc 100644
--- a/datafusion-cli/src/command.rs
+++ b/datafusion-cli/src/command.rs
@@ -39,7 +39,7 @@ pub enum Command {
     Quit,
     Help,
     ListTables,
-    DescribeTable(String),
+    DescribeTableStmt(String),
     ListFunctions,
     Include(Option<String>),
     SearchFunctions(String),
@@ -65,7 +65,7 @@ impl Command {
                 let batches = df.collect().await?;
                 print_options.print_batches(&batches, now)
             }
-            Self::DescribeTable(name) => {
+            Self::DescribeTableStmt(name) => {
                 let df = ctx.sql(&format!("SHOW COLUMNS FROM {}", name)).await?;
                 let batches = df.collect().await?;
                 print_options.print_batches(&batches, now)
@@ -125,7 +125,7 @@ impl Command {
         match self {
             Self::Quit => ("\\q", "quit datafusion-cli"),
             Self::ListTables => ("\\d", "list tables"),
-            Self::DescribeTable(_) => ("\\d name", "describe table"),
+            Self::DescribeTableStmt(_) => ("\\d name", "describe table"),
             Self::Help => ("\\?", "help"),
             Self::Include(_) => {
                 ("\\i filename", "reads input from the specified filename")
@@ -142,7 +142,7 @@ impl Command {
 
 const ALL_COMMANDS: [Command; 9] = [
     Command::ListTables,
-    Command::DescribeTable(String::new()),
+    Command::DescribeTableStmt(String::new()),
     Command::Quit,
     Command::Help,
     Command::Include(Some(String::new())),
@@ -183,7 +183,7 @@ impl FromStr for Command {
         Ok(match (c, arg) {
             ("q", None) => Self::Quit,
             ("d", None) => Self::ListTables,
-            ("d", Some(name)) => Self::DescribeTable(name.into()),
+            ("d", Some(name)) => Self::DescribeTableStmt(name.into()),
             ("?", None) => Self::Help,
             ("h", None) => Self::ListFunctions,
             ("h", Some(function)) => Self::SearchFunctions(function.into()),
diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs
index ec8fbd83a..1df9bf111 100644
--- a/datafusion/core/src/execution/context.rs
+++ b/datafusion/core/src/execution/context.rs
@@ -27,6 +27,7 @@ use crate::{
         optimizer::PhysicalOptimizerRule,
     },
 };
+use datafusion_expr::DescribeTable;
 pub use datafusion_physical_expr::execution_props::ExecutionProps;
 use datafusion_physical_expr::var_provider::is_system_variables;
 use parking_lot::RwLock;
@@ -42,8 +43,11 @@ use std::{
 };
 use std::{ops::ControlFlow, sync::Weak};
 
-use arrow::datatypes::{DataType, SchemaRef};
 use arrow::record_batch::RecordBatch;
+use arrow::{
+    array::StringBuilder,
+    datatypes::{DataType, Field, Schema, SchemaRef},
+};
 
 use crate::catalog::{
     catalog::{CatalogProvider, MemoryCatalogProvider},
@@ -360,6 +364,10 @@ impl SessionContext {
                 self.return_empty_dataframe()
             }
 
+            LogicalPlan::DescribeTable(DescribeTable { schema, .. }) => {
+                self.return_describe_table_dataframe(schema).await
+            }
+
             LogicalPlan::CreateCatalogSchema(CreateCatalogSchema {
                 schema_name,
                 if_not_exists,
@@ -442,6 +450,53 @@ impl SessionContext {
         Ok(DataFrame::new(self.state(), plan))
     }
 
+    // return an record_batch which describe table
+    async fn return_describe_table_record_batch(
+        &self,
+        schema: Arc<Schema>,
+    ) -> Result<RecordBatch> {
+        let record_batch_schema = Arc::new(Schema::new(vec![
+            Field::new("column_name", DataType::Utf8, false),
+            Field::new("data_type", DataType::Utf8, false),
+            Field::new("is_nullable", DataType::Utf8, false),
+        ]));
+
+        let mut column_names = StringBuilder::new();
+        let mut data_types = StringBuilder::new();
+        let mut is_nullables = StringBuilder::new();
+        for (_, field) in schema.fields().iter().enumerate() {
+            column_names.append_value(field.name());
+
+            // "System supplied type" --> Use debug format of the datatype
+            let data_type = field.data_type();
+            data_types.append_value(format!("{data_type:?}"));
+
+            // "YES if the column is possibly nullable, NO if it is known not nullable. "
+            let nullable_str = if field.is_nullable() { "YES" } else { "NO" };
+            is_nullables.append_value(nullable_str);
+        }
+
+        let record_batch = RecordBatch::try_new(
+            record_batch_schema,
+            vec![
+                Arc::new(column_names.finish()),
+                Arc::new(data_types.finish()),
+                Arc::new(is_nullables.finish()),
+            ],
+        )?;
+
+        Ok(record_batch)
+    }
+
+    // return an dataframe which describe file
+    async fn return_describe_table_dataframe(
+        &self,
+        schema: Arc<Schema>,
+    ) -> Result<DataFrame> {
+        let record_batch = self.return_describe_table_record_batch(schema).await?;
+        self.read_batch(record_batch)
+    }
+
     async fn create_external_table(
         &self,
         cmd: &CreateExternalTable,
@@ -1719,7 +1774,7 @@ impl SessionState {
             DFStatement::CreateExternalTable(table) => {
                 relations.insert(ObjectName(vec![Ident::from(table.name.as_str())]));
             }
-            DFStatement::DescribeTable(table) => {
+            DFStatement::DescribeTableStmt(table) => {
                 relations
                     .get_or_insert_with(&table.table_name, |_| table.table_name.clone());
             }
@@ -2058,7 +2113,6 @@ mod tests {
     use crate::test_util::parquet_test_data;
     use crate::variable::VarType;
     use arrow::array::ArrayRef;
-    use arrow::datatypes::*;
     use arrow::record_batch::RecordBatch;
     use async_trait::async_trait;
     use datafusion_expr::{create_udaf, create_udf, Expr, Volatility};
diff --git a/datafusion/core/src/physical_plan/planner.rs b/datafusion/core/src/physical_plan/planner.rs
index 8d5c3b7e1..c607b9791 100644
--- a/datafusion/core/src/physical_plan/planner.rs
+++ b/datafusion/core/src/physical_plan/planner.rs
@@ -1191,6 +1191,11 @@ impl DefaultPhysicalPlanner {
                         "Unsupported logical plan: SetVariable must be root of the plan".to_string(),
                     ))
                 }
+                LogicalPlan::DescribeTable(_) => {
+                    Err(DataFusionError::Internal(
+                        "Unsupported logical plan: DescribeTable must be root of the plan".to_string(),
+                    ))
+                }
                 LogicalPlan::Explain(_) => Err(DataFusionError::Internal(
                     "Unsupported logical plan: Explain must be root of the plan".to_string(),
                 )),
diff --git a/datafusion/core/tests/sqllogictests/test_files/describe.slt b/datafusion/core/tests/sqllogictests/test_files/describe.slt
new file mode 100644
index 000000000..bb5fdeb59
--- /dev/null
+++ b/datafusion/core/tests/sqllogictests/test_files/describe.slt
@@ -0,0 +1,64 @@
+# 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.
+
+
+##########
+# Describe internal tables when information_schema is true
+##########
+
+statement ok
+set datafusion.catalog.information_schema = true
+
+statement ok
+CREATE external table aggregate_simple(c1 real, c2 double, c3 boolean) STORED as CSV WITH HEADER ROW LOCATION 'tests/data/aggregate_simple.csv';
+
+query C1
+DESCRIBE aggregate_simple;
+----
+c1 Float32 NO
+c2 Float64 NO
+c3 Boolean NO
+
+statement ok
+DROP TABLE aggregate_simple;
+
+##########
+# Describe internal tables when information_schema is false
+##########
+
+statement ok
+set datafusion.catalog.information_schema = false
+
+statement ok
+CREATE external table aggregate_simple(c1 real, c2 double, c3 boolean) STORED as CSV WITH HEADER ROW LOCATION 'tests/data/aggregate_simple.csv';
+
+query C2
+DESCRIBE aggregate_simple;
+----
+c1 Float32 NO
+c2 Float64 NO
+c3 Boolean NO
+
+statement ok
+DROP TABLE aggregate_simple;
+
+##########
+# Describe file (currently we can only describe file in datafusion-cli, fix this after issue (#4850) has been done)
+##########
+
+statement error Error during planning: table 'datafusion.public.tests/data/aggregate_simple.csv' not found
+DESCRIBE 'tests/data/aggregate_simple.csv';
diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs
index baa8db8cc..f901fc879 100644
--- a/datafusion/expr/src/lib.rs
+++ b/datafusion/expr/src/lib.rs
@@ -71,11 +71,12 @@ pub use logical_plan::{
         build_join_schema, union, wrap_projection_for_join_if_necessary, UNNAMED_TABLE,
     },
     Aggregate, CreateCatalog, CreateCatalogSchema, CreateExternalTable,
-    CreateMemoryTable, CreateView, CrossJoin, Distinct, DmlStatement, DropTable,
-    DropView, EmptyRelation, Explain, Extension, Filter, Join, JoinConstraint, JoinType,
-    Limit, LogicalPlan, LogicalPlanBuilder, Partitioning, PlanType, PlanVisitor,
-    Projection, Repartition, SetVariable, Sort, StringifiedPlan, Subquery, SubqueryAlias,
-    TableScan, ToStringifiedPlan, Union, UserDefinedLogicalNode, Values, Window, WriteOp,
+    CreateMemoryTable, CreateView, CrossJoin, DescribeTable, Distinct, DmlStatement,
+    DropTable, DropView, EmptyRelation, Explain, Extension, Filter, Join, JoinConstraint,
+    JoinType, Limit, LogicalPlan, LogicalPlanBuilder, Partitioning, PlanType,
+    PlanVisitor, Projection, Repartition, SetVariable, Sort, StringifiedPlan, Subquery,
+    SubqueryAlias, TableScan, ToStringifiedPlan, Union, UserDefinedLogicalNode, Values,
+    Window, WriteOp,
 };
 pub use nullif::SUPPORTED_NULLIF_TYPES;
 pub use operator::Operator;
diff --git a/datafusion/expr/src/logical_plan/mod.rs b/datafusion/expr/src/logical_plan/mod.rs
index 0e5f3cd51..dd2ed1e42 100644
--- a/datafusion/expr/src/logical_plan/mod.rs
+++ b/datafusion/expr/src/logical_plan/mod.rs
@@ -23,11 +23,11 @@ mod plan;
 pub use builder::{table_scan, LogicalPlanBuilder};
 pub use plan::{
     Aggregate, Analyze, CreateCatalog, CreateCatalogSchema, CreateExternalTable,
-    CreateMemoryTable, CreateView, CrossJoin, Distinct, DmlStatement, DropTable,
-    DropView, EmptyRelation, Explain, Extension, Filter, Join, JoinConstraint, JoinType,
-    Limit, LogicalPlan, Partitioning, PlanType, PlanVisitor, Prepare, Projection,
-    Repartition, SetVariable, Sort, StringifiedPlan, Subquery, SubqueryAlias, TableScan,
-    ToStringifiedPlan, Union, Values, Window, WriteOp,
+    CreateMemoryTable, CreateView, CrossJoin, DescribeTable, Distinct, DmlStatement,
+    DropTable, DropView, EmptyRelation, Explain, Extension, Filter, Join, JoinConstraint,
+    JoinType, Limit, LogicalPlan, Partitioning, PlanType, PlanVisitor, Prepare,
+    Projection, Repartition, SetVariable, Sort, StringifiedPlan, Subquery, SubqueryAlias,
+    TableScan, ToStringifiedPlan, Union, Values, Window, WriteOp,
 };
 
 pub use display::display_schema;
diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs
index 944c3f4dd..785f84af1 100644
--- a/datafusion/expr/src/logical_plan/plan.rs
+++ b/datafusion/expr/src/logical_plan/plan.rs
@@ -121,6 +121,8 @@ pub enum LogicalPlan {
     Prepare(Prepare),
     /// Insert / Update / Delete
     Dml(DmlStatement),
+    /// Describe the schema of table
+    DescribeTable(DescribeTable),
 }
 
 impl LogicalPlan {
@@ -161,6 +163,9 @@ impl LogicalPlan {
             LogicalPlan::DropTable(DropTable { schema, .. }) => schema,
             LogicalPlan::DropView(DropView { schema, .. }) => schema,
             LogicalPlan::SetVariable(SetVariable { schema, .. }) => schema,
+            LogicalPlan::DescribeTable(DescribeTable { dummy_schema, .. }) => {
+                dummy_schema
+            }
             LogicalPlan::Dml(DmlStatement { table_schema, .. }) => table_schema,
         }
     }
@@ -221,6 +226,7 @@ impl LogicalPlan {
             | LogicalPlan::Prepare(Prepare { input, .. }) => input.all_schemas(),
             LogicalPlan::DropTable(_)
             | LogicalPlan::DropView(_)
+            | LogicalPlan::DescribeTable(_)
             | LogicalPlan::SetVariable(_) => vec![],
             LogicalPlan::Dml(DmlStatement { table_schema, .. }) => vec![table_schema],
         }
@@ -322,6 +328,7 @@ impl LogicalPlan {
             | LogicalPlan::Union(_)
             | LogicalPlan::Distinct(_)
             | LogicalPlan::Dml(_)
+            | LogicalPlan::DescribeTable(_)
             | LogicalPlan::Prepare(_) => Ok(()),
         }
     }
@@ -363,7 +370,8 @@ impl LogicalPlan {
             | LogicalPlan::CreateCatalog(_)
             | LogicalPlan::DropTable(_)
             | LogicalPlan::SetVariable(_)
-            | LogicalPlan::DropView(_) => vec![],
+            | LogicalPlan::DropView(_)
+            | LogicalPlan::DescribeTable(_) => vec![],
         }
     }
 
@@ -561,7 +569,8 @@ impl LogicalPlan {
             | LogicalPlan::CreateCatalog(_)
             | LogicalPlan::DropTable(_)
             | LogicalPlan::SetVariable(_)
-            | LogicalPlan::DropView(_) => true,
+            | LogicalPlan::DropView(_)
+            | LogicalPlan::DescribeTable(_) => true,
         };
         if !recurse {
             return Ok(false);
@@ -1170,6 +1179,9 @@ impl LogicalPlan {
                     }) => {
                         write!(f, "Prepare: {name:?} {data_types:?} ")
                     }
+                    LogicalPlan::DescribeTable(DescribeTable { .. }) => {
+                        write!(f, "DescribeTable")
+                    }
                 }
             }
         }
@@ -1625,6 +1637,15 @@ pub struct Prepare {
     pub input: Arc<LogicalPlan>,
 }
 
+/// Describe the schema of table
+#[derive(Clone)]
+pub struct DescribeTable {
+    /// Table schema
+    pub schema: Arc<Schema>,
+    /// Dummy schema
+    pub dummy_schema: DFSchemaRef,
+}
+
 /// Produces a relation with string representations of
 /// various parts of the plan
 #[derive(Clone)]
diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs
index c48a1c5fb..15ecb1e5c 100644
--- a/datafusion/expr/src/utils.rs
+++ b/datafusion/expr/src/utils.rs
@@ -738,6 +738,7 @@ pub fn from_plan(
             assert!(inputs.is_empty(), "{plan:?}  should have no inputs");
             Ok(plan.clone())
         }
+        LogicalPlan::DescribeTable(_) => Ok(plan.clone()),
     }
 }
 
diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs
index 2e7004b14..8351813e3 100644
--- a/datafusion/optimizer/src/common_subexpr_eliminate.rs
+++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs
@@ -237,6 +237,7 @@ impl OptimizerRule for CommonSubexprEliminate {
             | LogicalPlan::DropTable(_)
             | LogicalPlan::DropView(_)
             | LogicalPlan::SetVariable(_)
+            | LogicalPlan::DescribeTable(_)
             | LogicalPlan::Distinct(_)
             | LogicalPlan::Extension(_)
             | LogicalPlan::Dml(_)
diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs
index 6485e0c93..f3107d01b 100644
--- a/datafusion/optimizer/src/push_down_projection.rs
+++ b/datafusion/optimizer/src/push_down_projection.rs
@@ -409,6 +409,7 @@ fn optimize_plan(
         | LogicalPlan::DropTable(_)
         | LogicalPlan::DropView(_)
         | LogicalPlan::SetVariable(_)
+        | LogicalPlan::DescribeTable(_)
         | LogicalPlan::CrossJoin(_)
         | LogicalPlan::Dml(_)
         | LogicalPlan::Extension { .. }
diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs
index 54706c9fe..60fcc95e8 100644
--- a/datafusion/proto/src/logical_plan/mod.rs
+++ b/datafusion/proto/src/logical_plan/mod.rs
@@ -1343,11 +1343,14 @@ impl AsLogicalPlan for LogicalPlanNode {
                 "LogicalPlan serde is not yet implemented for DropView",
             )),
             LogicalPlan::SetVariable(_) => Err(proto_error(
-                "LogicalPlan serde is not yet implemented for DropView",
+                "LogicalPlan serde is not yet implemented for SetVariable",
             )),
             LogicalPlan::Dml(_) => Err(proto_error(
                 "LogicalPlan serde is not yet implemented for Dml",
             )),
+            LogicalPlan::DescribeTable(_) => Err(proto_error(
+                "LogicalPlan serde is not yet implemented for DescribeTable",
+            )),
         }
     }
 }
diff --git a/datafusion/sql/src/parser.rs b/datafusion/sql/src/parser.rs
index 2cc92d04c..d25268c87 100644
--- a/datafusion/sql/src/parser.rs
+++ b/datafusion/sql/src/parser.rs
@@ -80,7 +80,7 @@ impl fmt::Display for CreateExternalTable {
 
 /// DataFusion extension DDL for `DESCRIBE TABLE`
 #[derive(Debug, Clone, PartialEq, Eq)]
-pub struct DescribeTable {
+pub struct DescribeTableStmt {
     /// Table name
     pub table_name: ObjectName,
 }
@@ -95,7 +95,7 @@ pub enum Statement {
     /// Extension: `CREATE EXTERNAL TABLE`
     CreateExternalTable(CreateExternalTable),
     /// Extension: `DESCRIBE TABLE`
-    DescribeTable(DescribeTable),
+    DescribeTableStmt(DescribeTableStmt),
 }
 
 /// DataFusion SQL Parser based on [`sqlparser`]
@@ -210,7 +210,9 @@ impl<'a> DFParser<'a> {
     /// Parse a SQL `DESCRIBE` statement
     pub fn parse_describe(&mut self) -> Result<Statement, ParserError> {
         let table_name = self.parser.parse_object_name()?;
-        Ok(Statement::DescribeTable(DescribeTable { table_name }))
+        Ok(Statement::DescribeTableStmt(DescribeTableStmt {
+            table_name,
+        }))
     }
 
     /// Parse a SQL `CREATE` statementm handling `CREATE EXTERNAL TABLE`
diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs
index 5b7949d2d..fb50ef7bc 100644
--- a/datafusion/sql/src/statement.rs
+++ b/datafusion/sql/src/statement.rs
@@ -16,7 +16,7 @@
 // under the License.
 
 use crate::parser::{
-    CreateExternalTable, DFParser, DescribeTable, Statement as DFStatement,
+    CreateExternalTable, DFParser, DescribeTableStmt, Statement as DFStatement,
 };
 use crate::planner::{
     object_name_to_qualifier, object_name_to_table_reference, ContextProvider,
@@ -35,8 +35,8 @@ use datafusion_expr::utils::expr_to_columns;
 use datafusion_expr::{
     cast, col, CreateCatalog, CreateCatalogSchema,
     CreateExternalTable as PlanCreateExternalTable, CreateMemoryTable, CreateView,
-    DmlStatement, DropTable, DropView, Explain, Filter, LogicalPlan, LogicalPlanBuilder,
-    PlanType, SetVariable, ToStringifiedPlan, WriteOp,
+    DescribeTable, DmlStatement, DropTable, DropView, Explain, Filter, LogicalPlan,
+    LogicalPlanBuilder, PlanType, SetVariable, ToStringifiedPlan, WriteOp,
 };
 use sqlparser::ast;
 use sqlparser::ast::{
@@ -54,7 +54,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> {
         match statement {
             DFStatement::CreateExternalTable(s) => self.external_table_to_plan(s),
             DFStatement::Statement(s) => self.sql_statement_to_plan(*s),
-            DFStatement::DescribeTable(s) => self.describe_table_to_plan(s),
+            DFStatement::DescribeTableStmt(s) => self.describe_table_to_plan(s),
         }
     }
 
@@ -381,30 +381,23 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> {
         }
     }
 
-    fn describe_table_to_plan(&self, statement: DescribeTable) -> Result<LogicalPlan> {
-        let DescribeTable { table_name } = statement;
-
-        let where_clause = object_name_to_qualifier(&table_name);
+    fn describe_table_to_plan(
+        &self,
+        statement: DescribeTableStmt,
+    ) -> Result<LogicalPlan> {
+        let DescribeTableStmt { table_name } = statement;
         let table_ref = object_name_to_table_reference(table_name)?;
 
-        // check if table_name exists
-        let _ = self
+        let table_source = self
             .schema_provider
             .get_table_provider((&table_ref).into())?;
 
-        if self.has_table("information_schema", "tables") {
-            let sql = format!(
-                "SELECT column_name, data_type, is_nullable \
-                                FROM information_schema.columns WHERE {where_clause};"
-            );
-            let mut rewrite = DFParser::parse_sql(&sql)?;
-            self.statement_to_plan(rewrite.pop_front().unwrap())
-        } else {
-            Err(DataFusionError::Plan(
-                "DESCRIBE TABLE is not supported unless information_schema is enabled"
-                    .to_string(),
-            ))
-        }
+        let schema = table_source.schema();
+
+        Ok(LogicalPlan::DescribeTable(DescribeTable {
+            schema,
+            dummy_schema: DFSchemaRef::new(DFSchema::empty()),
+        }))
     }
 
     /// Generate a logical plan from a CREATE EXTERNAL TABLE statement