Skip to content

Commit 2c71845

Browse files
authored
fix: Native engine crashes on literal DateTrunc and TimestampTrunc (#3668)
1 parent d3b2007 commit 2c71845

4 files changed

Lines changed: 25 additions & 8 deletions

File tree

native/spark-expr/src/datetime_funcs/date_trunc.rs

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,9 @@
1616
// under the License.
1717

1818
use arrow::datatypes::DataType;
19-
use datafusion::common::{utils::take_function_args, DataFusionError, Result, ScalarValue::Utf8};
19+
use datafusion::common::{
20+
utils::take_function_args, DataFusionError, Result, ScalarValue, ScalarValue::Utf8,
21+
};
2022
use datafusion::logical_expr::{
2123
ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility,
2224
};
@@ -76,9 +78,14 @@ impl ScalarUDFImpl for SparkDateTrunc {
7678
let result = date_trunc_array_fmt_dyn(&date, &formats)?;
7779
Ok(ColumnarValue::Array(result))
7880
}
81+
(ColumnarValue::Scalar(date_scalar), ColumnarValue::Scalar(Utf8(Some(format)))) => {
82+
let date_arr = date_scalar.to_array()?;
83+
let result = date_trunc_dyn(&date_arr, format)?;
84+
let scalar = ScalarValue::try_from_array(&result, 0)?;
85+
Ok(ColumnarValue::Scalar(scalar))
86+
}
7987
_ => Err(DataFusionError::Execution(
80-
"Invalid input to function DateTrunc. Expected (PrimitiveArray<Date32>, Scalar) or \
81-
(PrimitiveArray<Date32>, StringArray)".to_string(),
88+
"Invalid input to function DateTrunc. Expected (Date32, Utf8)".to_string(),
8289
)),
8390
}
8491
}

native/spark-expr/src/datetime_funcs/timestamp_trunc.rs

Lines changed: 13 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@
1818
use crate::utils::array_with_timezone;
1919
use arrow::datatypes::{DataType, Schema, TimeUnit::Microsecond};
2020
use arrow::record_batch::RecordBatch;
21-
use datafusion::common::{DataFusionError, ScalarValue::Utf8};
21+
use datafusion::common::{DataFusionError, ScalarValue, ScalarValue::Utf8};
2222
use datafusion::logical_expr::ColumnarValue;
2323
use datafusion::physical_expr::PhysicalExpr;
2424
use std::hash::Hash;
@@ -130,10 +130,20 @@ impl PhysicalExpr for TimestampTruncExpr {
130130
let result = timestamp_trunc_array_fmt_dyn(&ts, &formats)?;
131131
Ok(ColumnarValue::Array(result))
132132
}
133+
(ColumnarValue::Scalar(ts_scalar), ColumnarValue::Scalar(Utf8(Some(format)))) => {
134+
let ts_arr = ts_scalar.to_array()?;
135+
let ts = array_with_timezone(
136+
ts_arr,
137+
tz.clone(),
138+
Some(&DataType::Timestamp(Microsecond, Some(tz.into()))),
139+
)?;
140+
let result = timestamp_trunc_dyn(&ts, format)?;
141+
let scalar = ScalarValue::try_from_array(&result, 0)?;
142+
Ok(ColumnarValue::Scalar(scalar))
143+
}
133144
_ => Err(DataFusionError::Execution(
134145
"Invalid input to function TimestampTrunc. \
135-
Expected (PrimitiveArray<TimestampMicrosecondType>, Scalar, String) or \
136-
(PrimitiveArray<TimestampMicrosecondType>, StringArray, String)"
146+
Expected (Timestamp, Utf8)"
137147
.to_string(),
138148
)),
139149
}

spark/src/test/resources/sql-tests/expressions/datetime/trunc_date.sql

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -33,5 +33,5 @@ query
3333
SELECT trunc(d, 'quarter') FROM test_trunc_date
3434

3535
-- literal arguments
36-
query ignore(https://github.com/apache/datafusion-comet/issues/3342)
36+
query
3737
SELECT trunc(date('2024-06-15'), 'year'), trunc(date('2024-06-15'), 'month'), trunc(date('2024-06-15'), 'quarter')

spark/src/test/resources/sql-tests/expressions/datetime/trunc_timestamp.sql

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -37,5 +37,5 @@ query
3737
SELECT date_trunc('hour', ts) FROM test_trunc_ts
3838

3939
-- literal arguments
40-
query ignore(https://github.com/apache/datafusion-comet/issues/3342)
40+
query
4141
SELECT date_trunc('year', timestamp('2024-06-15 10:30:45')), date_trunc('month', timestamp('2024-06-15 10:30:45')), date_trunc('day', timestamp('2024-06-15 10:30:45'))

0 commit comments

Comments
 (0)