-
Notifications
You must be signed in to change notification settings - Fork 240
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
Add regular expression support to string_split #4714
Conversation
Signed-off-by: Andy Grove <[email protected]>
sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala
Outdated
Show resolved
Hide resolved
sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala
Outdated
Show resolved
Hide resolved
case class GpuStringSplit(str: Expression, regex: Expression, limit: Expression, | ||
isRegExp: Boolean, pattern: String) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we get rid of the regex
expression completely? It is now useless since we use pattern
instead.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe this can be achieved if you override columnarEval
instead of doColumnar
similar to https://github.com/NVIDIA/spark-rapids/pull/4636/files#diff-a12810882b81a4eb395c03a80951f96ec080db793ffed6755739eeb2122840ccR1432
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It would be switching this from a GpuTernaryExpression
to a GpuBinaryExpression
. I personally don't see it as a big deal either way.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would prefer to stick with GpuTernaryExpression
to match Spark
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
But we don't have to, right? I don't see any benefit from keeping it a TernaryExpr
instead of just UnaryExpr
/GpuExpr
. I tried to implement GpuStringToMap
to inherit GpuExpression
and the evaluation function is super short: https://github.com/NVIDIA/spark-rapids/pull/4636/files#diff-a12810882b81a4eb395c03a80951f96ec080db793ffed6755739eeb2122840ccR1507-R1518
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Basically we have evaluated the literal delimiter pattern before calling to the Gpu override, thus we only pass in ONE input string expression.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Wouldn't I still need to pass in all of the expressions though so that I can implement children()
correctly?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, the original delimiter expression still needs to be passed in to initialize children
, but it is not used anywhere in the evaluation later on.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The delimiter expression already isn't used in the evaluation. It is only referenced in override def second: Expression = regex
which is just used to construct children
in final def children: Seq[Expression] = IndexedSeq(first, second, third)
.
I'm not against making the change and am curious to see what the benefits are but I would rather do this as a follow-on issue and review how similar regexp expressions are implemented since they all follow this same pattern.
sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala
Show resolved
Hide resolved
sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala
Show resolved
Hide resolved
...src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuRegExpReplaceMeta.scala
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nothing big, this is looking good.
case class GpuStringSplit(str: Expression, regex: Expression, limit: Expression, | ||
isRegExp: Boolean, pattern: String) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It would be switching this from a GpuTernaryExpression
to a GpuBinaryExpression
. I personally don't see it as a big deal either way.
build |
sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/GpuRegExpReplaceExec.scala
Outdated
Show resolved
Hide resolved
...plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuRegExpReplaceExec.scala
Outdated
Show resolved
Hide resolved
sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/GpuRegExpReplaceExec.scala
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I stopped reviewing after a bit because it looks like there might be some other code in here by accident.
sql-plugin/src/main/scala/com/nvidia/spark/rapids/conditionalExpressions.scala
Outdated
Show resolved
Hide resolved
sql-plugin/src/main/scala/org/apache/spark/sql/rapids/predicates.scala
Outdated
Show resolved
Hide resolved
sql-plugin/src/main/scala/org/apache/spark/sql/rapids/predicates.scala
Outdated
Show resolved
Hide resolved
This reverts commit c70390f.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks good to me
This PR adds Java binding for the new strings API `strings::split_re` and `strings::split_record_re`, which allows splitting strings by regular expression delimiters. In addition, the Java string split overloads with default split pattern (an empty string) are removed in this PR. That is because with default empty pattern the Java's split API produces different results than cudf. Finally, some cleanup has been perform automatically thanks to IntelliJ IDE. Depends on #10128. This is breaking change which is fixed by NVIDIA/spark-rapids#4714. Thus, it should be merged at the same time with NVIDIA/spark-rapids#4714. Authors: - Nghia Truong (https://github.com/ttnghia) - David Wendt (https://github.com/davidwendt) Approvers: - Robert (Bobby) Evans (https://github.com/revans2) - Andy Grove (https://github.com/andygrove) URL: #10139
build |
Signed-off-by: Andy Grove [email protected]
Closes #4003
Depends on rapidsai/cudf#10139
Follow-on issues:
Status: