You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/04/26 19:41:30 UTC

[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #2324: Implementing math power function for SQL

alamb commented on code in PR #2324:
URL: https://github.com/apache/arrow-datafusion/pull/2324#discussion_r859080080


##########
datafusion/expr/src/function.rs:
##########
@@ -505,6 +510,13 @@ pub fn signature(fun: &BuiltinScalarFunction) -> Signature {
             fun.volatility(),
         ),
         BuiltinScalarFunction::Random => Signature::exact(vec![], fun.volatility()),
+        BuiltinScalarFunction::Power => Signature::one_of(
+            vec![
+                TypeSignature::Exact(vec![DataType::Int64, DataType::Int64]),
+                TypeSignature::Exact(vec![DataType::Float64, DataType::Float64]),
+            ],

Review Comment:
   It appears that postgres supports `dp` (aka double precision, aka `Float64`) and `Numeric`
   
   https://www.postgresql.org/docs/12/functions-math.html
   
   I think for the initial implementation it would be fine to support `Int64` as well, but longer term we should add coercion rules to convert arguments to `Float64` before they are actually passed to `power`



##########
datafusion/physical-expr/src/math_expressions.rs:
##########
@@ -86,6 +88,33 @@ macro_rules! math_unary_function {
     };
 }
 
+macro_rules! downcast_arg {
+    ($ARG:expr, $NAME:expr, $ARRAY_TYPE:ident) => {{
+        $ARG.as_any().downcast_ref::<$ARRAY_TYPE>().ok_or_else(|| {
+            DataFusionError::Internal(format!(
+                "could not cast {} to {}",
+                $NAME,
+                type_name::<$ARRAY_TYPE>()
+            ))
+        })?
+    }};
+}
+
+macro_rules! make_function_inputs2 {
+    ($ARG1: expr, $ARG2: expr, $NAME1:expr, $NAME2: expr, $ARRAY_TYPE:ident, $FUNC: block) => {{
+        let arg1 = downcast_arg!($ARG1, $NAME1, $ARRAY_TYPE);
+        let arg2 = downcast_arg!($ARG2, $NAME2, $ARRAY_TYPE);
+
+        arg1.iter()
+            .zip(arg2.iter())
+            .map(|(a1, a2)| match (a1, a2) {
+                (Some(a1), Some(a2)) => Some($FUNC(a1, a2.try_into().unwrap())),

Review Comment:
   I wonder if `.unwrap()` is needed here -- it will cause a panic if the argument can't be converted to whatever $FUNC wants
   
   Does it work if you do 
   
   ```suggestion
                   (Some(a1), Some(a2)) => Some($FUNC(a1, a2.try_into()?)),
   ```
   Instead?



##########
datafusion/core/tests/sql/functions.rs:
##########
@@ -445,3 +445,169 @@ async fn case_builtin_math_expression() {
         assert_batches_sorted_eq!(expected, &results);
     }
 }
+
+#[tokio::test]
+async fn case_sensitive_identifiers_aggregates() {
+    let ctx = SessionContext::new();
+    ctx.register_table("t", table_with_sequence(1, 1).unwrap())
+        .unwrap();
+
+    let expected = vec![
+        "+----------+",
+        "| MAX(t.i) |",
+        "+----------+",
+        "| 1        |",
+        "+----------+",
+    ];
+
+    let results = plan_and_collect(&ctx, "SELECT max(i) FROM t")
+        .await
+        .unwrap();
+
+    assert_batches_sorted_eq!(expected, &results);
+
+    let results = plan_and_collect(&ctx, "SELECT MAX(i) FROM t")
+        .await
+        .unwrap();
+    assert_batches_sorted_eq!(expected, &results);
+
+    // Using double quotes allows specifying the function name with capitalization
+    let err = plan_and_collect(&ctx, "SELECT \"MAX\"(i) FROM t")
+        .await
+        .unwrap_err();
+    assert_eq!(
+        err.to_string(),
+        "Error during planning: Invalid function 'MAX'"
+    );
+
+    let results = plan_and_collect(&ctx, "SELECT \"max\"(i) FROM t")
+        .await
+        .unwrap();
+    assert_batches_sorted_eq!(expected, &results);
+}
+
+#[tokio::test]
+async fn test_power() -> Result<()> {
+    let schema = Arc::new(Schema::new(vec![
+        Field::new("i32", DataType::Int16, true),
+        Field::new("i64", DataType::Int64, true),
+        Field::new("f32", DataType::Float32, true),
+        Field::new("f64", DataType::Float64, true),
+    ]));
+
+    let data = RecordBatch::try_new(
+        schema.clone(),
+        vec![
+            Arc::new(Int16Array::from(vec![
+                Some(2),
+                Some(5),
+                Some(0),
+                Some(-14),
+                None,
+            ])),
+            Arc::new(Int64Array::from(vec![
+                Some(2),
+                Some(5),
+                Some(0),
+                Some(-14),
+                None,
+            ])),
+            Arc::new(Float32Array::from(vec![
+                Some(1.0),
+                Some(2.5),
+                Some(0.0),
+                Some(-14.5),
+                None,
+            ])),
+            Arc::new(Float64Array::from(vec![
+                Some(1.0),
+                Some(2.5),
+                Some(0.0),
+                Some(-14.5),
+                None,
+            ])),
+        ],
+    )?;
+
+    let table = MemTable::try_new(schema, vec![vec![data]])?;
+
+    let ctx = SessionContext::new();
+    ctx.register_table("test", Arc::new(table))?;
+    let sql = r"SELECT power(i32, 3) as power_i32,
+                 power(i64, 3) as power_i64,

Review Comment:
   A test that takes two columns in addition to a column and a scalar would also be good -- like `power(i64, i64)` perhaps



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org