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/23 14:36:44 UTC

[GitHub] [arrow-datafusion] comphead opened a new pull request, #2324: Implementing math power function for SQL

comphead opened a new pull request, #2324:
URL: https://github.com/apache/arrow-datafusion/pull/2324

   # Which issue does this PR close?
   
   Closes #1493
   
    # Rationale for this change
   Implementing power function for SQL
   
   # What changes are included in this PR?
   Implementing power function for SQL
   
   
   # Are there any user-facing changes?
   User can use math POWER function in SQL statements
   


-- 
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


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

Posted by GitBox <gi...@apache.org>.
xudong963 commented on code in PR #2324:
URL: https://github.com/apache/arrow-datafusion/pull/2324#discussion_r856931133


##########
datafusion/core/src/physical_plan/functions.rs:
##########
@@ -52,6 +52,200 @@ use datafusion_physical_expr::math_expressions;
 use datafusion_physical_expr::string_expressions;
 use std::sync::Arc;
 
+macro_rules! make_utf8_to_return_type {

Review Comment:
   Seems to be some commit-related problems



-- 
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


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

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2324:
URL: https://github.com/apache/arrow-datafusion/pull/2324#discussion_r860100540


##########
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() {

Review Comment:
   I still think this test is accidentally included in this PR. It currently exists in aggregates.rs: 
   
   https://github.com/apache/arrow-datafusion/blob/cc496f87caf8d13e4c17025e0d3b86643a21a243/datafusion/core/tests/sql/aggregates.rs#L1112



-- 
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


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

Posted by GitBox <gi...@apache.org>.
comphead commented on code in PR #2324:
URL: https://github.com/apache/arrow-datafusion/pull/2324#discussion_r860180923


##########
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() {

Review Comment:
   Fixed, probably a merge issue. Tried to be in sync with fast changing files



-- 
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


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

Posted by GitBox <gi...@apache.org>.
alamb commented on PR #2324:
URL: https://github.com/apache/arrow-datafusion/pull/2324#issuecomment-1112167233

   Thanks again all!


-- 
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


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

Posted by GitBox <gi...@apache.org>.
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


[GitHub] [arrow-datafusion] alamb merged pull request #2324: Implementing math power function for SQL

Posted by GitBox <gi...@apache.org>.
alamb merged PR #2324:
URL: https://github.com/apache/arrow-datafusion/pull/2324


-- 
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


[GitHub] [arrow-datafusion] comphead commented on pull request #2324: Implementing math power function for SQL

Posted by GitBox <gi...@apache.org>.
comphead commented on PR #2324:
URL: https://github.com/apache/arrow-datafusion/pull/2324#issuecomment-1109958674

   Hi @Ted-Jiang @xudong963, can you please review again, I'm looking forward to commence to other datafusion tasks


-- 
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


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

Posted by GitBox <gi...@apache.org>.
comphead commented on code in PR #2324:
URL: https://github.com/apache/arrow-datafusion/pull/2324#discussion_r859555520


##########
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 came up with `(Some(a1), Some(a2)) => Some($FUNC(a1, a2.try_into().ok()?)),`
   



-- 
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


[GitHub] [arrow-datafusion] comphead commented on pull request #2324: Implementing math power function for SQL

Posted by GitBox <gi...@apache.org>.
comphead commented on PR #2324:
URL: https://github.com/apache/arrow-datafusion/pull/2324#issuecomment-1107514476

   Implement math POWER function with signatures like [https://www.postgresql.org/docs/14/functions-math.html](url)
   
   
   Example usage
   >
           "SELECT power(i32, 3) as power_i32,
                    power(i64, 3) as power_i64,
                    power(f32, 3) as power_f32,
                    power(f64, 3) as power_f64,
                    power(2, 3) as power_int_scalar,
                    power(2.5, 3) as power_float_scalar
             FROM test";
   
   Output
   >
           "+-----------+-----------+-----------+-----------+------------------+--------------------+",
           "| power_i32 | power_i64 | power_f32 | power_f64 | power_int_scalar | power_float_scalar |",
           "+-----------+-----------+-----------+-----------+------------------+--------------------+",
           "| 8         | 8         | 1         | 1         | 8                | 15.625             |",
           "| 125       | 125       | 15.625    | 15.625    | 8                | 15.625             |",
           "| 0         | 0         | 0         | 0         | 8                | 15.625             |",
           "| -2744     | -2744     | -3048.625 | -3048.625 | 8                | 15.625             |",
           "|           |           |           |           | 8                | 15.625             |",
           "+-----------+-----------+-----------+-----------+------------------+--------------------+",


-- 
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


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

Posted by GitBox <gi...@apache.org>.
comphead commented on code in PR #2324:
URL: https://github.com/apache/arrow-datafusion/pull/2324#discussion_r859555520


##########
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 have removed the conversion, but expended the macros signature, so the macros user is responsible to provide correct type information. it will be checked on compile time.



##########
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:
   Done



-- 
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


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

Posted by GitBox <gi...@apache.org>.
comphead commented on code in PR #2324:
URL: https://github.com/apache/arrow-datafusion/pull/2324#discussion_r857028728


##########
datafusion/core/src/physical_plan/functions.rs:
##########
@@ -52,6 +52,200 @@ use datafusion_physical_expr::math_expressions;
 use datafusion_physical_expr::string_expressions;
 use std::sync::Arc;
 
+macro_rules! make_utf8_to_return_type {

Review Comment:
   Fixed that



-- 
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


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

Posted by GitBox <gi...@apache.org>.
Ted-Jiang commented on code in PR #2324:
URL: https://github.com/apache/arrow-datafusion/pull/2324#discussion_r857562140


##########
datafusion/expr/src/function.rs:
##########
@@ -217,6 +217,11 @@ pub fn return_type(
             }
         }),
 
+        BuiltinScalarFunction::Power => match &input_expr_types[0] {
+            DataType::Int32 | DataType::Int64 => Ok(DataType::Int64),

Review Comment:
   what about Int8,Int16, UInt....



-- 
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


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

Posted by GitBox <gi...@apache.org>.
comphead commented on code in PR #2324:
URL: https://github.com/apache/arrow-datafusion/pull/2324#discussion_r857744248


##########
datafusion/expr/src/function.rs:
##########
@@ -217,6 +217,11 @@ pub fn return_type(
             }
         }),
 
+        BuiltinScalarFunction::Power => match &input_expr_types[0] {
+            DataType::Int32 | DataType::Int64 => Ok(DataType::Int64),

Review Comment:
   Now its changed to be like in [https://www.postgresql.org/docs/14/functions-math.html](https://github.com/apache/arrow-datafusion/pull/url)
   
   So the signature is either i64 or f64. 
   If user column is another col type with less size like i32, f32, i16, etc it will work, however the return type is i64 or f64.
   Let me know if this is ok



-- 
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