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 2021/01/08 15:53:31 UTC

[GitHub] [arrow] ovr opened a new pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

ovr opened a new pull request #9139:
URL: https://github.com/apache/arrow/pull/9139


   …(md5, sha)
   
   Hello!
   
   I am still working on it, It's DRAFT because I am interested to indicate that I am working on it.
   
   Plan to support:
   
   - [ ] MD5
   - [ ] SHA224
   - [ ] SHA256
   - [ ] SHA384
   - [ ] SHA512
   
   Thanks


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

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



[GitHub] [arrow] ovr commented on a change in pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
ovr commented on a change in pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#discussion_r554425163



##########
File path: rust/datafusion/tests/sql.rs
##########
@@ -1859,19 +1859,33 @@ async fn crypto_expressions() -> Result<()> {
     let mut ctx = ExecutionContext::new();
     let sql = "SELECT
         md5('tom') AS md5_tom,
+        md5('') AS md5_empty_str,
+        md5(null) AS md5_null,
         sha224('tom') AS sha224_tom,
+        sha224('') AS sha224_empty_str,
+        sha224(null) AS sha224_null,
         sha256('tom') AS sha256_tom,
+        sha256('') AS sha256_empty_str,
         sha384('tom') AS sha348_tom,
-        sha512('tom') AS sha512_tom
+        sha384('') AS sha384_empty_str,
+        sha512('tom') AS sha512_tom,
+        sha512('') AS sha512_empty_str
     ";
     let actual = execute(&mut ctx, sql).await;
 
     let expected = vec![vec![
         "34b7da764b21d298ef307d04d8152dc5",
-        "BF6CB62649C42A9AE3876AB6F6D92AD36CB5414E495F8873292BE4D",
-        "E1608F75C5D7813F3D4031CB30BFB786507D98137538FF8E128A6FF74E84E643",
-        "96F5B68AA77848E4FDF5C1CB35DE2DBFAD6FFD7C25D9EA7C6C19B8A4D55A9187EB117C557883F58C16DFAC3E343",
-        "6E1B9B3FE84068E3751F7AD5E959D6F39AD0F8885D855166F55C659469D3C8B78118C44A2A49C72DDB481CD6D8731034E11CC0307BA843A9B3495CB8D3E"
+        "d41d8cd98f00b204e9800998ecf8427e",

Review comment:
       Tested with real PostgreSQL.
   
   ```sql
   select
           md5('tom') AS md5_tom,
           md5('') AS md5_empty_str,
           md5(null) AS md5_null,
           encode(sha224('tom'), 'hex') AS sha224_tom,
           encode(sha224(''), 'hex') AS sha224_empty_str,
           sha224(null) AS sha224_null;
   ```
   
   ```json
   [
     {
       "md5_tom": "34b7da764b21d298ef307d04d8152dc5",
       "md5_empty_str": "d41d8cd98f00b204e9800998ecf8427e",
       "md5_null": null,
       "sha224_tom": "0bf6cb62649c42a9ae3876ab6f6d92ad36cb5414e495f8873292be4d",
       "sha224_empty_str": "d14a028c2a3a2bc9476102bb288234c415a2b01f828ea62ac5b3e42f",
       "sha224_null": null
     }
   ]
   ```




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

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



[GitHub] [arrow] jorgecarleitao commented on a change in pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
jorgecarleitao commented on a change in pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#discussion_r554389768



##########
File path: rust/datafusion/src/physical_plan/crypto_expressions.rs
##########
@@ -0,0 +1,114 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Crypto expressions
+
+use md5::Md5;
+use sha2::{
+    digest::Output as SHA2DigestOutput, Digest as SHA2Digest, Sha224, Sha256, Sha384,
+    Sha512,
+};
+
+use crate::error::{DataFusionError, Result};
+use arrow::array::{
+    ArrayRef, BinaryBuilder, GenericBinaryArray, GenericStringArray,
+    StringOffsetSizeTrait,
+};
+
+fn md5_process(input: &str) -> String {
+    let mut digest = Md5::default();
+    digest.update(&input);
+
+    let mut result = String::new();
+
+    for byte in &digest.finalize() {
+        result.push_str(&format!("{:02x}", byte));
+    }
+
+    result
+}
+
+// It's not possible to return &[u8], because trait in trait without short lifetime
+fn sha_process<D: SHA2Digest + Default>(input: &str) -> SHA2DigestOutput<D> {
+    let mut digest = D::default();
+    digest.update(&input);
+
+    digest.finalize()
+}
+
+macro_rules! crypto_unary_string_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Utf8 string
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericStringArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            // first map is the iterator, second is for the `Option<_>`
+            Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())
+        }
+    };
+}
+
+macro_rules! crypto_unary_binary_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Binary
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericBinaryArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            let mut builder = BinaryBuilder::new(args.len());
+
+            for value in array.iter() {
+                builder
+                    .append_value($FUNC(value.unwrap()).as_slice())
+                    .unwrap();
+            }
+
+            Ok(builder.finish())
+        }
+    };
+}
+
+crypto_unary_string_function!(md5, md5_process);

Review comment:
       Aren't these always of a fixed size (e.g. `128-bit` for md5)?




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

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



[GitHub] [arrow] alamb commented on pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#issuecomment-756952094


   This PR probably needs to be rebased to pick up the fix for #9138 FYI
   


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

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



[GitHub] [arrow] alamb commented on pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#issuecomment-756952094


   This PR probably needs to be rebased to pick up the fix for #9138 FYI
   


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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#issuecomment-756887914


   https://issues.apache.org/jira/browse/ARROW-11188


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

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



[GitHub] [arrow] ovr commented on a change in pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
ovr commented on a change in pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#discussion_r554424637



##########
File path: rust/datafusion/src/physical_plan/crypto_expressions.rs
##########
@@ -0,0 +1,114 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Crypto expressions
+
+use md5::Md5;
+use sha2::{
+    digest::Output as SHA2DigestOutput, Digest as SHA2Digest, Sha224, Sha256, Sha384,
+    Sha512,
+};
+
+use crate::error::{DataFusionError, Result};
+use arrow::array::{
+    ArrayRef, BinaryBuilder, GenericBinaryArray, GenericStringArray,
+    StringOffsetSizeTrait,
+};
+
+fn md5_process(input: &str) -> String {
+    let mut digest = Md5::default();
+    digest.update(&input);
+
+    let mut result = String::new();
+
+    for byte in &digest.finalize() {
+        result.push_str(&format!("{:02x}", byte));
+    }
+
+    result
+}
+
+// It's not possible to return &[u8], because trait in trait without short lifetime
+fn sha_process<D: SHA2Digest + Default>(input: &str) -> SHA2DigestOutput<D> {
+    let mut digest = D::default();
+    digest.update(&input);
+
+    digest.finalize()
+}
+
+macro_rules! crypto_unary_string_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Utf8 string
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericStringArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            // first map is the iterator, second is for the `Option<_>`
+            Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())
+        }
+    };
+}
+
+macro_rules! crypto_unary_binary_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Binary
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericBinaryArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            let mut builder = BinaryBuilder::new(args.len());
+
+            for value in array.iter() {
+                builder
+                    .append_value($FUNC(value.unwrap()).as_slice())
+                    .unwrap();
+            }

Review comment:
       Yes, it will crash.
   
   Replaced this with `Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())` without as_slice
   And it works. Weird, Where `SHA2DigestOutput<D>` converted to slice.
   
   




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

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



[GitHub] [arrow] mqy commented on pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
mqy commented on pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#issuecomment-758012229


   Perhaps the PR description should be rewrite before merge.


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

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



[GitHub] [arrow] codecov-io commented on pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#issuecomment-757472748


   # [Codecov](https://codecov.io/gh/apache/arrow/pull/9139?src=pr&el=h1) Report
   > Merging [#9139](https://codecov.io/gh/apache/arrow/pull/9139?src=pr&el=desc) (871e18c) into [master](https://codecov.io/gh/apache/arrow/commit/08cccd68802c9ddc3ca0a5d4bad6e4ba382d74b4?el=desc) (08cccd6) will **decrease** coverage by `0.04%`.
   > The diff coverage is `66.66%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow/pull/9139/graphs/tree.svg?width=650&height=150&src=pr&token=LpTCFbqVT1)](https://codecov.io/gh/apache/arrow/pull/9139?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #9139      +/-   ##
   ==========================================
   - Coverage   81.81%   81.77%   -0.05%     
   ==========================================
     Files         214      215       +1     
     Lines       51373    51461      +88     
   ==========================================
   + Hits        42033    42083      +50     
   - Misses       9340     9378      +38     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow/pull/9139?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [rust/datafusion/src/logical\_plan/expr.rs](https://codecov.io/gh/apache/arrow/pull/9139/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9sb2dpY2FsX3BsYW4vZXhwci5ycw==) | `76.92% <ø> (ø)` | |
   | [rust/datafusion/src/physical\_plan/mod.rs](https://codecov.io/gh/apache/arrow/pull/9139/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL21vZC5ycw==) | `86.00% <ø> (ø)` | |
   | [rust/arrow/src/util/display.rs](https://codecov.io/gh/apache/arrow/pull/9139/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvdXRpbC9kaXNwbGF5LnJz) | `48.33% <33.33%> (-0.79%)` | :arrow_down: |
   | [rust/datafusion/src/physical\_plan/functions.rs](https://codecov.io/gh/apache/arrow/pull/9139/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL2Z1bmN0aW9ucy5ycw==) | `74.82% <60.00%> (-3.69%)` | :arrow_down: |
   | [...datafusion/src/physical\_plan/crypto\_expressions.rs](https://codecov.io/gh/apache/arrow/pull/9139/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL2NyeXB0b19leHByZXNzaW9ucy5ycw==) | `75.00% <75.00%> (ø)` | |
   | [rust/datafusion/tests/sql.rs](https://codecov.io/gh/apache/arrow/pull/9139/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3Rlc3RzL3NxbC5ycw==) | `99.82% <100.00%> (+<0.01%)` | :arrow_up: |
   | [rust/datafusion/src/datasource/csv.rs](https://codecov.io/gh/apache/arrow/pull/9139/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9kYXRhc291cmNlL2Nzdi5ycw==) | `65.00% <0.00%> (-16.25%)` | :arrow_down: |
   | [rust/datafusion/src/datasource/parquet.rs](https://codecov.io/gh/apache/arrow/pull/9139/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9kYXRhc291cmNlL3BhcnF1ZXQucnM=) | `94.77% <0.00%> (-1.44%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow/pull/9139?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow/pull/9139?src=pr&el=footer). Last update [08cccd6...871e18c](https://codecov.io/gh/apache/arrow/pull/9139?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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

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



[GitHub] [arrow] ovr commented on pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
ovr commented on pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#issuecomment-757169119


   Thank you, @jorgecarleitao, for your review. I've added tests + fix handling null values.
   
   I've compared it with PostgreSQL, and It works as expected.
   
   ```sql
   select
           md5('tom') AS md5_tom,
           md5('') AS md5_empty_str,
           md5(null) AS md5_null,
           encode(sha224('tom'), 'hex') AS sha224_tom,
           encode(sha224(''), 'hex') AS sha224_empty_str,
           sha224(null) AS sha224_null;
   ```
   
   ```json
   [
     {
       "md5_tom": "34b7da764b21d298ef307d04d8152dc5",
       "md5_empty_str": "d41d8cd98f00b204e9800998ecf8427e",
       "md5_null": null,
       "sha224_tom": "0bf6cb62649c42a9ae3876ab6f6d92ad36cb5414e495f8873292be4d",
       "sha224_empty_str": "d14a028c2a3a2bc9476102bb288234c415a2b01f828ea62ac5b3e42f",
       "sha224_null": null
     }
   ]
   ```
   
   Thanks


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

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



[GitHub] [arrow] ovr commented on a change in pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
ovr commented on a change in pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#discussion_r554424637



##########
File path: rust/datafusion/src/physical_plan/crypto_expressions.rs
##########
@@ -0,0 +1,114 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Crypto expressions
+
+use md5::Md5;
+use sha2::{
+    digest::Output as SHA2DigestOutput, Digest as SHA2Digest, Sha224, Sha256, Sha384,
+    Sha512,
+};
+
+use crate::error::{DataFusionError, Result};
+use arrow::array::{
+    ArrayRef, BinaryBuilder, GenericBinaryArray, GenericStringArray,
+    StringOffsetSizeTrait,
+};
+
+fn md5_process(input: &str) -> String {
+    let mut digest = Md5::default();
+    digest.update(&input);
+
+    let mut result = String::new();
+
+    for byte in &digest.finalize() {
+        result.push_str(&format!("{:02x}", byte));
+    }
+
+    result
+}
+
+// It's not possible to return &[u8], because trait in trait without short lifetime
+fn sha_process<D: SHA2Digest + Default>(input: &str) -> SHA2DigestOutput<D> {
+    let mut digest = D::default();
+    digest.update(&input);
+
+    digest.finalize()
+}
+
+macro_rules! crypto_unary_string_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Utf8 string
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericStringArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            // first map is the iterator, second is for the `Option<_>`
+            Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())
+        }
+    };
+}
+
+macro_rules! crypto_unary_binary_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Binary
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericBinaryArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            let mut builder = BinaryBuilder::new(args.len());
+
+            for value in array.iter() {
+                builder
+                    .append_value($FUNC(value.unwrap()).as_slice())
+                    .unwrap();
+            }

