You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by li...@apache.org on 2022/07/03 06:15:08 UTC

[arrow-datafusion] branch master updated: Improve ergonomics of physical expr `lit` (#2828)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new b47ab7c62 Improve ergonomics of physical expr `lit` (#2828)
b47ab7c62 is described below

commit b47ab7c622bc363d9f174bf72a53aabeef5a68f4
Author: Andrew Lamb <an...@nerdnetworks.org>
AuthorDate: Sun Jul 3 02:15:03 2022 -0400

    Improve ergonomics of physical expr `lit` (#2828)
    
    * Improve ergonomics of physical expr lit
    
    * Update usages of `lit`
    
    * Clippy
---
 .../src/physical_optimizer/aggregate_statistics.rs |   4 +-
 .../core/src/physical_plan/aggregates/mod.rs       |   2 +-
 datafusion/core/src/physical_plan/filter.rs        |  23 +-
 datafusion/physical-expr/src/expressions/case.rs   |  59 +-
 .../src/expressions/get_indexed_field.rs           |   2 +-
 .../physical-expr/src/expressions/in_list.rs       | 107 +---
 .../physical-expr/src/expressions/literal.rs       |  15 +-
 datafusion/physical-expr/src/functions.rs          | 691 ++++++---------------
 8 files changed, 252 insertions(+), 651 deletions(-)

diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs
index c75bf7e1b..1237d4911 100644
--- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs
+++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs
@@ -525,7 +525,7 @@ mod tests {
             expressions::binary(
                 expressions::col("a", &schema)?,
                 Operator::Gt,
-                expressions::lit(ScalarValue::from(1u32)),
+                expressions::lit(1u32),
                 &schema,
             )?,
             source,
@@ -568,7 +568,7 @@ mod tests {
             expressions::binary(
                 expressions::col("a", &schema)?,
                 Operator::Gt,
-                expressions::lit(ScalarValue::from(1u32)),
+                expressions::lit(1u32),
                 &schema,
             )?,
             source,
diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs
index 657b6281a..88eda63ed 100644
--- a/datafusion/core/src/physical_plan/aggregates/mod.rs
+++ b/datafusion/core/src/physical_plan/aggregates/mod.rs
@@ -718,7 +718,7 @@ mod tests {
         };
 
         let aggregates: Vec<Arc<dyn AggregateExpr>> = vec![Arc::new(Count::new(
-            lit(ScalarValue::Int8(Some(1))),
+            lit(1i8),
             "COUNT(1)".to_string(),
             DataType::Int64,
         ))];
diff --git a/datafusion/core/src/physical_plan/filter.rs b/datafusion/core/src/physical_plan/filter.rs
index b91c6bb9d..b4e3edaee 100644
--- a/datafusion/core/src/physical_plan/filter.rs
+++ b/datafusion/core/src/physical_plan/filter.rs
@@ -239,7 +239,6 @@ mod tests {
     use crate::physical_plan::ExecutionPlan;
     use crate::physical_plan::{collect, with_new_children_if_necessary};
     use crate::prelude::SessionContext;
-    use crate::scalar::ScalarValue;
     use crate::test;
     use crate::test_util;
     use datafusion_expr::Operator;
@@ -255,19 +254,9 @@ mod tests {
         let csv = test::scan_partitioned_csv(partitions)?;
 
         let predicate: Arc<dyn PhysicalExpr> = binary(
-            binary(
-                col("c2", &schema)?,
-                Operator::Gt,
-                lit(ScalarValue::from(1u32)),
-                &schema,
-            )?,
+            binary(col("c2", &schema)?, Operator::Gt, lit(1u32), &schema)?,
             Operator::And,
-            binary(
-                col("c2", &schema)?,
-                Operator::Lt,
-                lit(ScalarValue::from(4u32)),
-                &schema,
-            )?,
+            binary(col("c2", &schema)?, Operator::Lt, lit(4u32), &schema)?,
             &schema,
         )?;
 
@@ -292,12 +281,8 @@ mod tests {
         let partitions = 4;
         let input = test::scan_partitioned_csv(partitions)?;
 
-        let predicate: Arc<dyn PhysicalExpr> = binary(
-            col("c2", &schema)?,
-            Operator::Gt,
-            lit(ScalarValue::from(1u32)),
-            &schema,
-        )?;
+        let predicate: Arc<dyn PhysicalExpr> =
+            binary(col("c2", &schema)?, Operator::Gt, lit(1u32), &schema)?;
 
         let filter: Arc<dyn ExecutionPlan> =
             Arc::new(FilterExec::try_new(predicate, input.clone())?);
diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs
index 4c62c1d57..6e67ba4ad 100644
--- a/datafusion/physical-expr/src/expressions/case.rs
+++ b/datafusion/physical-expr/src/expressions/case.rs
@@ -316,10 +316,10 @@ mod tests {
         let schema = batch.schema();
 
         // CASE a WHEN 'foo' THEN 123 WHEN 'bar' THEN 456 END
-        let when1 = lit(ScalarValue::Utf8(Some("foo".to_string())));
-        let then1 = lit(ScalarValue::Int32(Some(123)));
-        let when2 = lit(ScalarValue::Utf8(Some("bar".to_string())));
-        let then2 = lit(ScalarValue::Int32(Some(456)));
+        let when1 = lit("foo");
+        let then1 = lit(123i32);
+        let when2 = lit("bar");
+        let then2 = lit(456i32);
 
         let expr = case(
             Some(col("a", &schema)?),
@@ -345,11 +345,11 @@ mod tests {
         let schema = batch.schema();
 
         // CASE a WHEN 'foo' THEN 123 WHEN 'bar' THEN 456 ELSE 999 END
-        let when1 = lit(ScalarValue::Utf8(Some("foo".to_string())));
-        let then1 = lit(ScalarValue::Int32(Some(123)));
-        let when2 = lit(ScalarValue::Utf8(Some("bar".to_string())));
-        let then2 = lit(ScalarValue::Int32(Some(456)));
-        let else_value = lit(ScalarValue::Int32(Some(999)));
+        let when1 = lit("foo");
+        let then1 = lit(123i32);
+        let when2 = lit("bar");
+        let then2 = lit(456i32);
+        let else_value = lit(999i32);
 
         let expr = case(
             Some(col("a", &schema)?),
@@ -376,10 +376,10 @@ mod tests {
         let schema = batch.schema();
 
         // CASE a when 0 THEN float64(null) ELSE 25.0 / cast(a, float64)  END
-        let when1 = lit(ScalarValue::Int32(Some(0)));
+        let when1 = lit(0i32);
         let then1 = lit(ScalarValue::Float64(None));
         let else_value = binary(
-            lit(ScalarValue::Float64(Some(25.0))),
+            lit(25.0f64),
             Operator::Divide,
             cast(col("a", &schema)?, &batch.schema(), Float64)?,
             &batch.schema(),
@@ -412,17 +412,17 @@ mod tests {
         let when1 = binary(
             col("a", &schema)?,
             Operator::Eq,
-            lit(ScalarValue::Utf8(Some("foo".to_string()))),
+            lit("foo"),
             &batch.schema(),
         )?;
-        let then1 = lit(ScalarValue::Int32(Some(123)));
+        let then1 = lit(123i32);
         let when2 = binary(
             col("a", &schema)?,
             Operator::Eq,
-            lit(ScalarValue::Utf8(Some("bar".to_string()))),
+            lit("bar"),
             &batch.schema(),
         )?;
-        let then2 = lit(ScalarValue::Int32(Some(456)));
+        let then2 = lit(456i32);
 
         let expr = case(None, &[(when1, then1), (when2, then2)], None)?;
         let result = expr.evaluate(&batch)?.into_array(batch.num_rows());
@@ -444,14 +444,9 @@ mod tests {
         let schema = batch.schema();
 
         // CASE WHEN a > 0 THEN 25.0 / cast(a, float64) ELSE float64(null) END
-        let when1 = binary(
-            col("a", &schema)?,
-            Operator::Gt,
-            lit(ScalarValue::Int32(Some(0))),
-            &batch.schema(),
-        )?;
+        let when1 = binary(col("a", &schema)?, Operator::Gt, lit(0i32), &batch.schema())?;
         let then1 = binary(
-            lit(ScalarValue::Float64(Some(25.0))),
+            lit(25.0f64),
             Operator::Divide,
             cast(col("a", &schema)?, &batch.schema(), Float64)?,
             &batch.schema(),
@@ -488,18 +483,18 @@ mod tests {
         let when1 = binary(
             col("a", &schema)?,
             Operator::Eq,
-            lit(ScalarValue::Utf8(Some("foo".to_string()))),
+            lit("foo"),
             &batch.schema(),
         )?;
-        let then1 = lit(ScalarValue::Int32(Some(123)));
+        let then1 = lit(123i32);
         let when2 = binary(
             col("a", &schema)?,
             Operator::Eq,
-            lit(ScalarValue::Utf8(Some("bar".to_string()))),
+            lit("bar"),
             &batch.schema(),
         )?;
-        let then2 = lit(ScalarValue::Int32(Some(456)));
-        let else_value = lit(ScalarValue::Int32(Some(999)));
+        let then2 = lit(456i32);
+        let else_value = lit(999i32);
 
         let expr = case(None, &[(when1, then1), (when2, then2)], Some(else_value))?;
         let result = expr.evaluate(&batch)?.into_array(batch.num_rows());
@@ -525,11 +520,11 @@ mod tests {
         let when = binary(
             col("a", &schema)?,
             Operator::Eq,
-            lit(ScalarValue::Utf8(Some("foo".to_string()))),
+            lit("foo"),
             &batch.schema(),
         )?;
-        let then = lit(ScalarValue::Float64(Some(123.3)));
-        let else_value = lit(ScalarValue::Int32(Some(999)));
+        let then = lit(123.3f64);
+        let else_value = lit(999i32);
 
         let expr = case(None, &[(when, then)], Some(else_value))?;
         let result = expr.evaluate(&batch)?.into_array(batch.num_rows());
@@ -555,7 +550,7 @@ mod tests {
         let when = binary(
             col("load4", &schema)?,
             Operator::Eq,
-            lit(ScalarValue::Float64(Some(1.77))),
+            lit(1.77f64),
             &batch.schema(),
         )?;
         let then = col("load4", &schema)?;
@@ -582,7 +577,7 @@ mod tests {
 
         // SELECT CASE load4 WHEN 1.77 THEN load4 END
         let expr = col("load4", &schema)?;
-        let when = lit(ScalarValue::Float64(Some(1.77)));
+        let when = lit(1.77f64);
         let then = col("load4", &schema)?;
 
         let expr = case(Some(expr), &[(when, then)], None)?;
diff --git a/datafusion/physical-expr/src/expressions/get_indexed_field.rs b/datafusion/physical-expr/src/expressions/get_indexed_field.rs
index 506ba96b6..bb43229d1 100644
--- a/datafusion/physical-expr/src/expressions/get_indexed_field.rs
+++ b/datafusion/physical-expr/src/expressions/get_indexed_field.rs
@@ -244,7 +244,7 @@ mod tests {
     #[test]
     fn get_indexed_field_invalid_scalar() -> Result<()> {
         let schema = list_schema("l");
-        let expr = lit(ScalarValue::Utf8(Some("a".to_string())));
+        let expr = lit("a");
         get_indexed_field_test_failure(schema, expr,  ScalarValue::Int64(Some(0)), "Execution error: get indexed field is only possible on lists with int64 indexes or struct with utf8 indexes. Tried Utf8 with Int64(0) index")
     }
 
diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs
index 729d066d6..392f382c6 100644
--- a/datafusion/physical-expr/src/expressions/in_list.rs
+++ b/datafusion/physical-expr/src/expressions/in_list.rs
@@ -780,10 +780,7 @@ mod tests {
         let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(a)])?;
 
         // expression: "a in ("a", "b")"
-        let list = vec![
-            lit(ScalarValue::Utf8(Some("a".to_string()))),
-            lit(ScalarValue::Utf8(Some("b".to_string()))),
-        ];
+        let list = vec![lit("a"), lit("b")];
         in_list!(
             batch,
             list,
@@ -794,10 +791,7 @@ mod tests {
         );
 
         // expression: "a not in ("a", "b")"
-        let list = vec![
-            lit(ScalarValue::Utf8(Some("a".to_string()))),
-            lit(ScalarValue::Utf8(Some("b".to_string()))),
-        ];
+        let list = vec![lit("a"), lit("b")];
         in_list!(
             batch,
             list,
@@ -808,11 +802,7 @@ mod tests {
         );
 
         // expression: "a not in ("a", "b")"
-        let list = vec![
-            lit(ScalarValue::Utf8(Some("a".to_string()))),
-            lit(ScalarValue::Utf8(Some("b".to_string()))),
-            lit(ScalarValue::Utf8(None)),
-        ];
+        let list = vec![lit("a"), lit("b"), lit(ScalarValue::Utf8(None))];
         in_list!(
             batch,
             list,
@@ -823,11 +813,7 @@ mod tests {
         );
 
         // expression: "a not in ("a", "b")"
-        let list = vec![
-            lit(ScalarValue::Utf8(Some("a".to_string()))),
-            lit(ScalarValue::Utf8(Some("b".to_string()))),
-            lit(ScalarValue::Utf8(None)),
-        ];
+        let list = vec![lit("a"), lit("b"), lit(ScalarValue::Utf8(None))];
         in_list!(
             batch,
             list,
@@ -848,10 +834,7 @@ mod tests {
         let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(a)])?;
 
         // expression: "a in (0, 1)"
-        let list = vec![
-            lit(ScalarValue::Int64(Some(0))),
-            lit(ScalarValue::Int64(Some(1))),
-        ];
+        let list = vec![lit(0i64), lit(1i64)];
         in_list!(
             batch,
             list,
@@ -862,10 +845,7 @@ mod tests {
         );
 
         // expression: "a not in (0, 1)"
-        let list = vec![
-            lit(ScalarValue::Int64(Some(0))),
-            lit(ScalarValue::Int64(Some(1))),
-        ];
+        let list = vec![lit(0i64), lit(1i64)];
         in_list!(
             batch,
             list,
@@ -876,11 +856,7 @@ mod tests {
         );
 
         // expression: "a in (0, 1, NULL)"
-        let list = vec![
-            lit(ScalarValue::Int64(Some(0))),
-            lit(ScalarValue::Int64(Some(1))),
-            lit(ScalarValue::Null),
-        ];
+        let list = vec![lit(0i64), lit(1i64), lit(ScalarValue::Null)];
         in_list!(
             batch,
             list,
@@ -891,11 +867,7 @@ mod tests {
         );
 
         // expression: "a not in (0, 1, NULL)"
-        let list = vec![
-            lit(ScalarValue::Int64(Some(0))),
-            lit(ScalarValue::Int64(Some(1))),
-            lit(ScalarValue::Null),
-        ];
+        let list = vec![lit(0i64), lit(1i64), lit(ScalarValue::Null)];
         in_list!(
             batch,
             list,
@@ -916,10 +888,7 @@ mod tests {
         let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(a)])?;
 
         // expression: "a in (0.0, 0.2)"
-        let list = vec![
-            lit(ScalarValue::Float64(Some(0.0))),
-            lit(ScalarValue::Float64(Some(0.1))),
-        ];
+        let list = vec![lit(0.0f64), lit(0.1f64)];
         in_list!(
             batch,
             list,
@@ -930,10 +899,7 @@ mod tests {
         );
 
         // expression: "a not in (0.0, 0.2)"
-        let list = vec![
-            lit(ScalarValue::Float64(Some(0.0))),
-            lit(ScalarValue::Float64(Some(0.1))),
-        ];
+        let list = vec![lit(0.0f64), lit(0.1f64)];
         in_list!(
             batch,
             list,
@@ -944,11 +910,7 @@ mod tests {
         );
 
         // expression: "a in (0.0, 0.2, NULL)"
-        let list = vec![
-            lit(ScalarValue::Float64(Some(0.0))),
-            lit(ScalarValue::Float64(Some(0.1))),
-            lit(ScalarValue::Null),
-        ];
+        let list = vec![lit(0.0f64), lit(0.1f64), lit(ScalarValue::Null)];
         in_list!(
             batch,
             list,
@@ -959,11 +921,7 @@ mod tests {
         );
 
         // expression: "a not in (0.0, 0.2, NULL)"
-        let list = vec![
-            lit(ScalarValue::Float64(Some(0.0))),
-            lit(ScalarValue::Float64(Some(0.1))),
-            lit(ScalarValue::Null),
-        ];
+        let list = vec![lit(0.0f64), lit(0.1f64), lit(ScalarValue::Null)];
         in_list!(
             batch,
             list,
@@ -984,7 +942,7 @@ mod tests {
         let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(a)])?;
 
         // expression: "a in (true)"
-        let list = vec![lit(ScalarValue::Boolean(Some(true)))];
+        let list = vec![lit(true)];
         in_list!(
             batch,
             list,
@@ -995,7 +953,7 @@ mod tests {
         );
 
         // expression: "a not in (true)"
-        let list = vec![lit(ScalarValue::Boolean(Some(true)))];
+        let list = vec![lit(true)];
         in_list!(
             batch,
             list,
@@ -1006,10 +964,7 @@ mod tests {
         );
 
         // expression: "a in (true, NULL)"
-        let list = vec![
-            lit(ScalarValue::Boolean(Some(true))),
-            lit(ScalarValue::Null),
-        ];
+        let list = vec![lit(true), lit(ScalarValue::Null)];
         in_list!(
             batch,
             list,
@@ -1020,10 +975,7 @@ mod tests {
         );
 
         // expression: "a not in (true, NULL)"
-        let list = vec![
-            lit(ScalarValue::Boolean(Some(true))),
-            lit(ScalarValue::Null),
-        ];
+        let list = vec![lit(true), lit(ScalarValue::Null)];
         in_list!(
             batch,
             list,
@@ -1049,10 +1001,7 @@ mod tests {
             RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(array)])?;
 
         // expression: "a in (100,200), the data type of list is INT32
-        let list = vec![
-            lit(ScalarValue::Int32(Some(100))),
-            lit(ScalarValue::Int32(Some(200))),
-        ];
+        let list = vec![lit(100i32), lit(200i32)];
         in_list!(
             batch,
             list,
@@ -1062,10 +1011,7 @@ mod tests {
             &schema
         );
         // expression: "a not in (100,200)
-        let list = vec![
-            lit(ScalarValue::Int32(Some(100))),
-            lit(ScalarValue::Int32(Some(200))),
-        ];
+        let list = vec![lit(100i32), lit(200i32)];
         in_list!(
             batch,
             list,
@@ -1077,14 +1023,11 @@ mod tests {
 
         // expression: "a in (200,NULL), the data type of list is INT32 AND NULL
         // TODO support: NULL data type to decimal in arrow-rs
-        // let list = vec![lit(ScalarValue::Int32(Some(100))), lit(ScalarValue::Null)];
+        // let list = vec![lit(100i32), lit(ScalarValue::Null)];
         // in_list!(batch, list, &false, vec![Some(true), None, Some(false)], col_a.clone(), &schema);
 
         // expression: "a in (200.5, 100), the data type of list is FLOAT32 and INT32
-        let list = vec![
-            lit(ScalarValue::Float32(Some(200.50f32))),
-            lit(ScalarValue::Int32(Some(100))),
-        ];
+        let list = vec![lit(200.50f32), lit(100i32)];
         in_list!(
             batch,
             list,
@@ -1095,10 +1038,7 @@ mod tests {
         );
 
         // expression: "a not in (200.5, 100), the data type of list is FLOAT32 and INT32
-        let list = vec![
-            lit(ScalarValue::Float32(Some(200.50f32))),
-            lit(ScalarValue::Int32(Some(101))),
-        ];
+        let list = vec![lit(200.50f32), lit(101i32)];
         in_list!(
             batch,
             list,
@@ -1110,10 +1050,7 @@ mod tests {
 
         // test the optimization: set
         // expression: "a in (99..300), the data type of list is INT32
-        let list = (99..300)
-            .into_iter()
-            .map(|v| lit(ScalarValue::Int32(Some(v))))
-            .collect::<Vec<_>>();
+        let list = (99i32..300).into_iter().map(lit).collect::<Vec<_>>();
 
         in_list!(
             batch,
diff --git a/datafusion/physical-expr/src/expressions/literal.rs b/datafusion/physical-expr/src/expressions/literal.rs
index b91e498a1..80a808cba 100644
--- a/datafusion/physical-expr/src/expressions/literal.rs
+++ b/datafusion/physical-expr/src/expressions/literal.rs
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-//! Literal expression
+//! Literal expressions for physical operations
 
 use std::any::Any;
 use std::sync::Arc;
@@ -28,7 +28,7 @@ use arrow::{
 use crate::PhysicalExpr;
 use datafusion_common::Result;
 use datafusion_common::ScalarValue;
-use datafusion_expr::ColumnarValue;
+use datafusion_expr::{ColumnarValue, Expr};
 
 /// Represents a literal value
 #[derive(Debug)]
@@ -74,8 +74,13 @@ impl PhysicalExpr for Literal {
 }
 
 /// Create a literal expression
-pub fn lit(value: ScalarValue) -> Arc<dyn PhysicalExpr> {
-    Arc::new(Literal::new(value))
+pub fn lit<T: datafusion_expr::Literal>(value: T) -> Arc<dyn PhysicalExpr> {
+    let scalar_value = if let Expr::Literal(v) = value.lit() {
+        v
+    } else {
+        unreachable!()
+    };
+    Arc::new(Literal::new(scalar_value))
 }
 
 #[cfg(test)]
@@ -93,7 +98,7 @@ mod tests {
         let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)])?;
 
         // create and evaluate a literal expression
-        let literal_expr = lit(ScalarValue::from(42i32));
+        let literal_expr = lit(42i32);
         assert_eq!("42", format!("{}", literal_expr));
 
         let literal_array = literal_expr.evaluate(&batch)?.into_array(batch.num_rows());
diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs
index 7c55a60a8..75604a532 100644
--- a/datafusion/physical-expr/src/functions.rs
+++ b/datafusion/physical-expr/src/functions.rs
@@ -765,25 +765,11 @@ mod tests {
 
     #[test]
     fn test_functions() -> Result<()> {
+        test_function!(Ascii, &[lit("x")], Ok(Some(120)), i32, Int32, Int32Array);
+        test_function!(Ascii, &[lit("ésoj")], Ok(Some(233)), i32, Int32, Int32Array);
         test_function!(
             Ascii,
-            &[lit(ScalarValue::Utf8(Some("x".to_string())))],
-            Ok(Some(120)),
-            i32,
-            Int32,
-            Int32Array
-        );
-        test_function!(
-            Ascii,
-            &[lit(ScalarValue::Utf8(Some("ésoj".to_string())))],
-            Ok(Some(233)),
-            i32,
-            Int32,
-            Int32Array
-        );
-        test_function!(
-            Ascii,
-            &[lit(ScalarValue::Utf8(Some("💯".to_string())))],
+            &[lit("💯")],
             Ok(Some(128175)),
             i32,
             Int32,
@@ -791,20 +777,13 @@ mod tests {
         );
         test_function!(
             Ascii,
-            &[lit(ScalarValue::Utf8(Some("💯a".to_string())))],
+            &[lit("💯a")],
             Ok(Some(128175)),
             i32,
             Int32,
             Int32Array
         );
-        test_function!(
-            Ascii,
-            &[lit(ScalarValue::Utf8(Some("".to_string())))],
-            Ok(Some(0)),
-            i32,
-            Int32,
-            Int32Array
-        );
+        test_function!(Ascii, &[lit("")], Ok(Some(0)), i32, Int32, Int32Array);
         test_function!(
             Ascii,
             &[lit(ScalarValue::Utf8(None))],
@@ -815,7 +794,7 @@ mod tests {
         );
         test_function!(
             BitLength,
-            &[lit(ScalarValue::Utf8(Some("chars".to_string())))],
+            &[lit("chars")],
             Ok(Some(40)),
             i32,
             Int32,
@@ -823,23 +802,16 @@ mod tests {
         );
         test_function!(
             BitLength,
-            &[lit(ScalarValue::Utf8(Some("josé".to_string())))],
+            &[lit("josé")],
             Ok(Some(40)),
             i32,
             Int32,
             Int32Array
         );
-        test_function!(
-            BitLength,
-            &[lit(ScalarValue::Utf8(Some("".to_string())))],
-            Ok(Some(0)),
-            i32,
-            Int32,
-            Int32Array
-        );
+        test_function!(BitLength, &[lit("")], Ok(Some(0)), i32, Int32, Int32Array);
         test_function!(
             Btrim,
-            &[lit(ScalarValue::Utf8(Some(" trim ".to_string())))],
+            &[lit(" trim ")],
             Ok(Some("trim")),
             &str,
             Utf8,
@@ -847,7 +819,7 @@ mod tests {
         );
         test_function!(
             Btrim,
-            &[lit(ScalarValue::Utf8(Some(" trim".to_string())))],
+            &[lit(" trim")],
             Ok(Some("trim")),
             &str,
             Utf8,
@@ -855,7 +827,7 @@ mod tests {
         );
         test_function!(
             Btrim,
-            &[lit(ScalarValue::Utf8(Some("trim ".to_string())))],
+            &[lit("trim ")],
             Ok(Some("trim")),
             &str,
             Utf8,
@@ -863,7 +835,7 @@ mod tests {
         );
         test_function!(
             Btrim,
-            &[lit(ScalarValue::Utf8(Some("\n trim \n".to_string())))],
+            &[lit("\n trim \n")],
             Ok(Some("\n trim \n")),
             &str,
             Utf8,
@@ -871,10 +843,7 @@ mod tests {
         );
         test_function!(
             Btrim,
-            &[
-                lit(ScalarValue::Utf8(Some("xyxtrimyyx".to_string()))),
-                lit(ScalarValue::Utf8(Some("xyz".to_string()))),
-            ],
+            &[lit("xyxtrimyyx"), lit("xyz"),],
             Ok(Some("trim")),
             &str,
             Utf8,
@@ -882,10 +851,7 @@ mod tests {
         );
         test_function!(
             Btrim,
-            &[
-                lit(ScalarValue::Utf8(Some("\nxyxtrimyyx\n".to_string()))),
-                lit(ScalarValue::Utf8(Some("xyz\n".to_string()))),
-            ],
+            &[lit("\nxyxtrimyyx\n"), lit("xyz\n"),],
             Ok(Some("trim")),
             &str,
             Utf8,
@@ -893,10 +859,7 @@ mod tests {
         );
         test_function!(
             Btrim,
-            &[
-                lit(ScalarValue::Utf8(None)),
-                lit(ScalarValue::Utf8(Some("xyz".to_string()))),
-            ],
+            &[lit(ScalarValue::Utf8(None)), lit("xyz"),],
             Ok(None),
             &str,
             Utf8,
@@ -904,10 +867,7 @@ mod tests {
         );
         test_function!(
             Btrim,
-            &[
-                lit(ScalarValue::Utf8(Some("xyxtrimyyx".to_string()))),
-                lit(ScalarValue::Utf8(None)),
-            ],
+            &[lit("xyxtrimyyx"), lit(ScalarValue::Utf8(None)),],
             Ok(None),
             &str,
             Utf8,
@@ -916,7 +876,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             CharacterLength,
-            &[lit(ScalarValue::Utf8(Some("chars".to_string())))],
+            &[lit("chars")],
             Ok(Some(5)),
             i32,
             Int32,
@@ -925,7 +885,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             CharacterLength,
-            &[lit(ScalarValue::Utf8(Some("josé".to_string())))],
+            &[lit("josé")],
             Ok(Some(4)),
             i32,
             Int32,
@@ -934,7 +894,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             CharacterLength,
-            &[lit(ScalarValue::Utf8(Some("".to_string())))],
+            &[lit("")],
             Ok(Some(0)),
             i32,
             Int32,
@@ -952,7 +912,7 @@ mod tests {
         #[cfg(not(feature = "unicode_expressions"))]
         test_function!(
             CharacterLength,
-            &[lit(ScalarValue::Utf8(Some("josé".to_string())))],
+            &[lit("josé")],
             Err(DataFusionError::Internal(
                 "function character_length requires compilation with feature flag: unicode_expressions.".to_string()
             )),
@@ -1022,11 +982,7 @@ mod tests {
         );
         test_function!(
             Concat,
-            &[
-                lit(ScalarValue::Utf8(Some("aa".to_string()))),
-                lit(ScalarValue::Utf8(Some("bb".to_string()))),
-                lit(ScalarValue::Utf8(Some("cc".to_string()))),
-            ],
+            &[lit("aa"), lit("bb"), lit("cc"),],
             Ok(Some("aabbcc")),
             &str,
             Utf8,
@@ -1034,11 +990,7 @@ mod tests {
         );
         test_function!(
             Concat,
-            &[
-                lit(ScalarValue::Utf8(Some("aa".to_string()))),
-                lit(ScalarValue::Utf8(None)),
-                lit(ScalarValue::Utf8(Some("cc".to_string()))),
-            ],
+            &[lit("aa"), lit(ScalarValue::Utf8(None)), lit("cc"),],
             Ok(Some("aacc")),
             &str,
             Utf8,
@@ -1054,12 +1006,7 @@ mod tests {
         );
         test_function!(
             ConcatWithSeparator,
-            &[
-                lit(ScalarValue::Utf8(Some("|".to_string()))),
-                lit(ScalarValue::Utf8(Some("aa".to_string()))),
-                lit(ScalarValue::Utf8(Some("bb".to_string()))),
-                lit(ScalarValue::Utf8(Some("cc".to_string()))),
-            ],
+            &[lit("|"), lit("aa"), lit("bb"), lit("cc"),],
             Ok(Some("aa|bb|cc")),
             &str,
             Utf8,
@@ -1067,10 +1014,7 @@ mod tests {
         );
         test_function!(
             ConcatWithSeparator,
-            &[
-                lit(ScalarValue::Utf8(Some("|".to_string()))),
-                lit(ScalarValue::Utf8(None)),
-            ],
+            &[lit("|"), lit(ScalarValue::Utf8(None)),],
             Ok(Some("")),
             &str,
             Utf8,
@@ -1080,9 +1024,9 @@ mod tests {
             ConcatWithSeparator,
             &[
                 lit(ScalarValue::Utf8(None)),
-                lit(ScalarValue::Utf8(Some("aa".to_string()))),
-                lit(ScalarValue::Utf8(Some("bb".to_string()))),
-                lit(ScalarValue::Utf8(Some("cc".to_string()))),
+                lit("aa"),
+                lit("bb"),
+                lit("cc"),
             ],
             Ok(None),
             &str,
@@ -1091,12 +1035,7 @@ mod tests {
         );
         test_function!(
             ConcatWithSeparator,
-            &[
-                lit(ScalarValue::Utf8(Some("|".to_string()))),
-                lit(ScalarValue::Utf8(Some("aa".to_string()))),
-                lit(ScalarValue::Utf8(None)),
-                lit(ScalarValue::Utf8(Some("cc".to_string()))),
-            ],
+            &[lit("|"), lit("aa"), lit(ScalarValue::Utf8(None)), lit("cc"),],
             Ok(Some("aa|cc")),
             &str,
             Utf8,
@@ -1144,28 +1083,14 @@ mod tests {
         );
         test_function!(
             InitCap,
-            &[lit(ScalarValue::Utf8(Some("hi THOMAS".to_string())))],
+            &[lit("hi THOMAS")],
             Ok(Some("Hi Thomas")),
             &str,
             Utf8,
             StringArray
         );
-        test_function!(
-            InitCap,
-            &[lit(ScalarValue::Utf8(Some("".to_string())))],
-            Ok(Some("")),
-            &str,
-            Utf8,
-            StringArray
-        );
-        test_function!(
-            InitCap,
-            &[lit(ScalarValue::Utf8(Some("".to_string())))],
-            Ok(Some("")),
-            &str,
-            Utf8,
-            StringArray
-        );
+        test_function!(InitCap, &[lit("")], Ok(Some("")), &str, Utf8, StringArray);
+        test_function!(InitCap, &[lit("")], Ok(Some("")), &str, Utf8, StringArray);
         test_function!(
             InitCap,
             &[lit(ScalarValue::Utf8(None))],
@@ -1177,10 +1102,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Left,
-            &[
-                lit(ScalarValue::Utf8(Some("abcde".to_string()))),
-                lit(ScalarValue::Int8(Some(2))),
-            ],
+            &[lit("abcde"), lit(ScalarValue::Int8(Some(2))),],
             Ok(Some("ab")),
             &str,
             Utf8,
@@ -1189,10 +1111,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Left,
-            &[
-                lit(ScalarValue::Utf8(Some("abcde".to_string()))),
-                lit(ScalarValue::Int64(Some(200))),
-            ],
+            &[lit("abcde"), lit(ScalarValue::Int64(Some(200))),],
             Ok(Some("abcde")),
             &str,
             Utf8,
@@ -1201,10 +1120,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Left,
-            &[
-                lit(ScalarValue::Utf8(Some("abcde".to_string()))),
-                lit(ScalarValue::Int64(Some(-2))),
-            ],
+            &[lit("abcde"), lit(ScalarValue::Int64(Some(-2))),],
             Ok(Some("abc")),
             &str,
             Utf8,
@@ -1213,10 +1129,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Left,
-            &[
-                lit(ScalarValue::Utf8(Some("abcde".to_string()))),
-                lit(ScalarValue::Int64(Some(-200))),
-            ],
+            &[lit("abcde"), lit(ScalarValue::Int64(Some(-200))),],
             Ok(Some("")),
             &str,
             Utf8,
@@ -1225,10 +1138,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Left,
-            &[
-                lit(ScalarValue::Utf8(Some("abcde".to_string()))),
-                lit(ScalarValue::Int64(Some(0))),
-            ],
+            &[lit("abcde"), lit(ScalarValue::Int64(Some(0))),],
             Ok(Some("")),
             &str,
             Utf8,
@@ -1249,10 +1159,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Left,
-            &[
-                lit(ScalarValue::Utf8(Some("abcde".to_string()))),
-                lit(ScalarValue::Int64(None)),
-            ],
+            &[lit("abcde"), lit(ScalarValue::Int64(None)),],
             Ok(None),
             &str,
             Utf8,
@@ -1261,10 +1168,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Left,
-            &[
-                lit(ScalarValue::Utf8(Some("joséésoj".to_string()))),
-                lit(ScalarValue::Int64(Some(5))),
-            ],
+            &[lit("joséésoj"), lit(ScalarValue::Int64(Some(5))),],
             Ok(Some("joséé")),
             &str,
             Utf8,
@@ -1273,10 +1177,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Left,
-            &[
-                lit(ScalarValue::Utf8(Some("joséésoj".to_string()))),
-                lit(ScalarValue::Int64(Some(-3))),
-            ],
+            &[lit("joséésoj"), lit(ScalarValue::Int64(Some(-3))),],
             Ok(Some("joséé")),
             &str,
             Utf8,
@@ -1286,7 +1187,7 @@ mod tests {
         test_function!(
             Left,
             &[
-                lit(ScalarValue::Utf8(Some("abcde".to_string()))),
+                lit("abcde"),
                 lit(ScalarValue::Int8(Some(2))),
             ],
             Err(DataFusionError::Internal(
@@ -1299,10 +1200,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Lpad,
-            &[
-                lit(ScalarValue::Utf8(Some("josé".to_string()))),
-                lit(ScalarValue::Int64(Some(5))),
-            ],
+            &[lit("josé"), lit(ScalarValue::Int64(Some(5))),],
             Ok(Some(" josé")),
             &str,
             Utf8,
@@ -1311,10 +1209,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Lpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(Some(5))),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(Some(5))),],
             Ok(Some("   hi")),
             &str,
             Utf8,
@@ -1323,10 +1218,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Lpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(Some(0))),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(Some(0))),],
             Ok(Some("")),
             &str,
             Utf8,
@@ -1335,10 +1227,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Lpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(None)),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(None)),],
             Ok(None),
             &str,
             Utf8,
@@ -1359,11 +1248,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Lpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(Some(5))),
-                lit(ScalarValue::Utf8(Some("xy".to_string()))),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(Some(5))), lit("xy"),],
             Ok(Some("xyxhi")),
             &str,
             Utf8,
@@ -1372,11 +1257,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Lpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(Some(21))),
-                lit(ScalarValue::Utf8(Some("abcdef".to_string()))),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(Some(21))), lit("abcdef"),],
             Ok(Some("abcdefabcdefabcdefahi")),
             &str,
             Utf8,
@@ -1385,11 +1266,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Lpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(Some(5))),
-                lit(ScalarValue::Utf8(Some(" ".to_string()))),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(Some(5))), lit(" "),],
             Ok(Some("   hi")),
             &str,
             Utf8,
@@ -1398,11 +1275,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Lpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(Some(5))),
-                lit(ScalarValue::Utf8(Some("".to_string()))),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(Some(5))), lit(""),],
             Ok(Some("hi")),
             &str,
             Utf8,
@@ -1414,7 +1287,7 @@ mod tests {
             &[
                 lit(ScalarValue::Utf8(None)),
                 lit(ScalarValue::Int64(Some(5))),
-                lit(ScalarValue::Utf8(Some("xy".to_string()))),
+                lit("xy"),
             ],
             Ok(None),
             &str,
@@ -1424,11 +1297,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Lpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(None)),
-                lit(ScalarValue::Utf8(Some("xy".to_string()))),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(None)), lit("xy"),],
             Ok(None),
             &str,
             Utf8,
@@ -1438,7 +1307,7 @@ mod tests {
         test_function!(
             Lpad,
             &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
+                lit("hi"),
                 lit(ScalarValue::Int64(Some(5))),
                 lit(ScalarValue::Utf8(None)),
             ],
@@ -1450,11 +1319,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Lpad,
-            &[
-                lit(ScalarValue::Utf8(Some("josé".to_string()))),
-                lit(ScalarValue::Int64(Some(10))),
-                lit(ScalarValue::Utf8(Some("xy".to_string()))),
-            ],
+            &[lit("josé"), lit(ScalarValue::Int64(Some(10))), lit("xy"),],
             Ok(Some("xyxyxyjosé")),
             &str,
             Utf8,
@@ -1463,11 +1328,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Lpad,
-            &[
-                lit(ScalarValue::Utf8(Some("josé".to_string()))),
-                lit(ScalarValue::Int64(Some(10))),
-                lit(ScalarValue::Utf8(Some("éñ".to_string()))),
-            ],
+            &[lit("josé"), lit(ScalarValue::Int64(Some(10))), lit("éñ"),],
             Ok(Some("éñéñéñjosé")),
             &str,
             Utf8,
@@ -1477,7 +1338,7 @@ mod tests {
         test_function!(
             Lpad,
             &[
-                lit(ScalarValue::Utf8(Some("josé".to_string()))),
+                lit("josé"),
                 lit(ScalarValue::Int64(Some(5))),
             ],
             Err(DataFusionError::Internal(
@@ -1489,7 +1350,7 @@ mod tests {
         );
         test_function!(
             Ltrim,
-            &[lit(ScalarValue::Utf8(Some(" trim".to_string())))],
+            &[lit(" trim")],
             Ok(Some("trim")),
             &str,
             Utf8,
@@ -1497,7 +1358,7 @@ mod tests {
         );
         test_function!(
             Ltrim,
-            &[lit(ScalarValue::Utf8(Some(" trim ".to_string())))],
+            &[lit(" trim ")],
             Ok(Some("trim ")),
             &str,
             Utf8,
@@ -1505,7 +1366,7 @@ mod tests {
         );
         test_function!(
             Ltrim,
-            &[lit(ScalarValue::Utf8(Some("trim ".to_string())))],
+            &[lit("trim ")],
             Ok(Some("trim ")),
             &str,
             Utf8,
@@ -1513,7 +1374,7 @@ mod tests {
         );
         test_function!(
             Ltrim,
-            &[lit(ScalarValue::Utf8(Some("trim".to_string())))],
+            &[lit("trim")],
             Ok(Some("trim")),
             &str,
             Utf8,
@@ -1521,7 +1382,7 @@ mod tests {
         );
         test_function!(
             Ltrim,
-            &[lit(ScalarValue::Utf8(Some("\n trim ".to_string())))],
+            &[lit("\n trim ")],
             Ok(Some("\n trim ")),
             &str,
             Utf8,
@@ -1538,7 +1399,7 @@ mod tests {
         #[cfg(feature = "crypto_expressions")]
         test_function!(
             MD5,
-            &[lit(ScalarValue::Utf8(Some("tom".to_string())))],
+            &[lit("tom")],
             Ok(Some("34b7da764b21d298ef307d04d8152dc5")),
             &str,
             Utf8,
@@ -1547,7 +1408,7 @@ mod tests {
         #[cfg(feature = "crypto_expressions")]
         test_function!(
             MD5,
-            &[lit(ScalarValue::Utf8(Some("".to_string())))],
+            &[lit("")],
             Ok(Some("d41d8cd98f00b204e9800998ecf8427e")),
             &str,
             Utf8,
@@ -1565,7 +1426,7 @@ mod tests {
         #[cfg(not(feature = "crypto_expressions"))]
         test_function!(
             MD5,
-            &[lit(ScalarValue::Utf8(Some("tom".to_string())))],
+            &[lit("tom")],
             Err(DataFusionError::Internal(
                 "function md5 requires compilation with feature flag: crypto_expressions.".to_string()
             )),
@@ -1575,7 +1436,7 @@ mod tests {
         );
         test_function!(
             OctetLength,
-            &[lit(ScalarValue::Utf8(Some("chars".to_string())))],
+            &[lit("chars")],
             Ok(Some(5)),
             i32,
             Int32,
@@ -1583,20 +1444,13 @@ mod tests {
         );
         test_function!(
             OctetLength,
-            &[lit(ScalarValue::Utf8(Some("josé".to_string())))],
+            &[lit("josé")],
             Ok(Some(5)),
             i32,
             Int32,
             Int32Array
         );
-        test_function!(
-            OctetLength,
-            &[lit(ScalarValue::Utf8(Some("".to_string())))],
-            Ok(Some(0)),
-            i32,
-            Int32,
-            Int32Array
-        );
+        test_function!(OctetLength, &[lit("")], Ok(Some(0)), i32, Int32, Int32Array);
         test_function!(
             OctetLength,
             &[lit(ScalarValue::Utf8(None))],
@@ -1608,11 +1462,7 @@ mod tests {
         #[cfg(feature = "regex_expressions")]
         test_function!(
             RegexpReplace,
-            &[
-                lit(ScalarValue::Utf8(Some("Thomas".to_string()))),
-                lit(ScalarValue::Utf8(Some(".[mN]a.".to_string()))),
-                lit(ScalarValue::Utf8(Some("M".to_string()))),
-            ],
+            &[lit("Thomas"), lit(".[mN]a."), lit("M"),],
             Ok(Some("ThM")),
             &str,
             Utf8,
@@ -1621,11 +1471,7 @@ mod tests {
         #[cfg(feature = "regex_expressions")]
         test_function!(
             RegexpReplace,
-            &[
-                lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))),
-                lit(ScalarValue::Utf8(Some("b..".to_string()))),
-                lit(ScalarValue::Utf8(Some("X".to_string()))),
-            ],
+            &[lit("foobarbaz"), lit("b.."), lit("X"),],
             Ok(Some("fooXbaz")),
             &str,
             Utf8,
@@ -1634,12 +1480,7 @@ mod tests {
         #[cfg(feature = "regex_expressions")]
         test_function!(
             RegexpReplace,
-            &[
-                lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))),
-                lit(ScalarValue::Utf8(Some("b..".to_string()))),
-                lit(ScalarValue::Utf8(Some("X".to_string()))),
-                lit(ScalarValue::Utf8(Some("g".to_string()))),
-            ],
+            &[lit("foobarbaz"), lit("b.."), lit("X"), lit("g"),],
             Ok(Some("fooXX")),
             &str,
             Utf8,
@@ -1648,12 +1489,7 @@ mod tests {
         #[cfg(feature = "regex_expressions")]
         test_function!(
             RegexpReplace,
-            &[
-                lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))),
-                lit(ScalarValue::Utf8(Some("b(..)".to_string()))),
-                lit(ScalarValue::Utf8(Some("X\\1Y".to_string()))),
-                lit(ScalarValue::Utf8(Some("g".to_string()))),
-            ],
+            &[lit("foobarbaz"), lit("b(..)"), lit("X\\1Y"), lit("g"),],
             Ok(Some("fooXarYXazY")),
             &str,
             Utf8,
@@ -1664,9 +1500,9 @@ mod tests {
             RegexpReplace,
             &[
                 lit(ScalarValue::Utf8(None)),
-                lit(ScalarValue::Utf8(Some("b(..)".to_string()))),
-                lit(ScalarValue::Utf8(Some("X\\1Y".to_string()))),
-                lit(ScalarValue::Utf8(Some("g".to_string()))),
+                lit("b(..)"),
+                lit("X\\1Y"),
+                lit("g"),
             ],
             Ok(None),
             &str,
@@ -1677,10 +1513,10 @@ mod tests {
         test_function!(
             RegexpReplace,
             &[
-                lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))),
+                lit("foobarbaz"),
                 lit(ScalarValue::Utf8(None)),
-                lit(ScalarValue::Utf8(Some("X\\1Y".to_string()))),
-                lit(ScalarValue::Utf8(Some("g".to_string()))),
+                lit("X\\1Y"),
+                lit("g"),
             ],
             Ok(None),
             &str,
@@ -1691,10 +1527,10 @@ mod tests {
         test_function!(
             RegexpReplace,
             &[
-                lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))),
-                lit(ScalarValue::Utf8(Some("b(..)".to_string()))),
+                lit("foobarbaz"),
+                lit("b(..)"),
                 lit(ScalarValue::Utf8(None)),
-                lit(ScalarValue::Utf8(Some("g".to_string()))),
+                lit("g"),
             ],
             Ok(None),
             &str,
@@ -1705,9 +1541,9 @@ mod tests {
         test_function!(
             RegexpReplace,
             &[
-                lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))),
-                lit(ScalarValue::Utf8(Some("b(..)".to_string()))),
-                lit(ScalarValue::Utf8(Some("X\\1Y".to_string()))),
+                lit("foobarbaz"),
+                lit("b(..)"),
+                lit("X\\1Y"),
                 lit(ScalarValue::Utf8(None)),
             ],
             Ok(None),
@@ -1718,12 +1554,7 @@ mod tests {
         #[cfg(feature = "regex_expressions")]
         test_function!(
             RegexpReplace,
-            &[
-                lit(ScalarValue::Utf8(Some("ABCabcABC".to_string()))),
-                lit(ScalarValue::Utf8(Some("(abc)".to_string()))),
-                lit(ScalarValue::Utf8(Some("X".to_string()))),
-                lit(ScalarValue::Utf8(Some("gi".to_string()))),
-            ],
+            &[lit("ABCabcABC"), lit("(abc)"), lit("X"), lit("gi"),],
             Ok(Some("XXX")),
             &str,
             Utf8,
@@ -1732,12 +1563,7 @@ mod tests {
         #[cfg(feature = "regex_expressions")]
         test_function!(
             RegexpReplace,
-            &[
-                lit(ScalarValue::Utf8(Some("ABCabcABC".to_string()))),
-                lit(ScalarValue::Utf8(Some("(abc)".to_string()))),
-                lit(ScalarValue::Utf8(Some("X".to_string()))),
-                lit(ScalarValue::Utf8(Some("i".to_string()))),
-            ],
+            &[lit("ABCabcABC"), lit("(abc)"), lit("X"), lit("i"),],
             Ok(Some("XabcABC")),
             &str,
             Utf8,
@@ -1747,9 +1573,9 @@ mod tests {
         test_function!(
             RegexpReplace,
             &[
-                lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))),
-                lit(ScalarValue::Utf8(Some("b..".to_string()))),
-                lit(ScalarValue::Utf8(Some("X".to_string()))),
+                lit("foobarbaz"),
+                lit("b.."),
+                lit("X"),
             ],
             Err(DataFusionError::Internal(
                 "function regexp_replace requires compilation with feature flag: regex_expressions.".to_string()
@@ -1760,10 +1586,7 @@ mod tests {
         );
         test_function!(
             Repeat,
-            &[
-                lit(ScalarValue::Utf8(Some("Pg".to_string()))),
-                lit(ScalarValue::Int64(Some(4))),
-            ],
+            &[lit("Pg"), lit(ScalarValue::Int64(Some(4))),],
             Ok(Some("PgPgPgPg")),
             &str,
             Utf8,
@@ -1782,10 +1605,7 @@ mod tests {
         );
         test_function!(
             Repeat,
-            &[
-                lit(ScalarValue::Utf8(Some("Pg".to_string()))),
-                lit(ScalarValue::Int64(None)),
-            ],
+            &[lit("Pg"), lit(ScalarValue::Int64(None)),],
             Ok(None),
             &str,
             Utf8,
@@ -1794,7 +1614,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Reverse,
-            &[lit(ScalarValue::Utf8(Some("abcde".to_string())))],
+            &[lit("abcde")],
             Ok(Some("edcba")),
             &str,
             Utf8,
@@ -1803,7 +1623,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Reverse,
-            &[lit(ScalarValue::Utf8(Some("loẅks".to_string())))],
+            &[lit("loẅks")],
             Ok(Some("skẅol")),
             &str,
             Utf8,
@@ -1812,7 +1632,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Reverse,
-            &[lit(ScalarValue::Utf8(Some("loẅks".to_string())))],
+            &[lit("loẅks")],
             Ok(Some("skẅol")),
             &str,
             Utf8,
@@ -1830,7 +1650,7 @@ mod tests {
         #[cfg(not(feature = "unicode_expressions"))]
         test_function!(
             Reverse,
-            &[lit(ScalarValue::Utf8(Some("abcde".to_string())))],
+            &[lit("abcde")],
             Err(DataFusionError::Internal(
                 "function reverse requires compilation with feature flag: unicode_expressions.".to_string()
             )),
@@ -1841,10 +1661,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Right,
-            &[
-                lit(ScalarValue::Utf8(Some("abcde".to_string()))),
-                lit(ScalarValue::Int8(Some(2))),
-            ],
+            &[lit("abcde"), lit(ScalarValue::Int8(Some(2))),],
             Ok(Some("de")),
             &str,
             Utf8,
@@ -1853,10 +1670,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Right,
-            &[
-                lit(ScalarValue::Utf8(Some("abcde".to_string()))),
-                lit(ScalarValue::Int64(Some(200))),
-            ],
+            &[lit("abcde"), lit(ScalarValue::Int64(Some(200))),],
             Ok(Some("abcde")),
             &str,
             Utf8,
@@ -1865,10 +1679,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Right,
-            &[
-                lit(ScalarValue::Utf8(Some("abcde".to_string()))),
-                lit(ScalarValue::Int64(Some(-2))),
-            ],
+            &[lit("abcde"), lit(ScalarValue::Int64(Some(-2))),],
             Ok(Some("cde")),
             &str,
             Utf8,
@@ -1877,10 +1688,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Right,
-            &[
-                lit(ScalarValue::Utf8(Some("abcde".to_string()))),
-                lit(ScalarValue::Int64(Some(-200))),
-            ],
+            &[lit("abcde"), lit(ScalarValue::Int64(Some(-200))),],
             Ok(Some("")),
             &str,
             Utf8,
@@ -1889,10 +1697,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Right,
-            &[
-                lit(ScalarValue::Utf8(Some("abcde".to_string()))),
-                lit(ScalarValue::Int64(Some(0))),
-            ],
+            &[lit("abcde"), lit(ScalarValue::Int64(Some(0))),],
             Ok(Some("")),
             &str,
             Utf8,
@@ -1913,10 +1718,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Right,
-            &[
-                lit(ScalarValue::Utf8(Some("abcde".to_string()))),
-                lit(ScalarValue::Int64(None)),
-            ],
+            &[lit("abcde"), lit(ScalarValue::Int64(None)),],
             Ok(None),
             &str,
             Utf8,
@@ -1925,10 +1727,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Right,
-            &[
-                lit(ScalarValue::Utf8(Some("joséésoj".to_string()))),
-                lit(ScalarValue::Int64(Some(5))),
-            ],
+            &[lit("joséésoj"), lit(ScalarValue::Int64(Some(5))),],
             Ok(Some("éésoj")),
             &str,
             Utf8,
@@ -1937,10 +1736,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Right,
-            &[
-                lit(ScalarValue::Utf8(Some("joséésoj".to_string()))),
-                lit(ScalarValue::Int64(Some(-3))),
-            ],
+            &[lit("joséésoj"), lit(ScalarValue::Int64(Some(-3))),],
             Ok(Some("éésoj")),
             &str,
             Utf8,
@@ -1950,7 +1746,7 @@ mod tests {
         test_function!(
             Right,
             &[
-                lit(ScalarValue::Utf8(Some("abcde".to_string()))),
+                lit("abcde"),
                 lit(ScalarValue::Int8(Some(2))),
             ],
             Err(DataFusionError::Internal(
@@ -1963,10 +1759,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Rpad,
-            &[
-                lit(ScalarValue::Utf8(Some("josé".to_string()))),
-                lit(ScalarValue::Int64(Some(5))),
-            ],
+            &[lit("josé"), lit(ScalarValue::Int64(Some(5))),],
             Ok(Some("josé ")),
             &str,
             Utf8,
@@ -1975,10 +1768,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Rpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(Some(5))),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(Some(5))),],
             Ok(Some("hi   ")),
             &str,
             Utf8,
@@ -1987,10 +1777,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Rpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(Some(0))),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(Some(0))),],
             Ok(Some("")),
             &str,
             Utf8,
@@ -1999,10 +1786,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Rpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(None)),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(None)),],
             Ok(None),
             &str,
             Utf8,
@@ -2023,11 +1807,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Rpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(Some(5))),
-                lit(ScalarValue::Utf8(Some("xy".to_string()))),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(Some(5))), lit("xy"),],
             Ok(Some("hixyx")),
             &str,
             Utf8,
@@ -2036,11 +1816,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Rpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(Some(21))),
-                lit(ScalarValue::Utf8(Some("abcdef".to_string()))),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(Some(21))), lit("abcdef"),],
             Ok(Some("hiabcdefabcdefabcdefa")),
             &str,
             Utf8,
@@ -2049,11 +1825,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Rpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(Some(5))),
-                lit(ScalarValue::Utf8(Some(" ".to_string()))),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(Some(5))), lit(" "),],
             Ok(Some("hi   ")),
             &str,
             Utf8,
@@ -2062,11 +1834,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Rpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(Some(5))),
-                lit(ScalarValue::Utf8(Some("".to_string()))),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(Some(5))), lit(""),],
             Ok(Some("hi")),
             &str,
             Utf8,
@@ -2078,7 +1846,7 @@ mod tests {
             &[
                 lit(ScalarValue::Utf8(None)),
                 lit(ScalarValue::Int64(Some(5))),
-                lit(ScalarValue::Utf8(Some("xy".to_string()))),
+                lit("xy"),
             ],
             Ok(None),
             &str,
@@ -2088,11 +1856,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Rpad,
-            &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
-                lit(ScalarValue::Int64(None)),
-                lit(ScalarValue::Utf8(Some("xy".to_string()))),
-            ],
+            &[lit("hi"), lit(ScalarValue::Int64(None)), lit("xy"),],
             Ok(None),
             &str,
             Utf8,
@@ -2102,7 +1866,7 @@ mod tests {
         test_function!(
             Rpad,
             &[
-                lit(ScalarValue::Utf8(Some("hi".to_string()))),
+                lit("hi"),
                 lit(ScalarValue::Int64(Some(5))),
                 lit(ScalarValue::Utf8(None)),
             ],
@@ -2114,11 +1878,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Rpad,
-            &[
-                lit(ScalarValue::Utf8(Some("josé".to_string()))),
-                lit(ScalarValue::Int64(Some(10))),
-                lit(ScalarValue::Utf8(Some("xy".to_string()))),
-            ],
+            &[lit("josé"), lit(ScalarValue::Int64(Some(10))), lit("xy"),],
             Ok(Some("joséxyxyxy")),
             &str,
             Utf8,
@@ -2127,11 +1887,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Rpad,
-            &[
-                lit(ScalarValue::Utf8(Some("josé".to_string()))),
-                lit(ScalarValue::Int64(Some(10))),
-                lit(ScalarValue::Utf8(Some("éñ".to_string()))),
-            ],
+            &[lit("josé"), lit(ScalarValue::Int64(Some(10))), lit("éñ"),],
             Ok(Some("josééñéñéñ")),
             &str,
             Utf8,
@@ -2141,7 +1897,7 @@ mod tests {
         test_function!(
             Rpad,
             &[
-                lit(ScalarValue::Utf8(Some("josé".to_string()))),
+                lit("josé"),
                 lit(ScalarValue::Int64(Some(5))),
             ],
             Err(DataFusionError::Internal(
@@ -2153,7 +1909,7 @@ mod tests {
         );
         test_function!(
             Rtrim,
-            &[lit(ScalarValue::Utf8(Some("trim ".to_string())))],
+            &[lit("trim ")],
             Ok(Some("trim")),
             &str,
             Utf8,
@@ -2161,7 +1917,7 @@ mod tests {
         );
         test_function!(
             Rtrim,
-            &[lit(ScalarValue::Utf8(Some(" trim ".to_string())))],
+            &[lit(" trim ")],
             Ok(Some(" trim")),
             &str,
             Utf8,
@@ -2169,7 +1925,7 @@ mod tests {
         );
         test_function!(
             Rtrim,
-            &[lit(ScalarValue::Utf8(Some(" trim \n".to_string())))],
+            &[lit(" trim \n")],
             Ok(Some(" trim \n")),
             &str,
             Utf8,
@@ -2177,7 +1933,7 @@ mod tests {
         );
         test_function!(
             Rtrim,
-            &[lit(ScalarValue::Utf8(Some(" trim".to_string())))],
+            &[lit(" trim")],
             Ok(Some(" trim")),
             &str,
             Utf8,
@@ -2185,7 +1941,7 @@ mod tests {
         );
         test_function!(
             Rtrim,
-            &[lit(ScalarValue::Utf8(Some("trim".to_string())))],
+            &[lit("trim")],
             Ok(Some("trim")),
             &str,
             Utf8,
@@ -2202,7 +1958,7 @@ mod tests {
         #[cfg(feature = "crypto_expressions")]
         test_function!(
             SHA224,
-            &[lit(ScalarValue::Utf8(Some("tom".to_string())))],
+            &[lit("tom")],
             Ok(Some(&[
                 11u8, 246u8, 203u8, 98u8, 100u8, 156u8, 66u8, 169u8, 174u8, 56u8, 118u8,
                 171u8, 111u8, 109u8, 146u8, 173u8, 54u8, 203u8, 84u8, 20u8, 228u8, 149u8,
@@ -2215,7 +1971,7 @@ mod tests {
         #[cfg(feature = "crypto_expressions")]
         test_function!(
             SHA224,
-            &[lit(ScalarValue::Utf8(Some("".to_string())))],
+            &[lit("")],
             Ok(Some(&[
                 209u8, 74u8, 2u8, 140u8, 42u8, 58u8, 43u8, 201u8, 71u8, 97u8, 2u8, 187u8,
                 40u8, 130u8, 52u8, 196u8, 21u8, 162u8, 176u8, 31u8, 130u8, 142u8, 166u8,
@@ -2237,7 +1993,7 @@ mod tests {
         #[cfg(not(feature = "crypto_expressions"))]
         test_function!(
             SHA224,
-            &[lit(ScalarValue::Utf8(Some("tom".to_string())))],
+            &[lit("tom")],
             Err(DataFusionError::Internal(
                 "function sha224 requires compilation with feature flag: crypto_expressions.".to_string()
             )),
@@ -2248,7 +2004,7 @@ mod tests {
         #[cfg(feature = "crypto_expressions")]
         test_function!(
             SHA256,
-            &[lit(ScalarValue::Utf8(Some("tom".to_string())))],
+            &[lit("tom")],
             Ok(Some(&[
                 225u8, 96u8, 143u8, 117u8, 197u8, 215u8, 129u8, 63u8, 61u8, 64u8, 49u8,
                 203u8, 48u8, 191u8, 183u8, 134u8, 80u8, 125u8, 152u8, 19u8, 117u8, 56u8,
@@ -2261,7 +2017,7 @@ mod tests {
         #[cfg(feature = "crypto_expressions")]
         test_function!(
             SHA256,
-            &[lit(ScalarValue::Utf8(Some("".to_string())))],
+            &[lit("")],
             Ok(Some(&[
                 227u8, 176u8, 196u8, 66u8, 152u8, 252u8, 28u8, 20u8, 154u8, 251u8, 244u8,
                 200u8, 153u8, 111u8, 185u8, 36u8, 39u8, 174u8, 65u8, 228u8, 100u8, 155u8,
@@ -2283,7 +2039,7 @@ mod tests {
         #[cfg(not(feature = "crypto_expressions"))]
         test_function!(
             SHA256,
-            &[lit(ScalarValue::Utf8(Some("tom".to_string())))],
+            &[lit("tom")],
             Err(DataFusionError::Internal(
                 "function sha256 requires compilation with feature flag: crypto_expressions.".to_string()
             )),
@@ -2294,7 +2050,7 @@ mod tests {
         #[cfg(feature = "crypto_expressions")]
         test_function!(
             SHA384,
-            &[lit(ScalarValue::Utf8(Some("tom".to_string())))],
+            &[lit("tom")],
             Ok(Some(&[
                 9u8, 111u8, 91u8, 104u8, 170u8, 119u8, 132u8, 142u8, 79u8, 223u8, 92u8,
                 28u8, 11u8, 53u8, 13u8, 226u8, 219u8, 250u8, 214u8, 15u8, 253u8, 124u8,
@@ -2309,7 +2065,7 @@ mod tests {
         #[cfg(feature = "crypto_expressions")]
         test_function!(
             SHA384,
-            &[lit(ScalarValue::Utf8(Some("".to_string())))],
+            &[lit("")],
             Ok(Some(&[
                 56u8, 176u8, 96u8, 167u8, 81u8, 172u8, 150u8, 56u8, 76u8, 217u8, 50u8,
                 126u8, 177u8, 177u8, 227u8, 106u8, 33u8, 253u8, 183u8, 17u8, 20u8, 190u8,
@@ -2333,7 +2089,7 @@ mod tests {
         #[cfg(not(feature = "crypto_expressions"))]
         test_function!(
             SHA384,
-            &[lit(ScalarValue::Utf8(Some("tom".to_string())))],
+            &[lit("tom")],
             Err(DataFusionError::Internal(
                 "function sha384 requires compilation with feature flag: crypto_expressions.".to_string()
             )),
@@ -2344,7 +2100,7 @@ mod tests {
         #[cfg(feature = "crypto_expressions")]
         test_function!(
             SHA512,
-            &[lit(ScalarValue::Utf8(Some("tom".to_string())))],
+            &[lit("tom")],
             Ok(Some(&[
                 110u8, 27u8, 155u8, 63u8, 232u8, 64u8, 104u8, 14u8, 55u8, 5u8, 31u8,
                 122u8, 213u8, 233u8, 89u8, 214u8, 243u8, 154u8, 208u8, 248u8, 136u8,
@@ -2360,7 +2116,7 @@ mod tests {
         #[cfg(feature = "crypto_expressions")]
         test_function!(
             SHA512,
-            &[lit(ScalarValue::Utf8(Some("".to_string())))],
+            &[lit("")],
             Ok(Some(&[
                 207u8, 131u8, 225u8, 53u8, 126u8, 239u8, 184u8, 189u8, 241u8, 84u8, 40u8,
                 80u8, 214u8, 109u8, 128u8, 7u8, 214u8, 32u8, 228u8, 5u8, 11u8, 87u8,
@@ -2385,7 +2141,7 @@ mod tests {
         #[cfg(not(feature = "crypto_expressions"))]
         test_function!(
             SHA512,
-            &[lit(ScalarValue::Utf8(Some("tom".to_string())))],
+            &[lit("tom")],
             Err(DataFusionError::Internal(
                 "function sha512 requires compilation with feature flag: crypto_expressions.".to_string()
             )),
@@ -2396,8 +2152,8 @@ mod tests {
         test_function!(
             SplitPart,
             &[
-                lit(ScalarValue::Utf8(Some("abc~@~def~@~ghi".to_string()))),
-                lit(ScalarValue::Utf8(Some("~@~".to_string()))),
+                lit("abc~@~def~@~ghi"),
+                lit("~@~"),
                 lit(ScalarValue::Int64(Some(2))),
             ],
             Ok(Some("def")),
@@ -2408,8 +2164,8 @@ mod tests {
         test_function!(
             SplitPart,
             &[
-                lit(ScalarValue::Utf8(Some("abc~@~def~@~ghi".to_string()))),
-                lit(ScalarValue::Utf8(Some("~@~".to_string()))),
+                lit("abc~@~def~@~ghi"),
+                lit("~@~"),
                 lit(ScalarValue::Int64(Some(20))),
             ],
             Ok(Some("")),
@@ -2420,8 +2176,8 @@ mod tests {
         test_function!(
             SplitPart,
             &[
-                lit(ScalarValue::Utf8(Some("abc~@~def~@~ghi".to_string()))),
-                lit(ScalarValue::Utf8(Some("~@~".to_string()))),
+                lit("abc~@~def~@~ghi"),
+                lit("~@~"),
                 lit(ScalarValue::Int64(Some(-1))),
             ],
             Err(DataFusionError::Execution(
@@ -2433,10 +2189,7 @@ mod tests {
         );
         test_function!(
             StartsWith,
-            &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
-                lit(ScalarValue::Utf8(Some("alph".to_string()))),
-            ],
+            &[lit("alphabet"), lit("alph"),],
             Ok(Some(true)),
             bool,
             Boolean,
@@ -2444,10 +2197,7 @@ mod tests {
         );
         test_function!(
             StartsWith,
-            &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
-                lit(ScalarValue::Utf8(Some("blph".to_string()))),
-            ],
+            &[lit("alphabet"), lit("blph"),],
             Ok(Some(false)),
             bool,
             Boolean,
@@ -2455,10 +2205,7 @@ mod tests {
         );
         test_function!(
             StartsWith,
-            &[
-                lit(ScalarValue::Utf8(None)),
-                lit(ScalarValue::Utf8(Some("alph".to_string()))),
-            ],
+            &[lit(ScalarValue::Utf8(None)), lit("alph"),],
             Ok(None),
             bool,
             Boolean,
@@ -2466,10 +2213,7 @@ mod tests {
         );
         test_function!(
             StartsWith,
-            &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
-                lit(ScalarValue::Utf8(None)),
-            ],
+            &[lit("alphabet"), lit(ScalarValue::Utf8(None)),],
             Ok(None),
             bool,
             Boolean,
@@ -2478,10 +2222,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Strpos,
-            &[
-                lit(ScalarValue::Utf8(Some("abc".to_string()))),
-                lit(ScalarValue::Utf8(Some("c".to_string()))),
-            ],
+            &[lit("abc"), lit("c"),],
             Ok(Some(3)),
             i32,
             Int32,
@@ -2490,10 +2231,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Strpos,
-            &[
-                lit(ScalarValue::Utf8(Some("josé".to_string()))),
-                lit(ScalarValue::Utf8(Some("é".to_string()))),
-            ],
+            &[lit("josé"), lit("é"),],
             Ok(Some(4)),
             i32,
             Int32,
@@ -2502,10 +2240,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Strpos,
-            &[
-                lit(ScalarValue::Utf8(Some("joséésoj".to_string()))),
-                lit(ScalarValue::Utf8(Some("so".to_string()))),
-            ],
+            &[lit("joséésoj"), lit("so"),],
             Ok(Some(6)),
             i32,
             Int32,
@@ -2514,10 +2249,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Strpos,
-            &[
-                lit(ScalarValue::Utf8(Some("joséésoj".to_string()))),
-                lit(ScalarValue::Utf8(Some("abc".to_string()))),
-            ],
+            &[lit("joséésoj"), lit("abc"),],
             Ok(Some(0)),
             i32,
             Int32,
@@ -2526,10 +2258,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Strpos,
-            &[
-                lit(ScalarValue::Utf8(None)),
-                lit(ScalarValue::Utf8(Some("abc".to_string()))),
-            ],
+            &[lit(ScalarValue::Utf8(None)), lit("abc"),],
             Ok(None),
             i32,
             Int32,
@@ -2538,10 +2267,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Strpos,
-            &[
-                lit(ScalarValue::Utf8(Some("joséésoj".to_string()))),
-                lit(ScalarValue::Utf8(None)),
-            ],
+            &[lit("joséésoj"), lit(ScalarValue::Utf8(None)),],
             Ok(None),
             i32,
             Int32,
@@ -2551,7 +2277,7 @@ mod tests {
         test_function!(
             Strpos,
             &[
-                lit(ScalarValue::Utf8(Some("joséésoj".to_string()))),
+                lit("joséésoj"),
                 lit(ScalarValue::Utf8(None)),
             ],
             Err(DataFusionError::Internal(
@@ -2564,10 +2290,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Substr,
-            &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
-                lit(ScalarValue::Int64(Some(0))),
-            ],
+            &[lit("alphabet"), lit(ScalarValue::Int64(Some(0))),],
             Ok(Some("alphabet")),
             &str,
             Utf8,
@@ -2576,10 +2299,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Substr,
-            &[
-                lit(ScalarValue::Utf8(Some("joséésoj".to_string()))),
-                lit(ScalarValue::Int64(Some(5))),
-            ],
+            &[lit("joséésoj"), lit(ScalarValue::Int64(Some(5))),],
             Ok(Some("ésoj")),
             &str,
             Utf8,
@@ -2588,10 +2308,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Substr,
-            &[
-                lit(ScalarValue::Utf8(Some("joséésoj".to_string()))),
-                lit(ScalarValue::Int64(Some(-5))),
-            ],
+            &[lit("joséésoj"), lit(ScalarValue::Int64(Some(-5))),],
             Ok(Some("joséésoj")),
             &str,
             Utf8,
@@ -2600,10 +2317,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Substr,
-            &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
-                lit(ScalarValue::Int64(Some(1))),
-            ],
+            &[lit("alphabet"), lit(ScalarValue::Int64(Some(1))),],
             Ok(Some("alphabet")),
             &str,
             Utf8,
@@ -2612,10 +2326,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Substr,
-            &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
-                lit(ScalarValue::Int64(Some(2))),
-            ],
+            &[lit("alphabet"), lit(ScalarValue::Int64(Some(2))),],
             Ok(Some("lphabet")),
             &str,
             Utf8,
@@ -2624,10 +2335,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Substr,
-            &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
-                lit(ScalarValue::Int64(Some(3))),
-            ],
+            &[lit("alphabet"), lit(ScalarValue::Int64(Some(3))),],
             Ok(Some("phabet")),
             &str,
             Utf8,
@@ -2636,10 +2344,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Substr,
-            &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
-                lit(ScalarValue::Int64(Some(-3))),
-            ],
+            &[lit("alphabet"), lit(ScalarValue::Int64(Some(-3))),],
             Ok(Some("alphabet")),
             &str,
             Utf8,
@@ -2648,10 +2353,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Substr,
-            &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
-                lit(ScalarValue::Int64(Some(30))),
-            ],
+            &[lit("alphabet"), lit(ScalarValue::Int64(Some(30))),],
             Ok(Some("")),
             &str,
             Utf8,
@@ -2660,10 +2362,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Substr,
-            &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
-                lit(ScalarValue::Int64(None)),
-            ],
+            &[lit("alphabet"), lit(ScalarValue::Int64(None)),],
             Ok(None),
             &str,
             Utf8,
@@ -2673,7 +2372,7 @@ mod tests {
         test_function!(
             Substr,
             &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
+                lit("alphabet"),
                 lit(ScalarValue::Int64(Some(3))),
                 lit(ScalarValue::Int64(Some(2))),
             ],
@@ -2686,7 +2385,7 @@ mod tests {
         test_function!(
             Substr,
             &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
+                lit("alphabet"),
                 lit(ScalarValue::Int64(Some(3))),
                 lit(ScalarValue::Int64(Some(20))),
             ],
@@ -2699,7 +2398,7 @@ mod tests {
         test_function!(
             Substr,
             &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
+                lit("alphabet"),
                 lit(ScalarValue::Int64(Some(0))),
                 lit(ScalarValue::Int64(Some(5))),
             ],
@@ -2713,7 +2412,7 @@ mod tests {
         test_function!(
             Substr,
             &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
+                lit("alphabet"),
                 lit(ScalarValue::Int64(Some(-5))),
                 lit(ScalarValue::Int64(Some(10))),
             ],
@@ -2727,7 +2426,7 @@ mod tests {
         test_function!(
             Substr,
             &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
+                lit("alphabet"),
                 lit(ScalarValue::Int64(Some(-5))),
                 lit(ScalarValue::Int64(Some(4))),
             ],
@@ -2741,7 +2440,7 @@ mod tests {
         test_function!(
             Substr,
             &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
+                lit("alphabet"),
                 lit(ScalarValue::Int64(Some(-5))),
                 lit(ScalarValue::Int64(Some(5))),
             ],
@@ -2754,7 +2453,7 @@ mod tests {
         test_function!(
             Substr,
             &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
+                lit("alphabet"),
                 lit(ScalarValue::Int64(None)),
                 lit(ScalarValue::Int64(Some(20))),
             ],
@@ -2767,7 +2466,7 @@ mod tests {
         test_function!(
             Substr,
             &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
+                lit("alphabet"),
                 lit(ScalarValue::Int64(Some(3))),
                 lit(ScalarValue::Int64(None)),
             ],
@@ -2780,7 +2479,7 @@ mod tests {
         test_function!(
             Substr,
             &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
+                lit("alphabet"),
                 lit(ScalarValue::Int64(Some(1))),
                 lit(ScalarValue::Int64(Some(-1))),
             ],
@@ -2795,7 +2494,7 @@ mod tests {
         test_function!(
             Substr,
             &[
-                lit(ScalarValue::Utf8(Some("joséésoj".to_string()))),
+                lit("joséésoj"),
                 lit(ScalarValue::Int64(Some(5))),
                 lit(ScalarValue::Int64(Some(2))),
             ],
@@ -2808,7 +2507,7 @@ mod tests {
         test_function!(
             Substr,
             &[
-                lit(ScalarValue::Utf8(Some("alphabet".to_string()))),
+                lit("alphabet"),
                 lit(ScalarValue::Int64(Some(0))),
             ],
             Err(DataFusionError::Internal(
@@ -2821,11 +2520,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Translate,
-            &[
-                lit(ScalarValue::Utf8(Some("12345".to_string()))),
-                lit(ScalarValue::Utf8(Some("143".to_string()))),
-                lit(ScalarValue::Utf8(Some("ax".to_string()))),
-            ],
+            &[lit("12345"), lit("143"), lit("ax"),],
             Ok(Some("a2x5")),
             &str,
             Utf8,
@@ -2834,11 +2529,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Translate,
-            &[
-                lit(ScalarValue::Utf8(None)),
-                lit(ScalarValue::Utf8(Some("143".to_string()))),
-                lit(ScalarValue::Utf8(Some("ax".to_string()))),
-            ],
+            &[lit(ScalarValue::Utf8(None)), lit("143"), lit("ax"),],
             Ok(None),
             &str,
             Utf8,
@@ -2847,11 +2538,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Translate,
-            &[
-                lit(ScalarValue::Utf8(Some("12345".to_string()))),
-                lit(ScalarValue::Utf8(None)),
-                lit(ScalarValue::Utf8(Some("ax".to_string()))),
-            ],
+            &[lit("12345"), lit(ScalarValue::Utf8(None)), lit("ax"),],
             Ok(None),
             &str,
             Utf8,
@@ -2860,11 +2547,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Translate,
-            &[
-                lit(ScalarValue::Utf8(Some("12345".to_string()))),
-                lit(ScalarValue::Utf8(Some("143".to_string()))),
-                lit(ScalarValue::Utf8(None)),
-            ],
+            &[lit("12345"), lit("143"), lit(ScalarValue::Utf8(None)),],
             Ok(None),
             &str,
             Utf8,
@@ -2873,11 +2556,7 @@ mod tests {
         #[cfg(feature = "unicode_expressions")]
         test_function!(
             Translate,
-            &[
-                lit(ScalarValue::Utf8(Some("é2íñ5".to_string()))),
-                lit(ScalarValue::Utf8(Some("éñí".to_string()))),
-                lit(ScalarValue::Utf8(Some("óü".to_string()))),
-            ],
+            &[lit("é2íñ5"), lit("éñí"), lit("óü"),],
             Ok(Some("ó2ü5")),
             &str,
             Utf8,
@@ -2887,9 +2566,9 @@ mod tests {
         test_function!(
             Translate,
             &[
-                lit(ScalarValue::Utf8(Some("12345".to_string()))),
-                lit(ScalarValue::Utf8(Some("143".to_string()))),
-                lit(ScalarValue::Utf8(Some("ax".to_string()))),
+                lit("12345"),
+                lit("143"),
+                lit("ax"),
             ],
             Err(DataFusionError::Internal(
                 "function translate requires compilation with feature flag: unicode_expressions.".to_string()
@@ -2900,7 +2579,7 @@ mod tests {
         );
         test_function!(
             Trim,
-            &[lit(ScalarValue::Utf8(Some(" trim ".to_string())))],
+            &[lit(" trim ")],
             Ok(Some("trim")),
             &str,
             Utf8,
@@ -2908,7 +2587,7 @@ mod tests {
         );
         test_function!(
             Trim,
-            &[lit(ScalarValue::Utf8(Some("trim ".to_string())))],
+            &[lit("trim ")],
             Ok(Some("trim")),
             &str,
             Utf8,
@@ -2916,7 +2595,7 @@ mod tests {
         );
         test_function!(
             Trim,
-            &[lit(ScalarValue::Utf8(Some(" trim".to_string())))],
+            &[lit(" trim")],
             Ok(Some("trim")),
             &str,
             Utf8,
@@ -2932,7 +2611,7 @@ mod tests {
         );
         test_function!(
             Upper,
-            &[lit(ScalarValue::Utf8(Some("upper".to_string())))],
+            &[lit("upper")],
             Ok(Some("UPPER")),
             &str,
             Utf8,
@@ -2940,7 +2619,7 @@ mod tests {
         );
         test_function!(
             Upper,
-            &[lit(ScalarValue::Utf8(Some("UPPER".to_string())))],
+            &[lit("UPPER")],
             Ok(Some("UPPER")),
             &str,
             Utf8,
@@ -3091,7 +2770,7 @@ mod tests {
         let execution_props = ExecutionProps::new();
 
         let col_value: ArrayRef = Arc::new(StringArray::from_slice(&["aaa-555"]));
-        let pattern = lit(ScalarValue::Utf8(Some(r".*-(\d*)".to_string())));
+        let pattern = lit(r".*-(\d*)");
         let columns: Vec<ArrayRef> = vec![col_value];
         let expr = create_physical_expr(
             &BuiltinScalarFunction::RegexpMatch,
@@ -3129,8 +2808,8 @@ mod tests {
         let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);
         let execution_props = ExecutionProps::new();
 
-        let col_value = lit(ScalarValue::Utf8(Some("aaa-555".to_string())));
-        let pattern = lit(ScalarValue::Utf8(Some(r".*-(\d*)".to_string())));
+        let col_value = lit("aaa-555");
+        let pattern = lit(r".*-(\d*)");
         let columns: Vec<ArrayRef> = vec![Arc::new(Int32Array::from_slice(&[1]))];
         let expr = create_physical_expr(
             &BuiltinScalarFunction::RegexpMatch,