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/09/07 16:58:39 UTC

[arrow-datafusion] branch main updated: Add backtrace to error messages (#7434)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 2ffda2a9a8 Add backtrace to error messages (#7434)
2ffda2a9a8 is described below

commit 2ffda2a9a893455e55cd773d9dd4f426a61d8cd3
Author: comphead <co...@users.noreply.github.com>
AuthorDate: Thu Sep 7 09:58:33 2023 -0700

    Add backtrace to error messages (#7434)
    
    * Adding back trace
    
    * fmt
    
    * fmt
    
    * tests
    
    * tests
    
    * tests
    
    * clippy
    
    * strip_backtrace
    
    * tests
    
    * moved backtrace
    
    * tests
    
    * tests
---
 datafusion-cli/src/helper.rs                       |   8 +-
 datafusion/common/src/column.rs                    |   4 +-
 datafusion/common/src/dfschema.rs                  |  16 +-
 datafusion/common/src/error.rs                     |  70 +++++--
 datafusion/common/src/scalar.rs                    |  10 +-
 datafusion/core/src/catalog/mod.rs                 |   2 +-
 datafusion/core/src/dataframe.rs                   |   6 +-
 datafusion/core/src/datasource/listing/table.rs    |   3 +-
 datafusion/core/src/datasource/memory.rs           |  22 +--
 .../core/src/datasource/physical_plan/csv.rs       |   4 +-
 .../core/src/datasource/physical_plan/json.rs      |   2 +-
 .../core/src/datasource/physical_plan/parquet.rs   |   2 +-
 datafusion/core/src/execution/context.rs           |   8 +-
 datafusion/core/src/physical_plan/stream.rs        |   2 +-
 datafusion/core/tests/config_from_env.rs           |   2 +-
 datafusion/core/tests/dataframe/mod.rs             |  10 +-
 datafusion/core/tests/sql/joins.rs                 |   2 +-
 datafusion/core/tests/sql/order.rs                 |   4 +-
 datafusion/core/tests/sql/sql_api.rs               |  10 +-
 datafusion/core/tests/sql/timestamp.rs             |   4 +-
 datafusion/execution/src/disk_manager.rs           |   2 +-
 datafusion/execution/src/memory_pool/pool.rs       |  10 +-
 datafusion/execution/src/object_store.rs           |  10 +-
 datafusion/expr/src/expr_rewriter/mod.rs           |   2 +-
 datafusion/expr/src/logical_plan/builder.rs        |   8 +-
 datafusion/expr/src/logical_plan/plan.rs           |  27 ++-
 datafusion/expr/src/type_coercion/aggregates.rs    |   6 +-
 datafusion/expr/src/type_coercion/binary.rs        |  14 +-
 datafusion/expr/src/window_frame.rs                |  10 +-
 datafusion/optimizer/src/analyzer/type_coercion.rs |  12 +-
 datafusion/optimizer/src/optimizer.rs              |   8 +-
 datafusion/physical-expr/src/aggregate/min_max.rs  |   9 +-
 datafusion/physical-expr/src/array_expressions.rs  |   2 +-
 .../physical-expr/src/datetime_expressions.rs      |  18 +-
 datafusion/physical-expr/src/expressions/column.rs |  27 ++-
 datafusion/physical-expr/src/functions.rs          |   2 +-
 datafusion/physical-expr/src/regex_expressions.rs  |   4 +-
 datafusion/sql/src/expr/identifier.rs              |   8 +-
 datafusion/sql/src/query.rs                        |   6 +-
 datafusion/sql/tests/sql_integration.rs            | 202 +++++++++++----------
 datafusion/sqllogictest/test_files/scalar.slt      |   4 +-
 41 files changed, 311 insertions(+), 271 deletions(-)

diff --git a/datafusion-cli/src/helper.rs b/datafusion-cli/src/helper.rs
index e4992122f9..69d412db5a 100644
--- a/datafusion-cli/src/helper.rs
+++ b/datafusion-cli/src/helper.rs
@@ -18,6 +18,7 @@
 //! Helper that helps with interactive editing, including multi-line parsing and validation,
 //! and auto-completion for file name during creating external table.
 
+use datafusion::common::sql_err;
 use datafusion::error::DataFusionError;
 use datafusion::sql::parser::{DFParser, Statement};
 use datafusion::sql::sqlparser::dialect::dialect_from_str;
@@ -162,9 +163,10 @@ pub fn unescape_input(input: &str) -> datafusion::error::Result<String> {
                     't' => '\t',
                     '\\' => '\\',
                     _ => {
-                        return Err(DataFusionError::SQL(ParserError::TokenizerError(
-                            format!("unsupported escape char: '\\{}'", next_char),
-                        )))
+                        return sql_err!(ParserError::TokenizerError(format!(
+                            "unsupported escape char: '\\{}'",
+                            next_char
+                        ),))
                     }
                 });
             }
diff --git a/datafusion/common/src/column.rs b/datafusion/common/src/column.rs
index 12510157db..2e729c128e 100644
--- a/datafusion/common/src/column.rs
+++ b/datafusion/common/src/column.rs
@@ -428,7 +428,7 @@ mod tests {
             )
             .expect_err("should've failed to find field");
         let expected = r#"Schema error: No field named z. Valid fields are t1.a, t1.b, t2.c, t2.d, t3.a, t3.b, t3.c, t3.d, t3.e."#;
-        assert_eq!(err.to_string(), expected);
+        assert_eq!(err.strip_backtrace(), expected);
 
         // ambiguous column reference
         let col = Column::from_name("a");
@@ -439,7 +439,7 @@ mod tests {
             )
             .expect_err("should've found ambiguous field");
         let expected = "Schema error: Ambiguous reference to unqualified field a";
-        assert_eq!(err.to_string(), expected);
+        assert_eq!(err.strip_backtrace(), expected);
 
         Ok(())
     }
diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs
index 64937eafa1..e015ef5c40 100644
--- a/datafusion/common/src/dfschema.rs
+++ b/datafusion/common/src/dfschema.rs
@@ -812,8 +812,8 @@ mod tests {
         // lookup with unqualified name "t1.c0"
         let err = schema.index_of_column(&col).unwrap_err();
         assert_eq!(
-            err.to_string(),
-            "Schema error: No field named \"t1.c0\". Valid fields are t1.c0, t1.c1.",
+            err.strip_backtrace(),
+            "Schema error: No field named \"t1.c0\". Valid fields are t1.c0, t1.c1."
         );
         Ok(())
     }
@@ -832,8 +832,8 @@ mod tests {
         // lookup with unqualified name "t1.c0"
         let err = schema.index_of_column(&col).unwrap_err();
         assert_eq!(
-            err.to_string(),
-            "Schema error: No field named \"t1.c0\". Valid fields are t1.\"CapitalColumn\", t1.\"field.with.period\".",
+            err.strip_backtrace(),
+            "Schema error: No field named \"t1.c0\". Valid fields are t1.\"CapitalColumn\", t1.\"field.with.period\"."
         );
         Ok(())
     }
@@ -916,8 +916,8 @@ mod tests {
         let right = DFSchema::try_from(test_schema_1())?;
         let join = left.join(&right);
         assert_eq!(
-            join.unwrap_err().to_string(),
-            "Schema error: Schema contains duplicate unqualified field name c0",
+            join.unwrap_err().strip_backtrace(),
+            "Schema error: Schema contains duplicate unqualified field name c0"
         );
         Ok(())
     }
@@ -993,12 +993,12 @@ mod tests {
 
         let col = Column::from_qualified_name("t1.c0");
         let err = schema.index_of_column(&col).unwrap_err();
-        assert_eq!(err.to_string(), "Schema error: No field named t1.c0.");
+        assert_eq!(err.strip_backtrace(), "Schema error: No field named t1.c0.");
 
         // the same check without qualifier
         let col = Column::from_name("c0");
         let err = schema.index_of_column(&col).err().unwrap();
-        assert_eq!("Schema error: No field named c0.", err.to_string());
+        assert_eq!(err.strip_backtrace(), "Schema error: No field named c0.");
     }
 
     #[test]
diff --git a/datafusion/common/src/error.rs b/datafusion/common/src/error.rs
index 140522d660..3b17c467ad 100644
--- a/datafusion/common/src/error.rs
+++ b/datafusion/common/src/error.rs
@@ -17,6 +17,7 @@
 
 //! DataFusion error types
 
+use std::backtrace::{Backtrace, BacktraceStatus};
 use std::error::Error;
 use std::fmt::{Display, Formatter};
 use std::io;
