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


##########
datafusion/functions/src/regex/regexpmatch.rs:
##########
@@ -0,0 +1,173 @@
+// 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.
+
+//! Encoding expressions
+use arrow::array::{Array, ArrayRef, OffsetSizeTrait};
+use arrow::compute::kernels::regexp;
+use arrow::datatypes::DataType;
+use arrow::datatypes::Field;
+use datafusion_common::ScalarValue;
+use datafusion_expr::TypeSignature::*;
+use datafusion_expr::{ScalarUDFImpl, Signature, Volatility};
+use std::any::Any;
+
+use datafusion_common::{arrow_datafusion_err, plan_err};
+use datafusion_common::{
+    cast::as_generic_string_array, internal_err, DataFusionError, Result,
+};
+use datafusion_expr::ColumnarValue;
+use std::sync::Arc;
+
+#[cfg(feature = "regex_expressions")]
+macro_rules! invoke_on_array_if_regex_expressions_feature_flag {

Review Comment:
   I don't think these macros are needed here because the entire module is not 
included if the `regex_expressions` module is not compiled in. 



##########
datafusion/functions/src/regex/regexpmatch.rs:
##########
@@ -0,0 +1,173 @@
+// 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.
+
+//! Encoding expressions
+use arrow::array::{Array, ArrayRef, OffsetSizeTrait};
+use arrow::compute::kernels::regexp;
+use arrow::datatypes::DataType;
+use arrow::datatypes::Field;
+use datafusion_common::ScalarValue;
+use datafusion_expr::TypeSignature::*;
+use datafusion_expr::{ScalarUDFImpl, Signature, Volatility};
+use std::any::Any;
+
+use datafusion_common::{arrow_datafusion_err, plan_err};
+use datafusion_common::{
+    cast::as_generic_string_array, internal_err, DataFusionError, Result,
+};
+use datafusion_expr::ColumnarValue;
+use std::sync::Arc;
+
+#[cfg(feature = "regex_expressions")]
+macro_rules! invoke_on_array_if_regex_expressions_feature_flag {
+    ($FUNC:ident, $T:tt, $NAME:expr) => {{
+        $FUNC::<$T>
+    }};
+}
+
+#[cfg(not(feature = "regex_expressions"))]
+macro_rules! invoke_on_array_if_regex_expressions_feature_flag {
+    ($FUNC:ident, $T:tt, $NAME:expr) => {
+        |_: &[ArrayRef]| -> Result<ArrayRef> {
+            internal_err!(
+                "function {} requires compilation with feature flag: 
regex_expressions.",
+                $NAME
+            )
+        }
+    };
+}
+#[derive(Debug)]
+pub(super) struct RegexpMatchFunc {
+    signature: Signature,
+}
+impl RegexpMatchFunc {
+    pub fn new() -> Self {
+        use DataType::*;
+        Self {
+            signature: Signature::one_of(
+                vec![
+                    Exact(vec![Utf8, Utf8]),
+                    Exact(vec![LargeUtf8, Utf8]),
+                    Exact(vec![Utf8, Utf8, Utf8]),
+                    Exact(vec![LargeUtf8, Utf8, Utf8]),
+                ],
+                Volatility::Immutable,
+            ),
+        }
+    }
+}
+
+impl ScalarUDFImpl for RegexpMatchFunc {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn name(&self) -> &str {
+        "regexp_match"
+    }
+
+    fn signature(&self) -> &Signature {
+        &self.signature
+    }
+
+    fn return_type(&self, arg_types: &[DataType]) -> Result<DataType> {
+        use DataType::*;
+
+        Ok(match &arg_types[0] {
+            LargeUtf8 => List(Arc::new(Field::new("item", LargeUtf8, true))),
+            Utf8 => List(Arc::new(Field::new("item", Utf8, true))),
+            Null => Null,
+            other => {
+                return plan_err!(
+                    "The regexp_match function can only accept strings. Got 
{other}"
+                );
+            }
+        })
+    }
+    fn invoke(&self, args: &[ColumnarValue]) -> Result<ColumnarValue> {
+        let len = args
+            .iter()
+            .fold(Option::<usize>::None, |acc, arg| match arg {
+                ColumnarValue::Scalar(_) => acc,
+                ColumnarValue::Array(a) => Some(a.len()),
+            });
+
+        let is_scalar = len.is_none();
+        let inferred_length = len.unwrap_or(1);
+        let args = args
+            .iter()
+            .map(|arg| arg.clone().into_array(inferred_length))
+            .collect::<Result<Vec<_>>>()?;
+
+        let result = regexp_match_func(&args);
+        if is_scalar {
+            // If all inputs are scalar, keeps output as scalar
+            let result = result.and_then(|arr| 
ScalarValue::try_from_array(&arr, 0));
+            result.map(ColumnarValue::Scalar)
+        } else {
+            result.map(ColumnarValue::Array)
+        }
+    }
+}
+fn regexp_match_func(args: &[ArrayRef]) -> Result<ArrayRef> {
+    match args[0].data_type() {
+        DataType::Utf8 => {
+            let func = invoke_on_array_if_regex_expressions_feature_flag!(
+                regexp_match,
+                i32,
+                "regexp_match"
+            );
+            func(args)
+        }
+        DataType::LargeUtf8 => {
+            let func = invoke_on_array_if_regex_expressions_feature_flag!(
+                regexp_match,
+                i64,
+                "regexp_match"
+            );
+            func(args)
+        }
+        other => {
+            internal_err!("Unsupported data type {other:?} for function 
regexp_match")
+        }
+    }
+}
+pub fn regexp_match<T: OffsetSizeTrait>(args: &[ArrayRef]) -> Result<ArrayRef> 
{
+    match args.len() {
+        2 => {
+            let values = as_generic_string_array::<T>(&args[0])?;
+            let regex = as_generic_string_array::<T>(&args[1])?;
+            regexp::regexp_match(values, regex, None)
+                .map_err(|e| arrow_datafusion_err!(e))
+        }
+        3 => {
+            let values = as_generic_string_array::<T>(&args[0])?;
+            let regex = as_generic_string_array::<T>(&args[1])?;
+            let flags = as_generic_string_array::<T>(&args[2])?;
+
+            if flags.iter().any(|s| s == Some("g")) {
+                return plan_err!("regexp_match() does not support the 
\"global\" option")
+            }
+
+            regexp::regexp_match(values, regex, Some(flags))
+                .map_err(|e| arrow_datafusion_err!(e))
+        }
+        other => internal_err!(

Review Comment:
   ```suggestion
           other => exec_err!(
   ```



##########
datafusion/core/tests/dataframe/dataframe_functions.rs:
##########
@@ -467,7 +467,7 @@ async fn test_fn_regexp_like() -> Result<()> {
 #[tokio::test]
 #[cfg(feature = "unicode_expressions")]
 async fn test_fn_regexp_match() -> Result<()> {
-    let expr = regexp_match(vec![col("a"), lit("[a-z]")]);
+    let expr = regexp_match(col("a"), lit("[a-z]"));

Review Comment:
   I think this is a good improvement, but it is an API change, so marking this 
PR as such



##########
datafusion/physical-expr/src/functions.rs:
##########
@@ -3191,89 +3167,89 @@ mod tests {
         Ok(())
     }
 
-    #[test]
-    #[cfg(feature = "regex_expressions")]
-    fn test_regexp_match() -> Result<()> {
-        use datafusion_common::cast::{as_list_array, as_string_array};
-        let schema = Schema::new(vec![Field::new("a", DataType::Utf8, false)]);
-        let execution_props = ExecutionProps::new();
-
-        let col_value: ArrayRef = Arc::new(StringArray::from(vec!["aaa-555"]));
-        let pattern = lit(r".*-(\d*)");
-        let columns: Vec<ArrayRef> = vec![col_value];
-        let expr = create_physical_expr_with_type_coercion(
-            &BuiltinScalarFunction::RegexpMatch,
-            &[col("a", &schema)?, pattern],
-            &schema,
-            &execution_props,
-        )?;
-
-        // type is correct
-        assert_eq!(
-            expr.data_type(&schema)?,
-            DataType::List(Arc::new(Field::new("item", DataType::Utf8, true)))
-        );
-
-        // evaluate works
-        let batch = RecordBatch::try_new(Arc::new(schema.clone()), columns)?;
-        let result = expr
-            .evaluate(&batch)?
-            .into_array(batch.num_rows())
-            .expect("Failed to convert to array");
-
-        // downcast works
-        let result = as_list_array(&result)?;
-        let first_row = result.value(0);
-        let first_row = as_string_array(&first_row)?;
-
-        // value is correct
-        let expected = "555".to_string();
-        assert_eq!(first_row.value(0), expected);
-
-        Ok(())
-    }
-
-    #[test]
-    #[cfg(feature = "regex_expressions")]
-    fn test_regexp_match_all_literals() -> Result<()> {
-        use datafusion_common::cast::{as_list_array, as_string_array};
-        let schema = Schema::new(vec![Field::new("a", DataType::Int32, 
false)]);
-        let execution_props = ExecutionProps::new();
-
-        let col_value = lit("aaa-555");
-        let pattern = lit(r".*-(\d*)");
-        let columns: Vec<ArrayRef> = vec![Arc::new(Int32Array::from(vec![1]))];
-        let expr = create_physical_expr_with_type_coercion(
-            &BuiltinScalarFunction::RegexpMatch,
-            &[col_value, pattern],
-            &schema,
-            &execution_props,
-        )?;
-
-        // type is correct
-        assert_eq!(
-            expr.data_type(&schema)?,
-            DataType::List(Arc::new(Field::new("item", DataType::Utf8, true)))
-        );
-
-        // evaluate works
-        let batch = RecordBatch::try_new(Arc::new(schema.clone()), columns)?;
-        let result = expr
-            .evaluate(&batch)?
-            .into_array(batch.num_rows())
-            .expect("Failed to convert to array");
-
-        // downcast works
-        let result = as_list_array(&result)?;
-        let first_row = result.value(0);
-        let first_row = as_string_array(&first_row)?;
-
-        // value is correct
-        let expected = "555".to_string();
-        assert_eq!(first_row.value(0), expected);
-
-        Ok(())
-    }
+    // #[test]

Review Comment:
   It looks to me like regexp_match is already tested in 
https://github.com/apache/arrow-datafusion/blob/c439bc73b6a9ba9efa4c8a9b5d2fb6111e660e74/datafusion/sqllogictest/test_files/regexp.slt#L128-L215
 so I think we can simply remove this test
   
   Bonus points would be to port it to .slt
   
   (there is no reason to  leave it in and commented out)



##########
datafusion/proto/src/logical_plan/from_proto.rs:
##########
@@ -536,7 +536,7 @@ impl From<&protobuf::ScalarFunction> for 
BuiltinScalarFunction {
             ScalarFunction::Lpad => Self::Lpad,
             ScalarFunction::Random => Self::Random,
             ScalarFunction::RegexpLike => Self::RegexpLike,
-            ScalarFunction::RegexpMatch => Self::RegexpMatch,
+            //ScalarFunction::RegexpMatch => Self::RegexpMatch,

Review Comment:
   ```suggestion
   ```
   
   I think removing it entirely would be better



##########
datafusion/proto/src/logical_plan/to_proto.rs:
##########
@@ -1518,7 +1518,7 @@ impl TryFrom<&BuiltinScalarFunction> for 
protobuf::ScalarFunction {
             BuiltinScalarFunction::Random => Self::Random,
             BuiltinScalarFunction::Uuid => Self::Uuid,
             BuiltinScalarFunction::RegexpLike => Self::RegexpLike,
-            BuiltinScalarFunction::RegexpMatch => Self::RegexpMatch,
+            //BuiltinScalarFunction::RegexpMatch => Self::RegexpMatch,

Review Comment:
   ```suggestion
   ```



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

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

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

Reply via email to