Skip to content

Commit

Permalink
Regexp_replace support regexp [databricks] (#4063)
Browse files Browse the repository at this point in the history
* Integrate regex parser and transpiler with regexp_replace

Signed-off-by: Andy Grove <andygrove@nvidia.com>

* revert change

* fix shim builds

* fix regression in qa_nightly_test

* update compatibility docs

* fix compilation error in 31xdb shim

* attempt to fix 312db compilation error

* Restore legacy behavior of using GpuStringReplace if the regex pattern is a literal string

* fix typo

* fix import in 301db shim

* Update docs/compatibility.md

Co-authored-by: Jason Lowe <jlowe@nvidia.com>

* Update docs/compatibility.md

Co-authored-by: Jason Lowe <jlowe@nvidia.com>

* Improve willNotWorkOnGpu messages

* Improve willNotWorkOnGpu messages

* Improve willNotWorkOnGpu messages

* update generated docs

* fix regression

* remove unused imports

* remove unused imports

* remove unused imports

* remove unused imports

* remove unused imports

Co-authored-by: Jason Lowe <jlowe@nvidia.com>
  • Loading branch information
andygrove and jlowe authored Nov 17, 2021
1 parent c0f59bd commit ade6591
Show file tree
Hide file tree
Showing 20 changed files with 665 additions and 171 deletions.
41 changes: 31 additions & 10 deletions docs/compatibility.md
Original file line number Diff line number Diff line change
Expand Up @@ -265,22 +265,43 @@ the end of the string. This will be fixed in a future release. The issue is

## Regular Expressions

### regexp_replace
The following Apache Spark regular expression functions and expressions are supported on the GPU:

The RAPIDS Accelerator for Apache Spark currently supports string literal matches, not wildcard
matches for the `regexp_replace` function and will fall back to CPU if a regular expression pattern
is provided.
- `RLIKE`
- `regexp`
- `regexp_like`
- `regexp_replace`

### RLike
These operations are disabled by default because of known incompatibilities between the Java regular expression
engine that Spark uses and the cuDF regular expression engine on the GPU, and also because the regular expression
kernels can potentially have high memory overhead.

The GPU implementation of `RLike` has the following known issues where behavior is not consistent with Apache Spark and
this expression is disabled by default. It can be enabled setting `spark.rapids.sql.expression.RLike=true`.
These operations can be enabled on the GPU with the following configuration settings:

- `$` does not match the end of string if the string ends with a line-terminator
- `spark.rapids.sql.expression.RLike=true` (for `RLIKE`, `regexp`, and `regexp_like`)
- `spark.rapids.sql.expression.RegExpReplace=true` for `regexp_replace`

Even when these expressions are enabled, there are instances where regular expression operations will fall back to
CPU when the RAPIDS Accelerator determines that a pattern is either unsupported or would produce incorrect results on the GPU.

Here are some examples of regular expression patterns that are not supported on the GPU and will fall back to the CPU.

- Lazy quantifiers, such as `a*?`
- Possessive quantifiers, such as `a*+`
- Character classes that use union, intersection, or subtraction semantics, such as `[a-d[m-p]]`, `[a-z&&[def]]`,
or `[a-z&&[^bc]]`
- Word and non-word boundaries, `\b` and `\B`
- Empty groups: `()`
- Regular expressions containing null characters (unless the pattern is a simple literal string)
- Beginning-of-line and end-of-line anchors (`^` and `$`) are not supported in some contexts, such as when combined
- with a choice (`^|a`) or when used anywhere in `regexp_replace` patterns.

In addition to these cases that can be detected, there is also one known issue that can cause incorrect results:

- `$` does not match the end of a string if the string ends with a line-terminator
([cuDF issue #9620](https://github.com/rapidsai/cudf/issues/9620))

`RLike` will fall back to CPU if any regular expressions are detected that are not supported on the GPU
or would produce different results on the GPU.
Work is ongoing to increase the range of regular expressions that can run on the GPU.

## Timestamps

Expand Down
4 changes: 2 additions & 2 deletions docs/configs.md
Original file line number Diff line number Diff line change
Expand Up @@ -259,10 +259,10 @@ Name | SQL Function(s) | Description | Default Value | Notes
<a name="sql.expression.PromotePrecision"></a>spark.rapids.sql.expression.PromotePrecision| |PromotePrecision before arithmetic operations between DecimalType data|true|None|
<a name="sql.expression.PythonUDF"></a>spark.rapids.sql.expression.PythonUDF| |UDF run in an external python process. Does not actually run on the GPU, but the transfer of data to/from it can be accelerated|true|None|
<a name="sql.expression.Quarter"></a>spark.rapids.sql.expression.Quarter|`quarter`|Returns the quarter of the year for date, in the range 1 to 4|true|None|
<a name="sql.expression.RLike"></a>spark.rapids.sql.expression.RLike|`rlike`|RLike|false|This is disabled by default because The GPU implementation of rlike is not compatible with Apache Spark. See the compatibility guide for more information.|
<a name="sql.expression.RLike"></a>spark.rapids.sql.expression.RLike|`rlike`|RLike|false|This is disabled by default because the implementation is not 100% compatible. See the compatibility guide for more information.|
<a name="sql.expression.Rand"></a>spark.rapids.sql.expression.Rand|`random`, `rand`|Generate a random column with i.i.d. uniformly distributed values in [0, 1)|true|None|
<a name="sql.expression.Rank"></a>spark.rapids.sql.expression.Rank|`rank`|Window function that returns the rank value within the aggregation window|true|None|
<a name="sql.expression.RegExpReplace"></a>spark.rapids.sql.expression.RegExpReplace|`regexp_replace`|RegExpReplace support for string literal input patterns|true|None|
<a name="sql.expression.RegExpReplace"></a>spark.rapids.sql.expression.RegExpReplace|`regexp_replace`|RegExpReplace support for string literal input patterns|false|This is disabled by default because the implementation is not 100% compatible. See the compatibility guide for more information.|
<a name="sql.expression.Remainder"></a>spark.rapids.sql.expression.Remainder|`%`, `mod`|Remainder or modulo|true|None|
<a name="sql.expression.Rint"></a>spark.rapids.sql.expression.Rint|`rint`|Rounds up a double value to the nearest double equal to an integer|true|None|
<a name="sql.expression.Round"></a>spark.rapids.sql.expression.Round|`round`|Round an expression to d decimal places using HALF_UP rounding mode|true|None|
Expand Down
6 changes: 3 additions & 3 deletions docs/supported_ops.md
Original file line number Diff line number Diff line change
Expand Up @@ -9638,7 +9638,7 @@ are limited.
<td rowSpan="3">RLike</td>
<td rowSpan="3">`rlike`</td>
<td rowSpan="3">RLike</td>
<td rowSpan="3">This is disabled by default because The GPU implementation of rlike is not compatible with Apache Spark. See the compatibility guide for more information.</td>
<td rowSpan="3">This is disabled by default because the implementation is not 100% compatible. See the compatibility guide for more information.</td>
<td rowSpan="3">project</td>
<td>str</td>
<td> </td>
Expand Down Expand Up @@ -9826,7 +9826,7 @@ are limited.
<td rowSpan="4">RegExpReplace</td>
<td rowSpan="4">`regexp_replace`</td>
<td rowSpan="4">RegExpReplace support for string literal input patterns</td>
<td rowSpan="4">None</td>
<td rowSpan="4">This is disabled by default because the implementation is not 100% compatible. See the compatibility guide for more information.</td>
<td rowSpan="4">project</td>
<td>str</td>
<td> </td>
Expand Down Expand Up @@ -9859,7 +9859,7 @@ are limited.
<td> </td>
<td> </td>
<td> </td>
<td><em>PS<br/>very limited regex support;<br/>Literal value only</em></td>
<td><em>PS<br/>Literal value only</em></td>
<td> </td>
<td> </td>
<td> </td>
Expand Down
3 changes: 2 additions & 1 deletion integration_tests/src/main/python/qa_nightly_select_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,8 @@ def idfn(val):
'spark.rapids.sql.hasNans': 'false',
'spark.rapids.sql.castStringToFloat.enabled': 'true',
'spark.rapids.sql.castFloatToIntegralTypes.enabled': 'true',
'spark.rapids.sql.castFloatToString.enabled': 'true'
'spark.rapids.sql.castFloatToString.enabled': 'true',
'spark.rapids.sql.expression.RegExpReplace': 'true'
}

_first_last_qa_conf = copy_and_update(_qa_conf, {
Expand Down
17 changes: 15 additions & 2 deletions integration_tests/src/main/python/string_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -337,7 +337,8 @@ def test_re_replace():
'REGEXP_REPLACE(a, "TEST", "PROD")',
'REGEXP_REPLACE(a, "TEST", "")',
'REGEXP_REPLACE(a, "TEST", "%^[]\ud720")',
'REGEXP_REPLACE(a, "TEST", NULL)'))
'REGEXP_REPLACE(a, "TEST", NULL)'),
conf={'spark.rapids.sql.expression.RegExpReplace': 'true'})

def test_re_replace_null():
gen = mk_str_gen('[\u0000 ]{0,2}TE[\u0000 ]{0,2}ST[\u0000 ]{0,2}')\
Expand All @@ -356,7 +357,8 @@ def test_re_replace_null():
'REGEXP_REPLACE(a, "\x00", "NULL")',
'REGEXP_REPLACE(a, "\0", "NULL")',
'REGEXP_REPLACE(a, "TE\u0000ST", "PROD")',
'REGEXP_REPLACE(a, "TE\u0000\u0000ST", "PROD")'))
'REGEXP_REPLACE(a, "TE\u0000\u0000ST", "PROD")'),
conf={'spark.rapids.sql.expression.RegExpReplace': 'true'})

def test_length():
gen = mk_str_gen('.{0,5}TEST[\ud720 A]{0,5}')
Expand Down Expand Up @@ -470,6 +472,17 @@ def test_like_complex_escape():
'a like "_oo"'),
conf={'spark.sql.parser.escapedStringLiterals': 'true'})

def test_regexp_replace():
gen = mk_str_gen('[abcd]{0,3}')
assert_gpu_and_cpu_are_equal_collect(
lambda spark: unary_op_df(spark, gen).selectExpr(
'regexp_replace(a, "a", "A")',
'regexp_replace(a, "[^xyz]", "A")',
'regexp_replace(a, "([^x])|([^y])", "A")',
'regexp_replace(a, "(?:aa)+", "A")',
'regexp_replace(a, "a|b|c", "A")'),
conf={'spark.rapids.sql.expression.RegExpReplace': 'true'})