Review comment:
       Yes, it will crash.
   
   Replaced this with `Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())` without calling `as_slice()` directly on `SHA2DigestOutput<D>` and It works. Wierd... How is it possible?)
   
   




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

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



[GitHub] [arrow] alamb closed pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect

Posted by GitBox <gi...@apache.org>.
alamb closed pull request #9139:
URL: https://github.com/apache/arrow/pull/9139


   


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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#issuecomment-756887914


   https://issues.apache.org/jira/browse/ARROW-11188


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

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



[GitHub] [arrow] mqy edited a comment on pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
mqy edited a comment on pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#issuecomment-758012229


   Perhaps the PR description should be rewritten before merge.


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

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



[GitHub] [arrow] jorgecarleitao commented on a change in pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
jorgecarleitao commented on a change in pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#discussion_r554389768



##########
File path: rust/datafusion/src/physical_plan/crypto_expressions.rs
##########
@@ -0,0 +1,114 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Crypto expressions
+
+use md5::Md5;
+use sha2::{
+    digest::Output as SHA2DigestOutput, Digest as SHA2Digest, Sha224, Sha256, Sha384,
+    Sha512,
+};
+
+use crate::error::{DataFusionError, Result};
+use arrow::array::{
+    ArrayRef, BinaryBuilder, GenericBinaryArray, GenericStringArray,
+    StringOffsetSizeTrait,
+};
+
+fn md5_process(input: &str) -> String {
+    let mut digest = Md5::default();
+    digest.update(&input);
+
+    let mut result = String::new();
+
+    for byte in &digest.finalize() {
+        result.push_str(&format!("{:02x}", byte));
+    }
+
+    result
+}
+
+// It's not possible to return &[u8], because trait in trait without short lifetime
+fn sha_process<D: SHA2Digest + Default>(input: &str) -> SHA2DigestOutput<D> {
+    let mut digest = D::default();
+    digest.update(&input);
+
+    digest.finalize()
+}
+
+macro_rules! crypto_unary_string_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Utf8 string
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericStringArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            // first map is the iterator, second is for the `Option<_>`
+            Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())
+        }
+    };
+}
+
+macro_rules! crypto_unary_binary_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Binary
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericBinaryArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            let mut builder = BinaryBuilder::new(args.len());
+
+            for value in array.iter() {
+                builder
+                    .append_value($FUNC(value.unwrap()).as_slice())
+                    .unwrap();
+            }
+
+            Ok(builder.finish())
+        }
+    };
+}
+
+crypto_unary_string_function!(md5, md5_process);

Review comment:
       Aren't these always of a fixed size (e.g. `128-bit` for md5)?

##########
File path: rust/datafusion/src/physical_plan/crypto_expressions.rs
##########
@@ -0,0 +1,114 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Crypto expressions
+
+use md5::Md5;
+use sha2::{
+    digest::Output as SHA2DigestOutput, Digest as SHA2Digest, Sha224, Sha256, Sha384,
+    Sha512,
+};
+
+use crate::error::{DataFusionError, Result};
+use arrow::array::{
+    ArrayRef, BinaryBuilder, GenericBinaryArray, GenericStringArray,
+    StringOffsetSizeTrait,
+};
+
+fn md5_process(input: &str) -> String {
+    let mut digest = Md5::default();
+    digest.update(&input);
+
+    let mut result = String::new();
+
+    for byte in &digest.finalize() {
+        result.push_str(&format!("{:02x}", byte));
+    }
+
+    result
+}
+
+// It's not possible to return &[u8], because trait in trait without short lifetime
+fn sha_process<D: SHA2Digest + Default>(input: &str) -> SHA2DigestOutput<D> {
+    let mut digest = D::default();
+    digest.update(&input);
+
+    digest.finalize()
+}
+
+macro_rules! crypto_unary_string_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Utf8 string
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericStringArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            // first map is the iterator, second is for the `Option<_>`
+            Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())
+        }
+    };
+}
+
+macro_rules! crypto_unary_binary_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Binary
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericBinaryArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            let mut builder = BinaryBuilder::new(args.len());
+
+            for value in array.iter() {
+                builder
+                    .append_value($FUNC(value.unwrap()).as_slice())
+                    .unwrap();
+            }

Review comment:
       This will crash on null values, no?
   
   Binary can also be built from an iterator, afai remember.

##########
File path: rust/datafusion/src/physical_plan/crypto_expressions.rs
##########
@@ -0,0 +1,114 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Crypto expressions
+
+use md5::Md5;
+use sha2::{
+    digest::Output as SHA2DigestOutput, Digest as SHA2Digest, Sha224, Sha256, Sha384,
+    Sha512,
+};
+
+use crate::error::{DataFusionError, Result};
+use arrow::array::{
+    ArrayRef, BinaryBuilder, GenericBinaryArray, GenericStringArray,
+    StringOffsetSizeTrait,
+};
+
+fn md5_process(input: &str) -> String {
+    let mut digest = Md5::default();
+    digest.update(&input);
+
+    let mut result = String::new();
+
+    for byte in &digest.finalize() {
+        result.push_str(&format!("{:02x}", byte));
+    }
+
+    result
+}
+
+// It's not possible to return &[u8], because trait in trait without short lifetime
+fn sha_process<D: SHA2Digest + Default>(input: &str) -> SHA2DigestOutput<D> {
+    let mut digest = D::default();
+    digest.update(&input);
+
+    digest.finalize()
+}
+
+macro_rules! crypto_unary_string_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Utf8 string
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericStringArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            // first map is the iterator, second is for the `Option<_>`
+            Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())
+        }
+    };
+}
+
+macro_rules! crypto_unary_binary_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Binary
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericBinaryArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            let mut builder = BinaryBuilder::new(args.len());
+
+            for value in array.iter() {
+                builder
+                    .append_value($FUNC(value.unwrap()).as_slice())
+                    .unwrap();
+            }
+
+            Ok(builder.finish())
+        }
+    };
+}
+
+crypto_unary_string_function!(md5, md5_process);
+crypto_unary_binary_function!(sha224, sha_process::<Sha224>);
+crypto_unary_binary_function!(sha256, sha_process::<Sha256>);
+crypto_unary_binary_function!(sha384, sha_process::<Sha384>);
+crypto_unary_binary_function!(sha512, sha_process::<Sha512>);

Review comment:
       IMO we should have a test of one of these functions here, with and without nulls, and with an empty string.




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

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



[GitHub] [arrow] ovr commented on pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
ovr commented on pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#issuecomment-756956692






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

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



[GitHub] [arrow] ovr commented on a change in pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
ovr commented on a change in pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#discussion_r554424667



##########
File path: rust/datafusion/src/physical_plan/crypto_expressions.rs
##########
@@ -0,0 +1,114 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Crypto expressions
+
+use md5::Md5;
+use sha2::{
+    digest::Output as SHA2DigestOutput, Digest as SHA2Digest, Sha224, Sha256, Sha384,
+    Sha512,
+};
+
+use crate::error::{DataFusionError, Result};
+use arrow::array::{
+    ArrayRef, BinaryBuilder, GenericBinaryArray, GenericStringArray,
+    StringOffsetSizeTrait,
+};
+
+fn md5_process(input: &str) -> String {
+    let mut digest = Md5::default();
+    digest.update(&input);
+
+    let mut result = String::new();
+
+    for byte in &digest.finalize() {
+        result.push_str(&format!("{:02x}", byte));
+    }
+
+    result
+}
+
+// It's not possible to return &[u8], because trait in trait without short lifetime
+fn sha_process<D: SHA2Digest + Default>(input: &str) -> SHA2DigestOutput<D> {
+    let mut digest = D::default();
+    digest.update(&input);
+
+    digest.finalize()
+}
+
+macro_rules! crypto_unary_string_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Utf8 string
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericStringArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            // first map is the iterator, second is for the `Option<_>`
+            Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())
+        }
+    };
+}
+
+macro_rules! crypto_unary_binary_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Binary
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericBinaryArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            let mut builder = BinaryBuilder::new(args.len());
+
+            for value in array.iter() {
+                builder
+                    .append_value($FUNC(value.unwrap()).as_slice())
+                    .unwrap();
+            }
+
+            Ok(builder.finish())
+        }
+    };
+}
+
+crypto_unary_string_function!(md5, md5_process);
+crypto_unary_binary_function!(sha224, sha_process::<Sha224>);
+crypto_unary_binary_function!(sha256, sha_process::<Sha256>);
+crypto_unary_binary_function!(sha384, sha_process::<Sha384>);
+crypto_unary_binary_function!(sha512, sha_process::<Sha512>);

