[GitHub] [spark] wangyum commented on a change in pull request #29999: [SPARK-33045][SQL] Support build-in function like_all and fix StackOverflowError issue.

2020-11-10 Thread GitBox


wangyum commented on a change in pull request #2:
URL: https://github.com/apache/spark/pull/2#discussion_r521135433



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
##
@@ -178,6 +180,86 @@ case class Like(left: Expression, right: Expression, 
escapeChar: Char)
   }
 }
 
+/**
+ * Optimized version of LIKE ALL, when all pattern values are literal.
+ */
+abstract class LikeAllBase extends UnaryExpression with ImplicitCastInputTypes 
with NullIntolerant {
+
+  protected def patterns: Seq[Any]
+
+  protected def isNotDefined: Boolean
+
+  override def inputTypes: Seq[DataType] = StringType :: Nil
+
+  override def dataType: DataType = BooleanType
+
+  override def nullable: Boolean = true
+
+  private lazy val hasNull: Boolean = patterns.contains(null)
+
+  private lazy val cache = patterns.filterNot(_ == null)
+.map(s => Pattern.compile(StringUtils.escapeLikeRegex(s.toString, '\\')))
+
+  override def eval(input: InternalRow): Any = {
+if (hasNull) {
+  null

Review comment:
   ```sql
   spark-sql> select 'a' like all ('%a%', null);
   NULL
   spark-sql> select 'a' not like all ('%a%', null);
   false
   spark-sql> select 'a' like any ('%a%', null);
   true
   spark-sql> select 'a' not like any ('%a%', null);
   NULL
   ```





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] wangyum commented on a change in pull request #29999: [SPARK-33045][SQL] Support build-in function like_all and fix StackOverflowError issue.

2020-11-10 Thread GitBox


wangyum commented on a change in pull request #2:
URL: https://github.com/apache/spark/pull/2#discussion_r521135433



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
##
@@ -178,6 +180,86 @@ case class Like(left: Expression, right: Expression, 
escapeChar: Char)
   }
 }
 
+/**
+ * Optimized version of LIKE ALL, when all pattern values are literal.
+ */
+abstract class LikeAllBase extends UnaryExpression with ImplicitCastInputTypes 
with NullIntolerant {
+
+  protected def patterns: Seq[Any]
+
+  protected def isNotDefined: Boolean
+
+  override def inputTypes: Seq[DataType] = StringType :: Nil
+
+  override def dataType: DataType = BooleanType
+
+  override def nullable: Boolean = true
+
+  private lazy val hasNull: Boolean = patterns.contains(null)
+
+  private lazy val cache = patterns.filterNot(_ == null)
+.map(s => Pattern.compile(StringUtils.escapeLikeRegex(s.toString, '\\')))
+
+  override def eval(input: InternalRow): Any = {
+if (hasNull) {
+  null

Review comment:
   ```sql
   spark-sql> select 'a' like all ('%a%', null);
   NULL
   spark-sql> select 'a' not like all ('%a%', null);
   false
   ```





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] wangyum commented on a change in pull request #29999: [SPARK-33045][SQL] Support build-in function like_all and fix StackOverflowError issue.

2020-11-10 Thread GitBox


wangyum commented on a change in pull request #2:
URL: https://github.com/apache/spark/pull/2#discussion_r521109327



##
File path: sql/core/src/test/resources/sql-functions/sql-expression-schema.md
##
@@ -346,4 +346,4 @@
 | org.apache.spark.sql.catalyst.expressions.xml.XPathList | xpath | SELECT 
xpath('b1b2b3c1c2','a/b/text()') | 
structb1b2b3c1c2, 
a/b/text()):array> |
 | org.apache.spark.sql.catalyst.expressions.xml.XPathLong | xpath_long | 
SELECT xpath_long('12', 'sum(a/b)') | 
struct12, sum(a/b)):bigint> |
 | org.apache.spark.sql.catalyst.expressions.xml.XPathShort | xpath_short | 
SELECT xpath_short('12', 'sum(a/b)') | 
struct12, sum(a/b)):smallint> |
-| org.apache.spark.sql.catalyst.expressions.xml.XPathString | xpath_string | 
SELECT xpath_string('bcc','a/c') | 
structbcc, a/c):string> |
\ No newline at end of file
+| org.apache.spark.sql.catalyst.expressions.xml.XPathString | xpath_string | 
SELECT xpath_string('bcc','a/c') | 
structbcc, a/c):string> |

Review comment:
   Revert this change?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] wangyum commented on a change in pull request #29999: [SPARK-33045][SQL] Support build-in function like_all and fix StackOverflowError issue.

2020-11-10 Thread GitBox


wangyum commented on a change in pull request #2:
URL: https://github.com/apache/spark/pull/2#discussion_r521108920



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##
@@ -1408,7 +1408,20 @@ class AstBuilder(conf: SQLConf) extends 
SqlBaseBaseVisitor[AnyRef] with Logging
   case Some(SqlBaseParser.ANY) | Some(SqlBaseParser.SOME) =>
 getLikeQuantifierExprs(ctx.expression).reduceLeft(Or)
   case Some(SqlBaseParser.ALL) =>