@@ -278,7 +279,9 @@ impl From<GenericError> for DataFusionError {
 impl Display for DataFusionError {
     fn fmt(&self, f: &mut Formatter) -> std::fmt::Result {
         match *self {
-            DataFusionError::ArrowError(ref desc) => write!(f, "Arrow error: {desc}"),
+            DataFusionError::ArrowError(ref desc) => {
+                write!(f, "Arrow error: {desc}")
+            }
             #[cfg(feature = "parquet")]
             DataFusionError::ParquetError(ref desc) => {
                 write!(f, "Parquet error: {desc}")
@@ -287,7 +290,9 @@ impl Display for DataFusionError {
             DataFusionError::AvroError(ref desc) => {
                 write!(f, "Avro error: {desc}")
             }
-            DataFusionError::IoError(ref desc) => write!(f, "IO error: {desc}"),
+            DataFusionError::IoError(ref desc) => {
+                write!(f, "IO error: {desc}")
+            }
             DataFusionError::SQL(ref desc) => {
                 write!(f, "SQL error: {desc:?}")
             }
@@ -298,7 +303,7 @@ impl Display for DataFusionError {
                 write!(f, "This feature is not implemented: {desc}")
             }
             DataFusionError::Internal(ref desc) => {
-                write!(f, "Internal error: {desc}. This was likely caused by a bug in DataFusion's \
+                write!(f, "Internal error: {desc}.\nThis was likely caused by a bug in DataFusion's \
                     code and we would welcome that you file an bug report in our issue tracker")
             }
             DataFusionError::Plan(ref desc) => {
@@ -404,6 +409,24 @@ impl DataFusionError {
     pub fn context(self, description: impl Into<String>) -> Self {
         Self::Context(description.into(), Box::new(self))
     }
+
+    pub fn strip_backtrace(&self) -> String {
+        self.to_string()
+            .split("\n\nbacktrace: ")
+            .collect::<Vec<&str>>()
+            .first()
+            .unwrap_or(&"")
+            .to_string()
+    }
+
+    pub fn get_back_trace() -> String {
+        let back_trace = Backtrace::capture();
+        if back_trace.status() == BacktraceStatus::Captured {
+            return format!("\n\nbacktrace: {}", back_trace);
+        }
+
+        "".to_string()
+    }
 }
 
 /// Unwrap an `Option` if possible. Otherwise return an `DataFusionError::Internal`.
@@ -444,7 +467,7 @@ macro_rules! make_error {
                 #[macro_export]
                 macro_rules! $NAME {
                     ($d($d args:expr),*) => {
-                        Err(DataFusionError::$ERR(format!($d($d args),*).into()))
+                        Err(DataFusionError::$ERR(format!("{}{}", format!($d($d args),*), DataFusionError::get_back_trace()).into()))
                     }
                 }
             }
@@ -464,6 +487,14 @@ make_error!(not_impl_err, NotImplemented);
 // Exposes a macro to create `DataFusionError::Execution`
 make_error!(exec_err, Execution);
 
+// Exposes a macro to create `DataFusionError::SQL`
+#[macro_export]
+macro_rules! sql_err {
+    ($ERR:expr) => {
+        Err(DataFusionError::SQL($ERR))
+    };
+}
+
 // To avoid compiler error when using macro in the same crate:
 // macros from the current crate cannot be referred to by absolute paths
 pub use exec_err as _exec_err;
@@ -478,18 +509,17 @@ mod test {
     use arrow::error::ArrowError;
 
     #[test]
-    fn arrow_error_to_datafusion() {
+    fn datafusion_error_to_arrow() {
         let res = return_arrow_error().unwrap_err();
-        assert_eq!(
-            res.to_string(),
-            "External error: Error during planning: foo"
-        );
+        assert!(res
+            .to_string()
+            .starts_with("External error: Error during planning: foo"));
     }
 
     #[test]
-    fn datafusion_error_to_arrow() {
+    fn arrow_error_to_datafusion() {
         let res = return_datafusion_error().unwrap_err();
-        assert_eq!(res.to_string(), "Arrow error: Schema error: bar");
+        assert_eq!(res.strip_backtrace(), "Arrow error: Schema error: bar");
     }
 
     #[test]
@@ -552,31 +582,37 @@ mod test {
     fn test_make_error_parse_input() {
         let res: Result<(), DataFusionError> = plan_err!("Err");
         let res = res.unwrap_err();
-        assert_eq!(res.to_string(), "Error during planning: Err");
+        assert_eq!(res.strip_backtrace(), "Error during planning: Err");
 
         let extra1 = "extra1";
         let extra2 = "extra2";
 
         let res: Result<(), DataFusionError> = plan_err!("Err {} {}", extra1, extra2);
         let res = res.unwrap_err();
-        assert_eq!(res.to_string(), "Error during planning: Err extra1 extra2");
+        assert_eq!(
+            res.strip_backtrace(),
+            "Error during planning: Err extra1 extra2"
+        );
 
         let res: Result<(), DataFusionError> =
             plan_err!("Err {:?} {:#?}", extra1, extra2);
         let res = res.unwrap_err();
         assert_eq!(
-            res.to_string(),
+            res.strip_backtrace(),
             "Error during planning: Err \"extra1\" \"extra2\""
         );
 
         let res: Result<(), DataFusionError> = plan_err!("Err {extra1} {extra2}");
         let res = res.unwrap_err();
-        assert_eq!(res.to_string(), "Error during planning: Err extra1 extra2");
+        assert_eq!(
+            res.strip_backtrace(),
+            "Error during planning: Err extra1 extra2"
+        );
 
         let res: Result<(), DataFusionError> = plan_err!("Err {extra1:?} {extra2:#?}");
         let res = res.unwrap_err();
         assert_eq!(
-            res.to_string(),
+            res.strip_backtrace(),
             "Error during planning: Err \"extra1\" \"extra2\""
         );
     }
@@ -599,7 +635,7 @@ mod test {
         let e = e.find_root();
 
         // DataFusionError does not implement Eq, so we use a string comparison + some cheap "same variant" test instead
-        assert_eq!(e.to_string(), exp.to_string(),);
+        assert_eq!(e.strip_backtrace(), exp.strip_backtrace());
         assert_eq!(std::mem::discriminant(e), std::mem::discriminant(&exp),)
     }
 }
diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs
index b0e3a75e7f..1f017ead59 100644
--- a/datafusion/common/src/scalar.rs
+++ b/datafusion/common/src/scalar.rs
@@ -3265,7 +3265,10 @@ mod tests {
     fn scalar_sub_trait_int32_overflow_test() {
         let int_value = ScalarValue::Int32(Some(i32::MAX));
         let int_value_2 = ScalarValue::Int32(Some(i32::MIN));
-        let err = int_value.sub_checked(&int_value_2).unwrap_err().to_string();
+        let err = int_value
+            .sub_checked(&int_value_2)
+            .unwrap_err()
+            .strip_backtrace();
         assert_eq!(
             err,
             "Arrow error: Compute error: Overflow happened on: 2147483647 - -2147483648"
@@ -3285,7 +3288,10 @@ mod tests {
     fn scalar_sub_trait_int64_overflow_test() {
         let int_value = ScalarValue::Int64(Some(i64::MAX));
         let int_value_2 = ScalarValue::Int64(Some(i64::MIN));
-        let err = int_value.sub_checked(&int_value_2).unwrap_err().to_string();
+        let err = int_value
+            .sub_checked(&int_value_2)
+            .unwrap_err()
+            .strip_backtrace();
         assert_eq!(err, "Arrow error: Compute error: Overflow happened on: 9223372036854775807 - -9223372036854775808")
     }
 
diff --git a/datafusion/core/src/catalog/mod.rs b/datafusion/core/src/catalog/mod.rs
index 5f3ca81dd7..6751edbd3a 100644
--- a/datafusion/core/src/catalog/mod.rs
+++ b/datafusion/core/src/catalog/mod.rs
@@ -237,7 +237,7 @@ mod tests {
 
         match catalog.register_schema("foo", schema) {
             Ok(_) => panic!("unexpected OK"),
-            Err(e) => assert_eq!(e.to_string(), "This feature is not implemented: Registering new schemas is not supported"),
+            Err(e) => assert_eq!(e.strip_backtrace(), "This feature is not implemented: Registering new schemas is not supported"),
         };
     }
 
diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe.rs
index 648b2340d4..2967b1823f 100644
--- a/datafusion/core/src/dataframe.rs
+++ b/datafusion/core/src/dataframe.rs
@@ -1502,7 +1502,7 @@ mod tests {
             // try to sort on some value not present in input to distinct
             .sort(vec![col("c2").sort(true, true)])
             .unwrap_err();
-        assert_eq!(err.to_string(), "Error during planning: For SELECT DISTINCT, ORDER BY expressions c2 must appear in select list");
+        assert_eq!(err.strip_backtrace(), "Error during planning: For SELECT DISTINCT, ORDER BY expressions c2 must appear in select list");
 
         Ok(())
     }
@@ -1560,7 +1560,7 @@ mod tests {
             .join_on(right, JoinType::Inner, [col("c1").eq(col("c1"))])
             .expect_err("join didn't fail check");
         let expected = "Schema error: Ambiguous reference to unqualified field c1";
-        assert_eq!(join.to_string(), expected);
+        assert_eq!(join.strip_backtrace(), expected);
 
         Ok(())
     }
@@ -1917,7 +1917,7 @@ mod tests {
             .with_column_renamed("c2", "AAA")
             .unwrap_err();
         let expected_err = "Schema error: Ambiguous reference to unqualified field c2";
-        assert_eq!(actual_err.to_string(), expected_err);
+        assert_eq!(actual_err.strip_backtrace(), expected_err);
 
         Ok(())
     }
diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs
index 7740bff210..07f0a9dc83 100644
--- a/datafusion/core/src/datasource/listing/table.rs
+++ b/datafusion/core/src/datasource/listing/table.rs
@@ -1862,7 +1862,8 @@ mod tests {
         )
         .await
         .expect_err("Example should fail!");
-        assert_eq!("Invalid or Unsupported Configuration: zstd compression requires specifying a level such as zstd(4)", format!("{e}"));
+        assert_eq!(e.strip_backtrace(), "Invalid or Unsupported Configuration: zstd compression requires specifying a level such as zstd(4)");
+
         Ok(())
     }
 
diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs
index 4599ae5c4a..337a8cabc2 100644
--- a/datafusion/core/src/datasource/memory.rs
+++ b/datafusion/core/src/datasource/memory.rs
@@ -435,12 +435,11 @@ mod tests {
             ],
         )?;
 
-        match MemTable::try_new(schema2, vec![vec![batch]]) {
-            Err(DataFusionError::Plan(e)) => {
-                assert_eq!("\"Mismatch between schema and batches\"", format!("{e:?}"))
-            }
-            _ => panic!("MemTable::new should have failed due to schema mismatch"),
-        }
+        let e = MemTable::try_new(schema2, vec![vec![batch]]).unwrap_err();
+        assert_eq!(
+            "Error during planning: Mismatch between schema and batches",
+            e.strip_backtrace()
+        );
 
         Ok(())
     }
@@ -466,12 +465,11 @@ mod tests {
             ],
         )?;
 
-        match MemTable::try_new(schema2, vec![vec![batch]]) {
-            Err(DataFusionError::Plan(e)) => {
-                assert_eq!("\"Mismatch between schema and batches\"", format!("{e:?}"))
-            }
-            _ => panic!("MemTable::new should have failed due to schema mismatch"),
-        }
+        let e = MemTable::try_new(schema2, vec![vec![batch]]).unwrap_err();
+        assert_eq!(
+            "Error during planning: Mismatch between schema and batches",
+            e.strip_backtrace()
+        );
 
         Ok(())
     }
diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs
index e692810381..ca0e388e56 100644
--- a/datafusion/core/src/datasource/physical_plan/csv.rs
+++ b/datafusion/core/src/datasource/physical_plan/csv.rs
@@ -835,7 +835,7 @@ mod tests {
 
         // errors due to https://github.com/apache/arrow-datafusion/issues/4918
         let mut it = csv.execute(0, task_ctx)?;
-        let err = it.next().await.unwrap().unwrap_err().to_string();
+        let err = it.next().await.unwrap().unwrap_err().strip_backtrace();
         assert_eq!(
             err,
             "Arrow error: Csv error: incorrect number of fields for line 1, expected 14 got 13"
@@ -1075,7 +1075,7 @@ mod tests {
             .write_csv(out_dir_url, DataFrameWriteOptions::new(), None)
             .await
             .expect_err("should fail because input file does not match inferred schema");
-        assert_eq!("Arrow error: Parser error: Error while parsing value d for column 0 at line 4", format!("{e}"));
+        assert_eq!(e.strip_backtrace(), "Arrow error: Parser error: Error while parsing value d for column 0 at line 4");
         Ok(())
     }
 
diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs
index ec5a4c1cef..04d161355e 100644
--- a/datafusion/core/src/datasource/physical_plan/json.rs
+++ b/datafusion/core/src/datasource/physical_plan/json.rs
@@ -794,7 +794,7 @@ mod tests {
             .write_json(out_dir_url, DataFrameWriteOptions::new())
             .await
             .expect_err("should fail because input file does not match inferred schema");
-        assert_eq!("Arrow error: Parser error: Error while parsing value d for column 0 at line 4", format!("{e}"));
+        assert_eq!(e.strip_backtrace(), "Arrow error: Parser error: Error while parsing value d for column 0 at line 4");
         Ok(())
     }
 
diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs
index 861a37a302..53e739b375 100644
--- a/datafusion/core/src/datasource/physical_plan/parquet.rs
+++ b/datafusion/core/src/datasource/physical_plan/parquet.rs
@@ -936,7 +936,7 @@ mod tests {
             .write_parquet(out_dir_url, DataFrameWriteOptions::new(), None)
             .await
             .expect_err("should fail because input file does not match inferred schema");
-        assert_eq!("Arrow error: Parser error: Error while parsing value d for column 0 at line 4", format!("{e}"));
+        assert_eq!(e.strip_backtrace(), "Arrow error: Parser error: Error while parsing value d for column 0 at line 4");
         Ok(())
     }
 
diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs
index bcbfb51570..12b0f27229 100644
--- a/datafusion/core/src/execution/context.rs
+++ b/datafusion/core/src/execution/context.rs
@@ -426,9 +426,8 @@ impl SessionContext {
     /// let err = ctx.sql_with_options("CREATE TABLE foo (x INTEGER)", options)
     ///   .await
     ///   .unwrap_err();
-    /// assert_eq!(
-    ///   err.to_string(),
-    ///   "Error during planning: DDL not supported: CreateMemoryTable"
+    /// assert!(
+    ///   err.to_string().starts_with("Error during planning: DDL not supported: CreateMemoryTable")
     /// );
     /// # Ok(())
     /// # }
@@ -2356,9 +2355,8 @@ mod tests {
         let ctx = SessionContext::new();
 
         let err = plan_and_collect(&ctx, "SElECT @=   X3").await.unwrap_err();
-
         assert_eq!(
-            err.to_string(),
+            err.strip_backtrace(),
             "Error during planning: variable [\"@=\"] has no type information"
         );
         Ok(())
diff --git a/datafusion/core/src/physical_plan/stream.rs b/datafusion/core/src/physical_plan/stream.rs
index 1147f28864..a688e5dcbd 100644
--- a/datafusion/core/src/physical_plan/stream.rs
+++ b/datafusion/core/src/physical_plan/stream.rs
@@ -450,7 +450,7 @@ mod test {
         // get the first result, which should be an error
         let first_batch = stream.next().await.unwrap();
         let first_err = first_batch.unwrap_err();
-        assert_eq!(first_err.to_string(), "Execution error: Test1");
+        assert_eq!(first_err.strip_backtrace(), "Execution error: Test1");
 
         // There should be no more batches produced (should not get the second error)
         assert!(stream.next().await.is_none());
diff --git a/datafusion/core/tests/config_from_env.rs b/datafusion/core/tests/config_from_env.rs
index a420f5c9f5..a5a5a4524e 100644
--- a/datafusion/core/tests/config_from_env.rs
+++ b/datafusion/core/tests/config_from_env.rs
@@ -36,7 +36,7 @@ fn from_env() {
 
     // for invalid testing
     env::set_var(env_key, "abc");
-    let err = ConfigOptions::from_env().unwrap_err().to_string();
+    let err = ConfigOptions::from_env().unwrap_err().strip_backtrace();
     assert_eq!(err, "Error parsing abc as usize\ncaused by\nExternal error: invalid digit found in string");
 
     env::remove_var(env_key);
diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs
index bda6e5d929..3e0a2ec826 100644
--- a/datafusion/core/tests/dataframe/mod.rs
+++ b/datafusion/core/tests/dataframe/mod.rs
@@ -482,7 +482,7 @@ async fn sort_on_distinct_unprojected_columns() -> Result<()> {
         .distinct()?
         .sort(vec![Expr::Sort(Sort::new(Box::new(col("b")), false, true))])
         .unwrap_err();
-    assert_eq!(err.to_string(), "Error during planning: For SELECT DISTINCT, ORDER BY expressions b must appear in select list");
+    assert_eq!(err.strip_backtrace(), "Error during planning: For SELECT DISTINCT, ORDER BY expressions b must appear in select list");
     Ok(())
 }
 
@@ -501,7 +501,7 @@ async fn sort_on_ambiguous_column() -> Result<()> {
         .unwrap_err();
 
     let expected = "Schema error: Ambiguous reference to unqualified field b";
-    assert_eq!(err.to_string(), expected);
+    assert_eq!(err.strip_backtrace(), expected);
     Ok(())
 }
 
@@ -520,7 +520,7 @@ async fn group_by_ambiguous_column() -> Result<()> {
         .unwrap_err();
 
     let expected = "Schema error: Ambiguous reference to unqualified field b";
-    assert_eq!(err.to_string(), expected);
+    assert_eq!(err.strip_backtrace(), expected);
     Ok(())
 }
 
@@ -539,7 +539,7 @@ async fn filter_on_ambiguous_column() -> Result<()> {
         .unwrap_err();
 
     let expected = "Schema error: Ambiguous reference to unqualified field b";
-    assert_eq!(err.to_string(), expected);
+    assert_eq!(err.strip_backtrace(), expected);
     Ok(())
 }
 
@@ -558,7 +558,7 @@ async fn select_ambiguous_column() -> Result<()> {
         .unwrap_err();
 
     let expected = "Schema error: Ambiguous reference to unqualified field b";
-    assert_eq!(err.to_string(), expected);
+    assert_eq!(err.strip_backtrace(), expected);
     Ok(())
 }
 
diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs
index 6ba8474ee1..1aaa01bf4f 100644
--- a/datafusion/core/tests/sql/joins.rs
+++ b/datafusion/core/tests/sql/joins.rs
@@ -231,7 +231,7 @@ async fn join_change_in_planner_without_sort_not_allowed() -> Result<()> {
     match df.create_physical_plan().await {
         Ok(_) => panic!("Expecting error."),
         Err(e) => {
-            assert_eq!(e.to_string(), "PipelineChecker\ncaused by\nError during planning: Join operation cannot operate on a non-prunable stream without enabling the 'allow_symmetric_joins_without_pruning' configuration flag")
+            assert_eq!(e.strip_backtrace(), "PipelineChecker\ncaused by\nError during planning: Join operation cannot operate on a non-prunable stream without enabling the 'allow_symmetric_joins_without_pruning' configuration flag")
         }
     }
     Ok(())
diff --git a/datafusion/core/tests/sql/order.rs b/datafusion/core/tests/sql/order.rs
index d676ac731f..c5497b4cc0 100644
--- a/datafusion/core/tests/sql/order.rs
+++ b/datafusion/core/tests/sql/order.rs
@@ -70,7 +70,7 @@ async fn create_external_table_with_ddl_ordered_non_cols() -> Result<()> {
         Ok(_) => panic!("Expecting error."),
         Err(e) => {
             assert_eq!(
-                e.to_string(),
+                e.strip_backtrace(),
                 "Error during planning: Column a is not in schema"
             )
         }
@@ -85,7 +85,7 @@ async fn create_external_table_with_ddl_ordered_without_schema() -> Result<()> {
     match ctx.state().create_logical_plan(sql).await {
         Ok(_) => panic!("Expecting error."),
         Err(e) => {
-            assert_eq!(e.to_string(), "Error during planning: Provide a schema before specifying the order while creating a table.")
+            assert_eq!(e.strip_backtrace(), "Error during planning: Provide a schema before specifying the order while creating a table.")
         }
     }
     Ok(())
diff --git a/datafusion/core/tests/sql/sql_api.rs b/datafusion/core/tests/sql/sql_api.rs
index 4f249a8656..d7adc9611b 100644
--- a/datafusion/core/tests/sql/sql_api.rs
+++ b/datafusion/core/tests/sql/sql_api.rs
@@ -30,7 +30,7 @@ async fn unsupported_ddl_returns_error() {
     let sql = "create view test_view as select * from test";
     let df = ctx.sql_with_options(sql, options).await;
     assert_eq!(
-        df.unwrap_err().to_string(),
+        df.unwrap_err().strip_backtrace(),
         "Error during planning: DDL not supported: CreateView"
     );
 
@@ -49,7 +49,7 @@ async fn unsupported_dml_returns_error() {
     let sql = "insert into test values (1)";
     let df = ctx.sql_with_options(sql, options).await;
     assert_eq!(
-        df.unwrap_err().to_string(),
+        df.unwrap_err().strip_backtrace(),
         "Error during planning: DML not supported: Insert Into"
     );
 
@@ -70,7 +70,7 @@ async fn unsupported_copy_returns_error() {
     let sql = format!("copy (values(1)) to '{}'", tmpfile.to_string_lossy());
     let df = ctx.sql_with_options(&sql, options).await;
     assert_eq!(
-        df.unwrap_err().to_string(),
+        df.unwrap_err().strip_backtrace(),
         "Error during planning: DML not supported: COPY"
     );
 
@@ -88,7 +88,7 @@ async fn unsupported_statement_returns_error() {
     let sql = "set datafusion.execution.batch_size = 5";
     let df = ctx.sql_with_options(sql, options).await;
     assert_eq!(
-        df.unwrap_err().to_string(),
+        df.unwrap_err().strip_backtrace(),
         "Error during planning: Statement not supported: SetVariable"
     );
 
@@ -110,7 +110,7 @@ async fn ddl_can_not_be_planned_by_session_state() {
     let plan = state.create_logical_plan(sql).await.unwrap();
     let physical_plan = state.create_physical_plan(&plan).await;
     assert_eq!(
-        physical_plan.unwrap_err().to_string(),
+        physical_plan.unwrap_err().strip_backtrace(),
         "This feature is not implemented: Unsupported logical plan: DropTable"
     );
 }
diff --git a/datafusion/core/tests/sql/timestamp.rs b/datafusion/core/tests/sql/timestamp.rs
index 611e7c2228..09bbd17542 100644
--- a/datafusion/core/tests/sql/timestamp.rs
+++ b/datafusion/core/tests/sql/timestamp.rs
@@ -795,7 +795,7 @@ async fn test_cast_to_time_with_time_zone_should_not_work() -> Result<()> {
     let results = plan_and_collect(&ctx, sql).await.unwrap_err();
 
     assert_eq!(
-        results.to_string(),
+        results.strip_backtrace(),
         "This feature is not implemented: Unsupported SQL type Time(None, WithTimeZone)"
     );
 
@@ -828,7 +828,7 @@ async fn test_cast_to_timetz_should_not_work() -> Result<()> {
     let results = plan_and_collect(&ctx, sql).await.unwrap_err();
 
     assert_eq!(
-        results.to_string(),
+        results.strip_backtrace(),
         "This feature is not implemented: Unsupported SQL type Time(None, Tz)"
     );
     Ok(())
diff --git a/datafusion/execution/src/disk_manager.rs b/datafusion/execution/src/disk_manager.rs
index e8d2ed9cc0..ecae698523 100644
--- a/datafusion/execution/src/disk_manager.rs
+++ b/datafusion/execution/src/disk_manager.rs
@@ -220,7 +220,7 @@ mod tests {
         let manager = DiskManager::try_new(config).unwrap();
         assert!(!manager.tmp_files_enabled());
         assert_eq!(
-            manager.create_tmp_file("Testing").unwrap_err().to_string(),
+            manager.create_tmp_file("Testing").unwrap_err().strip_backtrace(),
             "Resources exhausted: Memory Exhausted while Testing (DiskManager is disabled)",
         )
     }
diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs
index cc48c34b8e..fc49c5fa94 100644
--- a/datafusion/execution/src/memory_pool/pool.rs
+++ b/datafusion/execution/src/memory_pool/pool.rs
@@ -261,10 +261,10 @@ mod tests {
 
         assert_eq!(pool.reserved(), 4000);
 
-        let err = r2.try_grow(1).unwrap_err().to_string();
+        let err = r2.try_grow(1).unwrap_err().strip_backtrace();
         assert_eq!(err, "Resources exhausted: Failed to allocate additional 1 bytes for r2 with 2000 bytes already allocated - maximum available is 0");
 
-        let err = r2.try_grow(1).unwrap_err().to_string();
+        let err = r2.try_grow(1).unwrap_err().strip_backtrace();
         assert_eq!(err, "Resources exhausted: Failed to allocate additional 1 bytes for r2 with 2000 bytes already allocated - maximum available is 0");
 
         r1.shrink(1990);
@@ -289,12 +289,12 @@ mod tests {
             .with_can_spill(true)
             .register(&pool);
 
-        let err = r3.try_grow(70).unwrap_err().to_string();
+        let err = r3.try_grow(70).unwrap_err().strip_backtrace();
         assert_eq!(err, "Resources exhausted: Failed to allocate additional 70 bytes for r3 with 0 bytes already allocated - maximum available is 40");
 
         //Shrinking r2 to zero doesn't allow a3 to allocate more than 45
         r2.free();
-        let err = r3.try_grow(70).unwrap_err().to_string();
+        let err = r3.try_grow(70).unwrap_err().strip_backtrace();
         assert_eq!(err, "Resources exhausted: Failed to allocate additional 70 bytes for r3 with 0 bytes already allocated - maximum available is 40");
 
         // But dropping r2 does
@@ -307,7 +307,7 @@ mod tests {
         assert_eq!(pool.reserved(), 80);
 
         let mut r4 = MemoryConsumer::new("s4").register(&pool);
-        let err = r4.try_grow(30).unwrap_err().to_string();
+        let err = r4.try_grow(30).unwrap_err().strip_backtrace();
         assert_eq!(err, "Resources exhausted: Failed to allocate additional 30 bytes for s4 with 0 bytes already allocated - maximum available is 20");
     }
 }
diff --git a/datafusion/execution/src/object_store.rs b/datafusion/execution/src/object_store.rs
index 7bedf0c865..5a1cdb7690 100644
--- a/datafusion/execution/src/object_store.rs
+++ b/datafusion/execution/src/object_store.rs
@@ -234,20 +234,20 @@ mod tests {
         assert_eq!(url.as_str(), "s3://username:password@host:123/");
 
         let err = ObjectStoreUrl::parse("s3://bucket:invalid").unwrap_err();
-        assert_eq!(err.to_string(), "External error: invalid port number");
+        assert_eq!(err.strip_backtrace(), "External error: invalid port number");
 
         let err = ObjectStoreUrl::parse("s3://bucket?").unwrap_err();
-        assert_eq!(err.to_string(), "Execution error: ObjectStoreUrl must only contain scheme and authority, got: ?");
+        assert_eq!(err.strip_backtrace(), "Execution error: ObjectStoreUrl must only contain scheme and authority, got: ?");
 
         let err = ObjectStoreUrl::parse("s3://bucket?foo=bar").unwrap_err();
-        assert_eq!(err.to_string(), "Execution error: ObjectStoreUrl must only contain scheme and authority, got: ?foo=bar");
+        assert_eq!(err.strip_backtrace(), "Execution error: ObjectStoreUrl must only contain scheme and authority, got: ?foo=bar");
 
         let err = ObjectStoreUrl::parse("s3://host:123/foo").unwrap_err();
-        assert_eq!(err.to_string(), "Execution error: ObjectStoreUrl must only contain scheme and authority, got: /foo");
+        assert_eq!(err.strip_backtrace(), "Execution error: ObjectStoreUrl must only contain scheme and authority, got: /foo");
 
         let err =
             ObjectStoreUrl::parse("s3://username:password@host:123/foo").unwrap_err();
-        assert_eq!(err.to_string(), "Execution error: ObjectStoreUrl must only contain scheme and authority, got: /foo");
+        assert_eq!(err.strip_backtrace(), "Execution error: ObjectStoreUrl must only contain scheme and authority, got: /foo");
     }
 
     #[test]
diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs
index acb8d23c0d..1f04c80833 100644
--- a/datafusion/expr/src/expr_rewriter/mod.rs
+++ b/datafusion/expr/src/expr_rewriter/mod.rs
@@ -374,7 +374,7 @@ mod test {
         let error =
             normalize_col_with_schemas_and_ambiguity_check(expr, &[&schemas], &[])
                 .unwrap_err()
-                .to_string();
+                .strip_backtrace();
         assert_eq!(
             error,
             r#"Schema error: No field named b. Valid fields are "tableA".a."#
diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs
index eb7833504b..eb2123b7c2 100644
--- a/datafusion/expr/src/logical_plan/builder.rs
+++ b/datafusion/expr/src/logical_plan/builder.rs
@@ -1521,7 +1521,7 @@ mod tests {
         let err =
             LogicalPlanBuilder::scan("", table_source(&schema), projection).unwrap_err();
         assert_eq!(
-            err.to_string(),
+            err.strip_backtrace(),
             "Error during planning: table_name cannot be empty"
         );
     }
@@ -1650,8 +1650,8 @@ mod tests {
         let err_msg1 = plan1.clone().union(plan2.clone().build()?).unwrap_err();
         let err_msg2 = plan1.union_distinct(plan2.build()?).unwrap_err();
 
-        assert_eq!(err_msg1.to_string(), expected);
-        assert_eq!(err_msg2.to_string(), expected);
+        assert_eq!(err_msg1.strip_backtrace(), expected);
+        assert_eq!(err_msg2.strip_backtrace(), expected);
 
         Ok(())
     }
@@ -1875,7 +1875,7 @@ mod tests {
             LogicalPlanBuilder::intersect(plan1.build()?, plan2.build()?, true)
                 .unwrap_err();
 
-        assert_eq!(err_msg1.to_string(), expected);
+        assert_eq!(err_msg1.strip_backtrace(), expected);
 
         Ok(())
     }
diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs
index 9f2d90accc..c58ec92174 100644
--- a/datafusion/expr/src/logical_plan/plan.rs
+++ b/datafusion/expr/src/logical_plan/plan.rs
@@ -2597,14 +2597,11 @@ digraph {
             ..Default::default()
         };
         let plan = test_plan();
-        let res = plan.visit(&mut visitor);
-
-        if let Err(DataFusionError::NotImplemented(e)) = res {
-            assert_eq!("Error in pre_visit", e);
-        } else {
-            panic!("Expected an error");
-        }
-
+        let res = plan.visit(&mut visitor).unwrap_err();
+        assert_eq!(
+            "This feature is not implemented: Error in pre_visit",
+            res.strip_backtrace()
+        );
         assert_eq!(
             visitor.inner.strings,
             vec!["pre_visit Projection", "pre_visit Filter"]
@@ -2618,13 +2615,11 @@ digraph {
             ..Default::default()
         };
         let plan = test_plan();
-        let res = plan.visit(&mut visitor);
-        if let Err(DataFusionError::NotImplemented(e)) = res {
-            assert_eq!("Error in post_visit", e);
-        } else {
-            panic!("Expected an error");
-        }
-
+        let res = plan.visit(&mut visitor).unwrap_err();
+        assert_eq!(
+            "This feature is not implemented: Error in post_visit",
+            res.strip_backtrace()
+        );
         assert_eq!(
             visitor.inner.strings,
             vec![
@@ -2647,7 +2642,7 @@ digraph {
             })),
             empty_schema,
         );
-        assert_eq!("Error during planning: Projection has mismatch between number of expressions (1) and number of fields in schema (0)", format!("{}", p.err().unwrap()));
+        assert_eq!(p.err().unwrap().strip_backtrace(), "Error during planning: Projection has mismatch between number of expressions (1) and number of fields in schema (0)");
         Ok(())
     }
 
diff --git a/datafusion/expr/src/type_coercion/aggregates.rs b/datafusion/expr/src/type_coercion/aggregates.rs
index d087f2c030..261c406d5d 100644
--- a/datafusion/expr/src/type_coercion/aggregates.rs
+++ b/datafusion/expr/src/type_coercion/aggregates.rs
@@ -577,7 +577,7 @@ mod tests {
         let input_types = vec![DataType::Int64, DataType::Int32];
         let signature = fun.signature();
         let result = coerce_types(&fun, &input_types, &signature);
-        assert_eq!("Error during planning: The function Min expects 1 arguments, but 2 were provided", result.unwrap_err().to_string());
+        assert_eq!("Error during planning: The function Min expects 1 arguments, but 2 were provided", result.unwrap_err().strip_backtrace());
 
         // test input args is invalid data type for sum or avg
         let fun = AggregateFunction::Sum;
@@ -586,14 +586,14 @@ mod tests {
         let result = coerce_types(&fun, &input_types, &signature);
         assert_eq!(
             "Error during planning: The function Sum does not support inputs of type Utf8.",
-            result.unwrap_err().to_string()
+            result.unwrap_err().strip_backtrace()
         );
         let fun = AggregateFunction::Avg;
         let signature = fun.signature();
         let result = coerce_types(&fun, &input_types, &signature);
         assert_eq!(
             "Error during planning: The function Avg does not support inputs of type Utf8.",
-            result.unwrap_err().to_string()
+            result.unwrap_err().strip_backtrace()
         );
 
         // test count, array_agg, approx_distinct, min, max.
diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs
index fd456402dc..64f814cd95 100644
--- a/datafusion/expr/src/type_coercion/binary.rs
+++ b/datafusion/expr/src/type_coercion/binary.rs
@@ -779,8 +779,8 @@ fn null_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option<DataType> {
 mod tests {
     use arrow::datatypes::DataType;
 
+    use datafusion_common::assert_contains;
     use datafusion_common::Result;
-    use datafusion_common::{assert_contains, internal_err, DataFusionError};
 
     use crate::Operator;
 
@@ -791,15 +791,9 @@ mod tests {
         let result_type =
             get_input_types(&DataType::Float32, &Operator::Plus, &DataType::Utf8);
 
-        if let Err(DataFusionError::Plan(e)) = result_type {
-            assert_eq!(
-                e,
-                "Cannot coerce arithmetic expression Float32 + Utf8 to valid types"
-            );
-            Ok(())
-        } else {
-            internal_err!("Coercion should have returned an DataFusionError::Internal")
-        }
+        let e = result_type.unwrap_err();
+        assert_eq!(e.strip_backtrace(), "Error during planning: Cannot coerce arithmetic expression Float32 + Utf8 to valid types");
+        Ok(())
     }
 
     #[test]
diff --git a/datafusion/expr/src/window_frame.rs b/datafusion/expr/src/window_frame.rs
index 5c9cfcae66..5f161b85dd 100644
--- a/datafusion/expr/src/window_frame.rs
+++ b/datafusion/expr/src/window_frame.rs
@@ -23,7 +23,7 @@
 //! - An ending frame boundary,
 //! - An EXCLUDE clause.
 
-use datafusion_common::{plan_err, DataFusionError, Result, ScalarValue};
+use datafusion_common::{plan_err, sql_err, DataFusionError, Result, ScalarValue};
 use sqlparser::ast;
 use sqlparser::parser::ParserError::ParserError;
 use std::convert::{From, TryFrom};
@@ -241,9 +241,9 @@ pub fn convert_frame_bound_to_scalar_value(v: ast::Expr) -> Result<ScalarValue>
             let result = match *value {
                 ast::Expr::Value(ast::Value::SingleQuotedString(item)) => item,
                 e => {
-                    return Err(DataFusionError::SQL(ParserError(format!(
+                    return sql_err!(ParserError(format!(
                         "INTERVAL expression cannot be {e:?}"
-                    ))));
+                    )));
                 }
             };
             if let Some(leading_field) = leading_field {
@@ -332,7 +332,7 @@ mod tests {
         };
         let err = WindowFrame::try_from(window_frame).unwrap_err();
         assert_eq!(
-            err.to_string(),
+            err.strip_backtrace(),
             "Error during planning: Invalid window frame: start bound cannot be UNBOUNDED FOLLOWING".to_owned()
         );
 
@@ -343,7 +343,7 @@ mod tests {
         };
         let err = WindowFrame::try_from(window_frame).unwrap_err();
         assert_eq!(
-            err.to_string(),
+            err.strip_backtrace(),
             "Error during planning: Invalid window frame: end bound cannot be UNBOUNDED PRECEDING".to_owned()
         );
 
diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs
index c61a8d3350..6f258e6df3 100644
--- a/datafusion/optimizer/src/analyzer/type_coercion.rs
+++ b/datafusion/optimizer/src/analyzer/type_coercion.rs
@@ -873,8 +873,8 @@ mod test {
             .err()
             .unwrap();
         assert_eq!(
-            r#"Context("type_coercion", Plan("Coercion from [Utf8] to the signature Uniform(1, [Int32]) failed."))"#,
-            &format!("{err:?}")
+            "type_coercion\ncaused by\nError during planning: Coercion from [Utf8] to the signature Uniform(1, [Int32]) failed.",
+            err.strip_backtrace()
         );
         Ok(())
     }
@@ -943,8 +943,8 @@ mod test {
             .err()
             .unwrap();
         assert_eq!(
-            r#"Context("type_coercion", Plan("Coercion from [Utf8] to the signature Uniform(1, [Float64]) failed."))"#,
-            &format!("{err:?}")
+            "type_coercion\ncaused by\nError during planning: Coercion from [Utf8] to the signature Uniform(1, [Float64]) failed.",
+            err.strip_backtrace()
         );
         Ok(())
     }
@@ -1456,7 +1456,7 @@ mod test {
         };
         let err = coerce_case_expression(case, &schema).unwrap_err();
         assert_eq!(
-            err.to_string(),
+            err.strip_backtrace(),
             "Error during planning: \
             Failed to coerce case (Interval(MonthDayNano)) and \
             when ([Float32, Binary, Utf8]) to common types in \
@@ -1474,7 +1474,7 @@ mod test {
         };
         let err = coerce_case_expression(case, &schema).unwrap_err();
         assert_eq!(
-            err.to_string(),
+            err.strip_backtrace(),
             "Error during planning: \
             Failed to coerce then ([Date32, Float32, Binary]) and \
             else (Some(Timestamp(Nanosecond, None))) to common types \
diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs
index 3ce4ecf1c8..d3bdd47c5c 100644
--- a/datafusion/optimizer/src/optimizer.rs
+++ b/datafusion/optimizer/src/optimizer.rs
@@ -483,7 +483,7 @@ mod tests {
         assert_eq!(
             "Optimizer rule 'bad rule' failed\ncaused by\n\
             Error during planning: rule failed",
-            err.to_string()
+            err.strip_backtrace()
         );
     }
 
@@ -504,10 +504,10 @@ mod tests {
              DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"a\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, \
              DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"b\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, \
              DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"c\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }], \
-             metadata: {}, functional_dependencies: FunctionalDependencies { deps: [] } }. \
-             This was likely caused by a bug in DataFusion's code \
+             metadata: {}, functional_dependencies: FunctionalDependencies { deps: [] } }.\
+             \nThis was likely caused by a bug in DataFusion's code \
              and we would welcome that you file an bug report in our issue tracker",
-            err.to_string()
+            err.strip_backtrace()
         );
     }
 
diff --git a/datafusion/physical-expr/src/aggregate/min_max.rs b/datafusion/physical-expr/src/aggregate/min_max.rs
index db72c91ee1..14e515861d 100644
--- a/datafusion/physical-expr/src/aggregate/min_max.rs
+++ b/datafusion/physical-expr/src/aggregate/min_max.rs
@@ -1172,11 +1172,14 @@ mod tests {
 
         let right = ScalarValue::Decimal128(Some(124), 10, 3);
         let result = max(&left, &right);
-        let expect = DataFusionError::Internal(format!(
+        let err_msg = format!(
             "MIN/MAX is not expected to receive scalars of incompatible types {:?}",
             (Decimal128(Some(123), 10, 2), Decimal128(Some(124), 10, 3))
-        ));
-        assert_eq!(expect.to_string(), result.unwrap_err().to_string());
+        );
+        let expect = DataFusionError::Internal(err_msg);
+        assert!(expect
+            .strip_backtrace()
+            .starts_with(&result.unwrap_err().strip_backtrace()));
 
         // max batch
         let array: ArrayRef = Arc::new(
diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs
index 06432b615c..081345f8a5 100644
--- a/datafusion/physical-expr/src/array_expressions.rs
+++ b/datafusion/physical-expr/src/array_expressions.rs
@@ -3213,7 +3213,7 @@ mod tests {
 
         let array = array_append(&args);
 
-        assert_eq!(array.unwrap_err().to_string(), "Error during planning: array_append received incompatible types: '[Int64, Utf8]'.");
+        assert_eq!(array.unwrap_err().strip_backtrace(), "Error during planning: array_append received incompatible types: '[Int64, Utf8]'.");
     }
 
     fn return_array() -> ColumnarValue {
diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs
index 002332ab83..2db0818082 100644
--- a/datafusion/physical-expr/src/datetime_expressions.rs
+++ b/datafusion/physical-expr/src/datetime_expressions.rs
@@ -1061,7 +1061,7 @@ mod tests {
         let res =
             date_bin(&[ColumnarValue::Scalar(ScalarValue::IntervalDayTime(Some(1)))]);
         assert_eq!(
-            res.err().unwrap().to_string(),
+            res.err().unwrap().strip_backtrace(),
             "Execution error: DATE_BIN expected two or three arguments"
         );
 
@@ -1072,7 +1072,7 @@ mod tests {
             ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)),
         ]);
         assert_eq!(
-            res.err().unwrap().to_string(),
+            res.err().unwrap().strip_backtrace(),
             "Execution error: DATE_BIN expects stride argument to be an INTERVAL but got Interval(YearMonth)"
         );
 
@@ -1083,7 +1083,7 @@ mod tests {
             ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)),
         ]);
         assert_eq!(
-            res.err().unwrap().to_string(),
+            res.err().unwrap().strip_backtrace(),
             "Execution error: DATE_BIN stride must be non-zero"
         );
 
@@ -1094,7 +1094,7 @@ mod tests {
             ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)),
         ]);
         assert_eq!(
-            res.err().unwrap().to_string(),
+            res.err().unwrap().strip_backtrace(),
             "Execution error: DATE_BIN stride argument is too large"
         );
 
@@ -1105,7 +1105,7 @@ mod tests {
             ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)),
         ]);
         assert_eq!(
-            res.err().unwrap().to_string(),
+            res.err().unwrap().strip_backtrace(),
             "Execution error: DATE_BIN stride argument is too large"
         );
 
@@ -1116,7 +1116,7 @@ mod tests {
             ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)),
         ]);
         assert_eq!(
-            res.err().unwrap().to_string(),
+            res.err().unwrap().strip_backtrace(),
             "This feature is not implemented: DATE_BIN stride does not support combination of month, day and nanosecond intervals"
         );
 
@@ -1127,7 +1127,7 @@ mod tests {
             ColumnarValue::Scalar(ScalarValue::TimestampMicrosecond(Some(1), None)),
         ]);
         assert_eq!(
-            res.err().unwrap().to_string(),
+            res.err().unwrap().strip_backtrace(),
             "Execution error: DATE_BIN expects origin argument to be a TIMESTAMP with nanosececond precision but got Timestamp(Microsecond, None)"
         );
 
@@ -1146,7 +1146,7 @@ mod tests {
             ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)),
         ]);
         assert_eq!(
-            res.err().unwrap().to_string(),
+            res.err().unwrap().strip_backtrace(),
             "This feature is not implemented: DATE_BIN only supports literal values for the stride argument, not arrays"
         );
 
@@ -1158,7 +1158,7 @@ mod tests {
             ColumnarValue::Array(timestamps),
         ]);
         assert_eq!(
-            res.err().unwrap().to_string(),
+            res.err().unwrap().strip_backtrace(),
             "This feature is not implemented: DATE_BIN only supports literal values for the origin argument, not arrays"
         );
     }
diff --git a/datafusion/physical-expr/src/expressions/column.rs b/datafusion/physical-expr/src/expressions/column.rs
index 5955faa645..b7b5895db6 100644
--- a/datafusion/physical-expr/src/expressions/column.rs
+++ b/datafusion/physical-expr/src/expressions/column.rs
@@ -226,22 +226,20 @@ mod test {
     fn out_of_bounds_data_type() {
         let schema = Schema::new(vec![Field::new("foo", DataType::Utf8, true)]);
         let col = Column::new("id", 9);
-        let error = col.data_type(&schema).expect_err("error");
-        assert_eq!("Internal error: PhysicalExpr Column references column 'id' at index 9 (zero-based) \
-            but input schema only has 1 columns: [\"foo\"]. This was likely caused by a bug in \
-            DataFusion's code and we would welcome that you file an bug report in our issue tracker",
-           &format!("{error}"))
+        let error = col.data_type(&schema).expect_err("error").strip_backtrace();
+        assert!("Internal error: PhysicalExpr Column references column 'id' at index 9 (zero-based) \
+            but input schema only has 1 columns: [\"foo\"].\nThis was likely caused by a bug in \
+            DataFusion's code and we would welcome that you file an bug report in our issue tracker".starts_with(&error))
     }
 
     #[test]
     fn out_of_bounds_nullable() {
         let schema = Schema::new(vec![Field::new("foo", DataType::Utf8, true)]);
         let col = Column::new("id", 9);
-        let error = col.nullable(&schema).expect_err("error");
-        assert_eq!("Internal error: PhysicalExpr Column references column 'id' at index 9 (zero-based) \
-            but input schema only has 1 columns: [\"foo\"]. This was likely caused by a bug in \
-            DataFusion's code and we would welcome that you file an bug report in our issue tracker",
-                   &format!("{error}"))
+        let error = col.nullable(&schema).expect_err("error").strip_backtrace();
+        assert!("Internal error: PhysicalExpr Column references column 'id' at index 9 (zero-based) \
+            but input schema only has 1 columns: [\"foo\"].\nThis was likely caused by a bug in \
+            DataFusion's code and we would welcome that you file an bug report in our issue tracker".starts_with(&error))
     }
 
     #[test]
@@ -250,11 +248,10 @@ mod test {
         let data: StringArray = vec!["data"].into();
         let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(data)])?;
         let col = Column::new("id", 9);
-        let error = col.evaluate(&batch).expect_err("error");
-        assert_eq!("Internal error: PhysicalExpr Column references column 'id' at index 9 (zero-based) \
-            but input schema only has 1 columns: [\"foo\"]. This was likely caused by a bug in \
-            DataFusion's code and we would welcome that you file an bug report in our issue tracker",
-                   &format!("{error}"));
+        let error = col.evaluate(&batch).expect_err("error").strip_backtrace();
+        assert!("Internal error: PhysicalExpr Column references column 'id' at index 9 (zero-based) \
+            but input schema only has 1 columns: [\"foo\"].\nThis was likely caused by a bug in \
+            DataFusion's code and we would welcome that you file an bug report in our issue tracker".starts_with(&error));
         Ok(())
     }
 }
diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs
index 5081b817ce..233b903d11 100644
--- a/datafusion/physical-expr/src/functions.rs
+++ b/datafusion/physical-expr/src/functions.rs
@@ -1046,7 +1046,7 @@ mod tests {
                     match expr.evaluate(&batch) {
                         Ok(_) => assert!(false, "expected error"),
                         Err(error) => {
-                            assert_eq!(error.to_string(), expected_error.to_string());
+                            assert!(expected_error.strip_backtrace().starts_with(&error.strip_backtrace()));
                         }
                     }
                 }
diff --git a/datafusion/physical-expr/src/regex_expressions.rs b/datafusion/physical-expr/src/regex_expressions.rs
index 5aea70f75a..41cd019495 100644
--- a/datafusion/physical-expr/src/regex_expressions.rs
+++ b/datafusion/physical-expr/src/regex_expressions.rs
@@ -392,7 +392,7 @@ mod tests {
             regexp_match::<i32>(&[Arc::new(values), Arc::new(patterns), Arc::new(flags)])
                 .expect_err("unsupported flag should have failed");
 
-        assert_eq!(re_err.to_string(), "Error during planning: regexp_match() does not support the \"global\" option");
+        assert_eq!(re_err.strip_backtrace(), "Error during planning: regexp_match() does not support the \"global\" option");
     }
 
     #[test]
@@ -500,7 +500,7 @@ mod tests {
         ]);
         let pattern_err = re.expect_err("broken pattern should have failed");
         assert_eq!(
-            pattern_err.to_string(),
+            pattern_err.strip_backtrace(),
             "External error: regex parse error:\n    [\n    ^\nerror: unclosed character class"
         );
     }
diff --git a/datafusion/sql/src/expr/identifier.rs b/datafusion/sql/src/expr/identifier.rs
index 5a081136cc..5e03f14e53 100644
--- a/datafusion/sql/src/expr/identifier.rs
+++ b/datafusion/sql/src/expr/identifier.rs
@@ -429,10 +429,10 @@ mod test {
     #[test]
     fn test_form_identifier() -> Result<()> {
         let err = form_identifier(&[]).expect_err("empty identifiers didn't fail");
-        let expected = "Internal error: Incorrect number of identifiers: 0. \
+        let expected = "Internal error: Incorrect number of identifiers: 0.\n\
         This was likely caused by a bug in DataFusion's code and we would \
         welcome that you file an bug report in our issue tracker";
-        assert_eq!(err.to_string(), expected);
+        assert!(expected.starts_with(&err.strip_backtrace()));
 
         let ids = vec!["a".to_string()];
         let (qualifier, column) = form_identifier(&ids)?;
@@ -467,10 +467,10 @@ mod test {
             "e".to_string(),
         ])
         .expect_err("too many identifiers didn't fail");
-        let expected = "Internal error: Incorrect number of identifiers: 5. \
+        let expected = "Internal error: Incorrect number of identifiers: 5.\n\
         This was likely caused by a bug in DataFusion's code and we would \
         welcome that you file an bug report in our issue tracker";
-        assert_eq!(err.to_string(), expected);
+        assert!(expected.starts_with(&err.strip_backtrace()));
 
         Ok(())
     }
diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs
index 9cd141e43a..fc2a3fb9a5 100644
--- a/datafusion/sql/src/query.rs
+++ b/datafusion/sql/src/query.rs
@@ -20,7 +20,7 @@ use std::sync::Arc;
 use crate::planner::{ContextProvider, PlannerContext, SqlToRel};
 
 use datafusion_common::{
-    not_impl_err, plan_err, Constraints, DataFusionError, Result, ScalarValue,
+    not_impl_err, plan_err, sql_err, Constraints, DataFusionError, Result, ScalarValue,
 };
 use datafusion_expr::{
     CreateMemoryTable, DdlStatement, Expr, LogicalPlan, LogicalPlanBuilder,
@@ -61,9 +61,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> {
                 // A `WITH` block can't use the same name more than once
                 let cte_name = self.normalizer.normalize(cte.alias.name.clone());
                 if planner_context.contains_cte(&cte_name) {
-                    return Err(DataFusionError::SQL(ParserError(format!(
+                    return sql_err!(ParserError(format!(
                         "WITH query name {cte_name:?} specified more than once"
-                    ))));
+                    )));
                 }
                 // create logical plan & pass backreferencing CTEs
                 // CTE expr don't need extend outer_query_schema
diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs
index 154bd3f9a0..c1d1415dc7 100644
--- a/datafusion/sql/tests/sql_integration.rs
+++ b/datafusion/sql/tests/sql_integration.rs
@@ -96,7 +96,7 @@ fn parse_ident_normalization() {
         ),
         (
             "SELECT AGE FROM PERSON",
-            "Err(Plan(\"No table named: PERSON found\"))",
+            "Error during planning: No table named: PERSON found",
             false,
         ),
         (
@@ -121,7 +121,11 @@ fn parse_ident_normalization() {
                 enable_ident_normalization,
             },
         );
-        assert_eq!(expected, format!("{plan:?}"));
+        if plan.is_ok() {
+            assert_eq!(expected, format!("{plan:?}"));
+        } else {
+            assert_eq!(expected, plan.unwrap_err().strip_backtrace());
+        }
     }
 }
 
@@ -197,8 +201,8 @@ fn cast_to_invalid_decimal_type_precision_0() {
         let sql = "SELECT CAST(10 AS DECIMAL(0))";
         let err = logical_plan(sql).expect_err("query should have failed");
         assert_eq!(
-            r#"Plan("Decimal(precision = 0, scale = 0) should satisfy `0 < precision <= 38`, and `scale <= precision`.")"#,
-            format!("{err:?}")
+            "Error during planning: Decimal(precision = 0, scale = 0) should satisfy `0 < precision <= 38`, and `scale <= precision`.",
+            err.strip_backtrace()
         );
     }
 }
@@ -210,8 +214,8 @@ fn cast_to_invalid_decimal_type_precision_gt_38() {
         let sql = "SELECT CAST(10 AS DECIMAL(39))";
         let err = logical_plan(sql).expect_err("query should have failed");
         assert_eq!(
-            r#"Plan("Decimal(precision = 39, scale = 0) should satisfy `0 < precision <= 38`, and `scale <= precision`.")"#,
-            format!("{err:?}")
+            "Error during planning: Decimal(precision = 39, scale = 0) should satisfy `0 < precision <= 38`, and `scale <= precision`.",
+            err.strip_backtrace()
         );
     }
 }
@@ -223,8 +227,8 @@ fn cast_to_invalid_decimal_type_precision_lt_scale() {
         let sql = "SELECT CAST(10 AS DECIMAL(5, 10))";
         let err = logical_plan(sql).expect_err("query should have failed");
         assert_eq!(
-            r#"Plan("Decimal(precision = 5, scale = 10) should satisfy `0 < precision <= 38`, and `scale <= precision`.")"#,
-            format!("{err:?}")
+            "Error during planning: Decimal(precision = 5, scale = 10) should satisfy `0 < precision <= 38`, and `scale <= precision`.",
+            err.strip_backtrace()
         );
     }
 }
@@ -461,7 +465,7 @@ Dml: op=[Insert Into] table=[test_decimal]
 #[test]
 fn test_insert_schema_errors(#[case] sql: &str, #[case] error: &str) {
     let err = logical_plan(sql).unwrap_err();
-    assert_eq!(err.to_string(), error)
+    assert_eq!(err.strip_backtrace(), error)
 }
 
 #[test]
@@ -513,8 +517,8 @@ fn select_repeated_column() {
     let sql = "SELECT age, age FROM person";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-        r#"Plan("Projections require unique expression names but the expression \"person.age\" at position 0 and \"person.age\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.")"#,
-        format!("{err:?}")
+        "Error during planning: Projections require unique expression names but the expression \"person.age\" at position 0 and \"person.age\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.",
+        err.strip_backtrace()
     );
 }
 
@@ -523,8 +527,8 @@ fn select_wildcard_with_repeated_column() {
     let sql = "SELECT *, age FROM person";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-        r#"Plan("Projections require unique expression names but the expression \"person.age\" at position 3 and \"person.age\" at position 8 have the same name. Consider aliasing (\"AS\") one of them.")"#,
-        format!("{err:?}")
+        "Error during planning: Projections require unique expression names but the expression \"person.age\" at position 3 and \"person.age\" at position 8 have the same name. Consider aliasing (\"AS\") one of them.",
+        err.strip_backtrace()
     );
 }
 
@@ -710,8 +714,8 @@ fn table_with_column_alias_number_cols() {
                    FROM lineitem l (a, b)";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-        "Plan(\"Source table contains 3 columns but only 2 names given as column alias\")",
-        format!("{err:?}")
+        "Error during planning: Source table contains 3 columns but only 2 names given as column alias",
+        err.strip_backtrace()
     );
 }
 
@@ -720,8 +724,8 @@ fn select_with_ambiguous_column() {
     let sql = "SELECT id FROM person a, person b";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-        "SchemaError(AmbiguousReference { field: Column { relation: None, name: \"id\" } })",
-        format!("{err:?}")
+        "Schema error: Ambiguous reference to unqualified field id",
+        err.strip_backtrace()
     );
 }
 
@@ -874,8 +878,8 @@ fn select_with_having() {
                    HAVING age > 100 AND age < 200";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-            "Plan(\"HAVING clause references: person.age > Int64(100) AND person.age < Int64(200) must appear in the GROUP BY clause or be used in an aggregate function\")",
-            format!("{err:?}")
+            "Error during planning: HAVING clause references: person.age > Int64(100) AND person.age < Int64(200) must appear in the GROUP BY clause or be used in an aggregate function",
+            err.strip_backtrace()
         );
 }
 
@@ -886,8 +890,8 @@ fn select_with_having_referencing_column_not_in_select() {
                    HAVING first_name = 'M'";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-            "Plan(\"HAVING clause references: person.first_name = Utf8(\\\"M\\\") must appear in the GROUP BY clause or be used in an aggregate function\")",
-            format!("{err:?}")
+            "Error during planning: HAVING clause references: person.first_name = Utf8(\"M\") must appear in the GROUP BY clause or be used in an aggregate function",
+            err.strip_backtrace()
         );
 }
 
@@ -899,8 +903,8 @@ fn select_with_having_refers_to_invalid_column() {
                    HAVING first_name = 'M'";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-            "Plan(\"HAVING clause references non-aggregate values: Expression person.first_name could not be resolved from available columns: person.id, MAX(person.age)\")",
-            format!("{err:?}")
+            "Error during planning: HAVING clause references non-aggregate values: Expression person.first_name could not be resolved from available columns: person.id, MAX(person.age)",
+            err.strip_backtrace()
         );
 }
 
@@ -911,8 +915,8 @@ fn select_with_having_referencing_column_nested_in_select_expression() {
                    HAVING age > 100";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-            "Plan(\"HAVING clause references: person.age > Int64(100) must appear in the GROUP BY clause or be used in an aggregate function\")",
-            format!("{err:?}")
+            "Error during planning: HAVING clause references: person.age > Int64(100) must appear in the GROUP BY clause or be used in an aggregate function",
+            err.strip_backtrace()
         );
 }
 
@@ -923,8 +927,8 @@ fn select_with_having_with_aggregate_not_in_select() {
                    HAVING MAX(age) > 100";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-            "Plan(\"Projection references non-aggregate values: Expression person.first_name could not be resolved from available columns: MAX(person.age)\")",
-            format!("{err:?}")
+            "Error during planning: Projection references non-aggregate values: Expression person.first_name could not be resolved from available columns: MAX(person.age)",
+            err.strip_backtrace()
         );
 }
 
@@ -959,10 +963,8 @@ fn select_aggregate_with_having_referencing_column_not_in_select() {
                    HAVING first_name = 'M'";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-        "Plan(\"HAVING clause references non-aggregate values: \
-            Expression person.first_name could not be resolved from available columns: \
-            COUNT(*)\")",
-        format!("{err:?}")
+        "Error during planning: HAVING clause references non-aggregate values: Expression person.first_name could not be resolved from available columns: COUNT(*)",
+        err.strip_backtrace()
     );
 }
 
@@ -1082,10 +1084,8 @@ fn select_aggregate_with_group_by_with_having_referencing_column_not_in_group_by
                    HAVING MAX(age) > 10 AND last_name = 'M'";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-        "Plan(\"HAVING clause references non-aggregate values: \
-            Expression person.last_name could not be resolved from available columns: \
-            person.first_name, MAX(person.age)\")",
-        format!("{err:?}")
+        "Error during planning: HAVING clause references non-aggregate values: Expression person.last_name could not be resolved from available columns: person.first_name, MAX(person.age)",
+        err.strip_backtrace()
     );
 }
 
@@ -1252,8 +1252,8 @@ fn select_simple_aggregate_repeated_aggregate() {
     let sql = "SELECT MIN(age), MIN(age) FROM person";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-        r#"Plan("Projections require unique expression names but the expression \"MIN(person.age)\" at position 0 and \"MIN(person.age)\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.")"#,
-        format!("{err:?}")
+        "Error during planning: Projections require unique expression names but the expression \"MIN(person.age)\" at position 0 and \"MIN(person.age)\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.",
+        err.strip_backtrace()
     );
 }
 
@@ -1293,8 +1293,8 @@ fn select_simple_aggregate_repeated_aggregate_with_repeated_aliases() {
     let sql = "SELECT MIN(age) AS a, MIN(age) AS a FROM person";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-        r#"Plan("Projections require unique expression names but the expression \"MIN(person.age) AS a\" at position 0 and \"MIN(person.age) AS a\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.")"#,
-        format!("{err:?}")
+        "Error during planning: Projections require unique expression names but the expression \"MIN(person.age) AS a\" at position 0 and \"MIN(person.age) AS a\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.",
+        err.strip_backtrace()
     );
 }
 
@@ -1323,8 +1323,8 @@ fn select_simple_aggregate_with_groupby_with_aliases_repeated() {
     let sql = "SELECT state AS a, MIN(age) AS a FROM person GROUP BY state";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-        r#"Plan("Projections require unique expression names but the expression \"person.state AS a\" at position 0 and \"MIN(person.age) AS a\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.")"#,
-        format!("{err:?}")
+        "Error during planning: Projections require unique expression names but the expression \"person.state AS a\" at position 0 and \"MIN(person.age) AS a\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.",
+        err.strip_backtrace()
     );
 }
 
@@ -1344,7 +1344,7 @@ fn select_simple_aggregate_with_groupby_and_column_in_group_by_does_not_exist()
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!("Schema error: No field named doesnotexist. Valid fields are \"SUM(person.age)\", \
         person.id, person.first_name, person.last_name, person.age, person.state, \
-        person.salary, person.birth_date, person.\"😀\".", format!("{err}"));
+        person.salary, person.birth_date, person.\"😀\".", err.strip_backtrace());
 }
 
 #[test]
@@ -1359,8 +1359,8 @@ fn select_interval_out_of_range() {
     let sql = "SELECT INTERVAL '100000000000000000 day'";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-        "ArrowError(InvalidArgumentError(\"Unable to represent 100000000000000000 days in a signed 32-bit integer\"))",
-        format!("{err:?}")
+        "Arrow error: Invalid argument error: Unable to represent 100000000000000000 days in a signed 32-bit integer",
+        err.strip_backtrace(),
     );
 }
 
@@ -1371,8 +1371,8 @@ fn select_array_no_common_type() {
 
     // HashSet doesn't guarantee order
     assert_contains!(
-        err.to_string(),
-        r#"Arrays with different types are not supported: "#
+        err.strip_backtrace(),
+        "This feature is not implemented: Arrays with different types are not supported: "
     );
 }
 
@@ -1387,8 +1387,8 @@ fn recursive_ctes() {
         select * from numbers;";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-        r#"NotImplemented("Recursive CTEs are not supported")"#,
-        format!("{err:?}")
+        "This feature is not implemented: Recursive CTEs are not supported",
+        err.strip_backtrace()
     );
 }
 
@@ -1397,8 +1397,8 @@ fn select_array_non_literal_type() {
     let sql = "SELECT [now()]";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-        r#"NotImplemented("Arrays with elements other than literal are not supported: now()")"#,
-        format!("{err:?}")
+        "This feature is not implemented: Arrays with elements other than literal are not supported: now()",
+        err.strip_backtrace()
     );
 }
 
@@ -1433,15 +1433,15 @@ fn select_simple_aggregate_with_groupby_position_out_of_range() {
     let sql = "SELECT state, MIN(age) FROM person GROUP BY 0";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-            "Plan(\"Projection references non-aggregate values: Expression person.state could not be resolved from available columns: Int64(0), MIN(person.age)\")",
-            format!("{err:?}")
+        "Error during planning: Projection references non-aggregate values: Expression person.state could not be resolved from available columns: Int64(0), MIN(person.age)",
+            err.strip_backtrace()
         );
 
     let sql2 = "SELECT state, MIN(age) FROM person GROUP BY 5";
     let err2 = logical_plan(sql2).expect_err("query should have failed");
     assert_eq!(
-            "Plan(\"Projection references non-aggregate values: Expression person.state could not be resolved from available columns: Int64(5), MIN(person.age)\")",
-            format!("{err2:?}")
+        "Error during planning: Projection references non-aggregate values: Expression person.state could not be resolved from available columns: Int64(5), MIN(person.age)",
+            err2.strip_backtrace()
         );
 }
 
@@ -1460,8 +1460,8 @@ fn select_simple_aggregate_with_groupby_aggregate_repeated() {
     let sql = "SELECT state, MIN(age), MIN(age) FROM person GROUP BY state";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-        r#"Plan("Projections require unique expression names but the expression \"MIN(person.age)\" at position 1 and \"MIN(person.age)\" at position 2 have the same name. Consider aliasing (\"AS\") one of them.")"#,
-        format!("{err:?}")
+        "Error during planning: Projections require unique expression names but the expression \"MIN(person.age)\" at position 1 and \"MIN(person.age)\" at position 2 have the same name. Consider aliasing (\"AS\") one of them.",
+        err.strip_backtrace()
     );
 }
 
