Skip to content

Commit d5bd26e

Browse files
committed
update
1 parent 8a7c4ee commit d5bd26e

File tree

12 files changed

+93
-121
lines changed

12 files changed

+93
-121
lines changed

datafusion/core/src/physical_planner.rs

Lines changed: 4 additions & 46 deletions
Original file line numberDiff line numberDiff line change
@@ -2676,52 +2676,10 @@ mod tests {
26762676
let e = planner
26772677
.create_physical_plan(&logical_plan, &session_state)
26782678
.await
2679-
.expect_err("planning error");
2680-
2681-
insta::assert_snapshot!(e, @r#"
2682-
Error during planning: Extension planner for NoOp created an ExecutionPlan with mismatched schema. LogicalPlan schema: DFSchema { inner: Schema { fields: [Field { name: "a", data_type: Int32 }], metadata: {} }, field_qualifiers: [None], functional_dependencies: FunctionalDependencies { deps: [] } }, ExecutionPlan schema: Schema { fields: [Field { name: "b", data_type: Int32 }], metadata: {} }
2683-
2684-
backtrace: 0: std::backtrace::Backtrace::create
2685-
1: datafusion::physical_planner::DefaultPhysicalPlanner::map_logical_node_to_physical::{{closure}}
2686-
2: datafusion::physical_planner::DefaultPhysicalPlanner::task_helper::{{closure}}
2687-
3: <futures_util::stream::futures_unordered::FuturesUnordered<Fut> as futures_core::stream::Stream>::poll_next
2688-
4: futures_util::stream::stream::StreamExt::poll_next_unpin
2689-
5: <futures_util::stream::stream::buffer_unordered::BufferUnordered<St> as futures_core::stream::Stream>::poll_next
2690-
6: <S as futures_core::stream::TryStream>::try_poll_next
2691-
7: <futures_util::stream::try_stream::try_collect::TryCollect<St,C> as core::future::future::Future>::poll
2692-
8: datafusion::physical_planner::DefaultPhysicalPlanner::create_initial_plan::{{closure}}
2693-
9: <datafusion::physical_planner::DefaultPhysicalPlanner as datafusion::physical_planner::PhysicalPlanner>::create_physical_plan::{{closure}}
2694-
10: <core::pin::Pin<P> as core::future::future::Future>::poll
2695-
11: datafusion::physical_planner::tests::bad_extension_planner::{{closure}}
2696-
12: <core::pin::Pin<P> as core::future::future::Future>::poll
2697-
13: <core::pin::Pin<P> as core::future::future::Future>::poll
2698-
14: tokio::runtime::scheduler::current_thread::CoreGuard::block_on::{{closure}}::{{closure}}::{{closure}}
2699-
15: tokio::runtime::scheduler::current_thread::CoreGuard::block_on::{{closure}}::{{closure}}
2700-
16: tokio::runtime::scheduler::current_thread::Context::enter
2701-
17: tokio::runtime::scheduler::current_thread::CoreGuard::block_on::{{closure}}
2702-
18: tokio::runtime::scheduler::current_thread::CoreGuard::enter::{{closure}}
2703-
19: tokio::runtime::context::scoped::Scoped<T>::set
2704-
20: tokio::runtime::context::set_scheduler::{{closure}}
2705-
21: std::thread::local::LocalKey<T>::try_with
2706-
22: std::thread::local::LocalKey<T>::with
2707-
23: tokio::runtime::context::set_scheduler
2708-
24: tokio::runtime::scheduler::current_thread::CoreGuard::enter
2709-
25: tokio::runtime::scheduler::current_thread::CoreGuard::block_on
2710-
26: tokio::runtime::scheduler::current_thread::CurrentThread::block_on::{{closure}}
2711-
27: tokio::runtime::context::runtime::enter_runtime
2712-
28: tokio::runtime::scheduler::current_thread::CurrentThread::block_on
2713-
29: tokio::runtime::runtime::Runtime::block_on_inner
2714-
30: tokio::runtime::runtime::Runtime::block_on
2715-
31: datafusion::physical_planner::tests::bad_extension_planner
2716-
32: datafusion::physical_planner::tests::bad_extension_planner::{{closure}}
2717-
33: core::ops::function::FnOnce::call_once
2718-
34: test::__rust_begin_short_backtrace
2719-
35: test::run_test::{{closure}}
2720-
36: std::sys::backtrace::__rust_begin_short_backtrace
2721-
37: core::ops::function::FnOnce::call_once{{vtable.shim}}
2722-
38: std::sys::pal::unix::thread::Thread::new::thread_start
2723-
39: __pthread_cond_wait
2724-
"#);
2679+
.expect_err("planning error")
2680+
.strip_backtrace();
2681+
2682+
insta::assert_snapshot!(e, @r#"Error during planning: Extension planner for NoOp created an ExecutionPlan with mismatched schema. LogicalPlan schema: DFSchema { inner: Schema { fields: [Field { name: "a", data_type: Int32 }], metadata: {} }, field_qualifiers: [None], functional_dependencies: FunctionalDependencies { deps: [] } }, ExecutionPlan schema: Schema { fields: [Field { name: "b", data_type: Int32 }], metadata: {} }"#);
27252683
}
27262684

27272685
#[tokio::test]

datafusion/core/tests/physical_optimizer/test_utils.rs

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -151,13 +151,15 @@ pub fn schema() -> SchemaRef {
151151
]))
152152
}
153153

154+
#[expect(dead_code)]
154155
pub fn create_test_schema() -> Result<SchemaRef> {
155156
let nullable_column = Field::new("nullable_col", DataType::Int32, true);
156157
let non_nullable_column = Field::new("non_nullable_col", DataType::Int32, false);
157158
let schema = Arc::new(Schema::new(vec![nullable_column, non_nullable_column]));
158159
Ok(schema)
159160
}
160161

162+
#[expect(dead_code)]
161163
pub fn create_test_schema2() -> Result<SchemaRef> {
162164
let col_a = Field::new("col_a", DataType::Int32, true);
163165
let col_b = Field::new("col_b", DataType::Int32, true);
@@ -281,6 +283,7 @@ pub fn bounded_window_exec_with_partition(
281283
)
282284
}
283285

286+
#[expect(dead_code)]
284287
pub fn filter_exec(
285288
predicate: Arc<dyn PhysicalExpr>,
286289
input: Arc<dyn ExecutionPlan>,
@@ -326,6 +329,7 @@ pub fn repartition_exec(input: Arc<dyn ExecutionPlan>) -> Arc<dyn ExecutionPlan>
326329
Arc::new(RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap())
327330
}
328331

332+
#[expect(dead_code)]
329333
pub fn spr_repartition_exec(input: Arc<dyn ExecutionPlan>) -> Arc<dyn ExecutionPlan> {
330334
Arc::new(
331335
RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10))
@@ -334,6 +338,7 @@ pub fn spr_repartition_exec(input: Arc<dyn ExecutionPlan>) -> Arc<dyn ExecutionP
334338
)
335339
}
336340

341+
#[expect(dead_code)]
337342
pub fn aggregate_exec(input: Arc<dyn ExecutionPlan>) -> Arc<dyn ExecutionPlan> {
338343
let schema = input.schema();
339344
Arc::new(
@@ -391,13 +396,15 @@ pub fn projection_exec(
391396

392397
/// A test [`ExecutionPlan`] whose requirements can be configured.
393398
#[derive(Debug)]
399+
#[allow(dead_code)]
394400
pub struct RequirementsTestExec {
395401
required_input_ordering: Option<LexOrdering>,
396402
maintains_input_order: bool,
397403
input: Arc<dyn ExecutionPlan>,
398404
}
399405

400406
impl RequirementsTestExec {
407+
#[allow(dead_code)]
401408
pub fn new(input: Arc<dyn ExecutionPlan>) -> Self {
402409
Self {
403410
required_input_ordering: None,
@@ -548,6 +555,7 @@ pub fn stream_exec(schema: &SchemaRef) -> Arc<dyn ExecutionPlan> {
548555
}
549556

550557
/// Create an unbounded stream table with data ordering.
558+
#[expect(dead_code)]
551559
pub fn stream_exec_ordered(
552560
schema: &SchemaRef,
553561
ordering: LexOrdering,

datafusion/core/tests/sql/select.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -218,7 +218,7 @@ async fn test_parameter_invalid_types() -> Result<()> {
218218
.with_param_values(vec![ScalarValue::from(4_i32)])?
219219
.collect()
220220
.await;
221-
assert_snapshot!(results.unwrap_err().strip_backtrace(),
221+
assert_snapshot!(results.unwrap_err().strip_backtrace(),
222222
@r"
223223
type_coercion
224224
caused by

datafusion/functions/src/datetime/date_bin.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -687,7 +687,7 @@ mod tests {
687687
let res = invoke_date_bin_with_args(args, 1, return_field);
688688
assert_eq!(
689689
res.err().unwrap().strip_backtrace(),
690-
"Execution error: DATE_BIN expects origin argument to be a TIMESTAMP with nanosecond precision but got Timestamp(Microsecond, None)"
690+
"Execution error: DATE_BIN expects origin argument to be a TIMESTAMP with nanosecond precision but got Timestamp(µs)"
691691
);
692692

693693
args = vec![

datafusion/optimizer/src/analyzer/type_coercion.rs

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -2116,7 +2116,7 @@ mod test {
21162116
assert_analyzed_plan_eq!(
21172117
plan,
21182118
@r#"
2119-
Projection: CAST(Utf8("1998-03-18") AS Timestamp(Nanosecond, None)) = CAST(CAST(Utf8("1998-03-18") AS Date32) AS Timestamp(Nanosecond, None))
2119+
Projection: CAST(Utf8("1998-03-18") AS Timestamp(ns)) = CAST(CAST(Utf8("1998-03-18") AS Date32) AS Timestamp(ns))
21202120
EmptyRelation: rows=0
21212121
"#
21222122
)
@@ -2257,7 +2257,7 @@ mod test {
22572257
let err = coerce_case_expression(case, &schema).unwrap_err();
22582258
assert_snapshot!(
22592259
err.strip_backtrace(),
2260-
@"Error during planning: Failed to coerce then (Date32, Float32, Binary) and else (Timestamp(Nanosecond, None)) to common types in CASE WHEN expression"
2260+
@"Error during planning: Failed to coerce then (Date32, Float32, Binary) and else (Timestamp(ns)) to common types in CASE WHEN expression"
22612261
);
22622262

22632263
Ok(())
@@ -2464,7 +2464,7 @@ mod test {
24642464
assert_analyzed_plan_eq!(
24652465
plan,
24662466
@r#"
2467-
Projection: a = CAST(CAST(a AS Map(Field { name: "key_value", data_type: Struct([Field { name: "key", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, Field { name: "value", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }]), nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, false)) AS Map(Field { name: "entries", data_type: Struct([Field { name: "key", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, Field { name: "value", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }]), nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, false))
2467+
Projection: a = CAST(CAST(a AS Map(Field { "key_value": Struct("key": Utf8, "value": nullable Float64) }, false)) AS Map(Field { "entries": Struct("key": Utf8, "value": nullable Float64) }, false))
24682468
EmptyRelation: rows=0
24692469
"#
24702470
)
@@ -2487,7 +2487,7 @@ mod test {
24872487
assert_analyzed_plan_eq!(
24882488
plan,
24892489
@r#"
2490-
Projection: IntervalYearMonth("12") + CAST(Utf8("2000-01-01T00:00:00") AS Timestamp(Nanosecond, None))
2490+
Projection: IntervalYearMonth("12") + CAST(Utf8("2000-01-01T00:00:00") AS Timestamp(ns))
24912491
EmptyRelation: rows=0
24922492
"#
24932493
)
@@ -2512,7 +2512,7 @@ mod test {
25122512
assert_analyzed_plan_eq!(
25132513
plan,
25142514
@r#"
2515-
Projection: CAST(Utf8("1998-03-18") AS Timestamp(Nanosecond, None)) - CAST(Utf8("1998-03-18") AS Timestamp(Nanosecond, None))
2515+
Projection: CAST(Utf8("1998-03-18") AS Timestamp(ns)) - CAST(Utf8("1998-03-18") AS Timestamp(ns))
25162516
EmptyRelation: rows=0
25172517
"#
25182518
)

datafusion/optimizer/src/decorrelate_predicate_subquery.rs

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -1972,14 +1972,14 @@ mod tests {
19721972

19731973
assert_optimized_plan_equal!(
19741974
plan,
1975-
@r#"
1975+
@r"
19761976
Projection: test.b [b:UInt32]
19771977
LeftSemi Join: Filter: Boolean(true) [a:UInt32, b:UInt32, c:UInt32]
19781978
TableScan: test [a:UInt32, b:UInt32, c:UInt32]
19791979
SubqueryAlias: __correlated_sq_1 [arr:Int32;N]
19801980
Unnest: lists[sq.arr|depth=1] structs[] [arr:Int32;N]
1981-
TableScan: sq [arr:List(Field { name: "item", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} });N]
1982-
"#
1981+
TableScan: sq [arr:List(Field { data_type: Int32, nullable: true });N]
1982+
"
19831983
)
19841984
}
19851985

@@ -2007,14 +2007,14 @@ mod tests {
20072007

20082008
assert_optimized_plan_equal!(
20092009
plan,
2010-
@r#"
2010+
@r"
20112011
Projection: test.b [b:UInt32]
20122012
LeftSemi Join: Filter: __correlated_sq_1.a = test.b [a:UInt32, b:UInt32, c:UInt32]
20132013
TableScan: test [a:UInt32, b:UInt32, c:UInt32]
20142014
SubqueryAlias: __correlated_sq_1 [a:UInt32;N]
20152015
Unnest: lists[sq.a|depth=1] structs[] [a:UInt32;N]
2016-
TableScan: sq [a:List(Field { name: "item", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} });N]
2017-
"#
2016+
TableScan: sq [a:List(Field { data_type: UInt32, nullable: true });N]
2017+
"
20182018
)
20192019
}
20202020

datafusion/physical-expr/src/expressions/cast.rs

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -262,8 +262,8 @@ mod tests {
262262
},
263263
datatypes::*,
264264
};
265-
use datafusion_common::assert_contains;
266265
use datafusion_physical_expr_common::physical_expr::fmt_sql;
266+
use insta::assert_snapshot;
267267

268268
// runs an end-to-end test of physical type cast
269269
// 1. construct a record batch with a column "a" of type A
@@ -438,11 +438,8 @@ mod tests {
438438
)?;
439439
let expression =
440440
cast_with_options(col("a", &schema)?, &schema, Decimal128(6, 2), None)?;
441-
let e = expression.evaluate(&batch).unwrap_err(); // panics on OK
442-
assert_contains!(
443-
e.to_string(),
444-
"Arrow error: Invalid argument error: 12345679 is too large to store in a Decimal128 of precision 6. Max is 999999"
445-
);
441+
let e = expression.evaluate(&batch).unwrap_err().strip_backtrace(); // panics on OK
442+
assert_snapshot!(e, @"Arrow error: Invalid argument error: 123456.79 is too large to store in a Decimal128 of precision 6. Max is 9999.99");
446443

447444
let expression_safe = cast_with_options(
448445
col("a", &schema)?,

datafusion/physical-expr/src/expressions/dynamic_filters.rs

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -381,14 +381,14 @@ mod test {
381381
)
382382
.unwrap();
383383
let snap = dynamic_filter_1.snapshot().unwrap().unwrap();
384-
insta::assert_snapshot!(format!("{snap:?}"), @r#"BinaryExpr { left: Column { name: "a", index: 0 }, op: Eq, right: Literal { value: Int32(42), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, fail_on_overflow: false }"#);
384+
insta::assert_snapshot!(format!("{snap:?}"), @r#"BinaryExpr { left: Column { name: "a", index: 0 }, op: Eq, right: Literal { value: Int32(42), field: Field { name: "lit", data_type: Int32 } }, fail_on_overflow: false }"#);
385385
let dynamic_filter_2 = reassign_expr_columns(
386386
Arc::clone(&dynamic_filter) as Arc<dyn PhysicalExpr>,
387387
&filter_schema_2,
388388
)
389389
.unwrap();
390390
let snap = dynamic_filter_2.snapshot().unwrap().unwrap();
391-
insta::assert_snapshot!(format!("{snap:?}"), @r#"BinaryExpr { left: Column { name: "a", index: 1 }, op: Eq, right: Literal { value: Int32(42), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, fail_on_overflow: false }"#);
391+
insta::assert_snapshot!(format!("{snap:?}"), @r#"BinaryExpr { left: Column { name: "a", index: 1 }, op: Eq, right: Literal { value: Int32(42), field: Field { name: "lit", data_type: Int32 } }, fail_on_overflow: false }"#);
392392
// Both filters allow evaluating the same expression
393393
let batch_1 = RecordBatch::try_new(
394394
Arc::clone(&filter_schema_1),

datafusion/physical-expr/src/expressions/in_list.rs

Lines changed: 29 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -463,13 +463,13 @@ pub fn in_list(
463463

464464
#[cfg(test)]
465465
mod tests {
466-
467466
use super::*;
468467
use crate::expressions;
469468
use crate::expressions::{col, lit, try_cast};
470469
use datafusion_common::plan_err;
471470
use datafusion_expr::type_coercion::binary::comparison_coercion;
472471
use datafusion_physical_expr_common::physical_expr::fmt_sql;
472+
use insta::assert_snapshot;
473473
use itertools::Itertools as _;
474474

475475
type InListCastResult = (Arc<dyn PhysicalExpr>, Vec<Arc<dyn PhysicalExpr>>);
@@ -1443,7 +1443,7 @@ mod tests {
14431443
}
14441444

14451445
#[test]
1446-
fn test_fmt_sql() -> Result<()> {
1446+
fn test_fmt_sql_1() -> Result<()> {
14471447
let schema = Schema::new(vec![Field::new("a", DataType::Utf8, true)]);
14481448
let col_a = col("a", &schema)?;
14491449

@@ -1452,33 +1452,51 @@ mod tests {
14521452
let expr = in_list(Arc::clone(&col_a), list, &false, &schema)?;
14531453
let sql_string = fmt_sql(expr.as_ref()).to_string();
14541454
let display_string = expr.to_string();
1455-
assert_eq!(sql_string, "a IN (a, b)");
1456-
assert_eq!(display_string, "Use a@0 IN (SET) ([Literal { value: Utf8(\"a\"), field: Field { name: \"lit\", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8(\"b\"), field: Field { name: \"lit\", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }])");
1455+
assert_snapshot!(sql_string, @"a IN (a, b)");
1456+
assert_snapshot!(display_string, @r#"Use a@0 IN (SET) ([Literal { value: Utf8("a"), field: Field { name: "lit", data_type: Utf8 } }, Literal { value: Utf8("b"), field: Field { name: "lit", data_type: Utf8 } }])"#);
1457+
Ok(())
1458+
}
1459+
1460+
#[test]
1461+
fn test_fmt_sql_2() -> Result<()> {
1462+
let schema = Schema::new(vec![Field::new("a", DataType::Utf8, true)]);
1463+
let col_a = col("a", &schema)?;
14571464

14581465
// Test: a NOT IN ('a', 'b')
14591466
let list = vec![lit("a"), lit("b")];
14601467
let expr = in_list(Arc::clone(&col_a), list, &true, &schema)?;
14611468
let sql_string = fmt_sql(expr.as_ref()).to_string();
14621469
let display_string = expr.to_string();
1463-
assert_eq!(sql_string, "a NOT IN (a, b)");
1464-
assert_eq!(display_string, "a@0 NOT IN (SET) ([Literal { value: Utf8(\"a\"), field: Field { name: \"lit\", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8(\"b\"), field: Field { name: \"lit\", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }])");
1470+
assert_snapshot!(sql_string, @"a NOT IN (a, b)");
1471+
assert_snapshot!(display_string, @r#"a@0 NOT IN (SET) ([Literal { value: Utf8("a"), field: Field { name: "lit", data_type: Utf8 } }, Literal { value: Utf8("b"), field: Field { name: "lit", data_type: Utf8 } }])"#);
1472+
Ok(())
1473+
}
14651474

1475+
#[test]
1476+
fn test_fmt_sql_3() -> Result<()> {
1477+
let schema = Schema::new(vec![Field::new("a", DataType::Utf8, true)]);
1478+
let col_a = col("a", &schema)?;
14661479
// Test: a IN ('a', 'b', NULL)
14671480
let list = vec![lit("a"), lit("b"), lit(ScalarValue::Utf8(None))];
14681481
let expr = in_list(Arc::clone(&col_a), list, &false, &schema)?;
14691482
let sql_string = fmt_sql(expr.as_ref()).to_string();
14701483
let display_string = expr.to_string();
1471-
assert_eq!(sql_string, "a IN (a, b, NULL)");
1472-
assert_eq!(display_string, "Use a@0 IN (SET) ([Literal { value: Utf8(\"a\"), field: Field { name: \"lit\", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8(\"b\"), field: Field { name: \"lit\", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8(NULL), field: Field { name: \"lit\", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} } }])");
1484+
assert_snapshot!(sql_string, @"a IN (a, b, NULL)");
1485+
assert_snapshot!(display_string, @r#"Use a@0 IN (SET) ([Literal { value: Utf8("a"), field: Field { name: "lit", data_type: Utf8 } }, Literal { value: Utf8("b"), field: Field { name: "lit", data_type: Utf8 } }, Literal { value: Utf8(NULL), field: Field { name: "lit", data_type: Utf8, nullable: true } }])"#);
1486+
Ok(())
1487+
}
14731488

1489+
#[test]
1490+
fn test_fmt_sql_4() -> Result<()> {
1491+
let schema = Schema::new(vec![Field::new("a", DataType::Utf8, true)]);
1492+
let col_a = col("a", &schema)?;
14741493
// Test: a NOT IN ('a', 'b', NULL)
14751494
let list = vec![lit("a"), lit("b"), lit(ScalarValue::Utf8(None))];
14761495
let expr = in_list(Arc::clone(&col_a), list, &true, &schema)?;
14771496
let sql_string = fmt_sql(expr.as_ref()).to_string();
14781497
let display_string = expr.to_string();
1479-
assert_eq!(sql_string, "a NOT IN (a, b, NULL)");
1480-
assert_eq!(display_string, "a@0 NOT IN (SET) ([Literal { value: Utf8(\"a\"), field: Field { name: \"lit\", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8(\"b\"), field: Field { name: \"lit\", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8(NULL), field: Field { name: \"lit\", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} } }])");
1481-
1498+
assert_snapshot!(sql_string, @"a NOT IN (a, b, NULL)");
1499+
assert_snapshot!(display_string, @r#"a@0 NOT IN (SET) ([Literal { value: Utf8("a"), field: Field { name: "lit", data_type: Utf8 } }, Literal { value: Utf8("b"), field: Field { name: "lit", data_type: Utf8 } }, Literal { value: Utf8(NULL), field: Field { name: "lit", data_type: Utf8, nullable: true } }])"#);
14821500
Ok(())
14831501
}
14841502
}

0 commit comments

Comments
 (0)