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 2020/12/19 00:25:20 UTC

[GitHub] [arrow] seddonm1 opened a new pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

seddonm1 opened a new pull request #8966:
URL: https://github.com/apache/arrow/pull/8966


   This PR implements some of the basic string functions as per the ANSI SQL specification. To properly meet the ANSI specification work will need to be done on the `sqlparser` to support the verbose style that the ANSI spec has such as 
   ```sql
   trim(both 'xyz' from 'yxTomxx')
   ```


----------------------------------------------------------------
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] seddonm1 commented on a change in pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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



##########
File path: rust/datafusion/src/physical_plan/string_expressions.rs
##########
@@ -66,3 +71,73 @@ pub fn concatenate(args: &[ArrayRef]) -> Result<StringArray> {
     }
     Ok(builder.finish())
 }
+
+/// character_length returns number of characters in the string
+/// character_length('josé') = 4
+pub fn character_length(args: &[ArrayRef]) -> Result<Int32Array> {

Review comment:
       Agree and fixed.




----------------------------------------------------------------
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] seddonm1 commented on a change in pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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



##########
File path: rust/datafusion/src/physical_plan/string_expressions.rs
##########
@@ -66,3 +71,73 @@ pub fn concatenate(args: &[ArrayRef]) -> Result<StringArray> {
     }
     Ok(builder.finish())
 }