@@ -1518,8 +1518,8 @@ fn select_simple_aggregate_with_groupby_non_column_expression_nested_and_not_res
     let sql = "SELECT ((age + 1) / 2) * (age + 9), MIN(first_name) FROM person GROUP BY age + 1";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-            "Plan(\"Projection references non-aggregate values: Expression person.age could not be resolved from available columns: person.age + Int64(1), MIN(person.first_name)\")",
-            format!("{err:?}")
+        "Error during planning: Projection references non-aggregate values: Expression person.age could not be resolved from available columns: person.age + Int64(1), MIN(person.first_name)",
+            err.strip_backtrace()
         );
 }
 
@@ -1527,8 +1527,9 @@ fn select_simple_aggregate_with_groupby_non_column_expression_nested_and_not_res
 fn select_simple_aggregate_with_groupby_non_column_expression_and_its_column_selected() {
     let sql = "SELECT age, MIN(first_name) FROM person GROUP BY age + 1";
     let err = logical_plan(sql).expect_err("query should have failed");
-    assert_eq!("Plan(\"Projection references non-aggregate values: Expression person.age could not be resolved from available columns: person.age + Int64(1), MIN(person.first_name)\")",
-            format!("{err:?}")
+    assert_eq!(
+        "Error during planning: Projection references non-aggregate values: Expression person.age could not be resolved from available columns: person.age + Int64(1), MIN(person.first_name)",
+            err.strip_backtrace()
         );
 }
 
@@ -1793,10 +1794,8 @@ fn select_7480_2() {
     let sql = "SELECT c1, c13, MIN(c12) FROM aggregate_test_100 GROUP BY c1";
     let err = logical_plan(sql).expect_err("query should have failed");
     assert_eq!(
-        "Plan(\"Projection references non-aggregate values: \
-            Expression aggregate_test_100.c13 could not be resolved from available columns: \
-            aggregate_test_100.c1, MIN(aggregate_test_100.c12)\")",
-        format!("{err:?}")
+        "Error during planning: Projection references non-aggregate values: Expression aggregate_test_100.c13 could not be resolved from available columns: aggregate_test_100.c1, MIN(aggregate_test_100.c12)",
+        err.strip_backtrace()
     );
 }
 
@@ -1869,8 +1868,8 @@ fn create_external_table_with_compression_type() {
     for sql in sqls {
         let err = logical_plan(sql).expect_err("query should have failed");
         assert_eq!(
-            "Plan(\"File compression type cannot be set for PARQUET, AVRO, or ARROW files.\")",
-            format!("{err:?}")
+            "Error during planning: File compression type cannot be set for PARQUET, AVRO, or ARROW files.",
+            err.strip_backtrace()
         );
     }
 }
@@ -2853,7 +2852,7 @@ fn cte_use_same_name_multiple_times() {
     let expected =
         "SQL error: ParserError(\"WITH query name \\\"a\\\" specified more than once\")";
     let result = logical_plan(sql).err().unwrap();
-    assert_eq!(result.to_string(), expected);
+    assert_eq!(result.strip_backtrace(), expected);
 }
 
 #[test]
@@ -3116,7 +3115,7 @@ fn cte_unbalanced_number_of_columns() {
 
     let expected = "Error during planning: Source table contains 3 columns but only 1 names given as column alias";
     let result = logical_plan(sql).err().unwrap();
-    assert_eq!(result.to_string(), expected);
+    assert_eq!(result.strip_backtrace(), expected);
 }
 
 #[test]
@@ -3247,7 +3246,7 @@ fn order_by_ambiguous_name() {
     let expected = "Schema error: Ambiguous reference to unqualified field age";
 
     let err = logical_plan(sql).unwrap_err();
-    assert_eq!(err.to_string(), expected);
+    assert_eq!(err.strip_backtrace(), expected);
 }
 
 #[test]
@@ -3256,7 +3255,7 @@ fn group_by_ambiguous_name() {
     let expected = "Schema error: Ambiguous reference to unqualified field age";
 
     let err = logical_plan(sql).unwrap_err();
-    assert_eq!(err.to_string(), expected);
+    assert_eq!(err.strip_backtrace(), expected);
 }
 
 #[test]
@@ -3519,7 +3518,7 @@ fn test_select_distinct_order_by() {
     let result = logical_plan(sql);
     assert!(result.is_err());
     let err = result.err().unwrap();
-    assert_eq!(err.to_string(), expected);
+    assert_eq!(err.strip_backtrace(), expected);
 }
 
 #[rstest]
@@ -3546,7 +3545,7 @@ fn test_select_distinct_order_by() {
 #[test]
 fn test_select_unsupported_syntax_errors(#[case] sql: &str, #[case] error: &str) {
     let err = logical_plan(sql).unwrap_err();
-    assert_eq!(err.to_string(), error)
+    assert_eq!(err.strip_backtrace(), error)
 }
 
 #[test]
@@ -3603,7 +3602,7 @@ fn test_ambiguous_column_references_in_on_join() {
     let result = logical_plan(sql);
     assert!(result.is_err());
     let err = result.err().unwrap();
-    assert_eq!(err.to_string(), expected);
+    assert_eq!(err.strip_backtrace(), expected);
 }
 
 #[test]
@@ -3623,41 +3622,45 @@ fn test_ambiguous_column_references_with_in_using_join() {
 }
 
 #[test]
-#[should_panic(expected = "value: Plan(\"Invalid placeholder, not a number: $foo\"")]
 fn test_prepare_statement_to_plan_panic_param_format() {
     // param is not number following the $ sign
     // panic due to error returned from the parser
     let sql = "PREPARE my_plan(INT) AS SELECT id, age  FROM person WHERE age = $foo";
-    logical_plan(sql).unwrap();
+    assert_eq!(
+        logical_plan(sql).unwrap_err().strip_backtrace(),
+        "Error during planning: Invalid placeholder, not a number: $foo"
+    );
 }
 
 #[test]
-#[should_panic(
-    expected = "value: Plan(\"Invalid placeholder, zero is not a valid index: $0\""
-)]
 fn test_prepare_statement_to_plan_panic_param_zero() {
     // param is zero following the $ sign
     // panic due to error returned from the parser
     let sql = "PREPARE my_plan(INT) AS SELECT id, age  FROM person WHERE age = $0";
-    logical_plan(sql).unwrap();
+    assert_eq!(
+        logical_plan(sql).unwrap_err().strip_backtrace(),
+        "Error during planning: Invalid placeholder, zero is not a valid index: $0"
+    );
 }
 
 #[test]
