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::as_string_view_array;
28use datafusion_common::exec_err;
29use datafusion_common::plan_err;
30use datafusion_common::ScalarValue;
31use datafusion_common::{
32    cast::as_generic_string_array, internal_err, DataFusionError, Result,
33};
34use datafusion_expr::function::Hint;
35use datafusion_expr::ColumnarValue;
36use datafusion_expr::TypeSignature;
37use datafusion_expr::{Documentation, ScalarUDFImpl, Signature, Volatility};
38use datafusion_macros::user_doc;
39use regex::Regex;
40use std::any::Any;
41use std::collections::HashMap;
42use std::sync::{Arc, LazyLock};
43
44#[user_doc(
45    doc_section(label = "Regular Expression Functions"),
46    description = "Replaces substrings in a string that match a [regular expression](https://docs.rs/regex/latest/regex/#syntax).",
47    syntax_example = "regexp_replace(str, regexp, replacement[, flags])",
48    sql_example = r#"```sql
49> select regexp_replace('foobarbaz', 'b(..)', 'X\\1Y', 'g');
50+------------------------------------------------------------------------+
51| regexp_replace(Utf8("foobarbaz"),Utf8("b(..)"),Utf8("X\1Y"),Utf8("g")) |
52+------------------------------------------------------------------------+
53| fooXarYXazY                                                            |
54+------------------------------------------------------------------------+
55SELECT regexp_replace('aBc', '(b|d)', 'Ab\\1a', 'i');
56+-------------------------------------------------------------------+
57| regexp_replace(Utf8("aBc"),Utf8("(b|d)"),Utf8("Ab\1a"),Utf8("i")) |
58+-------------------------------------------------------------------+
59| aAbBac                                                            |
60+-------------------------------------------------------------------+
61```
62Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/regexp.rs)
63"#,
64    standard_argument(name = "str", prefix = "String"),
65    argument(
66        name = "regexp",
67        description = "Regular expression to match against.
68  Can be a constant, column, or function."
69    ),
70    argument(
71        name = "replacement",
72        description = "Replacement string expression to operate on. Can be a constant, column, or function, and any combination of operators."
73    ),
74    argument(
75        name = "flags",
76        description = r#"Optional regular expression flags that control the behavior of the regular expression. The following flags are supported:
77- **g**: (global) Search globally and don't return after the first match        
78- **i**: case-insensitive: letters match both upper and lower case
79- **m**: multi-line mode: ^ and $ match begin/end of line
80- **s**: allow . to match \n
81- **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used
82- **U**: swap the meaning of x* and x*?"#
83    )
84)]
85#[derive(Debug)]
86pub struct RegexpReplaceFunc {
87    signature: Signature,
88}
89impl Default for RegexpReplaceFunc {
90    fn default() -> Self {
91        Self::new()
92    }
93}
94
95impl RegexpReplaceFunc {
96    pub fn new() -> Self {
97        use DataType::*;
98        Self {
99            signature: Signature::one_of(
100                vec![
101                    TypeSignature::Exact(vec![Utf8, Utf8, Utf8]),
102                    TypeSignature::Exact(vec![Utf8View, Utf8, Utf8]),
103                    TypeSignature::Exact(vec![Utf8, Utf8, Utf8, Utf8]),
104                    TypeSignature::Exact(vec![Utf8View, Utf8, Utf8, Utf8]),
105                ],
106                Volatility::Immutable,
107            ),
108        }
109    }
110}
111
112impl ScalarUDFImpl for RegexpReplaceFunc {
113    fn as_any(&self) -> &dyn Any {
114        self
115    }
116
117    fn name(&self) -> &str {
118        "regexp_replace"
119    }
120
121    fn signature(&self) -> &Signature {
122        &self.signature
123    }
124
125    fn return_type(&self, arg_types: &[DataType]) -> Result<DataType> {
126        use DataType::*;
127        Ok(match &arg_types[0] {
128            LargeUtf8 | LargeBinary => LargeUtf8,
129            Utf8 | Binary => Utf8,
130            Utf8View | BinaryView => Utf8View,
131            Null => Null,
132            Dictionary(_, t) => match **t {
133                LargeUtf8 | LargeBinary => LargeUtf8,
134                Utf8 | Binary => Utf8,
135                Null => Null,
136                _ => {
137                    return plan_err!(
138                        "the regexp_replace can only accept strings but got {:?}",
139                        **t
140                    );
141                }
142            },
143            other => {
144                return plan_err!(
145                    "The regexp_replace function can only accept strings. Got {other}"
146                );
147            }
148        })
149    }
150
151    fn invoke_with_args(
152        &self,
153        args: datafusion_expr::ScalarFunctionArgs,
154    ) -> Result<ColumnarValue> {
155        let args = &args.args;
156
157        let len = args
158            .iter()
159            .fold(Option::<usize>::None, |acc, arg| match arg {
160                ColumnarValue::Scalar(_) => acc,
161                ColumnarValue::Array(a) => Some(a.len()),
162            });
163
164        let is_scalar = len.is_none();
165        let result = regexp_replace_func(args);
166        if is_scalar {
167            // If all inputs are scalar, keeps output as scalar
168            let result = result.and_then(|arr| ScalarValue::try_from_array(&arr, 0));
169            result.map(ColumnarValue::Scalar)
170        } else {
171            result.map(ColumnarValue::Array)
172        }
173    }
174
175    fn documentation(&self) -> Option<&Documentation> {
176        self.doc()
177    }
178}
179
180fn regexp_replace_func(args: &[ColumnarValue]) -> Result<ArrayRef> {
181    match args[0].data_type() {
182        DataType::Utf8 => specialize_regexp_replace::<i32>(args),
183        DataType::LargeUtf8 => specialize_regexp_replace::<i64>(args),
184        DataType::Utf8View => specialize_regexp_replace::<i32>(args),
185        other => {
186            internal_err!("Unsupported data type {other:?} for function regexp_replace")
187        }
188    }
189}
190
191/// replace POSIX capture groups (like \1) with Rust Regex group (like ${1})
192/// used by regexp_replace
193fn regex_replace_posix_groups(replacement: &str) -> String {
194    static CAPTURE_GROUPS_RE_LOCK: LazyLock<Regex> =
195        LazyLock::new(|| Regex::new(r"(\\)(\d*)").unwrap());
196    CAPTURE_GROUPS_RE_LOCK
197        .replace_all(replacement, "$${$2}")
198        .into_owned()
199}
200
201/// Replaces substring(s) matching a PCRE-like regular expression.
202///
203/// The full list of supported features and syntax can be found at
204/// <https://docs.rs/regex/latest/regex/#syntax>
205///
206/// Supported flags with the addition of 'g' can be found at
207/// <https://docs.rs/regex/latest/regex/#grouping-and-flags>
208///
209/// # Examples
210///
211/// ```ignore
212/// # use datafusion::prelude::*;
213/// # use datafusion::error::Result;
214/// # #[tokio::main]
215/// # async fn main() -> Result<()> {
216/// let ctx = SessionContext::new();
217/// let df = ctx.read_csv("tests/data/regex.csv", CsvReadOptions::new()).await?;
218///
219/// // use the regexp_replace function to replace substring(s) without flags
220/// let df = df.with_column(
221///     "a",
222///     regexp_replace(vec![col("values"), col("patterns"), col("replacement")])
223/// )?;
224/// // use the regexp_replace function to replace substring(s) with flags
225/// let df = df.with_column(
226///     "b",
227///     regexp_replace(vec![col("values"), col("patterns"), col("replacement"), col("flags")]),
228/// )?;
229///
230/// // literals can be used as well
231/// let df = df.with_column(
232///     "c",
233///     regexp_replace(vec![lit("foobarbequebaz"), lit("(bar)(beque)"), lit(r"\2")]),
234/// )?;
235///
236/// df.show().await?;
237///
238/// # Ok(())
239/// # }
240/// ```
241pub fn regexp_replace<'a, T: OffsetSizeTrait, V, B>(
242    string_array: V,
243    pattern_array: B,
244    replacement_array: B,
245    flags: Option<&ArrayRef>,
246) -> Result<ArrayRef>
247where
248    V: ArrayAccessor<Item = &'a str>,
249    B: 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 {
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) => {
311            let flags_array = as_generic_string_array::<T>(flags)?;
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, $T:ident, $EARLY_ABORT:ident, $ARRAY_SIZE:expr) => {{
402        let array = as_generic_string_array::<$T>($ARG)?;
403        if array.len() == 0 || array.is_null(0) {
404            return $EARLY_ABORT(array, $ARRAY_SIZE);
405        } else {
406            array.value(0)
407        }
408    }};
409}
410/// Special cased regex_replace implementation for the scenario where
411/// the pattern, replacement and flags are static (arrays that are derived
412/// from scalars). This means we can skip regex caching system and basically
413/// hold a single Regex object for the replace operation. This also speeds
414/// up the pre-processing time of the replacement string, since it only
415/// needs to processed once.
416fn _regexp_replace_static_pattern_replace<T: OffsetSizeTrait>(
417    args: &[ArrayRef],
418) -> Result<ArrayRef> {
419    let array_size = args[0].len();
420    let pattern = fetch_string_arg!(
421        &args[1],
422        "pattern",
423        i32,
424        _regexp_replace_early_abort,
425        array_size
426    );
427    let replacement = fetch_string_arg!(
428        &args[2],
429        "replacement",
430        i32,
431        _regexp_replace_early_abort,
432        array_size
433    );
434    let flags = match args.len() {
435        3 => None,
436        4 => Some(fetch_string_arg!(&args[3], "flags", i32, _regexp_replace_early_abort, array_size)),
437        other => {
438            return exec_err!(
439                "regexp_replace was called with {other} arguments. It requires at least 3 and at most 4."
440            )
441        }
442    };
443
444    // Embed the flag (if it exists) into the pattern. Limit will determine
445    // whether this is a global match (as in replace all) or just a single
446    // replace operation.
447    let (pattern, limit) = match flags {
448        Some("g") => (pattern.to_string(), 0),
449        Some(flags) => (
450            format!("(?{}){}", flags.to_string().replace('g', ""), pattern),
451            !flags.contains('g') as usize,
452        ),
453        None => (pattern.to_string(), 1),
454    };
455
456    let re =
457        Regex::new(&pattern).map_err(|err| DataFusionError::External(Box::new(err)))?;
458
459    // Replaces the posix groups in the replacement string
460    // with rust ones.
461    let replacement = regex_replace_posix_groups(replacement);
462
463    let string_array_type = args[0].data_type();
464    match string_array_type {
465        DataType::Utf8 | DataType::LargeUtf8 => {
466            let string_array = as_generic_string_array::<T>(&args[0])?;
467
468            // We are going to create the underlying string buffer from its parts
469            // to be able to re-use the existing null buffer for sparse arrays.
470            let mut vals = BufferBuilder::<u8>::new({
471                let offsets = string_array.value_offsets();
472                (offsets[string_array.len()] - offsets[0])
473                    .to_usize()
474                    .unwrap()
475            });
476            let mut new_offsets = BufferBuilder::<T>::new(string_array.len() + 1);
477            new_offsets.append(T::zero());
478
479            string_array.iter().for_each(|val| {
480                if let Some(val) = val {
481                    let result = re.replacen(val, limit, replacement.as_str());
482                    vals.append_slice(result.as_bytes());
483                }
484                new_offsets.append(T::from_usize(vals.len()).unwrap());
485            });
486
487            let data = ArrayDataBuilder::new(GenericStringArray::<T>::DATA_TYPE)
488                .len(string_array.len())
489                .nulls(string_array.nulls().cloned())
490                .buffers(vec![new_offsets.finish(), vals.finish()])
491                .build()?;
492            let result_array = GenericStringArray::<T>::from(data);
493            Ok(Arc::new(result_array) as ArrayRef)
494        }
495        DataType::Utf8View => {
496            let string_view_array = as_string_view_array(&args[0])?;
497
498            let mut builder = StringViewBuilder::with_capacity(string_view_array.len());
499
500            for val in string_view_array.iter() {
501                if let Some(val) = val {
502                    let result = re.replacen(val, limit, replacement.as_str());
503                    builder.append_value(result);
504                } else {
505                    builder.append_null();
506                }
507            }
508
509            let result = builder.finish();
510            Ok(Arc::new(result) as ArrayRef)
511        }
512        _ => unreachable!(
513            "Invalid data type for regexp_replace: {}",
514            string_array_type
515        ),
516    }
517}
518
519/// Determine which implementation of the regexp_replace to use based
520/// on the given set of arguments.
521pub fn specialize_regexp_replace<T: OffsetSizeTrait>(
522    args: &[ColumnarValue],
523) -> Result<ArrayRef> {
524    // This will serve as a dispatch table where we can
525    // leverage it in order to determine whether the scalarity
526    // of the given set of arguments fits a better specialized
527    // function.
528    let (is_source_scalar, is_pattern_scalar, is_replacement_scalar, is_flags_scalar) = (
529        matches!(args[0], ColumnarValue::Scalar(_)),
530        matches!(args[1], ColumnarValue::Scalar(_)),
531        matches!(args[2], ColumnarValue::Scalar(_)),
532        // The forth argument (flags) is optional; so in the event that
533        // it is not available, we'll claim that it is scalar.
534        matches!(args.get(3), Some(ColumnarValue::Scalar(_)) | None),
535    );
536    let len = args
537        .iter()
538        .fold(Option::<usize>::None, |acc, arg| match arg {
539            ColumnarValue::Scalar(_) => acc,
540            ColumnarValue::Array(a) => Some(a.len()),
541        });
542    let inferred_length = len.unwrap_or(1);
543    match (
544        is_source_scalar,
545        is_pattern_scalar,
546        is_replacement_scalar,
547        is_flags_scalar,
548    ) {
549        // This represents a very hot path for the case where the there is
550        // a single pattern that is being matched against and a single replacement.
551        // This is extremely important to specialize on since it removes the overhead
552        // of DF's in-house regex pattern cache (since there will be at most a single
553        // pattern) and the pre-processing of the same replacement pattern at each
554        // query.
555        //
556        // The flags needs to be a scalar as well since each pattern is actually
557        // constructed with the flags embedded into the pattern itself. This means
558        // even if the pattern itself is scalar, if the flags are an array then
559        // we will create many regexes and it is best to use the implementation
560        // that caches it. If there are no flags, we can simply ignore it here,
561        // and let the specialized function handle it.
562        (_, true, true, true) => {
563            let hints = [
564                Hint::Pad,
565                Hint::AcceptsSingular,
566                Hint::AcceptsSingular,
567                Hint::AcceptsSingular,
568            ];
569            let args = args
570                .iter()
571                .zip(hints.iter().chain(std::iter::repeat(&Hint::Pad)))
572                .map(|(arg, hint)| {
573                    // Decide on the length to expand this scalar to depending
574                    // on the given hints.
575                    let expansion_len = match hint {
576                        Hint::AcceptsSingular => 1,
577                        Hint::Pad => inferred_length,
578                    };
579                    arg.to_array(expansion_len)
580                })
581                .collect::<Result<Vec<_>>>()?;
582            _regexp_replace_static_pattern_replace::<T>(&args)
583        }
584
585        // If there are no specialized implementations, we'll fall back to the
586        // generic implementation.
587        (_, _, _, _) => {
588            let args = args
589                .iter()
590                .map(|arg| arg.to_array(inferred_length))
591                .collect::<Result<Vec<_>>>()?;
592
593            match args[0].data_type() {
594                DataType::Utf8View => {
595                    let string_array = args[0].as_string_view();
596                    let pattern_array = args[1].as_string::<i32>();
597                    let replacement_array = args[2].as_string::<i32>();
598                    regexp_replace::<i32, _, _>(
599                        string_array,
600                        pattern_array,
601                        replacement_array,
602                        args.get(3),
603                    )
604                }
605                DataType::Utf8 => {
606                    let string_array = args[0].as_string::<i32>();
607                    let pattern_array = args[1].as_string::<i32>();
608                    let replacement_array = args[2].as_string::<i32>();
609                    regexp_replace::<i32, _, _>(
610                        string_array,
611                        pattern_array,
612                        replacement_array,
613                        args.get(3),
614                    )
615                }
616                DataType::LargeUtf8 => {
617                    let string_array = args[0].as_string::<i64>();
618                    let pattern_array = args[1].as_string::<i64>();
619                    let replacement_array = args[2].as_string::<i64>();
620                    regexp_replace::<i64, _, _>(
621                        string_array,
622                        pattern_array,
623                        replacement_array,
624                        args.get(3),
625                    )
626                }
627                other => {
628                    exec_err!(
629                        "Unsupported data type {other:?} for function regex_replace"
630                    )
631                }
632            }
633        }
634    }
635}
636#[cfg(test)]
637mod tests {
638    use arrow::array::*;
639
640    use super::*;
641
642    macro_rules! static_pattern_regexp_replace {
643        ($name:ident, $T:ty, $O:ty) => {
644            #[test]
645            fn $name() {
646                let values = vec!["abc", "acd", "abcd1234567890123", "123456789012abc"];
647                let patterns = vec!["b"; 4];
648                let replacement = vec!["foo"; 4];
649                let expected =
650                    vec!["afooc", "acd", "afoocd1234567890123", "123456789012afooc"];
651
652                let values = <$T>::from(values);
653                let patterns = StringArray::from(patterns);
654                let replacements = StringArray::from(replacement);
655                let expected = <$T>::from(expected);
656
657                let re = _regexp_replace_static_pattern_replace::<$O>(&[
658                    Arc::new(values),
659                    Arc::new(patterns),
660                    Arc::new(replacements),
661                ])
662                .unwrap();
663
664                assert_eq!(re.as_ref(), &expected);
665            }
666        };
667    }
668
669    static_pattern_regexp_replace!(string_array, StringArray, i32);
670    static_pattern_regexp_replace!(string_view_array, StringViewArray, i32);
671    static_pattern_regexp_replace!(large_string_array, LargeStringArray, i64);
672
673    macro_rules! static_pattern_regexp_replace_with_flags {
674        ($name:ident, $T:ty, $O: ty) => {
675            #[test]
676            fn $name() {
677                let values = vec![
678                    "abc",
679                    "aBc",
680                    "acd",
681                    "abcd1234567890123",
682                    "aBcd1234567890123",
683                    "123456789012abc",
684                    "123456789012aBc",
685                ];
686                let expected = vec![
687                    "afooc",
688                    "afooc",
689                    "acd",
690                    "afoocd1234567890123",
691                    "afoocd1234567890123",
692                    "123456789012afooc",
693                    "123456789012afooc",
694                ];
695
696                let values = <$T>::from(values);
697                let patterns = StringArray::from(vec!["b"; 7]);
698                let replacements = StringArray::from(vec!["foo"; 7]);
699                let flags = StringArray::from(vec!["i"; 5]);
700                let expected = <$T>::from(expected);
701
702                let re = _regexp_replace_static_pattern_replace::<$O>(&[
703                    Arc::new(values),
704                    Arc::new(patterns),
705                    Arc::new(replacements),
706                    Arc::new(flags),
707                ])
708                .unwrap();
709
710                assert_eq!(re.as_ref(), &expected);
711            }
712        };
713    }
714
715    static_pattern_regexp_replace_with_flags!(string_array_with_flags, StringArray, i32);
716    static_pattern_regexp_replace_with_flags!(
717        string_view_array_with_flags,
718        StringViewArray,
719        i32
720    );
721    static_pattern_regexp_replace_with_flags!(
722        large_string_array_with_flags,
723        LargeStringArray,
724        i64
725    );
726
727    #[test]
728    fn test_static_pattern_regexp_replace_early_abort() {
729        let values = StringArray::from(vec!["abc"; 5]);
730        let patterns = StringArray::from(vec![None::<&str>; 5]);
731        let replacements = StringArray::from(vec!["foo"; 5]);
732        let expected = StringArray::from(vec![None::<&str>; 5]);
733
734        let re = _regexp_replace_static_pattern_replace::<i32>(&[
735            Arc::new(values),
736            Arc::new(patterns),
737            Arc::new(replacements),
738        ])
739        .unwrap();
740
741        assert_eq!(re.as_ref(), &expected);
742    }
743
744    #[test]
745    fn test_static_pattern_regexp_replace_early_abort_when_empty() {
746        let values = StringArray::from(Vec::<Option<&str>>::new());
747        let patterns = StringArray::from(Vec::<Option<&str>>::new());
748        let replacements = StringArray::from(Vec::<Option<&str>>::new());
749        let expected = StringArray::from(Vec::<Option<&str>>::new());
750
751        let re = _regexp_replace_static_pattern_replace::<i32>(&[
752            Arc::new(values),
753            Arc::new(patterns),
754            Arc::new(replacements),
755        ])
756        .unwrap();
757
758        assert_eq!(re.as_ref(), &expected);
759    }
760
761    #[test]
762    fn test_static_pattern_regexp_replace_early_abort_flags() {
763        let values = StringArray::from(vec!["abc"; 5]);
764        let patterns = StringArray::from(vec!["a"; 5]);
765        let replacements = StringArray::from(vec!["foo"; 5]);
766        let flags = StringArray::from(vec![None::<&str>; 5]);
767        let expected = StringArray::from(vec![None::<&str>; 5]);
768
769        let re = _regexp_replace_static_pattern_replace::<i32>(&[
770            Arc::new(values),
771            Arc::new(patterns),
772            Arc::new(replacements),
773            Arc::new(flags),
774        ])
775        .unwrap();
776
777        assert_eq!(re.as_ref(), &expected);
778    }
779
780    #[test]
781    fn test_static_pattern_regexp_replace_pattern_error() {
782        let values = StringArray::from(vec!["abc"; 5]);
783        // Deliberately using an invalid pattern to see how the single pattern
784        // error is propagated on regexp_replace.
785        let patterns = StringArray::from(vec!["["; 5]);
786        let replacements = StringArray::from(vec!["foo"; 5]);
787
788        let re = _regexp_replace_static_pattern_replace::<i32>(&[
789            Arc::new(values),
790            Arc::new(patterns),
791            Arc::new(replacements),
792        ]);
793        let pattern_err = re.expect_err("broken pattern should have failed");
794        assert_eq!(
795            pattern_err.strip_backtrace(),
796            "External error: regex parse error:\n    [\n    ^\nerror: unclosed character class"
797        );
798    }
799
800    #[test]
801    fn test_static_pattern_regexp_replace_with_null_buffers() {
802        let values = StringArray::from(vec![
803            Some("a"),
804            None,
805            Some("b"),
806            None,
807            Some("a"),
808            None,
809            None,
810            Some("c"),
811        ]);
812        let patterns = StringArray::from(vec!["a"; 1]);
813        let replacements = StringArray::from(vec!["foo"; 1]);
814        let expected = StringArray::from(vec![
815            Some("foo"),
816            None,
817            Some("b"),
818            None,
819            Some("foo"),
820            None,
821            None,
822            Some("c"),
823        ]);
824
825        let re = _regexp_replace_static_pattern_replace::<i32>(&[
826            Arc::new(values),
827            Arc::new(patterns),
828            Arc::new(replacements),
829        ])
830        .unwrap();
831
832        assert_eq!(re.as_ref(), &expected);
833        assert_eq!(re.null_count(), 4);
834    }
835
836    #[test]
837    fn test_static_pattern_regexp_replace_with_sliced_null_buffer() {
838        let values = StringArray::from(vec![
839            Some("a"),
840            None,
841            Some("b"),
842            None,
843            Some("a"),
844            None,
845            None,
846            Some("c"),
847        ]);
848        let values = values.slice(2, 5);
849        let patterns = StringArray::from(vec!["a"; 1]);
850        let replacements = StringArray::from(vec!["foo"; 1]);
851        let expected = StringArray::from(vec![Some("b"), None, Some("foo"), None, None]);
852
853        let re = _regexp_replace_static_pattern_replace::<i32>(&[
854            Arc::new(values),
855            Arc::new(patterns),
856            Arc::new(replacements),
857        ])
858        .unwrap();
859        assert_eq!(re.as_ref(), &expected);
860        assert_eq!(re.null_count(), 3);
861    }
862}