+
+/// character_length returns number of characters in the string
+/// character_length('josé') = 4
+pub fn character_length(args: &[ArrayRef]) -> Result<Int32Array> {
+    let num_rows = args[0].len();
+    let string_args =
+        &args[0]
+            .as_any()
+            .downcast_ref::<StringArray>()
+            .ok_or_else(|| {
+                DataFusionError::Internal(
+                    "could not cast input to StringArray".to_string(),
+                )
+            })?;
+
+    let result = (0..num_rows)
+        .map(|i| {
+            if string_args.is_null(i) {
+                // NB: Since we use the same null bitset as the input,
+                // the output for this value will be ignored, but we
+                // need some value in the array we are building.
+                Ok(0)
+            } else {
+                Ok(string_args.value(i).chars().count() as i32)
+            }
+        })
+        .collect::<Result<Vec<_>>>()?;
+
+    let data = ArrayData::new(
+        DataType::Int32,
+        num_rows,
+        Some(string_args.null_count()),
+        string_args.data().null_buffer().cloned(),
+        0,
+        vec![Buffer::from(result.to_byte_slice())],
+        vec![],
+    );
+
+    Ok(Int32Array::from(Arc::new(data)))
+}
+
+macro_rules! string_unary_function {
+    ($NAME:ident, $FUNC:ident) => {
+        /// string function that accepts utf8 and returns utf8
+        pub fn $NAME(args: &[ArrayRef]) -> Result<StringArray> {
+            let string_args = &args[0]
+                .as_any()
+                .downcast_ref::<StringArray>()
+                .ok_or_else(|| {
+                    DataFusionError::Internal(
+                        "could not cast input to StringArray".to_string(),
+                    )
+                })?;
+
+            let mut builder = StringBuilder::new(args.len());
+            for index in 0..args[0].len() {
+                if string_args.is_null(index) {
+                    builder.append_null()?;
+                } else {
+                    builder.append_value(&string_args.value(index).$FUNC())?;
+                }
+            }
+            Ok(builder.finish())

Review comment:
       Thanks @jorgecarleitao . Your code makes a lot of sense and the macro is much cleaner however I am stuck at the next bit which is how to pass in `T`. I can do it in `functions::create_physical_expr` like below but this does not feel correct.
   
   ```rust
   BuiltinScalarFunction::Lower => |args| match args[0].data_type() {
       DataType::Utf8 => Ok(Arc::new(string_expressions::lower::<i32>(args)?)),
       DataType::LargeUtf8 => Ok(Arc::new(string_expressions::lower::<i64>(args)?)),
       other => Err(DataFusionError::Internal(format!(
           "Unsupported data type {:?} for function lower",
           other,
       ))),
   },
   ```




----------------------------------------------------------------
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] seddonm1 commented on a change in pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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



##########
File path: rust/datafusion/tests/sql.rs
##########
@@ -1826,3 +1826,21 @@ async fn csv_between_expr_negated() -> Result<()> {
     assert_eq!(expected, actual);
     Ok(())
 }
+
+#[tokio::test]
+async fn string_expressions() -> Result<()> {
+    let mut ctx = ExecutionContext::new();
+    register_aggregate_csv(&mut ctx)?;
+    let sql = "SELECT
+        char_length('josé') AS char_length
+        ,character_length('josé') AS character_length
+        ,lower('TOM') AS lower
+        ,upper('tom') AS upper
+        ,trim(' tom ') AS trim

Review comment:
       Yes. 
   
   I have raised https://issues.apache.org/jira/browse/ARROW-10970 to provide the ability support SQL like:
   
   ```sql
   SELECT char_length(NULL) AS char_length_null
   ```
   
   So i can see if I can add that too.




----------------------------------------------------------------
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 #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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


   # [Codecov](https://codecov.io/gh/apache/arrow/pull/8966?src=pr&el=h1) Report
   > Merging [#8966](https://codecov.io/gh/apache/arrow/pull/8966?src=pr&el=desc) (8267501) into [master](https://codecov.io/gh/apache/arrow/commit/519e9da4fc1698f686525f4226295f3680a3f3db?el=desc) (519e9da) will **decrease** coverage by `0.06%`.
   > The diff coverage is `84.48%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow/pull/8966/graphs/tree.svg?width=650&height=150&src=pr&token=LpTCFbqVT1)](https://codecov.io/gh/apache/arrow/pull/8966?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #8966      +/-   ##
   ==========================================
   - Coverage   83.26%   83.19%   -0.07%     
   ==========================================
     Files         196      199       +3     
     Lines       48192    48718     +526     
   ==========================================
   + Hits        40125    40529     +404     
   - Misses       8067     8189     +122     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow/pull/8966?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [rust/datafusion/src/logical\_plan/expr.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9sb2dpY2FsX3BsYW4vZXhwci5ycw==) | `76.76% <ø> (ø)` | |
   | [rust/datafusion/src/physical\_plan/functions.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL2Z1bmN0aW9ucy5ycw==) | `81.73% <80.00%> (-0.30%)` | :arrow_down: |
   | [...datafusion/src/physical\_plan/string\_expressions.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL3N0cmluZ19leHByZXNzaW9ucy5ycw==) | `85.00% <84.21%> (-0.72%)` | :arrow_down: |
   | [rust/datafusion/src/sql/planner.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9zcWwvcGxhbm5lci5ycw==) | `84.81% <100.00%> (+3.08%)` | :arrow_up: |
   | [rust/datafusion/tests/sql.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3Rlc3RzL3NxbC5ycw==) | `99.83% <100.00%> (+<0.01%)` | :arrow_up: |
   | [rust/arrow/src/compute/kernels/arithmetic.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvY29tcHV0ZS9rZXJuZWxzL2FyaXRobWV0aWMucnM=) | `89.79% <0.00%> (-9.23%)` | :arrow_down: |
   | [rust/datafusion/src/optimizer/filter\_push\_down.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9vcHRpbWl6ZXIvZmlsdGVyX3B1c2hfZG93bi5ycw==) | `97.65% <0.00%> (-1.74%)` | :arrow_down: |
   | [rust/datafusion/src/logical\_plan/builder.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9sb2dpY2FsX3BsYW4vYnVpbGRlci5ycw==) | `90.10% <0.00%> (-1.00%)` | :arrow_down: |
   | [rust/parquet/src/encodings/encoding.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9lbmNvZGluZ3MvZW5jb2RpbmcucnM=) | `95.24% <0.00%> (-0.20%)` | :arrow_down: |
   | [rust/datafusion/src/physical\_plan/planner.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL3BsYW5uZXIucnM=) | `80.45% <0.00%> (-0.10%)` | :arrow_down: |
   | ... and [17 more](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow/pull/8966?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/8966?src=pr&el=footer). Last update [5819943...8267501](https://codecov.io/gh/apache/arrow/pull/8966?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] seddonm1 commented on pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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


   @jorgecarleitao Thanks for your comments (they really help me learn) and have done a major refactor. 
   
   Please pay close attention to the comments here: https://github.com/apache/arrow/pull/8966#discussion_r546304152 as I do not want to make decisions like that on my own.


----------------------------------------------------------------
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 #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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



##########
File path: rust/datafusion/src/physical_plan/string_expressions.rs
##########
@@ -66,3 +71,73 @@ pub fn concatenate(args: &[ArrayRef]) -> Result<StringArray> {
     }
     Ok(builder.finish())
 }
+
+/// character_length returns number of characters in the string
+/// character_length('josé') = 4
+pub fn character_length(args: &[ArrayRef]) -> Result<Int32Array> {
+    let num_rows = args[0].len();
+    let string_args =
+        &args[0]
+            .as_any()
+            .downcast_ref::<StringArray>()
+            .ok_or_else(|| {
+                DataFusionError::Internal(
+                    "could not cast input to StringArray".to_string(),
+                )
+            })?;
+
+    let result = (0..num_rows)
+        .map(|i| {
+            if string_args.is_null(i) {
+                // NB: Since we use the same null bitset as the input,
+                // the output for this value will be ignored, but we
+                // need some value in the array we are building.
+                Ok(0)
+            } else {
+                Ok(string_args.value(i).chars().count() as i32)
+            }
+        })
+        .collect::<Result<Vec<_>>>()?;
+
+    let data = ArrayData::new(
+        DataType::Int32,
+        num_rows,
+        Some(string_args.null_count()),
+        string_args.data().null_buffer().cloned(),
+        0,
+        vec![Buffer::from(result.to_byte_slice())],
+        vec![],
+    );
+
+    Ok(Int32Array::from(Arc::new(data)))
+}
+
+macro_rules! string_unary_function {
+    ($NAME:ident, $FUNC:ident) => {
+        /// string function that accepts utf8 and returns utf8
+        pub fn $NAME(args: &[ArrayRef]) -> Result<StringArray> {
+            let string_args = &args[0]
+                .as_any()
+                .downcast_ref::<StringArray>()
+                .ok_or_else(|| {
+                    DataFusionError::Internal(
+                        "could not cast input to StringArray".to_string(),
+                    )
+                })?;
+
+            let mut builder = StringBuilder::new(args.len());
+            for index in 0..args[0].len() {
+                if string_args.is_null(index) {
+                    builder.append_null()?;
+                } else {
+                    builder.append_value(&string_args.value(index).$FUNC())?;
+                }
+            }
+            Ok(builder.finish())

Review comment:
       The Arrow crate implements efficient `IntoIter` and `FromIter` that generally make the code simpler to read and more performant because it performs less bound checks. I.e. something like
   
   ```rust
   string_args.iter().map(|x| x.map(|x| x.$FUNC())).collect()
   // (first map is the iterator, second is for the `Option<_>`
   ```
   
   will probably work.

##########
File path: rust/datafusion/tests/sql.rs
##########
@@ -1826,3 +1826,21 @@ async fn csv_between_expr_negated() -> Result<()> {
     assert_eq!(expected, actual);
     Ok(())
 }
+
+#[tokio::test]
+async fn string_expressions() -> Result<()> {
+    let mut ctx = ExecutionContext::new();
+    register_aggregate_csv(&mut ctx)?;
+    let sql = "SELECT
+        char_length('josé') AS char_length
+        ,character_length('josé') AS character_length
+        ,lower('TOM') AS lower
+        ,upper('tom') AS upper
+        ,trim(' tom ') AS trim

Review comment:
       I think that this does not cover the null cases.

##########
File path: rust/datafusion/src/physical_plan/functions.rs
##########
@@ -203,6 +216,10 @@ pub fn return_type(
             }
         }),
         BuiltinScalarFunction::Concat => Ok(DataType::Utf8),
+        BuiltinScalarFunction::CharacterLength => Ok(DataType::Int32),
+        BuiltinScalarFunction::Lower => Ok(DataType::Utf8),
+        BuiltinScalarFunction::Upper => Ok(DataType::Utf8),
+        BuiltinScalarFunction::Trim => Ok(DataType::Utf8),

Review comment:
       Could you add functions to `logical_plan::expr` and `prelude` to expose these new functions also there, so that they can also be used in the DataFrame API?

##########
File path: rust/datafusion/tests/sql.rs
##########
@@ -1826,3 +1826,21 @@ async fn csv_between_expr_negated() -> Result<()> {
     assert_eq!(expected, actual);
     Ok(())
 }
+
+#[tokio::test]
+async fn string_expressions() -> Result<()> {
+    let mut ctx = ExecutionContext::new();
+    register_aggregate_csv(&mut ctx)?;

Review comment:
       Do we need to register this CSV?

##########
File path: rust/datafusion/src/physical_plan/functions.rs
##########
@@ -280,6 +309,18 @@ fn signature(fun: &BuiltinScalarFunction) -> Signature {
             Signature::Uniform(1, vec![DataType::Utf8, DataType::LargeUtf8])
         }
         BuiltinScalarFunction::Concat => Signature::Variadic(vec![DataType::Utf8]),
+        BuiltinScalarFunction::CharacterLength => {
+            Signature::Uniform(1, vec![DataType::Utf8, DataType::LargeUtf8])

Review comment:
       The signature states that `LargeUtf8` is supported, but the implementation only supports `Utf8`.
   
   If we only use `DataType::Utf8` here, the planner will coerce any `LargeUtf8` to `Utf8`. :)

##########
File path: rust/datafusion/src/physical_plan/string_expressions.rs
##########
@@ -66,3 +71,73 @@ pub fn concatenate(args: &[ArrayRef]) -> Result<StringArray> {
     }
     Ok(builder.finish())
 }
+
+/// character_length returns number of characters in the string
+/// character_length('josé') = 4
+pub fn character_length(args: &[ArrayRef]) -> Result<Int32Array> {
+    let num_rows = args[0].len();
+    let string_args =
+        &args[0]
+            .as_any()
+            .downcast_ref::<StringArray>()
+            .ok_or_else(|| {
+                DataFusionError::Internal(
+                    "could not cast input to StringArray".to_string(),
+                )
+            })?;
+
+    let result = (0..num_rows)

Review comment:
       I would try to use the from and to iterator here also. It should make this code simpler also.

##########
File path: rust/datafusion/src/physical_plan/string_expressions.rs
##########
@@ -66,3 +71,73 @@ pub fn concatenate(args: &[ArrayRef]) -> Result<StringArray> {
     }
     Ok(builder.finish())
 }
+
+/// character_length returns number of characters in the string
+/// character_length('josé') = 4
+pub fn character_length(args: &[ArrayRef]) -> Result<Int32Array> {

Review comment:
       Any reason to use `Int32` instead of `UInt32`?

##########
File path: rust/datafusion/src/physical_plan/functions.rs
##########
@@ -118,6 +118,14 @@ pub enum BuiltinScalarFunction {
     Length,
     /// concat
     Concat,
+    /// character_length
+    CharacterLength,
+    /// lower
+    Lower,
+    /// upper
+    Upper,
+    /// ltrim

Review comment:
       Is this `ltrim` or trim? The test seems to indicate `trim`.




----------------------------------------------------------------
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] seddonm1 commented on a change in pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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



##########
File path: rust/datafusion/src/physical_plan/string_expressions.rs
##########
@@ -66,3 +71,73 @@ pub fn concatenate(args: &[ArrayRef]) -> Result<StringArray> {
     }
     Ok(builder.finish())
 }
+
+/// character_length returns number of characters in the string
+/// character_length('josé') = 4
+pub fn character_length(args: &[ArrayRef]) -> Result<Int32Array> {
+    let num_rows = args[0].len();
+    let string_args =
+        &args[0]
+            .as_any()
+            .downcast_ref::<StringArray>()
+            .ok_or_else(|| {
+                DataFusionError::Internal(
+                    "could not cast input to StringArray".to_string(),
+                )
+            })?;
+
+    let result = (0..num_rows)
+        .map(|i| {
+            if string_args.is_null(i) {
+                // NB: Since we use the same null bitset as the input,
+                // the output for this value will be ignored, but we
+                // need some value in the array we are building.
+                Ok(0)
+            } else {
+                Ok(string_args.value(i).chars().count() as i32)
+            }
+        })
+        .collect::<Result<Vec<_>>>()?;
+
+    let data = ArrayData::new(
+        DataType::Int32,
+        num_rows,
+        Some(string_args.null_count()),
+        string_args.data().null_buffer().cloned(),
+        0,
+        vec![Buffer::from(result.to_byte_slice())],
+        vec![],
+    );
+
+    Ok(Int32Array::from(Arc::new(data)))
+}
+
+macro_rules! string_unary_function {
+    ($NAME:ident, $FUNC:ident) => {
+        /// string function that accepts utf8 and returns utf8
+        pub fn $NAME(args: &[ArrayRef]) -> Result<StringArray> {
+            let string_args = &args[0]
+                .as_any()
+                .downcast_ref::<StringArray>()
+                .ok_or_else(|| {
+                    DataFusionError::Internal(
+                        "could not cast input to StringArray".to_string(),
+                    )
+                })?;
+
+            let mut builder = StringBuilder::new(args.len());
+            for index in 0..args[0].len() {
+                if string_args.is_null(index) {
+                    builder.append_null()?;
+                } else {
+                    builder.append_value(&string_args.value(index).$FUNC())?;
+                }
+            }
+            Ok(builder.finish())

Review comment:
       Ok you can disregard this comment as it is exactly how `length` is implemented. Technically we should not expose `length` as a SQL function but we could rename the alias to be `char_length` and `character_length` based on the ANSI SQL spec: https://jakewheat.github.io/sql-overview/sql-2016-foundation-grammar.html#char-length-expression
   
   




----------------------------------------------------------------
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] seddonm1 commented on a change in pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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



##########
File path: rust/datafusion/src/physical_plan/functions.rs
##########
@@ -118,6 +118,14 @@ pub enum BuiltinScalarFunction {
     Length,
     /// concat
     Concat,
+    /// character_length
+    CharacterLength,
+    /// lower
+    Lower,
+    /// upper
+    Upper,
+    /// ltrim

Review comment:
       Yes, sorry, I had actually implemented quite a few more functions (more `ltrim`, `rtrim` and `position`) but then removed them as technically `ltrim` and `rtrim` is not part of the ANSI standard but are Postgres additions (and others).
   
   I think the decision on SQL dialect to support is required before continuing too much further.




----------------------------------------------------------------
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] andygrove commented on pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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


   This looks great. Thanks @seddonm1 


----------------------------------------------------------------
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 #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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



##########
File path: rust/datafusion/src/physical_plan/string_expressions.rs
##########
@@ -66,3 +71,73 @@ pub fn concatenate(args: &[ArrayRef]) -> Result<StringArray> {
     }
     Ok(builder.finish())
 }
+
+/// character_length returns number of characters in the string
+/// character_length('josé') = 4
+pub fn character_length(args: &[ArrayRef]) -> Result<Int32Array> {
+    let num_rows = args[0].len();
+    let string_args =
+        &args[0]
+            .as_any()
+            .downcast_ref::<StringArray>()
+            .ok_or_else(|| {
+                DataFusionError::Internal(
+                    "could not cast input to StringArray".to_string(),
+                )
+            })?;
+
+    let result = (0..num_rows)
+        .map(|i| {
+            if string_args.is_null(i) {
+                // NB: Since we use the same null bitset as the input,
+                // the output for this value will be ignored, but we
+                // need some value in the array we are building.
+                Ok(0)
+            } else {
+                Ok(string_args.value(i).chars().count() as i32)
+            }
+        })
+        .collect::<Result<Vec<_>>>()?;
+
+    let data = ArrayData::new(
+        DataType::Int32,
+        num_rows,
+        Some(string_args.null_count()),
+        string_args.data().null_buffer().cloned(),
+        0,
+        vec![Buffer::from(result.to_byte_slice())],
+        vec![],
+    );
+
+    Ok(Int32Array::from(Arc::new(data)))
+}
+
+macro_rules! string_unary_function {
+    ($NAME:ident, $FUNC:ident) => {
+        /// string function that accepts utf8 and returns utf8
+        pub fn $NAME(args: &[ArrayRef]) -> Result<StringArray> {
+            let string_args = &args[0]
+                .as_any()
+                .downcast_ref::<StringArray>()
+                .ok_or_else(|| {
+                    DataFusionError::Internal(
+                        "could not cast input to StringArray".to_string(),
+                    )
+                })?;
+
+            let mut builder = StringBuilder::new(args.len());
+            for index in 0..args[0].len() {
+                if string_args.is_null(index) {
+                    builder.append_null()?;
+                } else {
+                    builder.append_value(&string_args.value(index).$FUNC())?;
+                }
+            }
+            Ok(builder.finish())

Review comment:
       The trick is to use the generic `GenericStringArray`, whose `StringArray` and `LargeStringArray` are concrete types of. Something like
   
   ```rust
   fn op<T: StringOffsetSizeTrait>(array: GenericStringArray<T>) -> GenericStringArray<T> {
        let array = array.downcast::<GenericStringArray<T>>().unwrap();
   
        array.iter().map(|x| x.map(|x| x.$FUNC())).collect()
   }
   ```
   
   The FromIter and ToIterator are implemented for the generic struct and thus the compiler should be able to resolve these for both `T: i32` and `T: i64`.




----------------------------------------------------------------
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] seddonm1 commented on a change in pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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



##########
File path: rust/datafusion/tests/sql.rs
##########
@@ -1826,3 +1826,21 @@ async fn csv_between_expr_negated() -> Result<()> {
     assert_eq!(expected, actual);
     Ok(())
 }
+
+#[tokio::test]
+async fn string_expressions() -> Result<()> {
+    let mut ctx = ExecutionContext::new();
+    register_aggregate_csv(&mut ctx)?;
+    let sql = "SELECT
+        char_length('josé') AS char_length
+        ,character_length('josé') AS character_length
+        ,lower('TOM') AS lower
+        ,upper('tom') AS upper
+        ,trim(' tom ') AS trim

Review comment:
       @andygrove copying you in due to decision:
   
   I have now added the `NULL` value to both the test cases and the planner.
   
   This is where things get interesting. For this statement:
   
   ```sql
   SELECT NULL
   ```
   
   Spark implements a special `NullType` for this return type but that creates a lot of side effects for things like the Parquet writer and JDBC drivers do not support this type.
   
   I tested Postgres:
   
   ```sql
   CREATE TABLE test AS
   SELECT NULL;
   ```
   
   The DDL for this table shows that column as a `text` type so that is why I have applied the default `utf8` type to `Value(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] github-actions[bot] commented on pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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


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


----------------------------------------------------------------
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] seddonm1 commented on a change in pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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



##########
File path: rust/datafusion/src/physical_plan/string_expressions.rs
##########
@@ -66,3 +71,73 @@ pub fn concatenate(args: &[ArrayRef]) -> Result<StringArray> {
     }
     Ok(builder.finish())
 }
+
+/// character_length returns number of characters in the string
+/// character_length('josé') = 4
+pub fn character_length(args: &[ArrayRef]) -> Result<Int32Array> {
+    let num_rows = args[0].len();
+    let string_args =
+        &args[0]
+            .as_any()
+            .downcast_ref::<StringArray>()
+            .ok_or_else(|| {
+                DataFusionError::Internal(
+                    "could not cast input to StringArray".to_string(),
+                )
+            })?;
+
+    let result = (0..num_rows)
+        .map(|i| {
+            if string_args.is_null(i) {
+                // NB: Since we use the same null bitset as the input,
+                // the output for this value will be ignored, but we
+                // need some value in the array we are building.
+                Ok(0)
+            } else {
+                Ok(string_args.value(i).chars().count() as i32)
+            }
+        })
+        .collect::<Result<Vec<_>>>()?;
+
+    let data = ArrayData::new(
+        DataType::Int32,
+        num_rows,
+        Some(string_args.null_count()),
+        string_args.data().null_buffer().cloned(),
+        0,
+        vec![Buffer::from(result.to_byte_slice())],
+        vec![],
+    );
+
+    Ok(Int32Array::from(Arc::new(data)))
+}
+
+macro_rules! string_unary_function {
+    ($NAME:ident, $FUNC:ident) => {
+        /// string function that accepts utf8 and returns utf8
+        pub fn $NAME(args: &[ArrayRef]) -> Result<StringArray> {
+            let string_args = &args[0]
+                .as_any()
+                .downcast_ref::<StringArray>()
+                .ok_or_else(|| {
+                    DataFusionError::Internal(
+                        "could not cast input to StringArray".to_string(),
+                    )
+                })?;
+
+            let mut builder = StringBuilder::new(args.len());
+            for index in 0..args[0].len() {
+                if string_args.is_null(index) {
+                    builder.append_null()?;
+                } else {
+                    builder.append_value(&string_args.value(index).$FUNC())?;
+                }
+            }
+            Ok(builder.finish())

Review comment:
       Thanks @jorgecarleitao . Your code makes a lot of sense and the macro is much cleaner however I am stuck at the next bit which is how to pass in `T`. I can do it in `functions::create_physical_expr` like below but this does not feel correct.
   
   ```rust
   BuiltinScalarFunction::Lower => {
       |args| {
           match args[0].data_type() {
               DataType::Utf8 => {
                   Ok(Arc::new(string_expressions::lower::<i32>(args)?))
               }
               DataType::LargeUtf8 => {
                   Ok(Arc::new(string_expressions::lower::<i64>(args)?))
               }
               other => Err(DataFusionError::Internal(format!(
                   "Unsupported data type {:?} for function lower",
                   other,
               ))),
           }
       }
   }
   ```




----------------------------------------------------------------
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 pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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


   @seddonm1 , the API for built-in functions is relatively new and WIP. If you felt that it did not suit the needs or that it could be simpler / easier to use, please raise that concern. We anticipate that it will be more used as time progresses, and it is useful to check its design from time to time to make sure that its assumptions still hold.


----------------------------------------------------------------
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] seddonm1 commented on a change in pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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



##########
File path: rust/datafusion/src/physical_plan/functions.rs
##########
@@ -280,6 +309,18 @@ fn signature(fun: &BuiltinScalarFunction) -> Signature {
             Signature::Uniform(1, vec![DataType::Utf8, DataType::LargeUtf8])
         }
         BuiltinScalarFunction::Concat => Signature::Variadic(vec![DataType::Utf8]),
+        BuiltinScalarFunction::CharacterLength => {
+            Signature::Uniform(1, vec![DataType::Utf8, DataType::LargeUtf8])

Review comment:
       I have reworked the macros significantly to now support Utf8 and LargeUtf8 functions going forward. It would be trivial to add more functions like `ltrim` which support both types.




----------------------------------------------------------------
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] seddonm1 commented on pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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


   > @seddonm1 , the API for built-in functions is relatively new and WIP. If you felt that it did not suit the needs or that it could be simpler / easier to use, please raise that concern. We anticipate that it will be more used as time progresses, and it is useful to check its design from time to time to make sure that its assumptions still hold.
   
   The instructions in the README are definitely enough information to easily add these kind of functions (I updated slightly) and I think the API (like the `Variadic` and `Uniform` signatures) is really intuitive - so I think you did a great job.
   
   The big question at a project level is which dialect of SQL to support. Adding `ltrim` for example is trivial but that is a Postgres specific function: https://www.postgresql.org/docs/13/functions-string.html#FUNCTIONS-STRING-OTHER (its in lots of other implementations too). If we decide Postgres support then already the DataFusion `concat` implementation does not align with Postgres in how it handles `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] seddonm1 commented on pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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


   > Thanks a lot, @seddonm1 , really clean implementation.
   
   Thanks @jorgecarleitao and thanks for your patience 👍 


----------------------------------------------------------------
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] seddonm1 commented on a change in pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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



##########
File path: rust/datafusion/tests/sql.rs
##########
@@ -1826,3 +1826,21 @@ async fn csv_between_expr_negated() -> Result<()> {
     assert_eq!(expected, actual);
     Ok(())
 }
+
+#[tokio::test]
+async fn string_expressions() -> Result<()> {
+    let mut ctx = ExecutionContext::new();
+    register_aggregate_csv(&mut ctx)?;

Review comment:
       correct. removed.




----------------------------------------------------------------
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] seddonm1 commented on a change in pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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



##########
File path: rust/datafusion/src/physical_plan/string_expressions.rs
##########
@@ -66,3 +71,73 @@ pub fn concatenate(args: &[ArrayRef]) -> Result<StringArray> {
     }
     Ok(builder.finish())
 }
+
+/// character_length returns number of characters in the string
+/// character_length('josé') = 4
+pub fn character_length(args: &[ArrayRef]) -> Result<Int32Array> {
+    let num_rows = args[0].len();
+    let string_args =
+        &args[0]
+            .as_any()
+            .downcast_ref::<StringArray>()
+            .ok_or_else(|| {
+                DataFusionError::Internal(
+                    "could not cast input to StringArray".to_string(),
+                )
+            })?;
+
+    let result = (0..num_rows)
+        .map(|i| {
+            if string_args.is_null(i) {
+                // NB: Since we use the same null bitset as the input,
+                // the output for this value will be ignored, but we
+                // need some value in the array we are building.
+                Ok(0)
+            } else {
+                Ok(string_args.value(i).chars().count() as i32)
+            }
+        })
+        .collect::<Result<Vec<_>>>()?;
+
+    let data = ArrayData::new(
+        DataType::Int32,
+        num_rows,
+        Some(string_args.null_count()),
+        string_args.data().null_buffer().cloned(),
+        0,
+        vec![Buffer::from(result.to_byte_slice())],
+        vec![],
+    );
+
+    Ok(Int32Array::from(Arc::new(data)))
+}
+
+macro_rules! string_unary_function {
+    ($NAME:ident, $FUNC:ident) => {
+        /// string function that accepts utf8 and returns utf8
+        pub fn $NAME(args: &[ArrayRef]) -> Result<StringArray> {
+            let string_args = &args[0]
+                .as_any()
+                .downcast_ref::<StringArray>()
+                .ok_or_else(|| {
+                    DataFusionError::Internal(
+                        "could not cast input to StringArray".to_string(),
+                    )
+                })?;
+
+            let mut builder = StringBuilder::new(args.len());
+            for index in 0..args[0].len() {
+                if string_args.is_null(index) {
+                    builder.append_null()?;
+                } else {
+                    builder.append_value(&string_args.value(index).$FUNC())?;
+                }
+            }
+            Ok(builder.finish())

Review comment:
       This did work well but I have struggled to make it work with code that supports both `Utf8` and `LargeUtf8` types as the code does now. Maybe you could help 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] jorgecarleitao closed pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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


   


----------------------------------------------------------------
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 edited a comment on pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #8966:
URL: https://github.com/apache/arrow/pull/8966#issuecomment-748546622


   # [Codecov](https://codecov.io/gh/apache/arrow/pull/8966?src=pr&el=h1) Report
   > Merging [#8966](https://codecov.io/gh/apache/arrow/pull/8966?src=pr&el=desc) (2f26b23) into [master](https://codecov.io/gh/apache/arrow/commit/519e9da4fc1698f686525f4226295f3680a3f3db?el=desc) (519e9da) will **decrease** coverage by `0.09%`.
   > The diff coverage is `74.46%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow/pull/8966/graphs/tree.svg?width=650&height=150&src=pr&token=LpTCFbqVT1)](https://codecov.io/gh/apache/arrow/pull/8966?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #8966      +/-   ##
   ==========================================
   - Coverage   83.26%   83.16%   -0.10%     
   ==========================================
     Files         196      200       +4     
     Lines       48192    48992     +800     
   ==========================================
   + Hits        40125    40743     +618     
   - Misses       8067     8249     +182     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow/pull/8966?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [rust/datafusion/src/logical\_plan/expr.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9sb2dpY2FsX3BsYW4vZXhwci5ycw==) | `76.76% <ø> (ø)` | |
   | [rust/datafusion/src/physical\_plan/functions.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL2Z1bmN0aW9ucy5ycw==) | `79.34% <65.71%> (-2.68%)` | :arrow_down: |
   | [...datafusion/src/physical\_plan/string\_expressions.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL3N0cmluZ19leHByZXNzaW9ucy5ycw==) | `87.50% <100.00%> (+1.78%)` | :arrow_up: |
   | [rust/datafusion/src/sql/planner.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9zcWwvcGxhbm5lci5ycw==) | `84.81% <100.00%> (+3.08%)` | :arrow_up: |
   | [rust/datafusion/tests/sql.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3Rlc3RzL3NxbC5ycw==) | `99.83% <100.00%> (+<0.01%)` | :arrow_up: |
   | [rust/arrow/src/compute/kernels/arithmetic.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvY29tcHV0ZS9rZXJuZWxzL2FyaXRobWV0aWMucnM=) | `89.79% <0.00%> (-9.23%)` | :arrow_down: |
   | [rust/arrow/src/compute/kernels/sort.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvY29tcHV0ZS9rZXJuZWxzL3NvcnQucnM=) | `93.42% <0.00%> (-2.34%)` | :arrow_down: |
   | [rust/datafusion/src/optimizer/filter\_push\_down.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9vcHRpbWl6ZXIvZmlsdGVyX3B1c2hfZG93bi5ycw==) | `97.65% <0.00%> (-1.74%)` | :arrow_down: |
   | [rust/datafusion/src/logical\_plan/builder.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9sb2dpY2FsX3BsYW4vYnVpbGRlci5ycw==) | `90.10% <0.00%> (-1.00%)` | :arrow_down: |
   | [rust/arrow/src/compute/util.rs](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvY29tcHV0ZS91dGlsLnJz) | `98.93% <0.00%> (-0.44%)` | :arrow_down: |
   | ... and [22 more](https://codecov.io/gh/apache/arrow/pull/8966/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow/pull/8966?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/8966?src=pr&el=footer). Last update [5819943...2f26b23](https://codecov.io/gh/apache/arrow/pull/8966?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] seddonm1 commented on pull request #8966: ARROW-10969: [Rust][DataFusion] Implement basic String ANSI SQL Functions

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


   > Overall, this looks really good. Thanks a lot for taking the time to implement these.
   > 
   > I left a lot of comments, but they are all pretty small, so please do not take their number by the correctness of this: you understood and used all the APIs really well, they are just considering edge cases and small consistency improvements.
   
   Thanks @jorgecarleitao and no problem with the number of comments.
   
   I will work through these and let you know.


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