-#[should_panic(expected = "value: SQL(ParserError(\"Expected AS, found: SELECT\"))")]
 fn test_prepare_statement_to_plan_panic_prepare_wrong_syntax() {
     // param is not number following the $ sign
     // panic due to error returned from the parser
     let sql = "PREPARE AS SELECT id, age  FROM person WHERE age = $foo";
-    logical_plan(sql).unwrap();
+    assert_eq!(
+        logical_plan(sql).unwrap_err().strip_backtrace(),
+        "SQL error: ParserError(\"Expected AS, found: SELECT\")"
+    )
 }
 
 #[test]
-#[should_panic(
-    expected = "value: SchemaError(FieldNotFound { field: Column { relation: None, name: \"id\" }, valid_fields: [] })"
-)]
 fn test_prepare_statement_to_plan_panic_no_relation_and_constant_param() {
     let sql = "PREPARE my_plan(INT) AS SELECT id + $1";
-    logical_plan(sql).unwrap();
+    assert_eq!(
+        logical_plan(sql).unwrap_err().strip_backtrace(),
+        "Schema error: No field named id."
+    )
 }
 
 #[test]
@@ -3729,41 +3732,48 @@ fn test_prepare_statement_to_plan_no_param() {
 }
 
 #[test]
-#[should_panic(expected = "value: Plan(\"Expected 1 parameters, got 0\")")]
 fn test_prepare_statement_to_plan_one_param_no_value_panic() {
     // no embedded parameter but still declare it
     let sql = "PREPARE my_plan(INT) AS SELECT id, age  FROM person WHERE age = 10";
     let plan = logical_plan(sql).unwrap();
     // declare 1 param but provide 0
     let param_values = vec![];
-    let expected_plan = "whatever";
-    prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan);
+    assert_eq!(
+        plan.with_param_values(param_values)
+            .unwrap_err()
+            .strip_backtrace(),
+        "Error during planning: Expected 1 parameters, got 0"
+    );
 }
 
 #[test]
