diff --git a/datafusion-examples/README.md b/datafusion-examples/README.md index df14db00845d..5b1a5e24853c 100644 --- a/datafusion-examples/README.md +++ b/datafusion-examples/README.md @@ -52,21 +52,22 @@ cargo run --example csv_sql - [`dataframe_output.rs`](examples/dataframe_output.rs): Examples of methods which write data out from a DataFrame - [`dataframe_in_memory.rs`](examples/dataframe_in_memory.rs): Run a query using a DataFrame against data in memory - [`deserialize_to_struct.rs`](examples/deserialize_to_struct.rs): Convert query results into rust structs using serde -- [`expr_api.rs`](examples/expr_api.rs): Create, execute, simplify and anaylze `Expr`s +- [`expr_api.rs`](examples/expr_api.rs): Create, execute, simplify and analyze `Expr`s - [`flight_sql_server.rs`](examples/flight/flight_sql_server.rs): Run DataFusion as a standalone process and execute SQL queries from JDBC clients +- [`make_date.rs`](examples/make_date.rs): Examples of using the make_date function - [`memtable.rs`](examples/memtable.rs): Create an query data in memory using SQL and `RecordBatch`es - [`parquet_sql.rs`](examples/parquet_sql.rs): Build and run a query plan from a SQL statement against a local Parquet file - [`parquet_sql_multiple_files.rs`](examples/parquet_sql_multiple_files.rs): Build and run a query plan from a SQL statement against multiple local Parquet files - [`query-aws-s3.rs`](examples/external_dependency/query-aws-s3.rs): Configure `object_store` and run a query against files stored in AWS S3 - [`query-http-csv.rs`](examples/query-http-csv.rs): Configure `object_store` and run a query against files vi HTTP +- [`regexp.rs`](examples/regexp.rs): Examples of using regular expression functions - [`rewrite_expr.rs`](examples/rewrite_expr.rs): Define and invoke a custom Query Optimizer pass +- [`to_timestamp.rs`](examples/to_timestamp.rs): Examples of using to_timestamp functions - [`simple_udf.rs`](examples/simple_udf.rs): Define and invoke a User Defined Scalar Function (UDF) - [`advanced_udf.rs`](examples/advanced_udf.rs): Define and invoke a more complicated User Defined Scalar Function (UDF) - [`simple_udaf.rs`](examples/simple_udaf.rs): Define and invoke a User Defined Aggregate Function (UDAF) - [`advanced_udaf.rs`](examples/advanced_udaf.rs): Define and invoke a more complicated User Defined Aggregate Function (UDAF) - [`simple_udfw.rs`](examples/simple_udwf.rs): Define and invoke a User Defined Window Function (UDWF) -- [`make_date.rs`](examples/make_date.rs): Examples of using the make_date function -- [`to_timestamp.rs`](examples/to_timestamp.rs): Examples of using the to_timestamp functions - [`advanced_udwf.rs`](examples/advanced_udwf.rs): Define and invoke a more complicated User Defined Window Function (UDWF) ## Distributed diff --git a/datafusion-examples/examples/regexp.rs b/datafusion-examples/examples/regexp.rs new file mode 100644 index 000000000000..02e74bae22af --- /dev/null +++ b/datafusion-examples/examples/regexp.rs @@ -0,0 +1,345 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// 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. + +use datafusion::error::Result; +use datafusion::prelude::*; +use datafusion_common::{assert_batches_eq, assert_contains}; + +/// This example demonstrates how to use the regexp_* functions +/// +/// the full list of supported features and +/// syntax can be found at +/// https://docs.rs/regex/latest/regex/#syntax +/// +/// Supported flags can be found at +/// https://docs.rs/regex/latest/regex/#grouping-and-flags +#[tokio::main] +async fn main() -> Result<()> { + let ctx = SessionContext::new(); + ctx.register_csv( + "examples", + "../../datafusion/physical-expr/tests/data/regex.csv", + CsvReadOptions::new(), + ) + .await?; + + // + // + //regexp_like examples + // + // + // regexp_like format is (regexp_like(text, regex[, flags]) + // + + // use sql and regexp_like function to test col 'values', against patterns in col 'patterns' without flags + let result = ctx + .sql("select regexp_like(values, patterns) from examples") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+------------------------------------------------+", + "| regexp_like(examples.values,examples.patterns) |", + "+------------------------------------------------+", + "| true |", + "| true |", + "| false |", + "| false |", + "| false |", + "| false |", + "| true |", + "| true |", + "| true |", + "| true |", + "| true |", + "+------------------------------------------------+", + ], + &result + ); + + // use sql and regexp_like function to test col 'values', against patterns in col 'patterns' with flags + let result = ctx + .sql("select regexp_like(values, patterns, flags) from examples") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+---------------------------------------------------------------+", + "| regexp_like(examples.values,examples.patterns,examples.flags) |", + "+---------------------------------------------------------------+", + "| true |", + "| true |", + "| true |", + "| false |", + "| false |", + "| false |", + "| true |", + "| true |", + "| true |", + "| true |", + "| true |", + "+---------------------------------------------------------------+", + ], + &result + ); + + // literals work as well + // to match against the entire input use ^ and $ in the regex + let result = ctx + .sql("select regexp_like('John Smith', '^.*Smith$'), regexp_like('Smith Jones', '^Smith.*$')") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+---------------------------------------------------+----------------------------------------------------+", + "| regexp_like(Utf8(\"John Smith\"),Utf8(\"^.*Smith$\")) | regexp_like(Utf8(\"Smith Jones\"),Utf8(\"^Smith.*$\")) |", + "+---------------------------------------------------+----------------------------------------------------+", + "| true | true |", + "+---------------------------------------------------+----------------------------------------------------+", + ], + &result + ); + + // look-around and back references are not supported for performance + // reasons. + // Note that an error may not always be returned but the result + // if returned will always be false + let result = ctx + .sql(r"select regexp_like('(?<=[A-Z]\w )Smith', 'John Smith', 'i') as a") + .await? + .collect() + .await; + + assert!(result.is_ok()); + let result = result.unwrap(); + assert_eq!(result.len(), 1); + + assert_batches_eq!( + &[ + "+-------+", + "| a |", + "+-------+", + "| false |", + "+-------+", + ], + &result + ); + + // invalid flags will result in an error + let result = ctx + .sql(r"select regexp_like('\b4(?!000)\d\d\d\b', 4010, 'g')") + .await? + .collect() + .await; + + let expected = "regexp_like() does not support the \"global\" option"; + assert_contains!(result.unwrap_err().to_string(), expected); + + // there is a size limit on the regex during regex compilation + let result = ctx + .sql("select regexp_like('aaaaa', 'a{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}')") + .await? + .collect() + .await; + + let expected = "Regular expression did not compile: CompiledTooBig"; + assert_contains!(result.unwrap_err().to_string(), expected); + + // + // + //regexp_match examples + // + // + // regexp_match format is (regexp_match(text, regex[, flags]) + // + + let _ = ctx.table("examples").await?; + + // use sql and regexp_match function to test col 'values', against patterns in col 'patterns' without flags + let result = ctx + .sql("select regexp_match(values, patterns) from examples") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+-------------------------------------------------+", + "| regexp_match(examples.values,examples.patterns) |", + "+-------------------------------------------------+", + "| [a] |", + "| [A] |", + "| |", + "| |", + "| |", + "| |", + "| [010] |", + "| [Düsseldorf] |", + "| [Москва] |", + "| [Köln] |", + "| [اليوم] |", + "+-------------------------------------------------+", + ], + &result + ); + + // use dataframe and regexp_match function to test col 'values', against patterns in col 'patterns' with flags + let result = ctx + .sql("select regexp_match(values, patterns, flags) from examples") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+----------------------------------------------------------------+", + "| regexp_match(examples.values,examples.patterns,examples.flags) |", + "+----------------------------------------------------------------+", + "| [a] |", + "| [A] |", + "| [B] |", + "| |", + "| |", + "| |", + "| [010] |", + "| [Düsseldorf] |", + "| [Москва] |", + "| [Köln] |", + "| [اليوم] |", + "+----------------------------------------------------------------+", + ], + &result + ); + + // literals work as well + // to match against the entire input use ^ and $ in the regex + let result = ctx + .sql("select regexp_match('John Smith', '^.*Smith$'), regexp_match('Smith Jones', '^Smith.*$')") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+----------------------------------------------------+-----------------------------------------------------+", + "| regexp_match(Utf8(\"John Smith\"),Utf8(\"^.*Smith$\")) | regexp_match(Utf8(\"Smith Jones\"),Utf8(\"^Smith.*$\")) |", + "+----------------------------------------------------+-----------------------------------------------------+", + "| [John Smith] | [Smith Jones] |", + "+----------------------------------------------------+-----------------------------------------------------+", + ], + &result + ); + + // + // + //regexp_replace examples + // + // + // regexp_replace format is (regexp_replace(text, regex, replace[, flags]) + // + + // use regexp_replace function against tables + let result = ctx + .sql("SELECT regexp_replace(values, patterns, replacement, concat('g', flags)) FROM examples") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+---------------------------------------------------------------------------------------------------------+", + "| regexp_replace(examples.values,examples.patterns,examples.replacement,concat(Utf8(\"g\"),examples.flags)) |", + "+---------------------------------------------------------------------------------------------------------+", + "| bbabbbc |", + "| B |", + "| aec |", + "| AbC |", + "| aBC |", + "| 4000 |", + "| xyz |", + "| München |", + "| Moscow |", + "| Koln |", + "| Today |", + "+---------------------------------------------------------------------------------------------------------+", + ], + &result + ); + + // global flag example + let result = ctx + .sql("SELECT regexp_replace('foobarbaz', 'b(..)', 'X\\1Y', 'g')") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+------------------------------------------------------------------------+", + "| regexp_replace(Utf8(\"foobarbaz\"),Utf8(\"b(..)\"),Utf8(\"X\\1Y\"),Utf8(\"g\")) |", + "+------------------------------------------------------------------------+", + "| fooXarYXazY |", + "+------------------------------------------------------------------------+", + ], + &result + ); + + // without global flag + let result = ctx + .sql("SELECT regexp_replace('foobarbaz', 'b(..)', 'X\\1Y')") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+--------------------------------------------------------------+", + "| regexp_replace(Utf8(\"foobarbaz\"),Utf8(\"b(..)\"),Utf8(\"X\\1Y\")) |", + "+--------------------------------------------------------------+", + "| fooXarYbaz |", + "+--------------------------------------------------------------+", + ], + &result + ); + + // null regex means null result + let result = ctx + .sql("SELECT regexp_replace('foobarbaz', NULL, 'X\\1Y', 'g')") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+---------------------------------------------------------------+", + "| regexp_replace(Utf8(\"foobarbaz\"),NULL,Utf8(\"X\\1Y\"),Utf8(\"g\")) |", + "+---------------------------------------------------------------+", + "| |", + "+---------------------------------------------------------------+", + ], + &result + ); + + Ok(()) +} diff --git a/datafusion/core/tests/dataframe/dataframe_functions.rs b/datafusion/core/tests/dataframe/dataframe_functions.rs index 486ea712edeb..8e00366d2b4f 100644 --- a/datafusion/core/tests/dataframe/dataframe_functions.rs +++ b/datafusion/core/tests/dataframe/dataframe_functions.rs @@ -434,6 +434,27 @@ async fn test_fn_md5() -> Result<()> { Ok(()) } +#[tokio::test] +#[cfg(feature = "unicode_expressions")] +async fn test_fn_regexp_like() -> Result<()> { + let expr = regexp_like(vec![col("a"), lit("[a-z]")]); + + let expected = [ + "+-----------------------------------+", + "| regexp_like(test.a,Utf8(\"[a-z]\")) |", + "+-----------------------------------+", + "| true |", + "| true |", + "| true |", + "| true |", + "+-----------------------------------+", + ]; + + assert_fn_batches!(expr, expected); + + Ok(()) +} + #[tokio::test] #[cfg(feature = "unicode_expressions")] async fn test_fn_regexp_match() -> Result<()> { diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 4cdf0c4a11dd..9f0d5d776297 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -241,6 +241,10 @@ pub enum BuiltinScalarFunction { OctetLength, /// random Random, + /// regexp_like + RegexpLike, + /// regexp_match + RegexpMatch, /// regexp_replace RegexpReplace, /// repeat @@ -303,8 +307,6 @@ pub enum BuiltinScalarFunction { Upper, /// uuid Uuid, - /// regexp_match - RegexpMatch, /// arrow_typeof ArrowTypeof, /// overlay @@ -460,6 +462,8 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::NullIf => Volatility::Immutable, BuiltinScalarFunction::OctetLength => Volatility::Immutable, BuiltinScalarFunction::Radians => Volatility::Immutable, + BuiltinScalarFunction::RegexpLike => Volatility::Immutable, + BuiltinScalarFunction::RegexpMatch => Volatility::Immutable, BuiltinScalarFunction::RegexpReplace => Volatility::Immutable, BuiltinScalarFunction::Repeat => Volatility::Immutable, BuiltinScalarFunction::Replace => Volatility::Immutable, @@ -487,7 +491,6 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::Translate => Volatility::Immutable, BuiltinScalarFunction::Trim => Volatility::Immutable, BuiltinScalarFunction::Upper => Volatility::Immutable, - BuiltinScalarFunction::RegexpMatch => Volatility::Immutable, BuiltinScalarFunction::Struct => Volatility::Immutable, BuiltinScalarFunction::FromUnixtime => Volatility::Immutable, BuiltinScalarFunction::ArrowTypeof => Volatility::Immutable, @@ -819,13 +822,22 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::Upper => { utf8_to_str_type(&input_expr_types[0], "upper") } + BuiltinScalarFunction::RegexpLike => Ok(match input_expr_types[0] { + LargeUtf8 | Utf8 => Boolean, + Null => Null, + _ => { + return plan_err!( + "The regexp_like function can only accept strings." + ); + } + }), BuiltinScalarFunction::RegexpMatch => Ok(match input_expr_types[0] { LargeUtf8 => List(Arc::new(Field::new("item", LargeUtf8, true))), Utf8 => List(Arc::new(Field::new("item", Utf8, true))), Null => Null, _ => { return plan_err!( - "The regexp_extract function can only accept strings." + "The regexp_match function can only accept strings." ); } }), @@ -1230,17 +1242,15 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::Replace | BuiltinScalarFunction::Translate => { Signature::one_of(vec![Exact(vec![Utf8, Utf8, Utf8])], self.volatility()) } - BuiltinScalarFunction::RegexpReplace => Signature::one_of( + BuiltinScalarFunction::RegexpLike => Signature::one_of( vec![ + Exact(vec![Utf8, Utf8]), + Exact(vec![LargeUtf8, Utf8]), Exact(vec![Utf8, Utf8, Utf8]), - Exact(vec![Utf8, Utf8, Utf8, Utf8]), + Exact(vec![LargeUtf8, Utf8, Utf8]), ], self.volatility(), ), - - BuiltinScalarFunction::NullIf => { - Signature::uniform(2, SUPPORTED_NULLIF_TYPES.to_vec(), self.volatility()) - } BuiltinScalarFunction::RegexpMatch => Signature::one_of( vec![ Exact(vec![Utf8, Utf8]), @@ -1250,6 +1260,17 @@ impl BuiltinScalarFunction { ], self.volatility(), ), + BuiltinScalarFunction::RegexpReplace => Signature::one_of( + vec![ + Exact(vec![Utf8, Utf8, Utf8]), + Exact(vec![Utf8, Utf8, Utf8, Utf8]), + ], + self.volatility(), + ), + + BuiltinScalarFunction::NullIf => { + Signature::uniform(2, SUPPORTED_NULLIF_TYPES.to_vec(), self.volatility()) + } BuiltinScalarFunction::Pi => Signature::exact(vec![], self.volatility()), BuiltinScalarFunction::Random => Signature::exact(vec![], self.volatility()), BuiltinScalarFunction::Uuid => Signature::exact(vec![], self.volatility()), @@ -1491,6 +1512,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::FindInSet => &["find_in_set"], // regex functions + BuiltinScalarFunction::RegexpLike => &["regexp_like"], BuiltinScalarFunction::RegexpMatch => &["regexp_match"], BuiltinScalarFunction::RegexpReplace => &["regexp_replace"], diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 877066aabfed..8ed4fd3de2a3 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -859,15 +859,20 @@ nary_scalar_expr!( "fill up a string to the length by appending the characters" ); nary_scalar_expr!( - RegexpReplace, - regexp_replace, - "replace strings that match a regular expression" + RegexpLike, + regexp_like, + "matches a regular expression against a string and returns true or false if there was at least one match or not" ); nary_scalar_expr!( RegexpMatch, regexp_match, "matches a regular expression against a string and returns matched substrings." ); +nary_scalar_expr!( + RegexpReplace, + regexp_replace, + "replace strings that match a regular expression" +); nary_scalar_expr!( Btrim, btrim, @@ -1385,6 +1390,8 @@ mod test { test_scalar_expr!(Ltrim, ltrim, string); test_scalar_expr!(MD5, md5, string); test_scalar_expr!(OctetLength, octet_length, string); + test_nary_scalar_expr!(RegexpLike, regexp_like, string, pattern); + test_nary_scalar_expr!(RegexpLike, regexp_like, string, pattern, flags); test_nary_scalar_expr!(RegexpMatch, regexp_match, string, pattern); test_nary_scalar_expr!(RegexpMatch, regexp_match, string, pattern, flags); test_nary_scalar_expr!( diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index 029f4565c04a..509fe778cc93 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -74,8 +74,10 @@ uuid = { version = "^1.2", features = ["v4"] } [dev-dependencies] criterion = "0.5" +datafusion = { workspace = true } rand = { workspace = true } rstest = { workspace = true } +tokio = { version = "1.28", features = ["rt-multi-thread"] } [[bench]] harness = false @@ -88,3 +90,7 @@ name = "make_date" [[bench]] harness = false name = "to_timestamp" + +[[bench]] +harness = false +name = "regexp" diff --git a/datafusion/physical-expr/benches/regexp.rs b/datafusion/physical-expr/benches/regexp.rs new file mode 100644 index 000000000000..0371b6bf28a9 --- /dev/null +++ b/datafusion/physical-expr/benches/regexp.rs @@ -0,0 +1,133 @@ +// 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. + +extern crate criterion; + +use std::iter; +use std::sync::Arc; + +use arrow_array::builder::StringBuilder; +use arrow_array::{ArrayRef, StringArray}; +use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use rand::distributions::Alphanumeric; +use rand::rngs::ThreadRng; +use rand::seq::SliceRandom; +use rand::Rng; + +use datafusion_physical_expr::regex_expressions::{ + regexp_like, regexp_match, regexp_replace, +}; + +fn data(rng: &mut ThreadRng) -> StringArray { + let mut data: Vec = vec![]; + for _ in 0..1000 { + data.push( + rng.sample_iter(&Alphanumeric) + .take(7) + .map(char::from) + .collect(), + ); + } + + StringArray::from(data) +} + +fn regex(rng: &mut ThreadRng) -> StringArray { + let samples = vec![ + ".*([A-Z]{1}).*".to_string(), + "^(A).*".to_string(), + r#"[\p{Letter}-]+"#.to_string(), + r#"[\p{L}-]+"#.to_string(), + "[a-zA-Z]_[a-zA-Z]{2}".to_string(), + ]; + let mut data: Vec = vec![]; + for _ in 0..1000 { + data.push(samples.choose(rng).unwrap().to_string()); + } + + StringArray::from(data) +} + +fn flags(rng: &mut ThreadRng) -> StringArray { + let samples = vec![Some("i".to_string()), Some("im".to_string()), None]; + let mut sb = StringBuilder::new(); + for _ in 0..1000 { + let sample = samples.choose(rng).unwrap(); + if sample.is_some() { + sb.append_value(sample.clone().unwrap()); + } else { + sb.append_null(); + } + } + + sb.finish() +} + +fn criterion_benchmark(c: &mut Criterion) { + c.bench_function("regexp_like_1000", |b| { + let mut rng = rand::thread_rng(); + let data = Arc::new(data(&mut rng)) as ArrayRef; + let regex = Arc::new(regex(&mut rng)) as ArrayRef; + let flags = Arc::new(flags(&mut rng)) as ArrayRef; + + b.iter(|| { + black_box( + regexp_like::(&[data.clone(), regex.clone(), flags.clone()]) + .expect("regexp_like should work on valid values"), + ) + }) + }); + + c.bench_function("regexp_match_1000", |b| { + let mut rng = rand::thread_rng(); + let data = Arc::new(data(&mut rng)) as ArrayRef; + let regex = Arc::new(regex(&mut rng)) as ArrayRef; + let flags = Arc::new(flags(&mut rng)) as ArrayRef; + + b.iter(|| { + black_box( + regexp_match::(&[data.clone(), regex.clone(), flags.clone()]) + .expect("regexp_match should work on valid values"), + ) + }) + }); + + c.bench_function("regexp_replace_1000", |b| { + let mut rng = rand::thread_rng(); + let data = Arc::new(data(&mut rng)) as ArrayRef; + let regex = Arc::new(regex(&mut rng)) as ArrayRef; + let flags = Arc::new(flags(&mut rng)) as ArrayRef; + let replacement = + Arc::new(StringArray::from_iter_values(iter::repeat("XX").take(1000))) + as ArrayRef; + + b.iter(|| { + black_box( + regexp_replace::(&[ + data.clone(), + regex.clone(), + replacement.clone(), + flags.clone(), + ]) + .expect("regexp_replace should work on valid values"), + ) + }) + }); +} + +criterion_group!(benches, criterion_benchmark); +criterion_main!(benches); diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index cbd780a8fb32..e09c6ab2e3d1 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -608,6 +608,27 @@ pub fn create_physical_fun( _ => unreachable!(), }, }), + BuiltinScalarFunction::RegexpLike => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + let func = invoke_on_array_if_regex_expressions_feature_flag!( + regexp_like, + i32, + "regexp_like" + ); + make_scalar_function_inner(func)(args) + } + DataType::LargeUtf8 => { + let func = invoke_on_array_if_regex_expressions_feature_flag!( + regexp_like, + i64, + "regexp_like" + ); + make_scalar_function_inner(func)(args) + } + other => { + internal_err!("Unsupported data type {other:?} for function regexp_like") + } + }), BuiltinScalarFunction::RegexpMatch => { Arc::new(|args| match args[0].data_type() { DataType::Utf8 => { @@ -1011,7 +1032,7 @@ mod tests { datatypes::Field, record_batch::RecordBatch, }; - use datafusion_common::cast::as_uint64_array; + use datafusion_common::cast::{as_boolean_array, as_uint64_array}; use datafusion_common::{exec_err, plan_err}; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::type_coercion::functions::data_types; @@ -3109,6 +3130,74 @@ mod tests { Ok(()) } + #[test] + #[cfg(feature = "regex_expressions")] + fn test_regexp_like() -> Result<()> { + 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 = vec![col_value]; + let expr = create_physical_expr_with_type_coercion( + &BuiltinScalarFunction::RegexpLike, + &[col("a", &schema)?, pattern], + &schema, + &execution_props, + )?; + + // type is correct + assert_eq!(expr.data_type(&schema)?, DataType::Boolean); + + // 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"); + + let result = as_boolean_array(&result)?; + + // value is correct + assert!(result.value(0)); + + Ok(()) + } + + #[test] + #[cfg(feature = "regex_expressions")] + fn test_regexp_like_all_literals() -> Result<()> { + 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 = vec![Arc::new(Int32Array::from(vec![1]))]; + let expr = create_physical_expr_with_type_coercion( + &BuiltinScalarFunction::RegexpLike, + &[col_value, pattern], + &schema, + &execution_props, + )?; + + // type is correct + assert_eq!(expr.data_type(&schema)?, DataType::Boolean); + + // 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"); + + let result = as_boolean_array(&result)?; + + // value is correct + assert!(result.value(0)); + + Ok(()) + } + #[test] #[cfg(feature = "regex_expressions")] fn test_regexp_match() -> Result<()> { diff --git a/datafusion/physical-expr/src/regex_expressions.rs b/datafusion/physical-expr/src/regex_expressions.rs index 83733da86484..901410cc272c 100644 --- a/datafusion/physical-expr/src/regex_expressions.rs +++ b/datafusion/physical-expr/src/regex_expressions.rs @@ -26,7 +26,7 @@ use arrow::array::{ OffsetSizeTrait, }; -use datafusion_common::{arrow_datafusion_err, plan_err}; +use datafusion_common::{arrow_datafusion_err, exec_err, plan_err}; use datafusion_common::{ cast::as_generic_string_array, internal_err, DataFusionError, Result, }; @@ -54,7 +54,120 @@ macro_rules! fetch_string_arg { }}; } -/// extract a specific group from a string column, using a regular expression +/// Tests a string using a regular expression returning true if at +/// least one match, false otherwise. +/// +/// The full list of supported features and syntax can be found at +/// +/// +/// Supported flags can be found at +/// +/// +/// # Examples +/// +/// ``` +/// # use datafusion::prelude::*; +/// # use datafusion::error::Result; +/// # #[tokio::main] +/// # async fn main() -> Result<()> { +/// let ctx = SessionContext::new(); +/// let df = ctx.read_csv("tests/data/regex.csv", CsvReadOptions::new()).await?; +/// +/// // use the regexp_like function to test col 'values', +/// // against patterns in col 'patterns' without flags +/// let df = df.with_column( +/// "a", +/// regexp_like(vec![col("values"), col("patterns")]) +/// )?; +/// // use the regexp_like function to test col 'values', +/// // against patterns in col 'patterns' with flags +/// let df = df.with_column( +/// "b", +/// regexp_like(vec![col("values"), col("patterns"), col("flags")]) +/// )?; +/// // literals can be used as well with dataframe calls +/// let df = df.with_column( +/// "c", +/// regexp_like(vec![lit("foobarbequebaz"), lit("(bar)(beque)")]) +/// )?; +/// +/// df.show().await?; +/// +/// # Ok(()) +/// # } +/// ``` +pub fn regexp_like(args: &[ArrayRef]) -> Result { + match args.len() { + 2 => { + let values = as_generic_string_array::(&args[0])?; + let regex = as_generic_string_array::(&args[1])?; + let array = arrow_string::regexp::regexp_is_match_utf8(values, regex, None) + .map_err(|e| arrow_datafusion_err!(e))?; + + Ok(Arc::new(array) as ArrayRef) + } + 3 => { + let values = as_generic_string_array::(&args[0])?; + let regex = as_generic_string_array::(&args[1])?; + let flags = as_generic_string_array::(&args[2])?; + + if flags.iter().any(|s| s == Some("g")) { + return plan_err!("regexp_like() does not support the \"global\" option"); + } + + let array = arrow_string::regexp::regexp_is_match_utf8(values, regex, Some(flags)) + .map_err(|e| arrow_datafusion_err!(e))?; + + Ok(Arc::new(array) as ArrayRef) + } + other => exec_err!( + "regexp_like was called with {other} arguments. It requires at least 2 and at most 3." + ), + } +} + +/// Extract a specific group from a string column, using a regular expression. +/// +/// The full list of supported features and syntax can be found at +/// +/// +/// Supported flags can be found at +/// +/// +/// # Examples +/// +/// ``` +/// # use datafusion::prelude::*; +/// # use datafusion::error::Result; +/// # #[tokio::main] +/// # async fn main() -> Result<()> { +/// let ctx = SessionContext::new(); +/// let df = ctx.read_csv("tests/data/regex.csv", CsvReadOptions::new()).await?; +/// +/// // use the regexp_match function to test col 'values', +/// // against patterns in col 'patterns' without flags +/// let df = df.with_column( +/// "a", +/// regexp_match(vec![col("values"), col("patterns")]) +/// )?; +/// // use the regexp_match function to test col 'values', +/// // against patterns in col 'patterns' with flags +/// let df = df.with_column( +/// "b", +/// regexp_match(vec![col("values"), col("patterns"), col("flags")]), +/// )?; +/// +/// // literals can be used as well with dataframe calls +/// let df = df.with_column( +/// "c", +/// regexp_match(vec![lit("foobarbequebaz"), lit("(bar)(beque)")]), +/// )?; +/// +/// df.show().await?; +/// +/// # Ok(()) +/// # } +/// ``` pub fn regexp_match(args: &[ArrayRef]) -> Result { match args.len() { 2 => { @@ -93,9 +206,46 @@ fn regex_replace_posix_groups(replacement: &str) -> String { .into_owned() } -/// Replaces substring(s) matching a POSIX regular expression. +/// Replaces substring(s) matching a PCRE-like regular expression. +/// +/// The full list of supported features and syntax can be found at +/// +/// +/// Supported flags with the addition of 'g' can be found at +/// +/// +/// # Examples +/// +/// ``` +/// # use datafusion::prelude::*; +/// # use datafusion::error::Result; +/// # #[tokio::main] +/// # async fn main() -> Result<()> { +/// let ctx = SessionContext::new(); +/// let df = ctx.read_csv("tests/data/regex.csv", CsvReadOptions::new()).await?; +/// +/// // use the regexp_replace function to replace substring(s) without flags +/// let df = df.with_column( +/// "a", +/// regexp_replace(vec![col("values"), col("patterns"), col("replacement")]) +/// )?; +/// // use the regexp_replace function to replace substring(s) with flags +/// let df = df.with_column( +/// "b", +/// regexp_replace(vec![col("values"), col("patterns"), col("replacement"), col("flags")]), +/// )?; +/// +/// // literals can be used as well +/// let df = df.with_column( +/// "c", +/// regexp_replace(vec![lit("foobarbequebaz"), lit("(bar)(beque)"), lit(r"\2")]), +/// )?; +/// +/// df.show().await?; /// -/// example: `regexp_replace('Thomas', '.[mN]a.', 'M') = 'ThM'` +/// # Ok(()) +/// # } +/// ``` pub fn regexp_replace(args: &[ArrayRef]) -> Result { // Default implementation for regexp_replace, assumes all args are arrays // and args is a sequence of 3 or 4 elements. @@ -117,7 +267,7 @@ pub fn regexp_replace(args: &[ArrayRef]) -> Result (Some(string), Some(pattern), Some(replacement)) => { let replacement = regex_replace_posix_groups(replacement); - // if patterns hashmap already has regexp then use else else create and return + // if patterns hashmap already has regexp then use else create and return let re = match patterns.get(pattern) { Some(re) => Ok(re), None => { @@ -163,7 +313,7 @@ pub fn regexp_replace(args: &[ArrayRef]) -> Result (format!("(?{flags}){pattern}"), false) }; - // if patterns hashmap already has regexp then use else else create and return + // if patterns hashmap already has regexp then use else create and return let re = match patterns.get(&pattern) { Some(re) => Ok(re), None => { @@ -201,7 +351,7 @@ fn _regexp_replace_early_abort( input_array: &GenericStringArray, ) -> Result { // Mimicking the existing behavior of regexp_replace, if any of the scalar arguments - // are actuall null, then the result will be an array of the same size but with nulls. + // are actually null, then the result will be an array of the same size but with nulls. // // Also acts like an early abort mechanism when the input array is empty. Ok(new_null_array(input_array.data_type(), input_array.len())) @@ -332,10 +482,70 @@ pub fn specialize_regexp_replace( #[cfg(test)] mod tests { - use super::*; use arrow::array::*; + use datafusion_common::ScalarValue; + use super::*; + + #[test] + fn test_case_sensitive_regexp_like() { + let values = StringArray::from(vec!["abc"; 5]); + + let patterns = + StringArray::from(vec!["^(a)", "^(A)", "(b|d)", "(B|D)", "^(b|c)"]); + + let mut expected_builder: BooleanBuilder = BooleanBuilder::new(); + expected_builder.append_value(true); + expected_builder.append_value(false); + expected_builder.append_value(true); + expected_builder.append_value(false); + expected_builder.append_value(false); + let expected = expected_builder.finish(); + + let re = regexp_like::(&[Arc::new(values), Arc::new(patterns)]).unwrap(); + + assert_eq!(re.as_ref(), &expected); + } + + #[test] + fn test_case_insensitive_regexp_like() { + let values = StringArray::from(vec!["abc"; 5]); + let patterns = + StringArray::from(vec!["^(a)", "^(A)", "(b|d)", "(B|D)", "^(b|c)"]); + let flags = StringArray::from(vec!["i"; 5]); + + let mut expected_builder: BooleanBuilder = BooleanBuilder::new(); + expected_builder.append_value(true); + expected_builder.append_value(true); + expected_builder.append_value(true); + expected_builder.append_value(true); + expected_builder.append_value(false); + let expected = expected_builder.finish(); + + let re = + regexp_like::(&[Arc::new(values), Arc::new(patterns), Arc::new(flags)]) + .unwrap(); + + assert_eq!(re.as_ref(), &expected); + } + + #[test] + fn test_unsupported_global_flag_regexp_like() { + let values = StringArray::from(vec!["abc"]); + let patterns = StringArray::from(vec!["^(a)"]); + let flags = StringArray::from(vec!["g"]); + + let re_err = + regexp_like::(&[Arc::new(values), Arc::new(patterns), Arc::new(flags)]) + .expect_err("unsupported flag should have failed"); + + assert_eq!( + re_err.strip_backtrace(), + "Error during planning: regexp_like() does not support the \"global\" option" + ); + } + #[test] fn test_case_sensitive_regexp_match() { let values = StringArray::from(vec!["abc"; 5]); diff --git a/datafusion/physical-expr/tests/data/regex.csv b/datafusion/physical-expr/tests/data/regex.csv new file mode 100644 index 000000000000..b249c39522b6 --- /dev/null +++ b/datafusion/physical-expr/tests/data/regex.csv @@ -0,0 +1,12 @@ +values,patterns,replacement,flags +abc,^(a),bb\1bb,i +ABC,^(A).*,B,i +aBc,(b|d),e,i +AbC,(B|D),e, +aBC,^(b|c),d, +4000,\b4([1-9]\d\d|\d[1-9]\d|\d\d[1-9])\b,xyz, +4010,\b4([1-9]\d\d|\d[1-9]\d|\d\d[1-9])\b,xyz, +Düsseldorf,[\p{Letter}-]+,München, +Москва,[\p{L}-]+,Moscow, +Köln,[a-zA-Z]ö[a-zA-Z]{2},Koln, +اليوم,^\p{Arabic}+$,Today, \ No newline at end of file diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 667e53842e56..0781e913919d 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -677,6 +677,7 @@ enum ScalarFunction { InStr = 132; MakeDate = 133; ArrayReverse = 134; + RegexpLike = 135; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 5b7d27d0dff0..f2b383b2e8f9 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -22436,6 +22436,7 @@ impl serde::Serialize for ScalarFunction { Self::InStr => "InStr", Self::MakeDate => "MakeDate", Self::ArrayReverse => "ArrayReverse", + Self::RegexpLike => "RegexpLike", }; serializer.serialize_str(variant) } @@ -22580,6 +22581,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "InStr", "MakeDate", "ArrayReverse", + "RegexpLike", ]; struct GeneratedVisitor; @@ -22753,6 +22755,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "InStr" => Ok(ScalarFunction::InStr), "MakeDate" => Ok(ScalarFunction::MakeDate), "ArrayReverse" => Ok(ScalarFunction::ArrayReverse), + "RegexpLike" => Ok(ScalarFunction::RegexpLike), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index cdf4dadcf894..f421f75a2703 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2764,6 +2764,7 @@ pub enum ScalarFunction { InStr = 132, MakeDate = 133, ArrayReverse = 134, + RegexpLike = 135, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2905,6 +2906,7 @@ impl ScalarFunction { ScalarFunction::InStr => "InStr", ScalarFunction::MakeDate => "MakeDate", ScalarFunction::ArrayReverse => "ArrayReverse", + ScalarFunction::RegexpLike => "RegexpLike", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -3043,6 +3045,7 @@ impl ScalarFunction { "InStr" => Some(Self::InStr), "MakeDate" => Some(Self::MakeDate), "ArrayReverse" => Some(Self::ArrayReverse), + "RegexpLike" => Some(Self::RegexpLike), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 0689da803538..4bc8d5baf143 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -62,12 +62,12 @@ use datafusion_expr::{ instr, isnan, iszero, lcm, left, levenshtein, ln, log, log10, log2, logical_plan::{PlanType, StringifiedPlan}, lower, lpad, ltrim, md5, nanvl, now, nullif, octet_length, overlay, pi, power, - radians, random, regexp_match, regexp_replace, repeat, replace, reverse, right, - round, rpad, rtrim, sha224, sha256, sha384, sha512, signum, sin, sinh, split_part, - sqrt, starts_with, string_to_array, strpos, struct_fun, substr, substr_index, - substring, tan, tanh, to_hex, translate, trim, trunc, upper, uuid, AggregateFunction, - Between, BinaryExpr, BuiltInWindowFunction, BuiltinScalarFunction, Case, Cast, Expr, - GetFieldAccess, GetIndexedField, GroupingSet, + radians, random, regexp_like, regexp_match, regexp_replace, repeat, replace, reverse, + right, round, rpad, rtrim, sha224, sha256, sha384, sha512, signum, sin, sinh, + split_part, sqrt, starts_with, string_to_array, strpos, struct_fun, substr, + substr_index, substring, tan, tanh, to_hex, translate, trim, trunc, upper, uuid, + AggregateFunction, Between, BinaryExpr, BuiltInWindowFunction, BuiltinScalarFunction, + Case, Cast, Expr, GetFieldAccess, GetIndexedField, GroupingSet, GroupingSet::GroupingSets, JoinConstraint, JoinType, Like, Operator, TryCast, WindowFrame, WindowFrameBound, WindowFrameUnits, @@ -538,6 +538,8 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::Left => Self::Left, ScalarFunction::Lpad => Self::Lpad, ScalarFunction::Random => Self::Random, + ScalarFunction::RegexpLike => Self::RegexpLike, + ScalarFunction::RegexpMatch => Self::RegexpMatch, ScalarFunction::RegexpReplace => Self::RegexpReplace, ScalarFunction::Repeat => Self::Repeat, ScalarFunction::Replace => Self::Replace, @@ -561,7 +563,6 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::MakeDate => Self::MakeDate, ScalarFunction::Uuid => Self::Uuid, ScalarFunction::Translate => Self::Translate, - ScalarFunction::RegexpMatch => Self::RegexpMatch, ScalarFunction::Coalesce => Self::Coalesce, ScalarFunction::Pi => Self::Pi, ScalarFunction::Power => Self::Power, @@ -1638,7 +1639,7 @@ pub fn parse_expr( .map(|expr| parse_expr(expr, registry)) .collect::, _>>()?, )), - ScalarFunction::RegexpReplace => Ok(regexp_replace( + ScalarFunction::RegexpLike => Ok(regexp_like( args.to_owned() .iter() .map(|expr| parse_expr(expr, registry)) @@ -1650,6 +1651,12 @@ pub fn parse_expr( .map(|expr| parse_expr(expr, registry)) .collect::, _>>()?, )), + ScalarFunction::RegexpReplace => Ok(regexp_replace( + args.to_owned() + .iter() + .map(|expr| parse_expr(expr, registry)) + .collect::, _>>()?, + )), ScalarFunction::Btrim => Ok(btrim( args.to_owned() .iter() diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 4df7f9fb6bf3..9d520c7d0452 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1518,6 +1518,8 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::Lpad => Self::Lpad, BuiltinScalarFunction::Random => Self::Random, BuiltinScalarFunction::Uuid => Self::Uuid, + BuiltinScalarFunction::RegexpLike => Self::RegexpLike, + BuiltinScalarFunction::RegexpMatch => Self::RegexpMatch, BuiltinScalarFunction::RegexpReplace => Self::RegexpReplace, BuiltinScalarFunction::Repeat => Self::Repeat, BuiltinScalarFunction::Replace => Self::Replace, @@ -1538,7 +1540,6 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::CurrentTime => Self::CurrentTime, BuiltinScalarFunction::MakeDate => Self::MakeDate, BuiltinScalarFunction::Translate => Self::Translate, - BuiltinScalarFunction::RegexpMatch => Self::RegexpMatch, BuiltinScalarFunction::Coalesce => Self::Coalesce, BuiltinScalarFunction::Pi => Self::Pi, BuiltinScalarFunction::Power => Self::Power, diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index 1a2dcf3a91f9..9e4e3aa8185d 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -1570,96 +1570,6 @@ SELECT substring('alphabet' for 1); ---- a -query T -SELECT regexp_replace('ABCabcABC', '(abc)', 'X', 'gi'); ----- -XXX - -query T -SELECT regexp_replace('ABCabcABC', '(abc)', 'X', 'i'); ----- -XabcABC - -query T -SELECT regexp_replace('foobarbaz', 'b..', 'X', 'g'); ----- -fooXX - -query T -SELECT regexp_replace('foobarbaz', 'b..', 'X'); ----- -fooXbaz - -query T -SELECT regexp_replace('foobarbaz', 'b(..)', 'X\\1Y', 'g'); ----- -fooXarYXazY - -query T -SELECT regexp_replace('foobarbaz', 'b(..)', 'X\\1Y', NULL); ----- -NULL - -query T -SELECT regexp_replace('foobarbaz', 'b(..)', NULL, 'g'); ----- -NULL - -query T -SELECT regexp_replace('foobarbaz', NULL, 'X\\1Y', 'g'); ----- -NULL - -query T -SELECT regexp_replace('Thomas', '.[mN]a.', 'M'); ----- -ThM - -query ? -SELECT regexp_replace(NULL, 'b(..)', 'X\\1Y', 'g'); ----- -NULL - -query ? -SELECT regexp_match('foobarbequebaz', ''); ----- -[] - -query ? -SELECT regexp_match('foobarbequebaz', '(bar)(beque)'); ----- -[bar, beque] - -query ? -SELECT regexp_match('foobarbequebaz', '(ba3r)(bequ34e)'); ----- -NULL - -query ? -SELECT regexp_match('aaa-0', '.*-(\d)'); ----- -[0] - -query ? -SELECT regexp_match('bb-1', '.*-(\d)'); ----- -[1] - -query ? -SELECT regexp_match('aa', '.*-(\d)'); ----- -NULL - -query ? -SELECT regexp_match(NULL, '.*-(\d)'); ----- -NULL - -query ? -SELECT regexp_match('aaa-0', NULL); ----- -NULL - ##### csv_query_nullif_divide_by_0 diff --git a/datafusion/sqllogictest/test_files/regexp.slt b/datafusion/sqllogictest/test_files/regexp.slt new file mode 100644 index 000000000000..1e951e2962ff --- /dev/null +++ b/datafusion/sqllogictest/test_files/regexp.slt @@ -0,0 +1,303 @@ +# 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. + +statement ok +CREATE TABLE t (str varchar, pattern varchar, flags varchar) AS VALUES + ('abc', '^(a)', 'i'), + ('ABC', '^(A).*', 'i'), + ('aBc', '(b|d)', 'i'), + ('AbC', '(B|D)', null), + ('aBC', '^(b|c)', null), + ('4000', '\b4([1-9]\d\d|\d[1-9]\d|\d\d[1-9])\b', null), + ('4010', '\b4([1-9]\d\d|\d[1-9]\d|\d\d[1-9])\b', null), + ('Düsseldorf','[\p{Letter}-]+', null), + ('Москва', '[\p{L}-]+', null), + ('Köln', '[a-zA-Z]ö[a-zA-Z]{2}', null), + ('إسرائيل', '^\p{Arabic}+$', null); + +# +# regexp_like tests +# + +query B +SELECT regexp_like(str, pattern, flags) FROM t; +---- +true +true +true +false +false +false +true +true +true +true +true + +query B +SELECT regexp_like('foobarbequebaz', ''); +---- +true + +query B +SELECT regexp_like('', ''); +---- +true + +query B +SELECT regexp_like('foobarbequebaz', '(bar)(beque)'); +---- +true + +query B +SELECT regexp_like('fooBarb +eQuebaz', '(bar).*(que)', 'is'); +---- +true + +query B +SELECT regexp_like('foobarbequebaz', '(ba3r)(bequ34e)'); +---- +false + +query B +SELECT regexp_like('foobarbequebaz', '^.*(barbequ[0-9]*e).*$', 'm'); +---- +true + +query B +SELECT regexp_like('aaa-0', '.*-(\d)'); +---- +true + +query B +SELECT regexp_like('bb-1', '.*-(\d)'); +---- +true + +query B +SELECT regexp_like('aa', '.*-(\d)'); +---- +false + +query ? +SELECT regexp_like(NULL, '.*-(\d)'); +---- +NULL + +query B +SELECT regexp_like('aaa-0', NULL); +---- +NULL + +query ? +SELECT regexp_like(null, '.*-(\d)'); +---- +NULL + +query error Error during planning: regexp_like\(\) does not support the "global" option +SELECT regexp_like('bb-1', '.*-(\d)', 'g'); + +query error Error during planning: regexp_like\(\) does not support the "global" option +SELECT regexp_like('bb-1', '.*-(\d)', 'g'); + +query error Arrow error: Compute error: Regular expression did not compile: CompiledTooBig\(10485760\) +SELECT regexp_like('aaaaa', 'a{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}'); + +# look-around is not supported and will just return false +query B +SELECT regexp_like('(?<=[A-Z]\w )Smith', 'John Smith', 'i'); +---- +false + + +# +# regexp_match tests +# + +query ? +SELECT regexp_match(str, pattern, flags) FROM t; +---- +[a] +[A] +[B] +NULL +NULL +NULL +[010] +[Düsseldorf] +[Москва] +[Köln] +[إسرائيل] + +query ? +SELECT regexp_match('foobarbequebaz', ''); +---- +[] + +query ? +SELECT regexp_match('', ''); +---- +[] + +query ? +SELECT regexp_match('foobarbequebaz', '(bar)(beque)'); +---- +[bar, beque] + +query ? +SELECT regexp_match('fooBarb +eQuebaz', '(bar).*(que)', 'is'); +---- +[Bar, Que] + +query ? +SELECT regexp_match('foobarbequebaz', '(ba3r)(bequ34e)'); +---- +NULL + +query ? +SELECT regexp_match('foobarbequebaz', '^.*(barbequ[0-9]*e).*$', 'm'); +---- +[barbeque] + +query ? +SELECT regexp_match('aaa-0', '.*-(\d)'); +---- +[0] + +query ? +SELECT regexp_match('bb-1', '.*-(\d)'); +---- +[1] + +query ? +SELECT regexp_match('aa', '.*-(\d)'); +---- +NULL + +query ? +SELECT regexp_match(NULL, '.*-(\d)'); +---- +NULL + +query ? +SELECT regexp_match('aaa-0', NULL); +---- +NULL + +query ? +SELECT regexp_match(null, '.*-(\d)'); +---- +NULL + +query error Error during planning: regexp_match\(\) does not support the "global" option +SELECT regexp_match('bb-1', '.*-(\d)', 'g'); + +query error Error during planning: regexp_match\(\) does not support the "global" option +SELECT regexp_match('bb-1', '.*-(\d)', 'g'); + +query error Arrow error: Compute error: Regular expression did not compile: CompiledTooBig\(10485760\) +SELECT regexp_match('aaaaa', 'a{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}{5}'); + +# look-around is not supported and will just return null +query ? +SELECT regexp_match('(?<=[A-Z]\w )Smith', 'John Smith', 'i'); +---- +NULL + +# +# regexp_replace tests +# + +query T +SELECT regexp_replace(str, pattern, 'X', concat('g', flags)) FROM t; +---- +Xbc +X +aXc +AbC +aBC +4000 +X +X +X +X +X + +query T +SELECT regexp_replace('ABCabcABC', '(abc)', 'X', 'gi'); +---- +XXX + +query T +SELECT regexp_replace('ABCabcABC', '(abc)', 'X', 'i'); +---- +XabcABC + +query T +SELECT regexp_replace('foobarbaz', 'b..', 'X', 'g'); +---- +fooXX + +query T +SELECT regexp_replace('foobarbaz', 'b..', 'X'); +---- +fooXbaz + +query T +SELECT regexp_replace('foobarbaz', 'b(..)', 'X\\1Y', 'g'); +---- +fooXarYXazY + +query T +SELECT regexp_replace('foobarbaz', 'b(..)', 'X\\1Y', NULL); +---- +NULL + +query T +SELECT regexp_replace('foobarbaz', 'b(..)', NULL, 'g'); +---- +NULL + +query T +SELECT regexp_replace('foobarbaz', NULL, 'X\\1Y', 'g'); +---- +NULL + +query T +SELECT regexp_replace('Thomas', '.[mN]a.', 'M'); +---- +ThM + +query ? +SELECT regexp_replace(NULL, 'b(..)', 'X\\1Y', 'g'); +---- +NULL + +query T +SELECT regexp_replace('foobar', 'bar', 'xx', 'gi') +---- +fooxx + +query T +SELECT regexp_replace(arrow_cast('foobar', 'Dictionary(Int32, Utf8)'), 'bar', 'xx', 'gi') +---- +fooxx + +statement ok +drop table t; \ No newline at end of file diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index bbfbc6a8bd3e..89405f29ac30 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1259,18 +1259,71 @@ decode(expression, format) ## Regular Expression Functions -Apache DataFusion uses the POSIX regular expression syntax and -supports the following regular expression functions: +Apache DataFusion uses a [PCRE-like] regular expression [syntax] +(minus support for several features including look-around and backreferences). +The following regular expression functions are supported: +- [regexp_like](#regexp_like) - [regexp_match](#regexp_match) - [regexp_replace](#regexp_replace) +[pcre-like]: https://en.wikibooks.org/wiki/Regular_Expressions/Perl-Compatible_Regular_Expressions +[syntax]: https://docs.rs/regex/latest/regex/#syntax + +### `regexp_like` + +Returns true if a [regular expression] has at least one match in a string, +false otherwise. + +[regular expression]: https://docs.rs/regex/latest/regex/#syntax + +``` +regexp_like(str, regexp[, flags]) +``` + +#### Arguments + +- **str**: String expression to operate on. + Can be a constant, column, or function, and any combination of string operators. +- **regexp**: Regular expression to test against the string expression. + Can be a constant, column, or function. +- **flags**: Optional regular expression flags that control the behavior of the + regular expression. The following flags are supported: + - **i**: case-insensitive: letters match both upper and lower case + - **m**: multi-line mode: ^ and $ match begin/end of line + - **s**: allow . to match \n + - **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used + - **U**: swap the meaning of x* and x*? + +#### Example + +```sql +❯ select regexp_like('Köln', '[a-zA-Z]ö[a-zA-Z]{2}'); ++--------------------------------------------------------+ +| regexp_like(Utf8("Köln"),Utf8("[a-zA-Z]ö[a-zA-Z]{2}")) | ++--------------------------------------------------------+ +| true | ++--------------------------------------------------------+ +❯ SELECT regexp_like('aBc', '(b|d)', 'i'); ++--------------------------------------------------+ +| regexp_like(Utf8("aBc"),Utf8("(b|d)"),Utf8("i")) | ++--------------------------------------------------+ +| true | ++--------------------------------------------------+ +``` + +Additional examples can be found [here] + +[here]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/regexp.rs + ### `regexp_match` -Returns a list of regular expression matches in a string. +Returns a list of [regular expression] matches in a string. + +[regular expression]: https://docs.rs/regex/latest/regex/#syntax ``` -regexp_match(str, regexp) +regexp_match(str, regexp[, flags]) ``` #### Arguments @@ -1279,13 +1332,43 @@ regexp_match(str, regexp) Can be a constant, column, or function, and any combination of string operators. - **regexp**: Regular expression to match against. Can be a constant, column, or function. +- **flags**: Optional regular expression flags that control the behavior of the + regular expression. The following flags are supported: + - **i**: case-insensitive: letters match both upper and lower case + - **m**: multi-line mode: ^ and $ match begin/end of line + - **s**: allow . to match \n + - **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used + - **U**: swap the meaning of x* and x*? + +#### Example + +```sql +❯ select regexp_match('Köln', '[a-zA-Z]ö[a-zA-Z]{2}'); ++---------------------------------------------------------+ +| regexp_match(Utf8("Köln"),Utf8("[a-zA-Z]ö[a-zA-Z]{2}")) | ++---------------------------------------------------------+ +| [Köln] | ++---------------------------------------------------------+ +❯ SELECT regexp_match('aBc', '(b|d)', 'i'); ++---------------------------------------------------+ +| regexp_match(Utf8("aBc"),Utf8("(b|d)"),Utf8("i")) | ++---------------------------------------------------+ +| [B] | ++---------------------------------------------------+ +``` + +Additional examples can be found [here] + +[here]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/regexp.rs ### `regexp_replace` -Replaces substrings in a string that match a regular expression. +Replaces substrings in a string that match a [regular expression]. + +[regular expression]: https://docs.rs/regex/latest/regex/#syntax ``` -regexp_replace(str, regexp, replacement, flags) +regexp_replace(str, regexp, replacement[, flags]) ``` #### Arguments @@ -1296,10 +1379,35 @@ regexp_replace(str, regexp, replacement, flags) Can be a constant, column, or function. - **replacement**: Replacement string expression. Can be a constant, column, or function, and any combination of string operators. -- **flags**: Regular expression flags that control the behavior of the - regular expression. The following flags are supported. - - **g**: (global) Search globally and don't return after the first match. - - **i**: (insensitive) Ignore case when matching. +- **flags**: Optional regular expression flags that control the behavior of the + regular expression. The following flags are supported: + - **g**: (global) Search globally and don't return after the first match + - **i**: case-insensitive: letters match both upper and lower case + - **m**: multi-line mode: ^ and $ match begin/end of line + - **s**: allow . to match \n + - **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used + - **U**: swap the meaning of x* and x*? + +#### Example + +```sql +❯ SELECT regexp_replace('foobarbaz', 'b(..)', 'X\\1Y', 'g'); ++------------------------------------------------------------------------+ +| regexp_replace(Utf8("foobarbaz"),Utf8("b(..)"),Utf8("X\1Y"),Utf8("g")) | ++------------------------------------------------------------------------+ +| fooXarYXazY | ++------------------------------------------------------------------------+ +❯ SELECT regexp_replace('aBc', '(b|d)', 'Ab\\1a', 'i'); ++-------------------------------------------------------------------+ +| regexp_replace(Utf8("aBc"),Utf8("(b|d)"),Utf8("Ab\1a"),Utf8("i")) | ++-------------------------------------------------------------------+ +| aAbBac | ++-------------------------------------------------------------------+ +``` + +Additional examples can be found [here] + +[here]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/regexp.rs ### `position`