Review comment:
       Good catch, I added tests for nulls and empty strings by SQL execution.




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

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



[GitHub] [arrow] jorgecarleitao commented on a change in pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
jorgecarleitao commented on a change in pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#discussion_r554604954



##########
File path: rust/datafusion/Cargo.toml
##########
@@ -61,6 +61,8 @@ futures = "0.3"
 pin-project-lite= "^0.2.0"
 tokio = { version = "0.2", features = ["macros", "blocking", "rt-core", "rt-threaded", "sync"] }
 log = "^0.4"
+md-5 = "^0.9.1"

Review comment:
       I generally agree with you, @alamb. In this case, we want to support posgres dialect, so it makes sense to support these functions (and not implement these ourselves, as they are even security related).
   
   In general, as long as the crates are small, I do not see a major issue. Our expensive dependencies are Tokio, crossbeam, etc, specially because they really increase the compile time (e.g. compared to the arrow crate).
   
   We already offer a scalar UDF that has the same performance as our own expressions. So, I think that this is the most we can do here.




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

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



[GitHub] [arrow] ovr commented on a change in pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
ovr commented on a change in pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#discussion_r554424637



##########
File path: rust/datafusion/src/physical_plan/crypto_expressions.rs
##########
@@ -0,0 +1,114 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Crypto expressions
+
+use md5::Md5;
+use sha2::{
+    digest::Output as SHA2DigestOutput, Digest as SHA2Digest, Sha224, Sha256, Sha384,
+    Sha512,
+};
+
+use crate::error::{DataFusionError, Result};
+use arrow::array::{
+    ArrayRef, BinaryBuilder, GenericBinaryArray, GenericStringArray,
+    StringOffsetSizeTrait,
+};
+
+fn md5_process(input: &str) -> String {
+    let mut digest = Md5::default();
+    digest.update(&input);
+
+    let mut result = String::new();
+
+    for byte in &digest.finalize() {
+        result.push_str(&format!("{:02x}", byte));
+    }
+
+    result
+}
+
+// It's not possible to return &[u8], because trait in trait without short lifetime
+fn sha_process<D: SHA2Digest + Default>(input: &str) -> SHA2DigestOutput<D> {
+    let mut digest = D::default();
+    digest.update(&input);
+
+    digest.finalize()
+}
+
+macro_rules! crypto_unary_string_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Utf8 string
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericStringArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            // first map is the iterator, second is for the `Option<_>`
+            Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())
+        }
+    };
+}
+
+macro_rules! crypto_unary_binary_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Binary
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericBinaryArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            let mut builder = BinaryBuilder::new(args.len());
+
+            for value in array.iter() {
+                builder
+                    .append_value($FUNC(value.unwrap()).as_slice())
+                    .unwrap();
+            }

Review comment:
       Yes, it will crash.
   
   Replaced this with `Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())` without as_slice
   And it works. Weird, Where `SHA2DigestOutput<D>` converted to slice.
   
   

##########
File path: rust/datafusion/src/physical_plan/crypto_expressions.rs
##########
@@ -0,0 +1,114 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Crypto expressions
+
+use md5::Md5;
+use sha2::{
+    digest::Output as SHA2DigestOutput, Digest as SHA2Digest, Sha224, Sha256, Sha384,
+    Sha512,
+};
+
+use crate::error::{DataFusionError, Result};
+use arrow::array::{
+    ArrayRef, BinaryBuilder, GenericBinaryArray, GenericStringArray,
+    StringOffsetSizeTrait,
+};
+
+fn md5_process(input: &str) -> String {
+    let mut digest = Md5::default();
+    digest.update(&input);
+
+    let mut result = String::new();
+
+    for byte in &digest.finalize() {
+        result.push_str(&format!("{:02x}", byte));
+    }
+
+    result
+}
+
+// It's not possible to return &[u8], because trait in trait without short lifetime
+fn sha_process<D: SHA2Digest + Default>(input: &str) -> SHA2DigestOutput<D> {
+    let mut digest = D::default();
+    digest.update(&input);
+
+    digest.finalize()
+}
+
+macro_rules! crypto_unary_string_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Utf8 string
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericStringArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            // first map is the iterator, second is for the `Option<_>`
+            Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())
+        }
+    };
+}
+
+macro_rules! crypto_unary_binary_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Binary
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericBinaryArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            let mut builder = BinaryBuilder::new(args.len());
+
+            for value in array.iter() {
+                builder
+                    .append_value($FUNC(value.unwrap()).as_slice())
+                    .unwrap();
+            }
+
+            Ok(builder.finish())
+        }
+    };
+}
+
+crypto_unary_string_function!(md5, md5_process);
+crypto_unary_binary_function!(sha224, sha_process::<Sha224>);
+crypto_unary_binary_function!(sha256, sha_process::<Sha256>);
+crypto_unary_binary_function!(sha384, sha_process::<Sha384>);
+crypto_unary_binary_function!(sha512, sha_process::<Sha512>);

