Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: Add support for RLike #469

Closed
wants to merge 17 commits into from
8 changes: 8 additions & 0 deletions common/src/main/scala/org/apache/comet/CometConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -401,6 +401,14 @@ object CometConf extends ShimCometConf {
.booleanConf
.createWithDefault(false)

val COMET_REGEXP_ALLOW_INCOMPATIBLE: ConfigEntry[Boolean] =
conf("spark.comet.regexp.allowIncompatible")
.doc("Comet is not currently fully compatible with Spark for all regular expressions. " +
"Set this config to true to allow them anyway using Rust's regular expression engine. " +
"See compatibility guide for more information.")
.booleanConf
.createWithDefault(false)

}

object ConfigHelpers {
Expand Down
1 change: 1 addition & 0 deletions core/src/execution/datafusion/expressions/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ pub mod bloom_filter_might_contain;
pub mod correlation;
pub mod covariance;
pub mod negative;
pub mod regexp;
pub mod stats;
pub mod stddev;
pub mod strings;
Expand Down
144 changes: 144 additions & 0 deletions core/src/execution/datafusion/expressions/regexp.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,144 @@
/*
* 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 crate::{errors::CometError, execution::datafusion::expressions::utils::down_cast_any_ref};
use arrow_array::{builder::BooleanBuilder, Array, RecordBatch, StringArray};
use arrow_schema::{DataType, Schema};
use datafusion::logical_expr::ColumnarValue;
use datafusion_common::ScalarValue;
use datafusion_physical_expr::PhysicalExpr;
use regex::Regex;
use std::{
any::Any,
fmt::{Display, Formatter},
hash::Hasher,
sync::Arc,
};

#[derive(Debug, Hash)]
pub struct RLike {
child: Arc<dyn PhysicalExpr>,
pattern: Arc<dyn PhysicalExpr>,
}

impl RLike {
pub fn new(child: Arc<dyn PhysicalExpr>, pattern: Arc<dyn PhysicalExpr>) -> Self {
Self { child, pattern }
}
}

impl Display for RLike {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
write!(
f,
"RLike [child: {}, pattern: {}] ",
self.child, self.pattern
)
}
}

impl PartialEq<dyn Any> for RLike {
fn eq(&self, other: &dyn Any) -> bool {
down_cast_any_ref(other)
.downcast_ref::<Self>()
.map(|x| self.child.eq(&x.child) && self.pattern.eq(&x.pattern))
.unwrap_or(false)
}
}

impl PhysicalExpr for RLike {
kazuyukitanimura marked this conversation as resolved.
Show resolved Hide resolved
fn as_any(&self) -> &dyn Any {
self
}

fn data_type(&self, _input_schema: &Schema) -> datafusion_common::Result<DataType> {
Ok(DataType::Boolean)
}

fn nullable(&self, input_schema: &Schema) -> datafusion_common::Result<bool> {
self.child.nullable(input_schema)
}

fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result<ColumnarValue> {
if let ColumnarValue::Array(v) = self.child.evaluate(batch)? {
if let ColumnarValue::Scalar(ScalarValue::Utf8(Some(pattern))) =
self.pattern.evaluate(batch)?
{
// TODO cache Regex across invocations of evaluate() or create it in constructor
match Regex::new(&pattern) {
Ok(re) => {
let inputs = v
.as_any()
.downcast_ref::<StringArray>()
.expect("string array");
let mut builder = BooleanBuilder::with_capacity(inputs.len());
if inputs.is_nullable() {
for i in 0..inputs.len() {
if inputs.is_null(i) {
builder.append_null();
} else {
builder.append_value(re.is_match(inputs.value(i)));
}
}
} else {
for i in 0..inputs.len() {
builder.append_value(re.is_match(inputs.value(i)));
}
}
Ok(ColumnarValue::Array(Arc::new(builder.finish())))
}
Err(e) => Err(CometError::Internal(format!(
"Failed to compile regular expression: {e:?}"
))
.into()),
}
} else {
Err(
CometError::Internal("Only scalar regex patterns are supported".to_string())
.into(),
)
}
} else {
// this should be unreachable because Spark will evaluate regex expressions against
// literal strings as part of query planning
Err(CometError::Internal("Only columnar inputs are supported".to_string()).into())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it possible to encounter dictionary type?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point. Yes, it probably is.

}
}

fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> {
vec![self.child.clone()]
}

fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn PhysicalExpr>>,
) -> datafusion_common::Result<Arc<dyn PhysicalExpr>> {
assert!(children.len() == 2);
Ok(Arc::new(RLike::new(
children[0].clone(),
children[1].clone(),
)))
}

fn dyn_hash(&self, state: &mut dyn Hasher) {
use std::hash::Hash;
let mut s = state;
self.hash(&mut s);
}
}
7 changes: 7 additions & 0 deletions core/src/execution/datafusion/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ use crate::{
covariance::Covariance,
if_expr::IfExpr,
negative,
regexp::RLike,
scalar_funcs::create_comet_physical_fun,
stats::StatsType,
stddev::Stddev,
Expand Down Expand Up @@ -437,6 +438,12 @@ impl PhysicalPlanner {

Ok(Arc::new(Like::new(left, right)))
}
ExprStruct::Rlike(expr) => {
let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?;
let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?;

Ok(Arc::new(RLike::new(left, right)))
}
ExprStruct::CheckOverflow(expr) => {
let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?;
let data_type = to_arrow_datatype(expr.datatype.as_ref().unwrap());
Expand Down
10 changes: 5 additions & 5 deletions core/src/execution/proto/expr.proto
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ message Expr {
StartsWith startsWith = 27;
EndsWith endsWith = 28;
Contains contains = 29;
// RLike rlike = 30;
RLike rlike = 30;
ScalarFunc scalarFunc = 31;
EqualNullSafe eqNullSafe = 32;
NotEqualNullSafe neqNullSafe = 33;
Expand Down Expand Up @@ -374,10 +374,10 @@ message Like {
Expr right = 2;
}

// message RLike {
// Expr left = 1;
// Expr right = 2;
// }
message RLike {
Expr left = 1;
Expr right = 2;
}

message StartsWith {
Expr left = 1;
Expand Down
7 changes: 7 additions & 0 deletions docs/source/user-guide/compatibility.md
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,13 @@ be used in production.

There is an [epic](https://github.com/apache/datafusion-comet/issues/313) where we are tracking the work to fully implement ANSI support.

## Regular Expressions

Comet uses the [regex](https://crates.io/crates/regex) crate to evaluate regular expressions, and it is expected that
this will produce different results to Java's regular expression engine in some cases. It also lacks support for
features such as backreferences. For these reasons, regular expression support is disabled by default and can be
enabled by setting `spark.comet.regexp.allowIncompatible=true`.

## Cast

Cast operations in Comet fall into three levels of support:
Expand Down
1 change: 1 addition & 0 deletions docs/source/user-guide/configs.md
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ Comet provides the following configuration settings.
| spark.comet.memory.overhead.min | Minimum amount of additional memory to be allocated per executor process for Comet, in MiB. | 402653184b |
| spark.comet.nativeLoadRequired | Whether to require Comet native library to load successfully when Comet is enabled. If not, Comet will silently fallback to Spark when it fails to load the native lib. Otherwise, an error will be thrown and the Spark job will be aborted. | false |
| spark.comet.parquet.enable.directBuffer | Whether to use Java direct byte buffer when reading Parquet. By default, this is false | false |
| spark.comet.regexp.allowIncompatible | Comet is not currently fully compatible with Spark for all regular expressions. Set this config to true to allow them anyway using Rust's regular expression engine. See compatibility guide for more information. | false |
| spark.comet.rowToColumnar.supportedOperatorList | A comma-separated list of row-based operators that will be converted to columnar format when 'spark.comet.rowToColumnar.enabled' is true | Range,InMemoryTableScan |
| spark.comet.scan.enabled | Whether to enable Comet scan. When this is turned on, Spark will use Comet to read Parquet data source. Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. By default, this config is true. | true |
| spark.comet.scan.preFetch.enabled | Whether to enable pre-fetching feature of CometScan. By default is disabled. | false |
Expand Down
2 changes: 1 addition & 1 deletion docs/source/user-guide/expressions.md
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ The following Spark expressions are currently available:
- Substring
- Coalesce
- StringSpace
- Like
- Like/RLike
- Contains
- Startswith
- Endswith
Expand Down
7 changes: 7 additions & 0 deletions docs/templates/compatibility-template.md
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,13 @@ be used in production.

There is an [epic](https://github.com/apache/datafusion-comet/issues/313) where we are tracking the work to fully implement ANSI support.

## Regular Expressions

Comet uses the [regex](https://crates.io/crates/regex) crate to evaluate regular expressions, and it is expected that
this will produce different results to Java's regular expression engine in some cases. It also lacks support for
features such as backreferences. For these reasons, regular expression support is disabled by default and can be
enabled by setting `spark.comet.regexp.allowIncompatible=true`.

## Cast

Cast operations in Comet fall into three levels of support:
Expand Down
58 changes: 40 additions & 18 deletions spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1114,24 +1114,46 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim
None
}

// TODO waiting for arrow-rs update
// case RLike(left, right) =>
Comment on lines -1104 to -1105
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I forgot the details. I remember this was added by someone at the early development. Not sure if we have anything update in arrow-rs for it.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I did look at arrow-rs and there is support there but it seems to have more logic than we need (including some postgres compatibility).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

// val leftExpr = exprToProtoInternal(left, inputs)
// val rightExpr = exprToProtoInternal(right, inputs)
//
// if (leftExpr.isDefined && rightExpr.isDefined) {
// val builder = ExprOuterClass.RLike.newBuilder()
// builder.setLeft(leftExpr.get)
// builder.setRight(rightExpr.get)
//
// Some(
// ExprOuterClass.Expr
// .newBuilder()
// .setRlike(builder)
// .build())
// } else {
// None
// }
case RLike(left, right) =>
// for now, we assume that all regular expressions are incompatible with Spark but
// later we can add logic to determine if a pattern will produce the same results
// in Rust, or even transpile the pattern to work around differences between the JVM
// and Rust regular expression engines
if (CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.get()) {

// we currently only support scalar regex patterns
right match {
case Literal(_, DataTypes.StringType) =>
// supported
case _ =>
withInfo(expr, "Only scalar patterns are supported")
return None
}

val leftExpr = exprToProtoInternal(left, inputs)
val rightExpr = exprToProtoInternal(right, inputs)

if (leftExpr.isDefined && rightExpr.isDefined) {
val builder = ExprOuterClass.RLike.newBuilder()
builder.setLeft(leftExpr.get)
builder.setRight(rightExpr.get)

Some(
ExprOuterClass.Expr
.newBuilder()
.setRlike(builder)
.build())
} else {
withInfo(expr, left, right)
None
}
} else {
withInfo(
expr,
"Regular expressions are disabled. " +
s"Set ${CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key}=true to enable them.")
None
}

case StartsWith(left, right) =>
val leftExpr = exprToProtoInternal(left, inputs)
Expand Down
23 changes: 23 additions & 0 deletions spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -594,6 +594,29 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper {
}
}

test("rlike") {
val table = "rlike_names"
withTable(table) {
sql(s"create table $table(id int, name varchar(20)) using parquet")
sql(s"insert into $table values(1,'James Smith')")
sql(s"insert into $table values(2,'Michael Rose')")
sql(s"insert into $table values(3,'Robert Williams')")
sql(s"insert into $table values(4,'Rames Rose')")
sql(s"insert into $table values(5,'Rames rose')")

withSQLConf(CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key -> "true") {
val query = sql(s"select id from $table where name rlike 'R[a-z]+s [Rr]ose'")
checkSparkAnswerAndOperator(query)

// test that we fall back to Spark if the pattern is not a scalar value
val query2 = sql(s"select id from $table where name rlike name")
val (sparkPlan, cometPlan) = checkSparkAnswer(query2)
val explain = new ExtendedExplainInfo().generateExtendedInfo(cometPlan)
assert(explain == "Only scalar patterns are supported")
}
}
}

test("like with custom escape") {
val table = "names"
withTable(table) {
Expand Down
Loading