-getLikeQuantifierExprs(ctx.expression).reduceLeft(And)
+validate(!ctx.expression.isEmpty, "Expected something between '(' 
and ')'.", ctx)
+val expressions = ctx.expression.asScala.map(expression)
+if (expressions.size > 200 && expressions.forall(_.foldable)) {

Review comment:
   Could we add a config, thus we can add this config to `like-all.sql`  
add test it by `SQLQueryTestSuite`?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] wangyum commented on a change in pull request #29999: [SPARK-33045][SQL] Support build-in function like_all and fix StackOverflowError issue.

2020-11-10 Thread GitBox


wangyum commented on a change in pull request #2:
URL: https://github.com/apache/spark/pull/2#discussion_r521107867



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
##
@@ -178,6 +180,86 @@ case class Like(left: Expression, right: Expression, 
escapeChar: Char)
   }
 }
 
+/**
+ * Optimized version of LIKE ALL, when all pattern values are literal.
+ */
+abstract class LikeAllBase extends UnaryExpression with ImplicitCastInputTypes 
with NullIntolerant {
+
+  protected def patterns: Seq[Any]
+
+  protected def isNotDefined: Boolean
+
+  override def inputTypes: Seq[DataType] = StringType :: Nil
+
+  override def dataType: DataType = BooleanType
+
+  override def nullable: Boolean = true
+
+  private lazy val hasNull: Boolean = patterns.contains(null)
+
+  private lazy val cache = patterns.filterNot(_ == null)
+.map(s => Pattern.compile(StringUtils.escapeLikeRegex(s.toString, '\\')))
+
+  override def eval(input: InternalRow): Any = {
+if (hasNull) {
+  null

Review comment:
   `null` -> `false`?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] wangyum commented on a change in pull request #29999: [SPARK-33045][SQL] Support build-in function like_all and fix StackOverflowError issue.

2020-11-10 Thread GitBox


wangyum commented on a change in pull request #2:
URL: https://github.com/apache/spark/pull/2#discussion_r521106614



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
##
@@ -102,6 +102,8 @@ package object dsl {
 def like(other: Expression, escapeChar: Char = '\\'): Expression =
   Like(expr, other, escapeChar)
 def rlike(other: Expression): Expression = RLike(expr, other)
+def likeAll(others: Literal*): Expression = LikeAll(expr, 
others.map(_.eval(EmptyRow)))

Review comment:
   `others: Literal*` -> `others: String*`?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] wangyum commented on a change in pull request #29999: [SPARK-33045][SQL] Support build-in function like_all and fix StackOverflowError issue.

2020-11-09 Thread GitBox


wangyum commented on a change in pull request #2:
URL: https://github.com/apache/spark/pull/2#discussion_r519773515



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
##
@@ -178,6 +179,142 @@ case class Like(left: Expression, right: Expression, 
escapeChar: Char)
   }
 }
 
+abstract class LikeAllBase extends Expression with ImplicitCastInputTypes {

Review comment:
   Could we make it only support `Literal`, for example:
   ```scala
   case class LikeAll(child: Expression, isNotDefined: Boolean, seq: 
mutable.Buffer[Any])
 extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant {
   
 override def dataType: DataType = BooleanType
   
 override def inputTypes: Seq[DataType] = StringType :: Nil
   
 @transient private[this] lazy val hasNull: Boolean = seq.contains(null)
   
 @transient private lazy val cachedPattern = seq.filterNot(_ == null)
   .map(s => Pattern.compile(StringUtils.escapeLikeRegex(s.toString, '\\')))
   
 override protected def nullSafeEval(input1: Any): Any = {
   if (hasNull) {
 false
   } else {
 val str = input1.asInstanceOf[UTF8String].toString
 if (isNotDefined) {
   !cachedPattern.exists(p => p.matcher(str).matches())
 } else {
   cachedPattern.forall(p => p.matcher(str).matches())
 }
   }
 }
   
// TODO: codegen
   }
   ```
   
   ```scala
   val exps = ctx.expression.asScala.map(expression)
   validate(exps.nonEmpty, "Expected something between '(' and ')'.", ctx)
   if (exps.size > 10 && exps.forall(_.foldable)) {
  LikeAll(e, isNotDefined, exps.map(_.eval(EmptyRow)))
   } else {
 exps.map(p => invertIfNotDefined(Like(e, p))).reduceLeft(And)
   }
   ```





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] wangyum commented on a change in pull request #29999: [SPARK-33045][SQL] Support build-in function like_all and fix StackOverflowError issue.

2020-10-13 Thread GitBox


wangyum commented on a change in pull request #2:
URL: https://github.com/apache/spark/pull/2#discussion_r503836078



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
##
@@ -344,6 +344,8 @@ object FunctionRegistry {
 expression[Length]("length"),
 expression[Levenshtein]("levenshtein"),
 expression[Like]("like"),
+expression[LikeAll]("like_all"),
+expression[NotLikeAll]("not_like_all"),

Review comment:
   Do we need to register these UDFs to the `FunctionRegistry`?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] wangyum commented on a change in pull request #29999: [SPARK-33045][SQL] Support build-in function like_all and fix StackOverflowError issue.

2020-10-13 Thread GitBox


wangyum commented on a change in pull request #2:
URL: https://github.com/apache/spark/pull/2#discussion_r503832375



##
File path: sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql
##
@@ -31,3 +31,13 @@ SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 3);
 SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', -1);
 SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?([a-z]+)', 1);
 SELECT regexp_extract_all('a 2b 14m', '(\\d+)?([a-z]+)', 1);
+
+-- like_all
+SELECT like_all('foo', '%foo%', '%oo');

Review comment:
   We already have a test file: 
https://github.com/apache/spark/blob/b10263b8e5106409467e0115968bbaf0b9141cd1/sql/core/src/test/resources/sql-tests/inputs/like-all.sql





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org