datafusion_functions/regex/
regexpreplace.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! Regex expressions
19use arrow::array::ArrayDataBuilder;
20use arrow::array::BufferBuilder;
21use arrow::array::GenericStringArray;
22use arrow::array::StringViewBuilder;
23use arrow::array::{new_null_array, ArrayIter, AsArray};
24use arrow::array::{Array, ArrayRef, OffsetSizeTrait};
25use arrow::array::{ArrayAccessor, StringViewArray};
26use arrow::datatypes::DataType;
27use datafusion_common::cast::{
28    as_large_string_array, as_string_array, as_string_view_array,
29};
30use datafusion_common::exec_err;
31use datafusion_common::plan_err;
32use datafusion_common::ScalarValue;
33use datafusion_common::{
34    cast::as_generic_string_array, internal_err, DataFusionError, Result,
35};
36use datafusion_expr::function::Hint;
37use datafusion_expr::ColumnarValue;
38use datafusion_expr::TypeSignature;
39use datafusion_expr::{Documentation, ScalarUDFImpl, Signature, Volatility};
40use datafusion_macros::user_doc;
41use regex::Regex;
42use std::any::Any;
43use std::collections::HashMap;
44use std::sync::{Arc, LazyLock};
45
46#[user_doc(
47    doc_section(label = "Regular Expression Functions"),
48    description = "Replaces substrings in a string that match a [regular expression](https://docs.rs/regex/latest/regex/#syntax).",
49    syntax_example = "regexp_replace(str, regexp, replacement[, flags])",
50    sql_example = r#"```sql
51> select regexp_replace('foobarbaz', 'b(..)', 'X\\1Y', 'g');
52+------------------------------------------------------------------------+
53| regexp_replace(Utf8("foobarbaz"),Utf8("b(..)"),Utf8("X\1Y"),Utf8("g")) |
54+------------------------------------------------------------------------+
55| fooXarYXazY                                                            |
56+------------------------------------------------------------------------+
57SELECT regexp_replace('aBc', '(b|d)', 'Ab\\1a', 'i');
58+-------------------------------------------------------------------+
59| regexp_replace(Utf8("aBc"),Utf8("(b|d)"),Utf8("Ab\1a"),Utf8("i")) |
60+-------------------------------------------------------------------+
61| aAbBac                                                            |
62+-------------------------------------------------------------------+
63```
64Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/regexp.rs)
65"#,
66    standard_argument(name = "str", prefix = "String"),
67    argument(
68        name = "regexp",
69        description = "Regular expression to match against.
70  Can be a constant, column, or function."
71    ),
72    argument(
73        name = "replacement",
74        description = "Replacement string expression to operate on. Can be a constant, column, or function, and any combination of operators."
75    ),
76    argument(
77        name = "flags",
78        description = r#"Optional regular expression flags that control the behavior of the regular expression. The following flags are supported:
79- **g**: (global) Search globally and don't return after the first match        
80- **i**: case-insensitive: letters match both upper and lower case
81- **m**: multi-line mode: ^ and $ match begin/end of line
82- **s**: allow . to match \n
83- **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used
84- **U**: swap the meaning of x* and x*?"#
85    )
86)]
87#[derive(Debug, PartialEq, Eq, Hash)]
88pub struct RegexpReplaceFunc {
89    signature: Signature,
90}
91impl Default for RegexpReplaceFunc {
92    fn default() -> Self {
93        Self::new()
94    }
95}
96
97impl RegexpReplaceFunc {
98    pub fn new() -> Self {
99        use DataType::*;
100        use TypeSignature::*;
101        Self {
102            signature: Signature::one_of(
103                vec![
104                    Uniform(3, vec![Utf8View, LargeUtf8, Utf8]),
105                    Uniform(4, vec![Utf8View, LargeUtf8, Utf8]),
106                ],
107                Volatility::Immutable,
108            ),
109        }
110    }
111}
112
113impl ScalarUDFImpl for RegexpReplaceFunc {
114    fn as_any(&self) -> &dyn Any {
115        self
116    }
117
118    fn name(&self) -> &str {
119        "regexp_replace"
120    }
121
122    fn signature(&self) -> &Signature {
123        &self.signature
124    }
125
126    fn return_type(&self, arg_types: &[DataType]) -> Result<DataType> {
127        use DataType::*;
128        Ok(match &arg_types[0] {
129            LargeUtf8 | LargeBinary => LargeUtf8,
130            Utf8 | Binary => Utf8,
131            Utf8View | BinaryView => Utf8View,
132            Null => Null,
133            Dictionary(_, t) => match **t {
134                LargeUtf8 | LargeBinary => LargeUtf8,
135                Utf8 | Binary => Utf8,
136                Null => Null,
137                _ => {
138                    return plan_err!(
139                        "the regexp_replace can only accept strings but got {:?}",
140                        **t
141                    );
142                }
143            },
144            other => {
145                return plan_err!(
146                    "The regexp_replace function can only accept strings. Got {other}"
147                );
148            }
149        })
150    }
151
152    fn invoke_with_args(
153        &self,
154        args: datafusion_expr::ScalarFunctionArgs,
155    ) -> Result<ColumnarValue> {
156        let args = &args.args;
157
158        let len = args
159            .iter()
160            .fold(Option::<usize>::None, |acc, arg| match arg {
161                ColumnarValue::Scalar(_) => acc,
162                ColumnarValue::Array(a) => Some(a.len()),
163            });
164
165        let is_scalar = len.is_none();
166        let result = regexp_replace_func(args);
167        if is_scalar {
168            // If all inputs are scalar, keeps output as scalar
169            let result = result.and_then(|arr| ScalarValue::try_from_array(&arr, 0));
170            result.map(ColumnarValue::Scalar)
171        } else {
172            result.map(ColumnarValue::Array)
173        }
174    }
175
176    fn documentation(&self) -> Option<&Documentation> {
177        self.doc()
178    }
179}
180
181fn regexp_replace_func(args: &[ColumnarValue]) -> Result<ArrayRef> {
182    match args[0].data_type() {
183        DataType::Utf8 => specialize_regexp_replace::<i32>(args),
184        DataType::LargeUtf8 => specialize_regexp_replace::<i64>(args),
185        DataType::Utf8View => specialize_regexp_replace::<i32>(args),
186        other => {
187            internal_err!("Unsupported data type {other:?} for function regexp_replace")
188        }
189    }
190}
191
192/// replace POSIX capture groups (like \1) with Rust Regex group (like ${1})
193/// used by regexp_replace
194fn regex_replace_posix_groups(replacement: &str) -> String {
195    static CAPTURE_GROUPS_RE_LOCK: LazyLock<Regex> =
196        LazyLock::new(|| Regex::new(r"(\\)(\d*)").unwrap());
197    CAPTURE_GROUPS_RE_LOCK
198        .replace_all(replacement, "$${$2}")
199        .into_owned()
200}
201
202/// Replaces substring(s) matching a PCRE-like regular expression.
203///
204/// The full list of supported features and syntax can be found at
205/// <https://docs.rs/regex/latest/regex/#syntax>
206///
207/// Supported flags with the addition of 'g' can be found at
208/// <https://docs.rs/regex/latest/regex/#grouping-and-flags>
209///
210/// # Examples
211///
212/// ```ignore
213/// # use datafusion::prelude::*;
214/// # use datafusion::error::Result;
215/// # #[tokio::main]
216/// # async fn main() -> Result<()> {
217/// let ctx = SessionContext::new();
218/// let df = ctx.read_csv("tests/data/regex.csv", CsvReadOptions::new()).await?;
219///
220/// // use the regexp_replace function to replace substring(s) without flags
221/// let df = df.with_column(
222///     "a",
223///     regexp_replace(vec![col("values"), col("patterns"), col("replacement")])
224/// )?;
225/// // use the regexp_replace function to replace substring(s) with flags
226/// let df = df.with_column(
227///     "b",
228///     regexp_replace(vec![col("values"), col("patterns"), col("replacement"), col("flags")]),
229/// )?;
230///
231/// // literals can be used as well
232/// let df = df.with_column(
233///     "c",
234///     regexp_replace(vec![lit("foobarbequebaz"), lit("(bar)(beque)"), lit(r"\2")]),
235/// )?;
236///
237/// df.show().await?;
238///
239/// # Ok(())
240/// # }
241/// ```
242pub fn regexp_replace<'a, T: OffsetSizeTrait, U>(
243    string_array: U,
244    pattern_array: U,
245    replacement_array: U,
246    flags_array: Option<U>,
247) -> Result<ArrayRef>
248where
249    U: ArrayAccessor<Item = &'a str>,
250{
251    // Default implementation for regexp_replace, assumes all args are arrays
252    // and args is a sequence of 3 or 4 elements.
253
254    // creating Regex is expensive so create hashmap for memoization
255    let mut patterns: HashMap<String, Regex> = HashMap::new();
256
257    let datatype = string_array.data_type().to_owned();
258
259    let string_array_iter = ArrayIter::new(string_array);
260    let pattern_array_iter = ArrayIter::new(pattern_array);
261    let replacement_array_iter = ArrayIter::new(replacement_array);
262
263    match flags_array {
264        None => {
265            let result_iter = string_array_iter
266                .zip(pattern_array_iter)
267                .zip(replacement_array_iter)
268                .map(|((string, pattern), replacement)| {
269                    match (string, pattern, replacement) {
270                        (Some(string), Some(pattern), Some(replacement)) => {
271                            let replacement = regex_replace_posix_groups(replacement);
272                            // if patterns hashmap already has regexp then use else create and return
273                            let re = match patterns.get(pattern) {
274                                Some(re) => Ok(re),
275                                None => match Regex::new(pattern) {
276                                    Ok(re) => {
277                                        patterns.insert(pattern.to_string(), re);
278                                        Ok(patterns.get(pattern).unwrap())
279                                    }
280                                    Err(err) => {
281                                        Err(DataFusionError::External(Box::new(err)))
282                                    }
283                                },
284                            };
285
286                            Some(re.map(|re| re.replace(string, replacement.as_str())))
287                                .transpose()
288                        }
289                        _ => Ok(None),
290                    }
291                });
292
293            match datatype {
294                DataType::Utf8 | DataType::LargeUtf8 => {
295                    let result =
296                        result_iter.collect::<Result<GenericStringArray<T>>>()?;
297                    Ok(Arc::new(result) as ArrayRef)
298                }
299                DataType::Utf8View => {
300                    let result = result_iter.collect::<Result<StringViewArray>>()?;
301                    Ok(Arc::new(result) as ArrayRef)
302                }
303                other => {
304                    exec_err!(
305                        "Unsupported data type {other:?} for function regex_replace"
306                    )
307                }
308            }
309        }
310        Some(flags_array) => {
311            let flags_array_iter = ArrayIter::new(flags_array);
312
313            let result_iter = string_array_iter
314                .zip(pattern_array_iter)
315                .zip(replacement_array_iter)
316                .zip(flags_array_iter)
317                .map(|(((string, pattern), replacement), flags)| {
318                    match (string, pattern, replacement, flags) {
319                        (Some(string), Some(pattern), Some(replacement), Some(flags)) => {
320                            let replacement = regex_replace_posix_groups(replacement);
321
322                            // format flags into rust pattern
323                            let (pattern, replace_all) = if flags == "g" {
324                                (pattern.to_string(), true)
325                            } else if flags.contains('g') {
326                                (
327                                    format!(
328                                        "(?{}){}",
329                                        flags.to_string().replace('g', ""),
330                                        pattern
331                                    ),
332                                    true,
333                                )
334                            } else {
335                                (format!("(?{flags}){pattern}"), false)
336                            };
337
338                            // if patterns hashmap already has regexp then use else create and return
339                            let re = match patterns.get(&pattern) {
340                                Some(re) => Ok(re),
341                                None => match Regex::new(pattern.as_str()) {
342                                    Ok(re) => {
343                                        patterns.insert(pattern.clone(), re);
344                                        Ok(patterns.get(&pattern).unwrap())
345                                    }
346                                    Err(err) => {
347                                        Err(DataFusionError::External(Box::new(err)))
348                                    }
349                                },
350                            };
351
352                            Some(re.map(|re| {
353                                if replace_all {
354                                    re.replace_all(string, replacement.as_str())
355                                } else {
356                                    re.replace(string, replacement.as_str())
357                                }
358                            }))
359                            .transpose()
360                        }
361                        _ => Ok(None),
362                    }
363                });
364
365            match datatype {
366                DataType::Utf8 | DataType::LargeUtf8 => {
367                    let result =
368                        result_iter.collect::<Result<GenericStringArray<T>>>()?;
369                    Ok(Arc::new(result) as ArrayRef)
370                }
371                DataType::Utf8View => {
372                    let result = result_iter.collect::<Result<StringViewArray>>()?;
373                    Ok(Arc::new(result) as ArrayRef)
374                }
375                other => {
376                    exec_err!(
377                        "Unsupported data type {other:?} for function regex_replace"
378                    )
379                }
380            }
381        }
382    }
383}
384
385fn _regexp_replace_early_abort<T: ArrayAccessor>(
386    input_array: T,
387    sz: usize,
388) -> Result<ArrayRef> {
389    // Mimicking the existing behavior of regexp_replace, if any of the scalar arguments
390    // are actually null, then the result will be an array of the same size as the first argument with all nulls.
391    //
392    // Also acts like an early abort mechanism when the input array is empty.
393    Ok(new_null_array(input_array.data_type(), sz))
394}
395
396/// Get the first argument from the given string array.
397///
398/// Note: If the array is empty or the first argument is null,
399/// then calls the given early abort function.
400macro_rules! fetch_string_arg {
401    ($ARG:expr, $NAME:expr, $EARLY_ABORT:ident, $ARRAY_SIZE:expr) => {{
402        let string_array_type = ($ARG).data_type();
403        match string_array_type {
404            DataType::Utf8 => {
405                let array = as_string_array($ARG)?;
406                if array.len() == 0 || array.is_null(0) {
407                    return $EARLY_ABORT(array, $ARRAY_SIZE);
408                } else {
409                    array.value(0)
410                }
411            }
412            DataType::LargeUtf8 => {
413                let array = as_large_string_array($ARG)?;
414                if array.len() == 0 || array.is_null(0) {
415                    return $EARLY_ABORT(array, $ARRAY_SIZE);
416                } else {
417                    array.value(0)
418                }
419            }
420            DataType::Utf8View => {
421                let array = as_string_view_array($ARG)?;
422                if array.len() == 0 || array.is_null(0) {
423                    return $EARLY_ABORT(array, $ARRAY_SIZE);
424                } else {
425                    array.value(0)
426                }
427            }
428            _ => unreachable!(
429                "Invalid data type for regexp_replace: {}",
430                string_array_type
431            ),
432        }
433    }};
434}
435/// Special cased regex_replace implementation for the scenario where
436/// the pattern, replacement and flags are static (arrays that are derived
437/// from scalars). This means we can skip regex caching system and basically
438/// hold a single Regex object for the replace operation. This also speeds
439/// up the pre-processing time of the replacement string, since it only
440/// needs to processed once.
441fn _regexp_replace_static_pattern_replace<T: OffsetSizeTrait>(
442    args: &[ArrayRef],
443) -> Result<ArrayRef> {
444    let array_size = args[0].len();
445    let pattern =
446        fetch_string_arg!(&args[1], "pattern", _regexp_replace_early_abort, array_size);
447    let replacement = fetch_string_arg!(
448        &args[2],
449        "replacement",
450        _regexp_replace_early_abort,
451        array_size
452    );
453    let flags = match args.len() {
454        3 => None,
455        4 => Some(fetch_string_arg!(&args[3], "flags", _regexp_replace_early_abort, array_size)),
456        other => {
457            return exec_err!(
458                "regexp_replace was called with {other} arguments. It requires at least 3 and at most 4."
459            )
460        }
461    };
462
463    // Embed the flag (if it exists) into the pattern. Limit will determine
464    // whether this is a global match (as in replace all) or just a single
465    // replace operation.
466    let (pattern, limit) = match flags {
467        Some("g") => (pattern.to_string(), 0),
468        Some(flags) => (
469            format!("(?{}){}", flags.to_string().replace('g', ""), pattern),
470            !flags.contains('g') as usize,
471        ),
472        None => (pattern.to_string(), 1),
473    };
474
475    let re =
476        Regex::new(&pattern).map_err(|err| DataFusionError::External(Box::new(err)))?;
477
478    // Replaces the posix groups in the replacement string
479    // with rust ones.
480    let replacement = regex_replace_posix_groups(replacement);
481
482    let string_array_type = args[0].data_type();
483    match string_array_type {
484        DataType::Utf8 | DataType::LargeUtf8 => {
485            let string_array = as_generic_string_array::<T>(&args[0])?;
486
487            // We are going to create the underlying string buffer from its parts
488            // to be able to re-use the existing null buffer for sparse arrays.
489            let mut vals = BufferBuilder::<u8>::new({
490                let offsets = string_array.value_offsets();
491                (offsets[string_array.len()] - offsets[0])
492                    .to_usize()
493                    .unwrap()
494            });
495            let mut new_offsets = BufferBuilder::<T>::new(string_array.len() + 1);
496            new_offsets.append(T::zero());
497
498            string_array.iter().for_each(|val| {
499                if let Some(val) = val {
500                    let result = re.replacen(val, limit, replacement.as_str());
501                    vals.append_slice(result.as_bytes());
502                }
503                new_offsets.append(T::from_usize(vals.len()).unwrap());
504            });
505
506            let data = ArrayDataBuilder::new(GenericStringArray::<T>::DATA_TYPE)
507                .len(string_array.len())
508                .nulls(string_array.nulls().cloned())
509                .buffers(vec![new_offsets.finish(), vals.finish()])
510                .build()?;
511            let result_array = GenericStringArray::<T>::from(data);
512            Ok(Arc::new(result_array) as ArrayRef)
513        }
514        DataType::Utf8View => {
515            let string_view_array = as_string_view_array(&args[0])?;
516
517            let mut builder = StringViewBuilder::with_capacity(string_view_array.len());
518
519            for val in string_view_array.iter() {
520                if let Some(val) = val {
521                    let result = re.replacen(val, limit, replacement.as_str());
522                    builder.append_value(result);
523                } else {
524                    builder.append_null();
525                }
526            }
527
528            let result = builder.finish();
529            Ok(Arc::new(result) as ArrayRef)
530        }
531        _ => unreachable!(
532            "Invalid data type for regexp_replace: {}",
533            string_array_type
534        ),
535    }
536}
537
538/// Determine which implementation of the regexp_replace to use based
539/// on the given set of arguments.
540pub fn specialize_regexp_replace<T: OffsetSizeTrait>(
541    args: &[ColumnarValue],
542) -> Result<ArrayRef> {
543    // This will serve as a dispatch table where we can
544    // leverage it in order to determine whether the scalarity
545    // of the given set of arguments fits a better specialized
546    // function.
547    let (is_source_scalar, is_pattern_scalar, is_replacement_scalar, is_flags_scalar) = (
548        matches!(args[0], ColumnarValue::Scalar(_)),
549        matches!(args[1], ColumnarValue::Scalar(_)),
550        matches!(args[2], ColumnarValue::Scalar(_)),
551        // The forth argument (flags) is optional; so in the event that
552        // it is not available, we'll claim that it is scalar.
553        matches!(args.get(3), Some(ColumnarValue::Scalar(_)) | None),
554    );
555    let len = args
556        .iter()
557        .fold(Option::<usize>::None, |acc, arg| match arg {
558            ColumnarValue::Scalar(_) => acc,
559            ColumnarValue::Array(a) => Some(a.len()),
560        });
561    let inferred_length = len.unwrap_or(1);
562    match (
563        is_source_scalar,
564        is_pattern_scalar,
565        is_replacement_scalar,
566        is_flags_scalar,
567    ) {
568        // This represents a very hot path for the case where the there is
569        // a single pattern that is being matched against and a single replacement.
570        // This is extremely important to specialize on since it removes the overhead
571        // of DF's in-house regex pattern cache (since there will be at most a single
572        // pattern) and the pre-processing of the same replacement pattern at each
573        // query.
574        //
575        // The flags needs to be a scalar as well since each pattern is actually
576        // constructed with the flags embedded into the pattern itself. This means
577        // even if the pattern itself is scalar, if the flags are an array then
578        // we will create many regexes and it is best to use the implementation
579        // that caches it. If there are no flags, we can simply ignore it here,
580        // and let the specialized function handle it.
581        (_, true, true, true) => {
582            let hints = [
583                Hint::Pad,
584                Hint::AcceptsSingular,
585                Hint::AcceptsSingular,
586                Hint::AcceptsSingular,
587            ];
588            let args = args
589                .iter()
590                .zip(hints.iter().chain(std::iter::repeat(&Hint::Pad)))
591                .map(|(arg, hint)| {
592                    // Decide on the length to expand this scalar to depending
593                    // on the given hints.
594                    let expansion_len = match hint {
595                        Hint::AcceptsSingular => 1,
596                        Hint::Pad => inferred_length,
597                    };
598                    arg.to_array(expansion_len)
599                })
600                .collect::<Result<Vec<_>>>()?;
601            _regexp_replace_static_pattern_replace::<T>(&args)
602        }
603
604        // If there are no specialized implementations, we'll fall back to the
605        // generic implementation.
606        (_, _, _, _) => {
607            let args = args
608                .iter()
609                .map(|arg| arg.to_array(inferred_length))
610                .collect::<Result<Vec<_>>>()?;
611
612            match (
613                args[0].data_type(),
614                args[1].data_type(),
615                args[2].data_type(),
616                args.get(3).map(|a| a.data_type()),
617            ) {
618                (
619                    DataType::Utf8,
620                    DataType::Utf8,
621                    DataType::Utf8,
622                    Some(DataType::Utf8) | None,
623                ) => {
624                    let string_array = args[0].as_string::<i32>();
625                    let pattern_array = args[1].as_string::<i32>();
626                    let replacement_array = args[2].as_string::<i32>();
627                    let flags_array = args.get(3).map(|a| a.as_string::<i32>());
628                    regexp_replace::<i32, _>(
629                        string_array,
630                        pattern_array,
631                        replacement_array,
632                        flags_array,
633                    )
634                }
635                (
636                    DataType::Utf8View,
637                    DataType::Utf8View,
638                    DataType::Utf8View,
639                    Some(DataType::Utf8View) | None,
640                ) => {
641                    let string_array = args[0].as_string_view();
642                    let pattern_array = args[1].as_string_view();
643                    let replacement_array = args[2].as_string_view();
644                    let flags_array = args.get(3).map(|a| a.as_string_view());
645                    regexp_replace::<i32, _>(
646                        string_array,
647                        pattern_array,
648                        replacement_array,
649                        flags_array,
650                    )
651                }
652                (
653                    DataType::LargeUtf8,
654                    DataType::LargeUtf8,
655                    DataType::LargeUtf8,
656                    Some(DataType::LargeUtf8) | None,
657                ) => {
658                    let string_array = args[0].as_string::<i64>();
659                    let pattern_array = args[1].as_string::<i64>();
660                    let replacement_array = args[2].as_string::<i64>();
661                    let flags_array = args.get(3).map(|a| a.as_string::<i64>());
662                    regexp_replace::<i64, _>(
663                        string_array,
664                        pattern_array,
665                        replacement_array,
666                        flags_array,
667                    )
668                }
669                other => {
670                    exec_err!(
671                        "Unsupported data type {other:?} for function regex_replace"
672                    )
673                }
674            }
675        }
676    }
677}
678#[cfg(test)]
679mod tests {
680    use arrow::array::*;
681
682    use super::*;
683
684    macro_rules! static_pattern_regexp_replace {
685        ($name:ident, $T:ty, $O:ty) => {
686            #[test]
687            fn $name() {
688                let values = vec!["abc", "acd", "abcd1234567890123", "123456789012abc"];
689                let patterns = vec!["b"; 4];
690                let replacement = vec!["foo"; 4];
691                let expected =
692                    vec!["afooc", "acd", "afoocd1234567890123", "123456789012afooc"];
693
694                let values = <$T>::from(values);
695                let patterns = <$T>::from(patterns);
696                let replacements = <$T>::from(replacement);
697                let expected = <$T>::from(expected);
698
699                let re = _regexp_replace_static_pattern_replace::<$O>(&[
700                    Arc::new(values),
701                    Arc::new(patterns),
702                    Arc::new(replacements),
703                ])
704                .unwrap();
705
706                assert_eq!(re.as_ref(), &expected);
707            }
708        };
709    }
710
711    static_pattern_regexp_replace!(string_array, StringArray, i32);
712    static_pattern_regexp_replace!(string_view_array, StringViewArray, i32);
713    static_pattern_regexp_replace!(large_string_array, LargeStringArray, i64);
714
715    macro_rules! static_pattern_regexp_replace_with_flags {
716        ($name:ident, $T:ty, $O: ty) => {
717            #[test]
718            fn $name() {
719                let values = vec![
720                    "abc",
721                    "aBc",
722                    "acd",
723                    "abcd1234567890123",
724                    "aBcd1234567890123",
725                    "123456789012abc",
726                    "123456789012aBc",
727                ];
728                let expected = vec![
729                    "afooc",
730                    "afooc",
731                    "acd",
732                    "afoocd1234567890123",
733                    "afoocd1234567890123",
734                    "123456789012afooc",
735                    "123456789012afooc",
736                ];
737
738                let values = <$T>::from(values);
739                let patterns = StringArray::from(vec!["b"; 7]);
740                let replacements = StringArray::from(vec!["foo"; 7]);
741                let flags = StringArray::from(vec!["i"; 5]);
742                let expected = <$T>::from(expected);
743
744                let re = _regexp_replace_static_pattern_replace::<$O>(&[
745                    Arc::new(values),
746                    Arc::new(patterns),
747                    Arc::new(replacements),
748                    Arc::new(flags),
749                ])
750                .unwrap();
751
752                assert_eq!(re.as_ref(), &expected);
753            }
754        };
755    }
756
757    static_pattern_regexp_replace_with_flags!(string_array_with_flags, StringArray, i32);
758    static_pattern_regexp_replace_with_flags!(
759        string_view_array_with_flags,
760        StringViewArray,
761        i32
762    );
763    static_pattern_regexp_replace_with_flags!(
764        large_string_array_with_flags,
765        LargeStringArray,
766        i64
767    );
768
769    #[test]
770    fn test_static_pattern_regexp_replace_early_abort() {
771        let values = StringArray::from(vec!["abc"; 5]);
772        let patterns = StringArray::from(vec![None::<&str>; 5]);
773        let replacements = StringArray::from(vec!["foo"; 5]);
774        let expected = StringArray::from(vec![None::<&str>; 5]);
775
776        let re = _regexp_replace_static_pattern_replace::<i32>(&[
777            Arc::new(values),
778            Arc::new(patterns),
779            Arc::new(replacements),
780        ])
781        .unwrap();
782
783        assert_eq!(re.as_ref(), &expected);
784    }
785
786    #[test]
787    fn test_static_pattern_regexp_replace_early_abort_when_empty() {
788        let values = StringArray::from(Vec::<Option<&str>>::new());
789        let patterns = StringArray::from(Vec::<Option<&str>>::new());
790        let replacements = StringArray::from(Vec::<Option<&str>>::new());
791        let expected = StringArray::from(Vec::<Option<&str>>::new());
792
793        let re = _regexp_replace_static_pattern_replace::<i32>(&[
794            Arc::new(values),
795            Arc::new(patterns),
796            Arc::new(replacements),
797        ])
798        .unwrap();
799
800        assert_eq!(re.as_ref(), &expected);
801    }
802
803    #[test]
804    fn test_static_pattern_regexp_replace_early_abort_flags() {
805        let values = StringArray::from(vec!["abc"; 5]);
806        let patterns = StringArray::from(vec!["a"; 5]);
807        let replacements = StringArray::from(vec!["foo"; 5]);
808        let flags = StringArray::from(vec![None::<&str>; 5]);
809        let expected = StringArray::from(vec![None::<&str>; 5]);
810
811        let re = _regexp_replace_static_pattern_replace::<i32>(&[
812            Arc::new(values),
813            Arc::new(patterns),
814            Arc::new(replacements),
815            Arc::new(flags),
816        ])
817        .unwrap();
818
819        assert_eq!(re.as_ref(), &expected);
820    }
821
822    #[test]
823    fn test_static_pattern_regexp_replace_pattern_error() {
824        let values = StringArray::from(vec!["abc"; 5]);
825        // Deliberately using an invalid pattern to see how the single pattern
826        // error is propagated on regexp_replace.
827        let patterns = StringArray::from(vec!["["; 5]);
828        let replacements = StringArray::from(vec!["foo"; 5]);
829
830        let re = _regexp_replace_static_pattern_replace::<i32>(&[
831            Arc::new(values),
832            Arc::new(patterns),
833            Arc::new(replacements),
834        ]);
835        let pattern_err = re.expect_err("broken pattern should have failed");
836        assert_eq!(
837            pattern_err.strip_backtrace(),
838            "External error: regex parse error:\n    [\n    ^\nerror: unclosed character class"
839        );
840    }
841
842    #[test]
843    fn test_static_pattern_regexp_replace_with_null_buffers() {
844        let values = StringArray::from(vec![
845            Some("a"),
846            None,
847            Some("b"),
848            None,
849            Some("a"),
850            None,
851            None,
852            Some("c"),
853        ]);
854        let patterns = StringArray::from(vec!["a"; 1]);
855        let replacements = StringArray::from(vec!["foo"; 1]);
856        let expected = StringArray::from(vec![
857            Some("foo"),
858            None,
859            Some("b"),
860            None,
861            Some("foo"),
862            None,
863            None,
864            Some("c"),
865        ]);
866
867        let re = _regexp_replace_static_pattern_replace::<i32>(&[
868            Arc::new(values),
869            Arc::new(patterns),
870            Arc::new(replacements),
871        ])
872        .unwrap();
873
874        assert_eq!(re.as_ref(), &expected);
875        assert_eq!(re.null_count(), 4);
876    }
877
878    #[test]
879    fn test_static_pattern_regexp_replace_with_sliced_null_buffer() {
880        let values = StringArray::from(vec![
881            Some("a"),
882            None,
883            Some("b"),
884            None,
885            Some("a"),
886            None,
887            None,
888            Some("c"),
889        ]);
890        let values = values.slice(2, 5);
891        let patterns = StringArray::from(vec!["a"; 1]);
892        let replacements = StringArray::from(vec!["foo"; 1]);
893        let expected = StringArray::from(vec![Some("b"), None, Some("foo"), None, None]);
894
895        let re = _regexp_replace_static_pattern_replace::<i32>(&[
896            Arc::new(values),
897            Arc::new(patterns),
898            Arc::new(replacements),
899        ])
900        .unwrap();
901        assert_eq!(re.as_ref(), &expected);
902        assert_eq!(re.null_count(), 3);
903    }
904}