Review comment:
       Good catch, I added tests for nulls and empty strings by SQL execution.

##########
File path: rust/datafusion/src/physical_plan/crypto_expressions.rs
##########
@@ -0,0 +1,114 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Crypto expressions
+
+use md5::Md5;
+use sha2::{
+    digest::Output as SHA2DigestOutput, Digest as SHA2Digest, Sha224, Sha256, Sha384,
+    Sha512,
+};
+
+use crate::error::{DataFusionError, Result};
+use arrow::array::{
+    ArrayRef, BinaryBuilder, GenericBinaryArray, GenericStringArray,
+    StringOffsetSizeTrait,
+};
+
+fn md5_process(input: &str) -> String {
+    let mut digest = Md5::default();
+    digest.update(&input);
+
+    let mut result = String::new();
+
+    for byte in &digest.finalize() {
+        result.push_str(&format!("{:02x}", byte));
+    }
+
+    result
+}
+
+// It's not possible to return &[u8], because trait in trait without short lifetime
+fn sha_process<D: SHA2Digest + Default>(input: &str) -> SHA2DigestOutput<D> {
+    let mut digest = D::default();
+    digest.update(&input);
+
+    digest.finalize()
+}
+
+macro_rules! crypto_unary_string_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Utf8 string
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericStringArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            // first map is the iterator, second is for the `Option<_>`
+            Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())
+        }
+    };
+}
+
+macro_rules! crypto_unary_binary_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Binary
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericBinaryArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            let mut builder = BinaryBuilder::new(args.len());
+
+            for value in array.iter() {
+                builder
+                    .append_value($FUNC(value.unwrap()).as_slice())
+                    .unwrap();
+            }

Review comment:
       Yes, it will crash.
   
   Replaced this with `Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())` without calling `as_slice()` directly on `SHA2DigestOutput<D>` and It works. Wierd... How is it possible?)
   
   

##########
File path: rust/datafusion/tests/sql.rs
##########
@@ -1859,19 +1859,33 @@ async fn crypto_expressions() -> Result<()> {
     let mut ctx = ExecutionContext::new();
     let sql = "SELECT
         md5('tom') AS md5_tom,
+        md5('') AS md5_empty_str,
+        md5(null) AS md5_null,
         sha224('tom') AS sha224_tom,
+        sha224('') AS sha224_empty_str,
+        sha224(null) AS sha224_null,
         sha256('tom') AS sha256_tom,
+        sha256('') AS sha256_empty_str,
         sha384('tom') AS sha348_tom,
-        sha512('tom') AS sha512_tom
+        sha384('') AS sha384_empty_str,
+        sha512('tom') AS sha512_tom,
+        sha512('') AS sha512_empty_str
     ";
     let actual = execute(&mut ctx, sql).await;
 
     let expected = vec![vec![
         "34b7da764b21d298ef307d04d8152dc5",
-        "BF6CB62649C42A9AE3876AB6F6D92AD36CB5414E495F8873292BE4D",
-        "E1608F75C5D7813F3D4031CB30BFB786507D98137538FF8E128A6FF74E84E643",
-        "96F5B68AA77848E4FDF5C1CB35DE2DBFAD6FFD7C25D9EA7C6C19B8A4D55A9187EB117C557883F58C16DFAC3E343",
-        "6E1B9B3FE84068E3751F7AD5E959D6F39AD0F8885D855166F55C659469D3C8B78118C44A2A49C72DDB481CD6D8731034E11CC0307BA843A9B3495CB8D3E"
+        "d41d8cd98f00b204e9800998ecf8427e",

Review comment:
       Tested with real PostgreSQL.
   
   ```sql
   select
           md5('tom') AS md5_tom,
           md5('') AS md5_empty_str,
           md5(null) AS md5_null,
           encode(sha224('tom'), 'hex') AS sha224_tom,
           encode(sha224(''), 'hex') AS sha224_empty_str,
           sha224(null) AS sha224_null;
   ```
   
   ```json
   [
     {
       "md5_tom": "34b7da764b21d298ef307d04d8152dc5",
       "md5_empty_str": "d41d8cd98f00b204e9800998ecf8427e",
       "md5_null": null,
       "sha224_tom": "0bf6cb62649c42a9ae3876ab6f6d92ad36cb5414e495f8873292be4d",
       "sha224_empty_str": "d14a028c2a3a2bc9476102bb288234c415a2b01f828ea62ac5b3e42f",
       "sha224_null": null
     }
   ]
   ```




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

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



[GitHub] [arrow] alamb commented on pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#issuecomment-758242999


   I filed https://issues.apache.org/jira/browse/ARROW-11214 to track the feature flag idea


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

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



[GitHub] [arrow] alamb commented on pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#issuecomment-758243354


   Thanks again for the contribution @ovr !
   


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

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



[GitHub] [arrow] ovr commented on pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
ovr commented on pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#issuecomment-756956692


   Thank you @alamb for notice 👍 
   
    I've done with PR, marked it as ready for review and awaiting review from DF's team.


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

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



[GitHub] [arrow] jorgecarleitao commented on a change in pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
jorgecarleitao commented on a change in pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#discussion_r554389768



##########
File path: rust/datafusion/src/physical_plan/crypto_expressions.rs
##########
@@ -0,0 +1,114 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Crypto expressions
+
+use md5::Md5;
+use sha2::{
+    digest::Output as SHA2DigestOutput, Digest as SHA2Digest, Sha224, Sha256, Sha384,
+    Sha512,
+};
+
+use crate::error::{DataFusionError, Result};
+use arrow::array::{
+    ArrayRef, BinaryBuilder, GenericBinaryArray, GenericStringArray,
+    StringOffsetSizeTrait,
+};
+
+fn md5_process(input: &str) -> String {
+    let mut digest = Md5::default();
+    digest.update(&input);
+
+    let mut result = String::new();
+
+    for byte in &digest.finalize() {
+        result.push_str(&format!("{:02x}", byte));
+    }
+
+    result
+}
+
+// It's not possible to return &[u8], because trait in trait without short lifetime
+fn sha_process<D: SHA2Digest + Default>(input: &str) -> SHA2DigestOutput<D> {
+    let mut digest = D::default();
+    digest.update(&input);
+
+    digest.finalize()
+}
+
+macro_rules! crypto_unary_string_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Utf8 string
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericStringArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            // first map is the iterator, second is for the `Option<_>`
+            Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())
+        }
+    };
+}
+
+macro_rules! crypto_unary_binary_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Binary
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericBinaryArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            let mut builder = BinaryBuilder::new(args.len());
+
+            for value in array.iter() {
+                builder
+                    .append_value($FUNC(value.unwrap()).as_slice())
+                    .unwrap();
+            }
+
+            Ok(builder.finish())
+        }
+    };
+}
+
+crypto_unary_string_function!(md5, md5_process);

