diff --git a/datafusion-cli/src/functions.rs b/datafusion-cli/src/functions.rs index 99511e969386..dd56b0196dd5 100644 --- a/datafusion-cli/src/functions.rs +++ b/datafusion-cli/src/functions.rs @@ -214,6 +214,7 @@ pub fn display_all_functions() -> Result<()> { } /// PARQUET_META table function +#[derive(Debug)] struct ParquetMetadataTable { schema: SchemaRef, batch: RecordBatch, diff --git a/datafusion-examples/examples/simple_udtf.rs b/datafusion-examples/examples/simple_udtf.rs index fe7f37cc00e3..baa783fce9e4 100644 --- a/datafusion-examples/examples/simple_udtf.rs +++ b/datafusion-examples/examples/simple_udtf.rs @@ -73,6 +73,7 @@ async fn main() -> Result<()> { /// Usage: `read_csv(filename, [limit])` /// /// [`read_csv`]: https://duckdb.org/docs/data/csv/overview.html +#[derive(Debug)] struct LocalCsvTable { schema: SchemaRef, limit: Option, diff --git a/datafusion/catalog/src/table.rs b/datafusion/catalog/src/table.rs index 69fa81faf8e2..6c36d907acc3 100644 --- a/datafusion/catalog/src/table.rs +++ b/datafusion/catalog/src/table.rs @@ -17,6 +17,7 @@ use std::any::Any; use std::borrow::Cow; +use std::fmt::Debug; use std::sync::Arc; use crate::session::Session; @@ -31,7 +32,7 @@ use datafusion_physical_plan::ExecutionPlan; /// Source table #[async_trait] -pub trait TableProvider: Sync + Send { +pub trait TableProvider: Debug + Sync + Send { /// Returns the table provider as [`Any`](std::any::Any) so that it can be /// downcast to a specific implementation. fn as_any(&self) -> &dyn Any; @@ -193,6 +194,7 @@ pub trait TableProvider: Sync + Send { /// # use datafusion_expr::{Expr, TableProviderFilterPushDown, TableType}; /// # use datafusion_physical_plan::ExecutionPlan; /// // Define a struct that implements the TableProvider trait + /// #[derive(Debug)] /// struct TestDataSource {} /// /// #[async_trait] @@ -212,7 +214,7 @@ pub trait TableProvider: Sync + Send { /// // This example only supports a between expr with a single column named "c1". /// Expr::Between(between_expr) => { /// between_expr.expr - /// .try_into_col() + /// .try_as_col() /// .map(|column| { /// if column.name == "c1" { /// TableProviderFilterPushDown::Exact @@ -283,7 +285,7 @@ pub trait TableProvider: Sync + Send { /// For example, this can be used to create a table "on the fly" /// from a directory of files only when that name is referenced. #[async_trait] -pub trait TableProviderFactory: Sync + Send { +pub trait TableProviderFactory: Debug + Sync + Send { /// Create a TableProvider with the given url async fn create( &self, diff --git a/datafusion/core/src/catalog_common/information_schema.rs b/datafusion/core/src/catalog_common/information_schema.rs index d086ce900cc3..df4257504b1d 100644 --- a/datafusion/core/src/catalog_common/information_schema.rs +++ b/datafusion/core/src/catalog_common/information_schema.rs @@ -19,15 +19,15 @@ //! //! [Information Schema]: https://en.wikipedia.org/wiki/Information_schema -use async_trait::async_trait; -use datafusion_common::DataFusionError; -use std::{any::Any, sync::Arc}; - use arrow::{ array::{StringBuilder, UInt64Builder}, datatypes::{DataType, Field, Schema, SchemaRef}, record_batch::RecordBatch, }; +use async_trait::async_trait; +use datafusion_common::DataFusionError; +use std::fmt::{Debug, Formatter}; +use std::{any::Any, sync::Arc}; use crate::catalog::{CatalogProviderList, SchemaProvider, TableProvider}; use crate::datasource::streaming::StreamingTable; @@ -75,6 +75,15 @@ struct InformationSchemaConfig { catalog_list: Arc, } +impl Debug for InformationSchemaConfig { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("InformationSchemaConfig") + // TODO it would be great to print the catalog list here + // but that would require CatalogProviderList to implement Debug + .finish_non_exhaustive() + } +} + impl InformationSchemaConfig { /// Construct the `information_schema.tables` virtual table async fn make_tables( @@ -246,6 +255,7 @@ impl SchemaProvider for InformationSchemaProvider { } } +#[derive(Debug)] struct InformationSchemaTables { schema: SchemaRef, config: InformationSchemaConfig, @@ -337,6 +347,7 @@ impl InformationSchemaTablesBuilder { } } +#[derive(Debug)] struct InformationSchemaViews { schema: SchemaRef, config: InformationSchemaConfig, @@ -424,6 +435,7 @@ impl InformationSchemaViewBuilder { } } +#[derive(Debug)] struct InformationSchemaColumns { schema: SchemaRef, config: InformationSchemaConfig, @@ -640,6 +652,7 @@ impl InformationSchemaColumnsBuilder { } } +#[derive(Debug)] struct InformationSchemata { schema: SchemaRef, config: InformationSchemaConfig, @@ -741,6 +754,7 @@ impl PartitionStream for InformationSchemata { } } +#[derive(Debug)] struct InformationSchemaDfSettings { schema: SchemaRef, config: InformationSchemaConfig, diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 663d8bf022f0..b52dcaa2f7fd 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1873,6 +1873,7 @@ impl DataFrame { } } +#[derive(Debug)] struct DataFrameTableProvider { plan: LogicalPlan, } diff --git a/datafusion/core/src/datasource/cte_worktable.rs b/datafusion/core/src/datasource/cte_worktable.rs index d2da15c64f52..23f57b12ae08 100644 --- a/datafusion/core/src/datasource/cte_worktable.rs +++ b/datafusion/core/src/datasource/cte_worktable.rs @@ -36,6 +36,7 @@ use crate::datasource::{TableProvider, TableType}; /// The temporary working table where the previous iteration of a recursive query is stored /// Naming is based on PostgreSQL's implementation. /// See here for more details: www.postgresql.org/docs/11/queries-with.html#id-1.5.6.12.5.4 +#[derive(Debug)] pub struct CteWorkTable { /// The name of the CTE work table // WIP, see https://github.com/apache/datafusion/issues/462 diff --git a/datafusion/core/src/datasource/empty.rs b/datafusion/core/src/datasource/empty.rs index d831dd006031..bc5b82bd8c5b 100644 --- a/datafusion/core/src/datasource/empty.rs +++ b/datafusion/core/src/datasource/empty.rs @@ -32,6 +32,7 @@ use crate::physical_plan::{empty::EmptyExec, ExecutionPlan}; /// An empty plan that is useful for testing and generating plans /// without mapping them to actual data. +#[derive(Debug)] pub struct EmptyTable { schema: SchemaRef, partitions: usize, diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 6115af34e69e..2a35fddeb033 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -646,6 +646,7 @@ impl ListingOptions { /// # Ok(()) /// # } /// ``` +#[derive(Debug)] pub struct ListingTable { table_paths: Vec, /// File fields only diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index ef6d195cdaff..d30247e2c67a 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -293,6 +293,7 @@ impl StreamConfig { /// /// [Hadoop]: https://hadoop.apache.org/ /// [`ListingTable`]: crate::datasource::listing::ListingTable +#[derive(Debug)] pub struct StreamTable(Arc); impl StreamTable { @@ -370,6 +371,7 @@ impl TableProvider for StreamTable { } } +#[derive(Debug)] struct StreamRead(Arc); impl PartitionStream for StreamRead { diff --git a/datafusion/core/src/datasource/streaming.rs b/datafusion/core/src/datasource/streaming.rs index f9ded357b5a5..0a14cfefcdf2 100644 --- a/datafusion/core/src/datasource/streaming.rs +++ b/datafusion/core/src/datasource/streaming.rs @@ -32,6 +32,7 @@ use datafusion_expr::{Expr, TableType}; use log::debug; /// A [`TableProvider`] that streams a set of [`PartitionStream`] +#[derive(Debug)] pub struct StreamingTable { schema: SchemaRef, partitions: Vec>, diff --git a/datafusion/core/src/datasource/view.rs b/datafusion/core/src/datasource/view.rs index 947714c1e4f9..1ffe54e4b06c 100644 --- a/datafusion/core/src/datasource/view.rs +++ b/datafusion/core/src/datasource/view.rs @@ -36,6 +36,7 @@ use datafusion_optimizer::Analyzer; use crate::datasource::{TableProvider, TableType}; /// An implementation of `TableProvider` that uses another logical plan. +#[derive(Debug)] pub struct ViewTable { /// LogicalPlan of the view logical_plan: LogicalPlan, diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 4ca59da55bad..a6c3e075a460 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -1831,6 +1831,7 @@ mod tests { #[test] fn test_streaming_table_after_projection() -> Result<()> { + #[derive(Debug)] struct DummyStreamPartition { schema: SchemaRef, } diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 39a126a06bb6..08740daa0c8e 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -297,6 +297,7 @@ pub fn csv_exec_sorted( } // construct a stream partition for test purposes +#[derive(Debug)] pub(crate) struct TestStreamPartition { pub schema: SchemaRef, } diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index edccb3844c84..e03c18fec7c4 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -174,6 +174,7 @@ pub fn populate_csv_partitions( } /// TableFactory for tests +#[derive(Default, Debug)] pub struct TestTableFactory {} #[async_trait] @@ -191,6 +192,7 @@ impl TableProviderFactory for TestTableFactory { } /// TableProvider for testing purposes +#[derive(Debug)] pub struct TestTableProvider { /// URL of table files or folder pub url: String, diff --git a/datafusion/core/tests/custom_sources_cases/mod.rs b/datafusion/core/tests/custom_sources_cases/mod.rs index c12dd4e1b0ee..e1bd14105e23 100644 --- a/datafusion/core/tests/custom_sources_cases/mod.rs +++ b/datafusion/core/tests/custom_sources_cases/mod.rs @@ -71,7 +71,9 @@ macro_rules! TEST_CUSTOM_RECORD_BATCH { //--- Custom source dataframe tests ---// +#[derive(Debug)] struct CustomTableProvider; + #[derive(Debug, Clone)] struct CustomExecutionPlan { projection: Option>, diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index e91bb023ef38..09f7265d639a 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -142,7 +142,7 @@ impl ExecutionPlan for CustomPlan { } } -#[derive(Clone)] +#[derive(Clone, Debug)] struct CustomProvider { zero_batch: RecordBatch, one_batch: RecordBatch, diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index 69ef6058a2f6..ec66df45c7ba 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -777,6 +777,7 @@ fn batches_byte_size(batches: &[RecordBatch]) -> usize { batches.iter().map(|b| b.get_array_memory_size()).sum() } +#[derive(Debug)] struct DummyStreamPartition { schema: SchemaRef, batches: Vec, @@ -798,6 +799,7 @@ impl PartitionStream for DummyStreamPartition { } /// Wrapper over a TableProvider that can provide ordering information +#[derive(Debug)] struct SortedTableProvider { schema: SchemaRef, batches: Vec>, diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index 750544ecdec1..1b4c28d41d19 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -38,6 +38,7 @@ use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::{get_plan_string, ExecutionPlan, ExecutionPlanProperties}; use std::sync::Arc; +#[derive(Debug)] struct DummyStreamPartition { schema: SchemaRef, } diff --git a/datafusion/core/tests/user_defined/user_defined_table_functions.rs b/datafusion/core/tests/user_defined/user_defined_table_functions.rs index 5fd3b7a03384..fe57752db52e 100644 --- a/datafusion/core/tests/user_defined/user_defined_table_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_table_functions.rs @@ -105,6 +105,7 @@ async fn test_deregister_udtf() -> Result<()> { Ok(()) } +#[derive(Debug)] struct SimpleCsvTable { schema: SchemaRef, exprs: Vec, diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 9dc8b214420b..b02e4fb5738d 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -18,6 +18,7 @@ //! Generic plans for deferred execution: [`StreamingTableExec`] and [`PartitionStream`] use std::any::Any; +use std::fmt::Debug; use std::sync::Arc; use super::{DisplayAs, DisplayFormatType, ExecutionMode, PlanProperties}; @@ -42,7 +43,7 @@ use log::debug; /// Combined with [`StreamingTableExec`], you can use this trait to implement /// [`ExecutionPlan`] for a custom source with less boiler plate than /// implementing `ExecutionPlan` directly for many use cases. -pub trait PartitionStream: Send + Sync { +pub trait PartitionStream: Debug + Send + Sync { /// Returns the schema of this partition fn schema(&self) -> &SchemaRef; diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index f5b4a096018f..4da43b313403 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -126,6 +126,7 @@ pub fn mem_exec(partitions: usize) -> MemoryExec { } // construct a stream partition for test purposes +#[derive(Debug)] pub struct TestPartitionStream { pub schema: SchemaRef, pub batches: Vec, diff --git a/datafusion/sqllogictest/src/test_context.rs b/datafusion/sqllogictest/src/test_context.rs index 19016d328f4c..d3ee720467b6 100644 --- a/datafusion/sqllogictest/src/test_context.rs +++ b/datafusion/sqllogictest/src/test_context.rs @@ -207,6 +207,7 @@ pub async fn register_partition_table(test_ctx: &mut TestContext) { // registers a LOCAL TEMPORARY table. pub async fn register_temp_table(ctx: &SessionContext) { + #[derive(Debug)] struct TestTable(TableType); #[async_trait]