def test_rlike():
gen = mk_str_gen('[abcd]{1,3}')
assert_gpu_and_cpu_are_equal_collect(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
/*
* Copyright (c) 2021, NVIDIA CORPORATION.
*
* Licensed 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.
*/
package com.nvidia.spark.rapids.shims.v2

import com.nvidia.spark.rapids.{CudfRegexTranspiler, DataFromReplacementRule, GpuExpression, GpuOverrides, RapidsConf, RapidsMeta, RegexUnsupportedException, TernaryExprMeta}

import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, RegExpReplace}
import org.apache.spark.sql.rapids.{GpuRegExpReplace, GpuStringReplace}
import org.apache.spark.sql.types.DataTypes
import org.apache.spark.unsafe.types.UTF8String

class GpuRegExpReplaceMeta(
expr: RegExpReplace,
conf: RapidsConf,
parent: Option[RapidsMeta[_, _, _]],
rule: DataFromReplacementRule)
extends TernaryExprMeta[RegExpReplace](expr, conf, parent, rule) {

override def tagExprForGpu(): Unit = {
expr.regexp match {
case Literal(null, _) =>
willNotWorkOnGpu(s"null pattern is not supported on GPU")
case Literal(s: UTF8String, DataTypes.StringType) =>
val pattern = s.toString
if (pattern.isEmpty) {
willNotWorkOnGpu(s"empty pattern is not supported on GPU")
}

if (GpuOverrides.isSupportedStringReplacePattern(expr.regexp)) {
// use GpuStringReplace
} else {
try {
new CudfRegexTranspiler(replace = true).transpile(pattern)
} catch {
case e: RegexUnsupportedException =>
willNotWorkOnGpu(e.getMessage)
}
}

case _ =>
willNotWorkOnGpu(s"non-literal pattern is not supported on GPU")
}
}

override def convertToGpu(
lhs: Expression,
regexp: Expression,
rep: Expression): GpuExpression = {
if (GpuOverrides.isSupportedStringReplacePattern(expr.regexp)) {
GpuStringReplace(lhs, regexp, rep)
} else {
GpuRegExpReplace(lhs, regexp, rep)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ import org.apache.spark.sql.execution.joins._
import org.apache.spark.sql.execution.python._
import org.apache.spark.sql.execution.window.WindowExecBase
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.rapids.{GpuAbs, GpuAverage, GpuFileSourceScanExec, GpuStringReplace, GpuTimeSub}
import org.apache.spark.sql.rapids.{GpuAbs, GpuAverage, GpuFileSourceScanExec, GpuTimeSub}
import org.apache.spark.sql.rapids.execution.{GpuBroadcastNestedLoopJoinExecBase, GpuShuffleExchangeExecBase, JoinTypeChecks, SerializeBatchDeserializeHostBuffer, SerializeConcatHostBuffersDeserializeBatch, TrampolineUtil}
import org.apache.spark.sql.rapids.execution.python._
import org.apache.spark.sql.rapids.execution.python.shims.v2._
Expand Down Expand Up @@ -330,20 +330,11 @@ abstract class SparkBaseShims extends Spark30XShims {
"RegExpReplace support for string literal input patterns",
ExprChecks.projectOnly(TypeSig.STRING, TypeSig.STRING,
Seq(ParamCheck("str", TypeSig.STRING, TypeSig.STRING),
ParamCheck("regex", TypeSig.lit(TypeEnum.STRING)
.withPsNote(TypeEnum.STRING, "very limited regex support"), TypeSig.STRING),
ParamCheck("regex", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING),
ParamCheck("rep", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING))),
(a, conf, p, r) => new TernaryExprMeta[RegExpReplace](a, conf, p, r) {
override def tagExprForGpu(): Unit = {
if (!GpuOverrides.isSupportedStringReplacePattern(a.regexp)) {
willNotWorkOnGpu(
"Only non-null, non-empty String literals that are not regex patterns " +
"are supported by RegExpReplace on the GPU")
}
}
override def convertToGpu(lhs: Expression, regexp: Expression,
rep: Expression): GpuExpression = GpuStringReplace(lhs, regexp, rep)
}),
(a, conf, p, r) => new GpuRegExpReplaceMeta(a, conf, p, r)).disabledByDefault(
"the implementation is not 100% compatible. " +
"See the compatibility guide for more information."),
GpuScalaUDFMeta.exprMeta
).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
/*
* Copyright (c) 2021, NVIDIA CORPORATION.
*
* Licensed 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.
*/
package com.nvidia.spark.rapids.shims.v2

import com.nvidia.spark.rapids._

import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, RegExpReplace}
import org.apache.spark.sql.rapids.{GpuRegExpReplace, GpuStringReplace}
import org.apache.spark.sql.types.DataTypes
import org.apache.spark.unsafe.types.UTF8String