Review comment:
       Aren't these always of a fixed size (e.g. `128-bit` for md5)?

##########
File path: rust/datafusion/src/physical_plan/crypto_expressions.rs
##########
@@ -0,0 +1,114 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Crypto expressions
+
+use md5::Md5;
+use sha2::{
+    digest::Output as SHA2DigestOutput, Digest as SHA2Digest, Sha224, Sha256, Sha384,
+    Sha512,
+};
+
+use crate::error::{DataFusionError, Result};
+use arrow::array::{
+    ArrayRef, BinaryBuilder, GenericBinaryArray, GenericStringArray,
+    StringOffsetSizeTrait,
+};
+
+fn md5_process(input: &str) -> String {
+    let mut digest = Md5::default();
+    digest.update(&input);
+
+    let mut result = String::new();
+
+    for byte in &digest.finalize() {
+        result.push_str(&format!("{:02x}", byte));
+    }
+
+    result
+}
+
+// It's not possible to return &[u8], because trait in trait without short lifetime
+fn sha_process<D: SHA2Digest + Default>(input: &str) -> SHA2DigestOutput<D> {
+    let mut digest = D::default();
+    digest.update(&input);
+
+    digest.finalize()
+}
+
+macro_rules! crypto_unary_string_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Utf8 string
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericStringArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            // first map is the iterator, second is for the `Option<_>`
+            Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())
+        }
+    };
+}
+
+macro_rules! crypto_unary_binary_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Binary
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericBinaryArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            let mut builder = BinaryBuilder::new(args.len());
+
+            for value in array.iter() {
+                builder
+                    .append_value($FUNC(value.unwrap()).as_slice())
+                    .unwrap();
+            }

Review comment:
       This will crash on null values, no?
   
   Binary can also be built from an iterator, afai remember.

##########
File path: rust/datafusion/src/physical_plan/crypto_expressions.rs
##########
@@ -0,0 +1,114 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Crypto expressions
+
+use md5::Md5;
+use sha2::{
+    digest::Output as SHA2DigestOutput, Digest as SHA2Digest, Sha224, Sha256, Sha384,
+    Sha512,
+};
+
+use crate::error::{DataFusionError, Result};
+use arrow::array::{
+    ArrayRef, BinaryBuilder, GenericBinaryArray, GenericStringArray,
+    StringOffsetSizeTrait,
+};
+
+fn md5_process(input: &str) -> String {
+    let mut digest = Md5::default();
+    digest.update(&input);
+
+    let mut result = String::new();
+
+    for byte in &digest.finalize() {
+        result.push_str(&format!("{:02x}", byte));
+    }
+
+    result
+}
+
+// It's not possible to return &[u8], because trait in trait without short lifetime
+fn sha_process<D: SHA2Digest + Default>(input: &str) -> SHA2DigestOutput<D> {
+    let mut digest = D::default();
+    digest.update(&input);
+
+    digest.finalize()
+}
+
+macro_rules! crypto_unary_string_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Utf8 string
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericStringArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            // first map is the iterator, second is for the `Option<_>`
+            Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())
+        }
+    };
+}
+
+macro_rules! crypto_unary_binary_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Binary
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericBinaryArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            let mut builder = BinaryBuilder::new(args.len());
+
+            for value in array.iter() {
+                builder
+                    .append_value($FUNC(value.unwrap()).as_slice())
+                    .unwrap();
+            }
+
+            Ok(builder.finish())
+        }
+    };
+}
+
+crypto_unary_string_function!(md5, md5_process);
+crypto_unary_binary_function!(sha224, sha_process::<Sha224>);
+crypto_unary_binary_function!(sha256, sha_process::<Sha256>);
+crypto_unary_binary_function!(sha384, sha_process::<Sha384>);
+crypto_unary_binary_function!(sha512, sha_process::<Sha512>);

