Skip to content

Commit b5dfdbe

Browse files
authored
feat: add metadata to literal expressions (#16170) (#16315)
1 parent c76c1f0 commit b5dfdbe

File tree

94 files changed

+724
-421
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

94 files changed

+724
-421
lines changed

datafusion-cli/src/functions.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -322,7 +322,7 @@ pub struct ParquetMetadataFunc {}
322322
impl TableFunctionImpl for ParquetMetadataFunc {
323323
fn call(&self, exprs: &[Expr]) -> Result<Arc<dyn TableProvider>> {
324324
let filename = match exprs.first() {
325-
Some(Expr::Literal(ScalarValue::Utf8(Some(s)))) => s, // single quote: parquet_metadata('x.parquet')
325+
Some(Expr::Literal(ScalarValue::Utf8(Some(s)), _)) => s, // single quote: parquet_metadata('x.parquet')
326326
Some(Expr::Column(Column { name, .. })) => name, // double quote: parquet_metadata("x.parquet")
327327
_ => {
328328
return plan_err!(

datafusion-examples/examples/expr_api.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -65,7 +65,7 @@ async fn main() -> Result<()> {
6565
let expr2 = Expr::BinaryExpr(BinaryExpr::new(
6666
Box::new(col("a")),
6767
Operator::Plus,
68-
Box::new(Expr::Literal(ScalarValue::Int32(Some(5)))),
68+
Box::new(Expr::Literal(ScalarValue::Int32(Some(5)), None)),
6969
));
7070
assert_eq!(expr, expr2);
7171

datafusion-examples/examples/optimizer_rule.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -171,7 +171,7 @@ fn is_binary_eq(binary_expr: &BinaryExpr) -> bool {
171171

172172
/// Return true if the expression is a literal or column reference
173173
fn is_lit_or_col(expr: &Expr) -> bool {
174-
matches!(expr, Expr::Column(_) | Expr::Literal(_))
174+
matches!(expr, Expr::Column(_) | Expr::Literal(_, _))
175175
}
176176

177177
/// A simple user defined filter function

datafusion-examples/examples/simple_udtf.rs

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -133,7 +133,8 @@ struct LocalCsvTableFunc {}
133133

134134
impl TableFunctionImpl for LocalCsvTableFunc {
135135
fn call(&self, exprs: &[Expr]) -> Result<Arc<dyn TableProvider>> {
136-
let Some(Expr::Literal(ScalarValue::Utf8(Some(ref path)))) = exprs.first() else {
136+
let Some(Expr::Literal(ScalarValue::Utf8(Some(ref path)), _)) = exprs.first()
137+
else {
137138
return plan_err!("read_csv requires at least one string argument");
138139
};
139140

@@ -145,7 +146,7 @@ impl TableFunctionImpl for LocalCsvTableFunc {
145146
let info = SimplifyContext::new(&execution_props);
146147
let expr = ExprSimplifier::new(info).simplify(expr.clone())?;
147148

148-
if let Expr::Literal(ScalarValue::Int64(Some(limit))) = expr {
149+
if let Expr::Literal(ScalarValue::Int64(Some(limit)), _) = expr {
149150
Ok(limit as usize)
150151
} else {
151152
plan_err!("Limit must be an integer")

datafusion/catalog-listing/src/helpers.rs

Lines changed: 8 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -61,7 +61,7 @@ pub fn expr_applicable_for_cols(col_names: &[&str], expr: &Expr) -> bool {
6161
Ok(TreeNodeRecursion::Stop)
6262
}
6363
}
64-
Expr::Literal(_)
64+
Expr::Literal(_, _)
6565
| Expr::Alias(_)
6666
| Expr::OuterReferenceColumn(_, _)
6767
| Expr::ScalarVariable(_, _)
@@ -346,8 +346,8 @@ fn populate_partition_values<'a>(
346346
{
347347
match op {
348348
Operator::Eq => match (left.as_ref(), right.as_ref()) {
349-
(Expr::Column(Column { ref name, .. }), Expr::Literal(val))
350-
| (Expr::Literal(val), Expr::Column(Column { ref name, .. })) => {
349+
(Expr::Column(Column { ref name, .. }), Expr::Literal(val, _))
350+
| (Expr::Literal(val, _), Expr::Column(Column { ref name, .. })) => {
351351
if partition_values
352352
.insert(name, PartitionValue::Single(val.to_string()))
353353
.is_some()
@@ -984,7 +984,7 @@ mod tests {
984984
assert_eq!(
985985
evaluate_partition_prefix(
986986
partitions,
987-
&[col("a").eq(Expr::Literal(ScalarValue::Date32(Some(3))))],
987+
&[col("a").eq(Expr::Literal(ScalarValue::Date32(Some(3)), None))],
988988
),
989989
Some(Path::from("a=1970-01-04")),
990990
);
@@ -993,9 +993,10 @@ mod tests {
993993
assert_eq!(
994994
evaluate_partition_prefix(
995995
partitions,
996-
&[col("a").eq(Expr::Literal(ScalarValue::Date64(Some(
997-
4 * 24 * 60 * 60 * 1000
998-
)))),],
996+
&[col("a").eq(Expr::Literal(
997+
ScalarValue::Date64(Some(4 * 24 * 60 * 60 * 1000)),
998+
None
999+
)),],
9991000
),
10001001
Some(Path::from("a=1970-01-05")),
10011002
);

datafusion/core/benches/map_query_sql.rs

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -71,8 +71,11 @@ fn criterion_benchmark(c: &mut Criterion) {
7171
let mut value_buffer = Vec::new();
7272

7373
for i in 0..1000 {
74-
key_buffer.push(Expr::Literal(ScalarValue::Utf8(Some(keys[i].clone()))));
75-
value_buffer.push(Expr::Literal(ScalarValue::Int32(Some(values[i]))));
74+
key_buffer.push(Expr::Literal(
75+
ScalarValue::Utf8(Some(keys[i].clone())),
76+
None,
77+
));
78+
value_buffer.push(Expr::Literal(ScalarValue::Int32(Some(values[i])), None));
7679
}
7780
c.bench_function("map_1000_1", |b| {
7881
b.iter(|| {

datafusion/core/src/dataframe/mod.rs

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1337,7 +1337,10 @@ impl DataFrame {
13371337
/// ```
13381338
pub async fn count(self) -> Result<usize> {
13391339
let rows = self
1340-
.aggregate(vec![], vec![count(Expr::Literal(COUNT_STAR_EXPANSION))])?
1340+
.aggregate(
1341+
vec![],
1342+
vec![count(Expr::Literal(COUNT_STAR_EXPANSION, None))],
1343+
)?
13411344
.collect()
13421345
.await?;
13431346
let len = *rows

datafusion/core/src/datasource/listing/table.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2230,7 +2230,7 @@ mod tests {
22302230
let filter_predicate = Expr::BinaryExpr(BinaryExpr::new(
22312231
Box::new(Expr::Column("column1".into())),
22322232
Operator::GtEq,
2233-
Box::new(Expr::Literal(ScalarValue::Int32(Some(0)))),
2233+
Box::new(Expr::Literal(ScalarValue::Int32(Some(0)), None)),
22342234
));
22352235

22362236
// Create a new batch of data to insert into the table

datafusion/core/src/execution/context/mod.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1214,7 +1214,7 @@ impl SessionContext {
12141214
let mut params: Vec<ScalarValue> = parameters
12151215
.into_iter()
12161216
.map(|e| match e {
1217-
Expr::Literal(scalar) => Ok(scalar),
1217+
Expr::Literal(scalar, _) => Ok(scalar),
12181218
_ => not_impl_err!("Unsupported parameter type: {}", e),
12191219
})
12201220
.collect::<Result<_>>()?;

datafusion/core/src/physical_planner.rs

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -2257,7 +2257,8 @@ mod tests {
22572257
// verify that the plan correctly casts u8 to i64
22582258
// the cast from u8 to i64 for literal will be simplified, and get lit(int64(5))
22592259
// the cast here is implicit so has CastOptions with safe=true
2260-
let expected = "BinaryExpr { left: Column { name: \"c7\", index: 2 }, op: Lt, right: Literal { value: Int64(5) }, fail_on_overflow: false }";
2260+
let expected = r#"BinaryExpr { left: Column { name: "c7", index: 2 }, op: Lt, right: Literal { value: Int64(5), field: Field { name: "5", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, fail_on_overflow: false }"#;
2261+
22612262
assert!(format!("{exec_plan:?}").contains(expected));
22622263
Ok(())
22632264
}
@@ -2282,7 +2283,7 @@ mod tests {
22822283
&session_state,
22832284
);
22842285

2285-
let expected = r#"Ok(PhysicalGroupBy { expr: [(Column { name: "c1", index: 0 }, "c1"), (Column { name: "c2", index: 1 }, "c2"), (Column { name: "c3", index: 2 }, "c3")], null_expr: [(Literal { value: Utf8(NULL) }, "c1"), (Literal { value: Int64(NULL) }, "c2"), (Literal { value: Int64(NULL) }, "c3")], groups: [[false, false, false], [true, false, false], [false, true, false], [false, false, true], [true, true, false], [true, false, true], [false, true, true], [true, true, true]] })"#;
2286+
let expected = r#"Ok(PhysicalGroupBy { expr: [(Column { name: "c1", index: 0 }, "c1"), (Column { name: "c2", index: 1 }, "c2"), (Column { name: "c3", index: 2 }, "c3")], null_expr: [(Literal { value: Utf8(NULL), field: Field { name: "NULL", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} } }, "c1"), (Literal { value: Int64(NULL), field: Field { name: "NULL", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} } }, "c2"), (Literal { value: Int64(NULL), field: Field { name: "NULL", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} } }, "c3")], groups: [[false, false, false], [true, false, false], [false, true, false], [false, false, true], [true, true, false], [true, false, true], [false, true, true], [true, true, true]] })"#;
22862287

22872288
assert_eq!(format!("{cube:?}"), expected);
22882289

@@ -2309,7 +2310,7 @@ mod tests {
23092310
&session_state,
23102311
);
23112312

2312-
let expected = r#"Ok(PhysicalGroupBy { expr: [(Column { name: "c1", index: 0 }, "c1"), (Column { name: "c2", index: 1 }, "c2"), (Column { name: "c3", index: 2 }, "c3")], null_expr: [(Literal { value: Utf8(NULL) }, "c1"), (Literal { value: Int64(NULL) }, "c2"), (Literal { value: Int64(NULL) }, "c3")], groups: [[true, true, true], [false, true, true], [false, false, true], [false, false, false]] })"#;
2313+
let expected = r#"Ok(PhysicalGroupBy { expr: [(Column { name: "c1", index: 0 }, "c1"), (Column { name: "c2", index: 1 }, "c2"), (Column { name: "c3", index: 2 }, "c3")], null_expr: [(Literal { value: Utf8(NULL), field: Field { name: "NULL", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} } }, "c1"), (Literal { value: Int64(NULL), field: Field { name: "NULL", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} } }, "c2"), (Literal { value: Int64(NULL), field: Field { name: "NULL", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} } }, "c3")], groups: [[true, true, true], [false, true, true], [false, false, true], [false, false, false]] })"#;
23132314

23142315
assert_eq!(format!("{rollup:?}"), expected);
23152316

@@ -2493,7 +2494,7 @@ mod tests {
24932494
let execution_plan = plan(&logical_plan).await?;
24942495
// verify that the plan correctly adds cast from Int64(1) to Utf8, and the const will be evaluated.
24952496

2496-
let expected = "expr: [(BinaryExpr { left: BinaryExpr { left: Column { name: \"c1\", index: 0 }, op: Eq, right: Literal { value: Utf8(\"a\") }, fail_on_overflow: false }, op: Or, right: BinaryExpr { left: Column { name: \"c1\", index: 0 }, op: Eq, right: Literal { value: Utf8(\"1\") }, fail_on_overflow: false }, fail_on_overflow: false }";
2497+
let expected = "expr: [(BinaryExpr { left: BinaryExpr { left: Column { name: \"c1\", index: 0 }, op: Eq, right: Literal { value: Utf8(\"a\"), field: Field { name: \"a\", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, fail_on_overflow: false }, op: Or, right: BinaryExpr { left: Column { name: \"c1\", index: 0 }, op: Eq, right: Literal { value: Utf8(\"1\"), field: Field { name: \"1\", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, fail_on_overflow: false }, fail_on_overflow: false }";
24972498

24982499
let actual = format!("{execution_plan:?}");
24992500
assert!(actual.contains(expected), "{}", actual);

datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -179,12 +179,12 @@ impl TableProvider for CustomProvider {
179179
match &filters[0] {
180180
Expr::BinaryExpr(BinaryExpr { right, .. }) => {
181181
let int_value = match &**right {
182-
Expr::Literal(ScalarValue::Int8(Some(i))) => *i as i64,
183-
Expr::Literal(ScalarValue::Int16(Some(i))) => *i as i64,
184-
Expr::Literal(ScalarValue::Int32(Some(i))) => *i as i64,
185-
Expr::Literal(ScalarValue::Int64(Some(i))) => *i,
182+
Expr::Literal(ScalarValue::Int8(Some(i)), _) => *i as i64,
183+
Expr::Literal(ScalarValue::Int16(Some(i)), _) => *i as i64,
184+
Expr::Literal(ScalarValue::Int32(Some(i)), _) => *i as i64,
185+
Expr::Literal(ScalarValue::Int64(Some(i)), _) => *i,
186186
Expr::Cast(Cast { expr, data_type: _ }) => match expr.deref() {
187-
Expr::Literal(lit_value) => match lit_value {
187+
Expr::Literal(lit_value, _) => match lit_value {
188188
ScalarValue::Int8(Some(v)) => *v as i64,
189189
ScalarValue::Int16(Some(v)) => *v as i64,
190190
ScalarValue::Int32(Some(v)) => *v as i64,

datafusion/core/tests/dataframe/mod.rs

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1210,7 +1210,7 @@ async fn join_on_filter_datatype() -> Result<()> {
12101210
let join = left.clone().join_on(
12111211
right.clone(),
12121212
JoinType::Inner,
1213-
Some(Expr::Literal(ScalarValue::Null)),
1213+
Some(Expr::Literal(ScalarValue::Null, None)),
12141214
)?;
12151215
assert_snapshot!(join.into_optimized_plan().unwrap(), @"EmptyRelation");
12161216

@@ -4527,7 +4527,10 @@ async fn consecutive_projection_same_schema() -> Result<()> {
45274527

45284528
// Add `t` column full of nulls
45294529
let df = df
4530-
.with_column("t", cast(Expr::Literal(ScalarValue::Null), DataType::Int32))
4530+
.with_column(
4531+
"t",
4532+
cast(Expr::Literal(ScalarValue::Null, None), DataType::Int32),
4533+
)
45314534
.unwrap();
45324535
df.clone().show().await.unwrap();
45334536

datafusion/core/tests/execution/logical_plan.rs

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -47,9 +47,9 @@ async fn count_only_nulls() -> Result<()> {
4747
let input = Arc::new(LogicalPlan::Values(Values {
4848
schema: input_schema,
4949
values: vec![
50-
vec![Expr::Literal(ScalarValue::Null)],
51-
vec![Expr::Literal(ScalarValue::Null)],
52-
vec![Expr::Literal(ScalarValue::Null)],
50+
vec![Expr::Literal(ScalarValue::Null, None)],
51+
vec![Expr::Literal(ScalarValue::Null, None)],
52+
vec![Expr::Literal(ScalarValue::Null, None)],
5353
],
5454
}));
5555
let input_col_ref = Expr::Column(Column {

datafusion/core/tests/expr_api/simplification.rs

Lines changed: 7 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -282,10 +282,13 @@ fn select_date_plus_interval() -> Result<()> {
282282

283283
let date_plus_interval_expr = to_timestamp_expr(ts_string)
284284
.cast_to(&DataType::Date32, schema)?
285-
+ Expr::Literal(ScalarValue::IntervalDayTime(Some(IntervalDayTime {
286-
days: 123,
287-
milliseconds: 0,
288-
})));
285+
+ Expr::Literal(
286+
ScalarValue::IntervalDayTime(Some(IntervalDayTime {
287+
days: 123,
288+
milliseconds: 0,
289+
})),
290+
None,
291+
);
289292

290293
let plan = LogicalPlanBuilder::from(table_scan.clone())
291294
.project(vec![date_plus_interval_expr])?

datafusion/core/tests/user_defined/expr_planner.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -56,7 +56,7 @@ impl ExprPlanner for MyCustomPlanner {
5656
}
5757
BinaryOperator::Question => {
5858
Ok(PlannerResult::Planned(Expr::Alias(Alias::new(
59-
Expr::Literal(ScalarValue::Boolean(Some(true))),
59+
Expr::Literal(ScalarValue::Boolean(Some(true)), None),
6060
None::<&str>,
6161
format!("{} ? {}", expr.left, expr.right),
6262
))))

datafusion/core/tests/user_defined/user_defined_plan.rs

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -912,11 +912,12 @@ impl MyAnalyzerRule {
912912
.map(|e| {
913913
e.transform(|e| {
914914
Ok(match e {
915-
Expr::Literal(ScalarValue::Int64(i)) => {
915+
Expr::Literal(ScalarValue::Int64(i), _) => {
916916
// transform to UInt64
917-
Transformed::yes(Expr::Literal(ScalarValue::UInt64(
918-
i.map(|i| i as u64),
919-
)))
917+
Transformed::yes(Expr::Literal(
918+
ScalarValue::UInt64(i.map(|i| i as u64)),
919+
None,
920+
))
920921
}
921922
_ => Transformed::no(e),
922923
})

datafusion/core/tests/user_defined/user_defined_scalar_functions.rs

Lines changed: 63 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ use std::collections::HashMap;
2020
use std::hash::{DefaultHasher, Hash, Hasher};
2121
use std::sync::Arc;
2222

23-
use arrow::array::{as_string_array, record_batch, Int8Array, UInt64Array};
23+
use arrow::array::{as_string_array, create_array, record_batch, Int8Array, UInt64Array};
2424
use arrow::array::{
2525
builder::BooleanBuilder, cast::AsArray, Array, ArrayRef, Float32Array, Float64Array,
2626
Int32Array, RecordBatch, StringArray,
@@ -42,9 +42,9 @@ use datafusion_common::{
4242
};
4343
use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo};
4444
use datafusion_expr::{
45-
Accumulator, ColumnarValue, CreateFunction, CreateFunctionBody, LogicalPlanBuilder,
46-
OperateFunctionArg, ReturnFieldArgs, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl,
47-
Signature, Volatility,
45+
lit_with_metadata, Accumulator, ColumnarValue, CreateFunction, CreateFunctionBody,
46+
LogicalPlanBuilder, OperateFunctionArg, ReturnFieldArgs, ScalarFunctionArgs,
47+
ScalarUDF, ScalarUDFImpl, Signature, Volatility,
4848
};
4949
use datafusion_functions_nested::range::range_udf;
5050
use parking_lot::Mutex;
@@ -1529,6 +1529,65 @@ async fn test_metadata_based_udf() -> Result<()> {
15291529
Ok(())
15301530
}
15311531

1532+
#[tokio::test]
1533+
async fn test_metadata_based_udf_with_literal() -> Result<()> {
1534+
let ctx = SessionContext::new();
1535+
let input_metadata: HashMap<String, String> =
1536+
[("modify_values".to_string(), "double_output".to_string())]
1537+
.into_iter()
1538+
.collect();
1539+
let df = ctx.sql("select 0;").await?.select(vec![
1540+
lit(5u64).alias_with_metadata("lit_with_doubling", Some(input_metadata.clone())),
1541+
lit(5u64).alias("lit_no_doubling"),
1542+
lit_with_metadata(5u64, Some(input_metadata))
1543+
.alias("lit_with_double_no_alias_metadata"),
1544+
])?;
1545+
1546+
let output_metadata: HashMap<String, String> =
1547+
[("output_metatype".to_string(), "custom_value".to_string())]
1548+
.into_iter()
1549+
.collect();
1550+
let custom_udf = ScalarUDF::from(MetadataBasedUdf::new(output_metadata.clone()));
1551+
1552+
let plan = LogicalPlanBuilder::from(df.into_optimized_plan()?)
1553+
.project(vec![
1554+
custom_udf
1555+
.call(vec![col("lit_with_doubling")])
1556+
.alias("doubled_output"),
1557+
custom_udf
1558+
.call(vec![col("lit_no_doubling")])
1559+
.alias("not_doubled_output"),
1560+
custom_udf
1561+
.call(vec![col("lit_with_double_no_alias_metadata")])
1562+
.alias("double_without_alias_metadata"),
1563+
])?
1564+
.build()?;
1565+
1566+
let actual = DataFrame::new(ctx.state(), plan).collect().await?;
1567+
1568+
let schema = Arc::new(Schema::new(vec![
1569+
Field::new("doubled_output", DataType::UInt64, false)
1570+
.with_metadata(output_metadata.clone()),
1571+
Field::new("not_doubled_output", DataType::UInt64, false)
1572+
.with_metadata(output_metadata.clone()),
1573+
Field::new("double_without_alias_metadata", DataType::UInt64, false)
1574+
.with_metadata(output_metadata.clone()),
1575+
]));
1576+
1577+
let expected = RecordBatch::try_new(
1578+
schema,
1579+
vec![
1580+
create_array!(UInt64, [10]),
1581+
create_array!(UInt64, [5]),
1582+
create_array!(UInt64, [10]),
1583+
],
1584+
)?;
1585+
1586+
assert_eq!(expected, actual[0]);
1587+
1588+
Ok(())
1589+
}
1590+
15321591
/// This UDF is to test extension handling, both on the input and output
15331592
/// sides. For the input, we will handle the data differently if there is
15341593
/// the canonical extension type Bool8. For the output we will add a

datafusion/core/tests/user_defined/user_defined_table_functions.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -205,7 +205,7 @@ impl TableFunctionImpl for SimpleCsvTableFunc {
205205
let mut filepath = String::new();
206206
for expr in exprs {
207207
match expr {
208-
Expr::Literal(ScalarValue::Utf8(Some(ref path))) => {
208+
Expr::Literal(ScalarValue::Utf8(Some(ref path)), _) => {
209209
filepath.clone_from(path);
210210
}
211211
expr => new_exprs.push(expr.clone()),

datafusion/datasource-parquet/src/row_filter.rs

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -557,6 +557,7 @@ mod test {
557557
// Test all should fail
558558
let expr = col("timestamp_col").lt(Expr::Literal(
559559
ScalarValue::TimestampNanosecond(Some(1), Some(Arc::from("UTC"))),
560+
None,
560561
));
561562
let expr = logical2physical(&expr, &table_schema);
562563
let schema_adapter_factory = Arc::new(DefaultSchemaAdapterFactory);
@@ -597,6 +598,7 @@ mod test {
597598
// Test all should pass
598599
let expr = col("timestamp_col").gt(Expr::Literal(
599600
ScalarValue::TimestampNanosecond(Some(0), Some(Arc::from("UTC"))),
601+
None,
600602
));
601603
let expr = logical2physical(&expr, &table_schema);
602604
let schema_adapter_factory = Arc::new(DefaultSchemaAdapterFactory);
@@ -660,7 +662,7 @@ mod test {
660662

661663
let expr = col("string_col")
662664
.is_not_null()
663-
.or(col("bigint_col").gt(Expr::Literal(ScalarValue::Int64(Some(5)))));
665+
.or(col("bigint_col").gt(Expr::Literal(ScalarValue::Int64(Some(5)), None)));
664666
let expr = logical2physical(&expr, &table_schema);
665667

666668
assert!(can_expr_be_pushed_down_with_schemas(&expr, &table_schema));

0 commit comments

Comments
 (0)