class GpuRegExpReplaceMeta(
expr: RegExpReplace,
conf: RapidsConf,
parent: Option[RapidsMeta[_, _, _]],
rule: DataFromReplacementRule)
extends TernaryExprMeta[RegExpReplace](expr, conf, parent, rule) {

override def tagExprForGpu(): Unit = {
expr.regexp match {
case Literal(null, _) =>
willNotWorkOnGpu(s"null pattern is not supported on GPU")
case Literal(s: UTF8String, DataTypes.StringType) =>
val pattern = s.toString
if (pattern.isEmpty) {
willNotWorkOnGpu(s"empty pattern is not supported on GPU")
}

if (GpuOverrides.isSupportedStringReplacePattern(expr.regexp)) {
// use GpuStringReplace
} else {
try {
new CudfRegexTranspiler(replace = true).transpile(pattern)
} catch {
case e: RegexUnsupportedException =>
willNotWorkOnGpu(e.getMessage)
}
}

case _ =>
willNotWorkOnGpu(s"non-literal pattern is not supported on GPU")
}
}

override def convertToGpu(
lhs: Expression,
regexp: Expression,
rep: Expression): GpuExpression = {
if (GpuOverrides.isSupportedStringReplacePattern(expr.regexp)) {
GpuStringReplace(lhs, regexp, rep)
} else {
GpuRegExpReplace(lhs, regexp, rep)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNes
import org.apache.spark.sql.execution.python.{AggregateInPandasExec, ArrowEvalPythonExec, FlatMapGroupsInPandasExec, MapInPandasExec, WindowInPandasExec}
import org.apache.spark.sql.execution.window.WindowExecBase
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.rapids.{GpuAbs, GpuAverage, GpuFileSourceScanExec, GpuStringReplace, GpuTimeSub}
import org.apache.spark.sql.rapids.{GpuAbs, GpuAverage, GpuFileSourceScanExec, GpuTimeSub}
import org.apache.spark.sql.rapids.execution.{GpuBroadcastNestedLoopJoinExecBase, GpuShuffleExchangeExecBase, JoinTypeChecks, SerializeBatchDeserializeHostBuffer, SerializeConcatHostBuffersDeserializeBatch}
import org.apache.spark.sql.rapids.execution.python._
import org.apache.spark.sql.rapids.execution.python.shims.v2._
Expand Down Expand Up @@ -293,20 +293,11 @@ abstract class SparkBaseShims extends Spark30XShims {
"RegExpReplace support for string literal input patterns",
ExprChecks.projectOnly(TypeSig.STRING, TypeSig.STRING,
Seq(ParamCheck("str", TypeSig.STRING, TypeSig.STRING),
ParamCheck("regex", TypeSig.lit(TypeEnum.STRING)
.withPsNote(TypeEnum.STRING, "very limited regex support"), TypeSig.STRING),
ParamCheck("regex", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING),
ParamCheck("rep", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING))),
(a, conf, p, r) => new TernaryExprMeta[RegExpReplace](a, conf, p, r) {
override def tagExprForGpu(): Unit = {
if (!GpuOverrides.isSupportedStringReplacePattern(a.regexp)) {
willNotWorkOnGpu(
"Only non-null, non-empty String literals that are not regex patterns " +
"are supported by RegExpReplace on the GPU")
}
}
override def convertToGpu(lhs: Expression, regexp: Expression,
rep: Expression): GpuExpression = GpuStringReplace(lhs, regexp, rep)
}),
(a, conf, p, r) => new GpuRegExpReplaceMeta(a, conf, p, r)).disabledByDefault(
"the implementation is not 100% compatible. " +
"See the compatibility guide for more information."),
GpuScalaUDFMeta.exprMeta
).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap
}
Expand Down
Loading

0 comments on commit ade6591

Please sign in to comment.