Review comment:
       IMO we should have a test of one of these functions here, with and without nulls, and with an empty string.

##########
File path: rust/datafusion/src/physical_plan/crypto_expressions.rs
##########
@@ -0,0 +1,114 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Crypto expressions
+
+use md5::Md5;
+use sha2::{
+    digest::Output as SHA2DigestOutput, Digest as SHA2Digest, Sha224, Sha256, Sha384,
+    Sha512,
+};
+
+use crate::error::{DataFusionError, Result};
+use arrow::array::{
+    ArrayRef, BinaryBuilder, GenericBinaryArray, GenericStringArray,
+    StringOffsetSizeTrait,
+};
+
+fn md5_process(input: &str) -> String {
+    let mut digest = Md5::default();
+    digest.update(&input);
+
+    let mut result = String::new();
+
+    for byte in &digest.finalize() {
+        result.push_str(&format!("{:02x}", byte));
+    }
+
+    result
+}
+
+// It's not possible to return &[u8], because trait in trait without short lifetime
+fn sha_process<D: SHA2Digest + Default>(input: &str) -> SHA2DigestOutput<D> {
+    let mut digest = D::default();
+    digest.update(&input);
+
+    digest.finalize()
+}
+
+macro_rules! crypto_unary_string_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Utf8 string
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericStringArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            // first map is the iterator, second is for the `Option<_>`
+            Ok(array.iter().map(|x| x.map(|x| $FUNC(x))).collect())
+        }
+    };
+}
+
+macro_rules! crypto_unary_binary_function {
+    ($NAME:ident, $FUNC:expr) => {
+        /// crypto function that accepts Utf8 or LargeUtf8 and returns Binary
+        pub fn $NAME<T: StringOffsetSizeTrait>(
+            args: &[ArrayRef],
+        ) -> Result<GenericBinaryArray<i32>> {
+            if args.len() != 1 {
+                return Err(DataFusionError::Internal(format!(
+                    "{:?} args were supplied but {} takes exactly one argument",
+                    args.len(),
+                    String::from(stringify!($NAME)),
+                )));
+            }
+
+            let array = args[0]
+                .as_any()
+                .downcast_ref::<GenericStringArray<T>>()
+                .unwrap();
+
+            let mut builder = BinaryBuilder::new(args.len());
+
+            for value in array.iter() {
+                builder
+                    .append_value($FUNC(value.unwrap()).as_slice())
+                    .unwrap();
+            }
+
+            Ok(builder.finish())
+        }
+    };
+}
+
+crypto_unary_string_function!(md5, md5_process);

Review comment:
       Aren't these always of a fixed size (e.g. `128-bit` for md5)?




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

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



[GitHub] [arrow] alamb commented on a change in pull request #9139: ARROW-11188: [Rust] Support crypto functions from PostgreSQL dialect …

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #9139:
URL: https://github.com/apache/arrow/pull/9139#discussion_r554557321



##########
File path: rust/datafusion/src/physical_plan/functions.rs
##########
@@ -392,23 +498,18 @@ fn signature(fun: &BuiltinScalarFunction) -> Signature {
 
     // for now, the list is small, as we do not have many built-in functions.
     match fun {
-        BuiltinScalarFunction::Length => {
-            Signature::Uniform(1, vec![DataType::Utf8, DataType::LargeUtf8])
-        }
         BuiltinScalarFunction::Concat => Signature::Variadic(vec![DataType::Utf8]),
-        BuiltinScalarFunction::Lower => {
-            Signature::Uniform(1, vec![DataType::Utf8, DataType::LargeUtf8])
-        }
-        BuiltinScalarFunction::Upper => {
-            Signature::Uniform(1, vec![DataType::Utf8, DataType::LargeUtf8])
-        }
-        BuiltinScalarFunction::Trim => {
-            Signature::Uniform(1, vec![DataType::Utf8, DataType::LargeUtf8])
-        }
-        BuiltinScalarFunction::Ltrim => {
-            Signature::Uniform(1, vec![DataType::Utf8, DataType::LargeUtf8])
-        }
-        BuiltinScalarFunction::Rtrim => {
+        BuiltinScalarFunction::Upper

Review comment:
       👍  nice cleanup

##########
File path: rust/datafusion/Cargo.toml
##########
@@ -61,6 +61,8 @@ futures = "0.3"
 pin-project-lite= "^0.2.0"
 tokio = { version = "0.2", features = ["macros", "blocking", "rt-core", "rt-threaded", "sync"] }
 log = "^0.4"
+md-5 = "^0.9.1"

Review comment:
       It seems to me like we might want to start offering a way to keep the number of required dependencies of DataFusion down. For example, in this case we could potentially put the use of crypto functions behind a feature flag. 
   
   I am not proposing to add the feature flag as part of this PR, but more like trying to set the general direction of allowing users to pick features that they need and not have to pay compilation time (or binary size) cost for those they don't
   
   What do you think @jorgecarleitao  and @andygrove 




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

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