-#[should_panic(
-    expected = "value: Plan(\"Expected parameter of type Int32, got Float64 at index 0\")"
-)]
 fn test_prepare_statement_to_plan_one_param_one_value_different_type_panic() {
     // no embedded parameter but still declare it
     let sql = "PREPARE my_plan(INT) AS SELECT id, age  FROM person WHERE age = 10";
     let plan = logical_plan(sql).unwrap();
     // declare 1 param but provide 0
     let param_values = vec![ScalarValue::Float64(Some(20.0))];
-    let expected_plan = "whatever";
-    prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan);
+    assert_eq!(
+        plan.with_param_values(param_values)
+            .unwrap_err()
+            .strip_backtrace(),
+        "Error during planning: Expected parameter of type Int32, got Float64 at index 0"
+    );
 }
 
 #[test]
-#[should_panic(expected = "value: Plan(\"Expected 0 parameters, got 1\")")]
 fn test_prepare_statement_to_plan_no_param_on_value_panic() {
     // no embedded parameter but still declare it
     let sql = "PREPARE my_plan AS SELECT id, age  FROM person WHERE age = 10";
     let plan = logical_plan(sql).unwrap();
     // declare 1 param but provide 0
     let param_values = vec![ScalarValue::Int32(Some(10))];
-    let expected_plan = "whatever";
-    prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan);
+    assert_eq!(
+        plan.with_param_values(param_values)
+            .unwrap_err()
+            .strip_backtrace(),
+        "Error during planning: Expected 0 parameters, got 1"
+    );
 }
 
 #[test]
diff --git a/datafusion/sqllogictest/test_files/scalar.slt b/datafusion/sqllogictest/test_files/scalar.slt
index ca52163758..9d5eb54816 100644
--- a/datafusion/sqllogictest/test_files/scalar.slt
+++ b/datafusion/sqllogictest/test_files/scalar.slt
@@ -1526,7 +1526,7 @@ SELECT not(true), not(false)
 ----
 false true
 
-query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nInternal error: NOT 'Literal \{ value: Int64\(1\) \}' can't be evaluated because the expression's type is Int64, not boolean or NULL\. This was likely caused by a bug in DataFusion's code and we would welcome that you file an bug report in our issue tracker
+query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nInternal error: NOT 'Literal \{ value: Int64\(1\) \}' can't be evaluated because the expression's type is Int64, not boolean or NULL
 SELECT not(1), not(0)
 
 query ?B
@@ -1534,7 +1534,7 @@ SELECT null, not(null)
 ----
 NULL NULL
 
-query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nInternal error: NOT 'Literal \{ value: Utf8\("hi"\) \}' can't be evaluated because the expression's type is Utf8, not boolean or NULL\. This was likely caused by a bug in DataFusion's code and we would welcome that you file an bug report in our issue tracker
+query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nInternal error: NOT 'Literal \{ value: Utf8\("hi"\) \}' can't be evaluated because the expression's type is Utf8, not boolean or NULL
 SELECT NOT('hi')
 
 # test_negative_expressions()