You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/10/05 21:34:39 UTC

[GitHub] [spark] karenfeng opened a new pull request #29947: [WIP][SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

karenfeng opened a new pull request #29947:
URL: https://github.com/apache/spark/pull/29947


   ## What changes were proposed in this pull request?
   
   Adds a SQL function `raise_error` which underlies the refactored `assert_true` function. `assert_true` now also (optionally) accepts a custom error message field.
   `raise_error` is exposed in SQL, Python, Scala, and R.
   `assert_true` was previously only exposed in SQL; it is now also exposed in Python, Scala, and R.
   
   ### Why are the changes needed?
   
   Improves usability of `assert_true` by clarifying error messaging, and adds the useful helper function `raise_error`.
   
   
   ### Does this PR introduce _any_ user-facing change?
   
   Yes:
   - Adds `raise_error` function to the SQL, Python, Scala, and R APIs.
   - Adds `assert_true` function to the SQL, Python and R APIs.
   
   
   ### How was this patch tested?
   
   Adds unit tests in SQL, Python, Scala, and R for `assert_true` and `raise_error`.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703919621


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34029/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] zero323 commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
zero323 commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705411778


   It seem like R functions haven't been included in `NAMESPACE`. Unless it is intentional they should be added around here
   
   https://github.com/apache/spark/blob/5effa8ea261ba59214afedc2853d1b248b330ca6/R/pkg/NAMESPACE#L231
   
   and  here
   
   https://github.com/apache/spark/blob/5effa8ea261ba59214afedc2853d1b248b330ca6/R/pkg/NAMESPACE#L363
   
   for `assert_true` and `raise_error` respectively.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704508852


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34069/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704500617


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r500687443



##########
File path: python/pyspark/sql/functions.py
##########
@@ -1592,6 +1592,57 @@ def xxhash64(*cols):
     return Column(jc)
 
 

Review comment:
       Now we should also add it into type hints and documentations at https://github.com/apache/spark/blob/master/python/docs/source/reference/pyspark.sql.rst#functions and https://github.com/apache/spark/blob/master/python/pyspark/sql/functions.pyi. cc @zero323 FYI
   
   




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704548341


   **[Test build #129465 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129465/testReport)** for PR 29947 at commit [`52e16ec`](https://github.com/apache/spark/commit/52e16ec95210634d0b44b9fbaae59ae090caa8b3).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] viirya commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r500711220



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
##########
@@ -53,51 +53,80 @@ case class PrintToStderr(child: Expression) extends UnaryExpression {
 }
 
 /**
- * A function throws an exception if 'condition' is not true.
+ * Throw with the result of an expression (used for debugging).
  */
 @ExpressionDescription(
-  usage = "_FUNC_(expr) - Throws an exception if `expr` is not true.",
+  usage = "_FUNC_(expr) - Throws an exception with `expr`.",
   examples = """
     Examples:
-      > SELECT _FUNC_(0 < 1);
-       NULL
+      > SELECT _FUNC_('custom error message');
+       java.lang.RuntimeException
+       custom error message
   """,
-  since = "2.0.0")
-case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
-
-  override def nullable: Boolean = true
-
-  override def inputTypes: Seq[DataType] = Seq(BooleanType)
+  since = "3.1.0")
+case class RaiseError(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
 
+  override def foldable: Boolean = false
   override def dataType: DataType = NullType
+  override def inputTypes: Seq[AbstractDataType] = Seq(StringType)
 
-  override def prettyName: String = "assert_true"
+  override def prettyName: String = "raise_error"
 
-  private val errMsg = s"'${child.simpleString(SQLConf.get.maxToStringFields)}' is not true!"
-
-  override def eval(input: InternalRow) : Any = {
-    val v = child.eval(input)
-    if (v == null || java.lang.Boolean.FALSE.equals(v)) {
-      throw new RuntimeException(errMsg)
-    } else {
-      null
+  override def eval(input: InternalRow): Any = {
+    val value = child.eval(input)
+    if (value == null) {
+      throw new RuntimeException("null")
     }
+    throw new RuntimeException(value.toString())
   }
 
+  // if (true) is to avoid codegen compilation exception that statement is unreachable
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
     val eval = child.genCode(ctx)
+    ExprCode(
+      code = code"""${eval.code}
+        |if (true) {
+        |  if (${eval.value} == null) {
+        |    throw new RuntimeException("null");
+        |  }
+        |  throw new RuntimeException(${eval.value}.toString());
+        |}""".stripMargin,
+      isNull = TrueLiteral,

Review comment:
       Oh, I see. It is `NullType`.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703925701






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703967401






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] maropu commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r499952130



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/functions.scala
##########
@@ -1399,6 +1399,56 @@ object functions {
     Column(parser.parseExpression(expr))
   }
 
+  /**
+   * Returns null if the condition is true, and throws an exception otherwise.
+   *
+   * @group normal_funcs

Review comment:
       `misc_funcs` instead?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703933731


   **[Test build #129426 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129426/testReport)** for PR 29947 at commit [`109af99`](https://github.com/apache/spark/commit/109af99cd6fb780d5ef2eca621b0032352d1fe61).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703936035


   **[Test build #129427 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129427/testReport)** for PR 29947 at commit [`c683d78`](https://github.com/apache/spark/commit/c683d78b517b3c9398fd53a3a74e567ded8e7a3d).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705316739


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34142/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703952531


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34033/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703952513


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34033/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704500632


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34068/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703912704


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703965926






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704570674






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] maropu commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501508474



##########
File path: python/pyspark/sql/functions.pyi
##########
@@ -137,6 +137,8 @@ def sha1(col: ColumnOrName) -> Column: ...
 def sha2(col: ColumnOrName, numBits: int) -> Column: ...
 def hash(*cols: ColumnOrName) -> Column: ...
 def xxhash64(*cols: ColumnOrName) -> Column: ...
+def assert_true(col: ColumnOrName, errMsg: Union[Column, str] = ...): ...

Review comment:
       @karenfeng Could you fix them above in followup?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] viirya commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r500710006



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
##########
@@ -53,51 +53,80 @@ case class PrintToStderr(child: Expression) extends UnaryExpression {
 }
 
 /**
- * A function throws an exception if 'condition' is not true.
+ * Throw with the result of an expression (used for debugging).
  */
 @ExpressionDescription(
-  usage = "_FUNC_(expr) - Throws an exception if `expr` is not true.",
+  usage = "_FUNC_(expr) - Throws an exception with `expr`.",
   examples = """
     Examples:
-      > SELECT _FUNC_(0 < 1);
-       NULL
+      > SELECT _FUNC_('custom error message');
+       java.lang.RuntimeException
+       custom error message
   """,
-  since = "2.0.0")
-case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
-
-  override def nullable: Boolean = true
-
-  override def inputTypes: Seq[DataType] = Seq(BooleanType)
+  since = "3.1.0")
+case class RaiseError(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
 
+  override def foldable: Boolean = false
   override def dataType: DataType = NullType
+  override def inputTypes: Seq[AbstractDataType] = Seq(StringType)
 
-  override def prettyName: String = "assert_true"
+  override def prettyName: String = "raise_error"
 
-  private val errMsg = s"'${child.simpleString(SQLConf.get.maxToStringFields)}' is not true!"
-
-  override def eval(input: InternalRow) : Any = {
-    val v = child.eval(input)
-    if (v == null || java.lang.Boolean.FALSE.equals(v)) {
-      throw new RuntimeException(errMsg)
-    } else {
-      null
+  override def eval(input: InternalRow): Any = {
+    val value = child.eval(input)
+    if (value == null) {
+      throw new RuntimeException("null")
     }
+    throw new RuntimeException(value.toString())
   }
 
+  // if (true) is to avoid codegen compilation exception that statement is unreachable
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
     val eval = child.genCode(ctx)
+    ExprCode(
+      code = code"""${eval.code}
+        |if (true) {
+        |  if (${eval.value} == null) {
+        |    throw new RuntimeException("null");
+        |  }
+        |  throw new RuntimeException(${eval.value}.toString());
+        |}""".stripMargin,
+      isNull = TrueLiteral,

Review comment:
       This looks not correct?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704570657


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34072/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704657117






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703947722


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34033/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705201441






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703904204


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129422/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] zero323 commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
zero323 commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501535748



##########
File path: R/pkg/R/functions.R
##########
@@ -826,6 +826,55 @@ setMethod("xxhash64",
             column(jc)
           })
 
+#' @details
+#' \code{assert_true}: Returns null if the input column is true; throws an exception
+#' with the provided error message otherwise.
+#'
+#' @param errMsg (optional) The error message to be thrown.
+#'
+#' @rdname column_misc_functions
+#' @aliases assert_true assert_true,Column-method
+#' @examples
+#' \dontrun{
+#' tmp <- mutate(df, v1 = assert_true(df$vs < 2),
+#'                   v2 = assert_true(df$vs < 2, "custom error message"),
+#'                   v3 = assert_true(df$vs < 2, df$vs))
+#' head(tmp)}
+#' @note assert_true since 3.1.0
+setMethod("assert_true",
+          signature(x = "Column"),
+          function(x, errMsg = NULL) {
+            jc <- if (is.null(errMsg)) {
+              callJStatic("org.apache.spark.sql.functions", "assert_true", x@jc)
+            } else {
+              if (is.character(errMsg) && length(errMsg) == 1) {

Review comment:
       Shouldn't we throw an exception if `length(errMsg) != 1`? Just in case user does something like this?
   
   ```R
   > assert_true(column("foo"), c("foo", "bar"))
   Error in invokeJava(isStatic = TRUE, className, methodName, ...) : 
     trying to get slot "jc" from an object of a basic class ("character") with no slots
   ```
   
   i.e.
   
   ```R
               } else {
                 if (is.character(errMsg) {
                   stopifnot(length(errMsg) == 1)
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704626565


   **[Test build #129462 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129462/testReport)** for PR 29947 at commit [`5f9e5d5`](https://github.com/apache/spark/commit/5f9e5d56de8cbab77913617be86380c9b4c6903a).
    * This patch **fails SparkR unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705190596


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34135/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705201441






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705316748






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] maropu commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r499946110



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala
##########
@@ -994,12 +994,12 @@ class TypeCoercionSuite extends AnalysisTest {
       If(Literal.create(null, BooleanType), Literal(1), Literal(1)))
 
     ruleTest(rule,
-      If(AssertTrue(trueLit), Literal(1), Literal(2)),
-      If(Cast(AssertTrue(trueLit), BooleanType), Literal(1), Literal(2)))
+      If(new AssertTrue(trueLit), Literal(1), Literal(2)),
+      If(Cast(new AssertTrue(trueLit), BooleanType), Literal(1), Literal(2)))

Review comment:
       How about defining a companion object to avoid the unenecesary changes?
   ```
   object AssertTrue {
     def apply(...): AssertTrue = {...}
   }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703933731


   **[Test build #129426 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129426/testReport)** for PR 29947 at commit [`109af99`](https://github.com/apache/spark/commit/109af99cd6fb780d5ef2eca621b0032352d1fe61).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703909483


   **[Test build #129423 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129423/testReport)** for PR 29947 at commit [`0ff60f4`](https://github.com/apache/spark/commit/0ff60f4bd16de40af275fe90e82a0d83afd605e6).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703967406


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129427/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704625559






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704627459


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703925701






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] karenfeng commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r500475319



##########
File path: sql/core/src/test/resources/sql-tests/inputs/sql-compatibility-functions.sql
##########
@@ -22,3 +22,12 @@ SELECT string(1, 2);
 -- SPARK-21555: RuntimeReplaceable used in group by
 CREATE TEMPORARY VIEW tempView1 AS VALUES (1, NAMED_STRUCT('col1', 'gamma', 'col2', 'delta')) AS T(id, st);
 SELECT nvl(st.col1, "value"), count(*) FROM from tempView1 GROUP BY nvl(st.col1, "value");
+
+-- Spark-32793: Rewrite AssertTrue with RaiseError
+SELECT assert_true(true), assert_true(boolean(1));

Review comment:
       Whoops, I parked these here on accident. I'll move it to `misc-functions`.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703967313


   **[Test build #129427 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129427/testReport)** for PR 29947 at commit [`c683d78`](https://github.com/apache/spark/commit/c683d78b517b3c9398fd53a3a74e567ded8e7a3d).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703923955


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34030/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] zero323 commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
zero323 commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501609185



##########
File path: R/pkg/R/functions.R
##########
@@ -826,6 +826,55 @@ setMethod("xxhash64",
             column(jc)
           })
 
+#' @details
+#' \code{assert_true}: Returns null if the input column is true; throws an exception
+#' with the provided error message otherwise.
+#'
+#' @param errMsg (optional) The error message to be thrown.
+#'
+#' @rdname column_misc_functions
+#' @aliases assert_true assert_true,Column-method
+#' @examples
+#' \dontrun{
+#' tmp <- mutate(df, v1 = assert_true(df$vs < 2),
+#'                   v2 = assert_true(df$vs < 2, "custom error message"),
+#'                   v3 = assert_true(df$vs < 2, df$vs))
+#' head(tmp)}
+#' @note assert_true since 3.1.0
+setMethod("assert_true",
+          signature(x = "Column"),
+          function(x, errMsg = NULL) {
+            jc <- if (is.null(errMsg)) {
+              callJStatic("org.apache.spark.sql.functions", "assert_true", x@jc)
+            } else {
+              if (is.character(errMsg) && length(errMsg) == 1) {

Review comment:
       In practice we make this check anyway, so it is only a question if we do something about it.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705157509






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705201418


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34135/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704465308


   **[Test build #129462 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129462/testReport)** for PR 29947 at commit [`5f9e5d5`](https://github.com/apache/spark/commit/5f9e5d56de8cbab77913617be86380c9b4c6903a).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705157495


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34131/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704563661


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34072/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] viirya commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r500711365



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
##########
@@ -53,51 +53,80 @@ case class PrintToStderr(child: Expression) extends UnaryExpression {
 }
 
 /**
- * A function throws an exception if 'condition' is not true.
+ * Throw with the result of an expression (used for debugging).
  */
 @ExpressionDescription(
-  usage = "_FUNC_(expr) - Throws an exception if `expr` is not true.",
+  usage = "_FUNC_(expr) - Throws an exception with `expr`.",
   examples = """
     Examples:
-      > SELECT _FUNC_(0 < 1);
-       NULL
+      > SELECT _FUNC_('custom error message');
+       java.lang.RuntimeException
+       custom error message
   """,
-  since = "2.0.0")
-case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
-
-  override def nullable: Boolean = true
-
-  override def inputTypes: Seq[DataType] = Seq(BooleanType)
+  since = "3.1.0")
+case class RaiseError(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {

Review comment:
       Oh, looks like it is true.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704460343


   **[Test build #129461 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129461/testReport)** for PR 29947 at commit [`923ac46`](https://github.com/apache/spark/commit/923ac46ce2529b1b57811aa7cc2fa9299d65a7b6).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705269452


   **[Test build #129530 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129530/testReport)** for PR 29947 at commit [`e921f66`](https://github.com/apache/spark/commit/e921f66aca3829f50fddc90e66c168abd1f9ad11).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704570674






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] karenfeng commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501203125



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/functions.scala
##########
@@ -2318,6 +2318,56 @@ object functions {
     new XxHash64(cols.map(_.expr))
   }
 
+  /**
+   * Returns null if the condition is true, and throws an exception otherwise.
+   *
+   * @group misc_funcs
+   * @since 3.1.0
+   */
+  def assert_true(c: Column): Column = withExpr {
+    new AssertTrue(c.expr)
+  }
+
+  /**
+   * Returns null if the condition is true; throws an exception with the error message otherwise.
+   *
+   * @group misc_funcs
+   * @since 3.1.0
+   */
+  def assert_true(c: Column, e: Column): Column = withExpr {
+    new AssertTrue(c.expr, e.expr)
+  }
+
+  /**
+   * Returns null if the condition is true; throws an exception with the error message otherwise.
+   *
+   * @group misc_funcs
+   * @since 3.1.0
+   */
+  def assert_true(c: Column, e: String): Column = withExpr {

Review comment:
       I see - I can expose only Column inputs, and the user can wrap any literal strings.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703930252






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r500686612



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/functions.scala
##########
@@ -2318,6 +2318,56 @@ object functions {
     new XxHash64(cols.map(_.expr))
   }
 
+  /**
+   * Returns null if the condition is true, and throws an exception otherwise.
+   *
+   * @group misc_funcs
+   * @since 3.1.0
+   */
+  def assert_true(c: Column): Column = withExpr {
+    new AssertTrue(c.expr)
+  }
+
+  /**
+   * Returns null if the condition is true; throws an exception with the error message otherwise.
+   *
+   * @group misc_funcs
+   * @since 3.1.0
+   */
+  def assert_true(c: Column, e: Column): Column = withExpr {
+    new AssertTrue(c.expr, e.expr)
+  }
+
+  /**
+   * Returns null if the condition is true; throws an exception with the error message otherwise.
+   *
+   * @group misc_funcs
+   * @since 3.1.0
+   */
+  def assert_true(c: Column, e: String): Column = withExpr {

Review comment:
       I would avoid exposing `String` overridden ones. See also https://github.com/apache/spark/blob/1b60ff5afea0637f74c5f064642225b35b13b069/sql/core/src/main/scala/org/apache/spark/sql/functions.scala#L58-L60




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] viirya commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r500709556



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
##########
@@ -53,51 +53,80 @@ case class PrintToStderr(child: Expression) extends UnaryExpression {
 }
 
 /**
- * A function throws an exception if 'condition' is not true.
+ * Throw with the result of an expression (used for debugging).
  */
 @ExpressionDescription(
-  usage = "_FUNC_(expr) - Throws an exception if `expr` is not true.",
+  usage = "_FUNC_(expr) - Throws an exception with `expr`.",
   examples = """
     Examples:
-      > SELECT _FUNC_(0 < 1);
-       NULL
+      > SELECT _FUNC_('custom error message');
+       java.lang.RuntimeException
+       custom error message
   """,
-  since = "2.0.0")
-case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
-
-  override def nullable: Boolean = true
-
-  override def inputTypes: Seq[DataType] = Seq(BooleanType)
+  since = "3.1.0")
+case class RaiseError(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
 
+  override def foldable: Boolean = false
   override def dataType: DataType = NullType
+  override def inputTypes: Seq[AbstractDataType] = Seq(StringType)
 
-  override def prettyName: String = "assert_true"
+  override def prettyName: String = "raise_error"
 
-  private val errMsg = s"'${child.simpleString(SQLConf.get.maxToStringFields)}' is not true!"
-
-  override def eval(input: InternalRow) : Any = {
-    val v = child.eval(input)
-    if (v == null || java.lang.Boolean.FALSE.equals(v)) {
-      throw new RuntimeException(errMsg)
-    } else {
-      null
+  override def eval(input: InternalRow): Any = {
+    val value = child.eval(input)
+    if (value == null) {
+      throw new RuntimeException("null")

Review comment:
       Yeah, `RuntimeException(null)` looks weird.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501561059



##########
File path: python/pyspark/sql/functions.pyi
##########
@@ -137,6 +137,8 @@ def sha1(col: ColumnOrName) -> Column: ...
 def sha2(col: ColumnOrName, numBits: int) -> Column: ...
 def hash(*cols: ColumnOrName) -> Column: ...
 def xxhash64(*cols: ColumnOrName) -> Column: ...
+def assert_true(col: ColumnOrName, errMsg: Union[Column, str] = ...): ...

Review comment:
       I think doing `Column` is fine.

##########
File path: R/pkg/R/functions.R
##########
@@ -826,6 +826,55 @@ setMethod("xxhash64",
             column(jc)
           })
 
+#' @details
+#' \code{assert_true}: Returns null if the input column is true; throws an exception
+#' with the provided error message otherwise.
+#'
+#' @param errMsg (optional) The error message to be thrown.
+#'
+#' @rdname column_misc_functions
+#' @aliases assert_true assert_true,Column-method
+#' @examples
+#' \dontrun{
+#' tmp <- mutate(df, v1 = assert_true(df$vs < 2),
+#'                   v2 = assert_true(df$vs < 2, "custom error message"),
+#'                   v3 = assert_true(df$vs < 2, df$vs))
+#' head(tmp)}
+#' @note assert_true since 3.1.0
+setMethod("assert_true",
+          signature(x = "Column"),
+          function(x, errMsg = NULL) {
+            jc <- if (is.null(errMsg)) {
+              callJStatic("org.apache.spark.sql.functions", "assert_true", x@jc)
+            } else {
+              if (is.character(errMsg) && length(errMsg) == 1) {

Review comment:
       Yeah, more checks should be fine.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501561059



##########
File path: python/pyspark/sql/functions.pyi
##########
@@ -137,6 +137,8 @@ def sha1(col: ColumnOrName) -> Column: ...
 def sha2(col: ColumnOrName, numBits: int) -> Column: ...
 def hash(*cols: ColumnOrName) -> Column: ...
 def xxhash64(*cols: ColumnOrName) -> Column: ...
+def assert_true(col: ColumnOrName, errMsg: Union[Column, str] = ...): ...

Review comment:
       I think doing `Column` is fine.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] maropu commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r499951796



##########
File path: sql/core/src/test/resources/sql-tests/inputs/sql-compatibility-functions.sql
##########
@@ -22,3 +22,12 @@ SELECT string(1, 2);
 -- SPARK-21555: RuntimeReplaceable used in group by
 CREATE TEMPORARY VIEW tempView1 AS VALUES (1, NAMED_STRUCT('col1', 'gamma', 'col2', 'delta')) AS T(id, st);
 SELECT nvl(st.col1, "value"), count(*) FROM from tempView1 GROUP BY nvl(st.col1, "value");
+
+-- Spark-32793: Rewrite AssertTrue with RaiseError
+SELECT assert_true(true), assert_true(boolean(1));

Review comment:
       These tests are related to database compatibility?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704500617






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501421228



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
##########
@@ -1659,7 +1660,6 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession {
       StructType(Seq(StructField("a", IntegerType, nullable = true))))
   }
 

Review comment:
       ```suggestion
   
   
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705244079


   **[Test build #129526 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129526/testReport)** for PR 29947 at commit [`e5ad9e0`](https://github.com/apache/spark/commit/e5ad9e0cfa097c285ae5d484db08f261512abaea).
    * This patch **fails SparkR unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] viirya commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r500708582



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
##########
@@ -53,51 +53,80 @@ case class PrintToStderr(child: Expression) extends UnaryExpression {
 }
 
 /**
- * A function throws an exception if 'condition' is not true.
+ * Throw with the result of an expression (used for debugging).
  */
 @ExpressionDescription(
-  usage = "_FUNC_(expr) - Throws an exception if `expr` is not true.",
+  usage = "_FUNC_(expr) - Throws an exception with `expr`.",
   examples = """
     Examples:
-      > SELECT _FUNC_(0 < 1);
-       NULL
+      > SELECT _FUNC_('custom error message');
+       java.lang.RuntimeException
+       custom error message
   """,
-  since = "2.0.0")
-case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
-
-  override def nullable: Boolean = true
-
-  override def inputTypes: Seq[DataType] = Seq(BooleanType)
+  since = "3.1.0")
+case class RaiseError(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {

Review comment:
       Define `nullable`? Based on current `eval`, I think it is should be 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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704508882






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703930252






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703904195


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704460343


   **[Test build #129461 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129461/testReport)** for PR 29947 at commit [`923ac46`](https://github.com/apache/spark/commit/923ac46ce2529b1b57811aa7cc2fa9299d65a7b6).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704625559


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704499655


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34069/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] maropu commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r499954268



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
##########
@@ -53,7 +53,45 @@ case class PrintToStderr(child: Expression) extends UnaryExpression {
 }
 
 /**
- * A function throws an exception if 'condition' is not true.
+ * Throw with the result of an expression (used for debugging).
+ */
+@ExpressionDescription(
+  usage = "_FUNC_(expr) - Throws an exception with `expr`.",
+  examples = """
+    Examples:
+      > SELECT _FUNC_('custom error message');
+       java.lang.RuntimeException
+       custom error message
+  """,
+  since = "3.1.0")
+case class RaiseError(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
+
+  override def foldable: Boolean = false
+  override def dataType: DataType = NullType
+  override def inputTypes: Seq[AbstractDataType] = Seq(StringType)
+
+  override def prettyName: String = "raise_error"
+
+  override def nullSafeEval(input: Any): Unit =
+    throw new RuntimeException(input.toString())
+
+  // if (true) is to avoid codegen compilation exception that statement is unreachable
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+    val eval = child.genCode(ctx)
+    ExprCode(
+      code = code"""${eval.code}
+                   |if (true) {
+                   |  throw new RuntimeException(${eval.value}.toString());
+                   |}
+                 """.stripMargin,

Review comment:
       nit format:
   ```
         code = code"""${eval.code}
           |if (true) {
           |  throw new RuntimeException(${eval.value}.toString());
           |}""".stripMargin,
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] zero323 commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
zero323 commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501502832



##########
File path: python/pyspark/sql/functions.pyi
##########
@@ -137,6 +137,8 @@ def sha1(col: ColumnOrName) -> Column: ...
 def sha2(col: ColumnOrName, numBits: int) -> Column: ...
 def hash(*cols: ColumnOrName) -> Column: ...
 def xxhash64(*cols: ColumnOrName) -> Column: ...
+def assert_true(col: ColumnOrName, errMsg: Union[Column, str] = ...): ...

Review comment:
       Two small notes (sorry for being late):
   
   - I think we should annotate return type for `assert_true` - it will type check because of implicit `Any`, but I think it is better to avoid such cases
   
      ```python
      def assert_true(col: ColumnOrName, errMsg: Union[Column, str] = ...) -> Column: ...
      ```
   
   - For `def raise_error` I'd use [`NoReturn`](https://docs.python.org/3/library/typing.html#typing.NoReturn):
   
       ```python
       from typing import NoReturn
   
       def raise_error(errMsg: Union[Column, str]) -> NoReturn: ...
       ```

##########
File path: python/pyspark/sql/functions.pyi
##########
@@ -137,6 +137,8 @@ def sha1(col: ColumnOrName) -> Column: ...
 def sha2(col: ColumnOrName, numBits: int) -> Column: ...
 def hash(*cols: ColumnOrName) -> Column: ...
 def xxhash64(*cols: ColumnOrName) -> Column: ...
+def assert_true(col: ColumnOrName, errMsg: Union[Column, str] = ...): ...

Review comment:
       > * For `def raise_error` I'd use [`NoReturn`](https://docs.python.org/3/library/typing.html#typing.NoReturn):
   
   This might indicate intention here, though technically speaking it's still a `Column`, so
   
   ```python
   def raise_error(errMsg: Union[Column, str]) -> Column: ...
   ```
   
   is still correct (and literal one). Do you have any thoughts about it @HyukjinKwon?

##########
File path: R/pkg/R/functions.R
##########
@@ -826,6 +826,55 @@ setMethod("xxhash64",
             column(jc)
           })
 
+#' @details
+#' \code{assert_true}: Returns null if the input column is true; throws an exception
+#' with the provided error message otherwise.
+#'
+#' @param errMsg (optional) The error message to be thrown.
+#'
+#' @rdname column_misc_functions
+#' @aliases assert_true assert_true,Column-method
+#' @examples
+#' \dontrun{
+#' tmp <- mutate(df, v1 = assert_true(df$vs < 2),
+#'                   v2 = assert_true(df$vs < 2, "custom error message"),
+#'                   v3 = assert_true(df$vs < 2, df$vs))
+#' head(tmp)}
+#' @note assert_true since 3.1.0
+setMethod("assert_true",
+          signature(x = "Column"),
+          function(x, errMsg = NULL) {
+            jc <- if (is.null(errMsg)) {
+              callJStatic("org.apache.spark.sql.functions", "assert_true", x@jc)
+            } else {
+              if (is.character(errMsg) && length(errMsg) == 1) {

Review comment:
       Shouldn't we throw an exception if `length(errMsg) != 1`? Just in case user does something like this?
   
   ```
   > assert_true(column("foo"), c("foo", "bar"))
   Error in invokeJava(isStatic = TRUE, className, methodName, ...) : 
     trying to get slot "jc" from an object of a basic class ("character") with no slots
   ```

##########
File path: R/pkg/R/functions.R
##########
@@ -826,6 +826,55 @@ setMethod("xxhash64",
             column(jc)
           })
 
+#' @details
+#' \code{assert_true}: Returns null if the input column is true; throws an exception
+#' with the provided error message otherwise.
+#'
+#' @param errMsg (optional) The error message to be thrown.
+#'
+#' @rdname column_misc_functions
+#' @aliases assert_true assert_true,Column-method
+#' @examples
+#' \dontrun{
+#' tmp <- mutate(df, v1 = assert_true(df$vs < 2),
+#'                   v2 = assert_true(df$vs < 2, "custom error message"),
+#'                   v3 = assert_true(df$vs < 2, df$vs))
+#' head(tmp)}
+#' @note assert_true since 3.1.0
+setMethod("assert_true",
+          signature(x = "Column"),
+          function(x, errMsg = NULL) {
+            jc <- if (is.null(errMsg)) {
+              callJStatic("org.apache.spark.sql.functions", "assert_true", x@jc)
+            } else {
+              if (is.character(errMsg) && length(errMsg) == 1) {

Review comment:
       Shouldn't we throw an exception if `length(errMsg) != 1`? Just in case user does something like this?
   
   ```R
   > assert_true(column("foo"), c("foo", "bar"))
   Error in invokeJava(isStatic = TRUE, className, methodName, ...) : 
     trying to get slot "jc" from an object of a basic class ("character") with no slots
   ```

##########
File path: R/pkg/R/functions.R
##########
@@ -826,6 +826,55 @@ setMethod("xxhash64",
             column(jc)
           })
 
+#' @details
+#' \code{assert_true}: Returns null if the input column is true; throws an exception
+#' with the provided error message otherwise.
+#'
+#' @param errMsg (optional) The error message to be thrown.
+#'
+#' @rdname column_misc_functions
+#' @aliases assert_true assert_true,Column-method
+#' @examples
+#' \dontrun{
+#' tmp <- mutate(df, v1 = assert_true(df$vs < 2),
+#'                   v2 = assert_true(df$vs < 2, "custom error message"),
+#'                   v3 = assert_true(df$vs < 2, df$vs))
+#' head(tmp)}
+#' @note assert_true since 3.1.0
+setMethod("assert_true",
+          signature(x = "Column"),
+          function(x, errMsg = NULL) {
+            jc <- if (is.null(errMsg)) {
+              callJStatic("org.apache.spark.sql.functions", "assert_true", x@jc)
+            } else {
+              if (is.character(errMsg) && length(errMsg) == 1) {

Review comment:
       Shouldn't we throw an exception if `length(errMsg) != 1`? Just in case user does something like this?
   
   ```R
   > assert_true(column("foo"), c("foo", "bar"))
   Error in invokeJava(isStatic = TRUE, className, methodName, ...) : 
     trying to get slot "jc" from an object of a basic class ("character") with no slots
   ```
   
   i.e.
   
   ```R
               } else {
                 if (is.character(errMsg) {
                   stopifnot(length(errMsg) == 1)
   ```

##########
File path: R/pkg/R/functions.R
##########
@@ -826,6 +826,55 @@ setMethod("xxhash64",
             column(jc)
           })
 
+#' @details
+#' \code{assert_true}: Returns null if the input column is true; throws an exception
+#' with the provided error message otherwise.
+#'
+#' @param errMsg (optional) The error message to be thrown.
+#'
+#' @rdname column_misc_functions
+#' @aliases assert_true assert_true,Column-method
+#' @examples
+#' \dontrun{
+#' tmp <- mutate(df, v1 = assert_true(df$vs < 2),
+#'                   v2 = assert_true(df$vs < 2, "custom error message"),
+#'                   v3 = assert_true(df$vs < 2, df$vs))
+#' head(tmp)}
+#' @note assert_true since 3.1.0
+setMethod("assert_true",
+          signature(x = "Column"),
+          function(x, errMsg = NULL) {
+            jc <- if (is.null(errMsg)) {
+              callJStatic("org.apache.spark.sql.functions", "assert_true", x@jc)
+            } else {
+              if (is.character(errMsg) && length(errMsg) == 1) {

Review comment:
       Shouldn't we throw an exception if `length(errMsg) != 1`? Just in case user does something like this?
   
   ```R
   > assert_true(column("foo"), c("foo", "bar"))
   Error in invokeJava(isStatic = TRUE, className, methodName, ...) : 
     trying to get slot "jc" from an object of a basic class ("character") with no slots
   ```
   
   i.e.
   
   ```R
              ...
               } else {
                 if (is.character(errMsg) {
                   stopifnot(length(errMsg) == 1)
              ...
   ```

##########
File path: R/pkg/R/functions.R
##########
@@ -826,6 +826,55 @@ setMethod("xxhash64",
             column(jc)
           })
 
+#' @details
+#' \code{assert_true}: Returns null if the input column is true; throws an exception
+#' with the provided error message otherwise.
+#'
+#' @param errMsg (optional) The error message to be thrown.
+#'
+#' @rdname column_misc_functions
+#' @aliases assert_true assert_true,Column-method
+#' @examples
+#' \dontrun{
+#' tmp <- mutate(df, v1 = assert_true(df$vs < 2),
+#'                   v2 = assert_true(df$vs < 2, "custom error message"),
+#'                   v3 = assert_true(df$vs < 2, df$vs))
+#' head(tmp)}
+#' @note assert_true since 3.1.0
+setMethod("assert_true",
+          signature(x = "Column"),
+          function(x, errMsg = NULL) {
+            jc <- if (is.null(errMsg)) {
+              callJStatic("org.apache.spark.sql.functions", "assert_true", x@jc)
+            } else {
+              if (is.character(errMsg) && length(errMsg) == 1) {

Review comment:
       In practice we make this check anyway, so it is only a question if we do something about it.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705435481


   @zero323 it's my bad that I rushed. Can you make a quick followup if you're available? I think @karenfeng lives in US timezone and probably is sleeping :-)


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705157509






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703952523


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703965931


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129426/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705244702






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] viirya commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r500713239



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
##########
@@ -53,51 +53,80 @@ case class PrintToStderr(child: Expression) extends UnaryExpression {
 }
 
 /**
- * A function throws an exception if 'condition' is not true.
+ * Throw with the result of an expression (used for debugging).
  */
 @ExpressionDescription(
-  usage = "_FUNC_(expr) - Throws an exception if `expr` is not true.",
+  usage = "_FUNC_(expr) - Throws an exception with `expr`.",
   examples = """
     Examples:
-      > SELECT _FUNC_(0 < 1);
-       NULL
+      > SELECT _FUNC_('custom error message');
+       java.lang.RuntimeException
+       custom error message
   """,
-  since = "2.0.0")
-case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
-
-  override def nullable: Boolean = true
-
-  override def inputTypes: Seq[DataType] = Seq(BooleanType)
+  since = "3.1.0")
+case class RaiseError(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
 
+  override def foldable: Boolean = false
   override def dataType: DataType = NullType
+  override def inputTypes: Seq[AbstractDataType] = Seq(StringType)
 
-  override def prettyName: String = "assert_true"
+  override def prettyName: String = "raise_error"
 
-  private val errMsg = s"'${child.simpleString(SQLConf.get.maxToStringFields)}' is not true!"
-
-  override def eval(input: InternalRow) : Any = {
-    val v = child.eval(input)
-    if (v == null || java.lang.Boolean.FALSE.equals(v)) {
-      throw new RuntimeException(errMsg)
-    } else {
-      null
+  override def eval(input: InternalRow): Any = {
+    val value = child.eval(input)
+    if (value == null) {
+      throw new RuntimeException("null")
     }
+    throw new RuntimeException(value.toString())
   }
 
+  // if (true) is to avoid codegen compilation exception that statement is unreachable
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
     val eval = child.genCode(ctx)
+    ExprCode(
+      code = code"""${eval.code}
+        |if (true) {
+        |  if (${eval.value} == null) {

Review comment:
       We should use `eval.isNull`?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705170731


   **[Test build #129530 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129530/testReport)** for PR 29947 at commit [`e921f66`](https://github.com/apache/spark/commit/e921f66aca3829f50fddc90e66c168abd1f9ad11).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705435481






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] zero323 commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
zero323 commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501535748



##########
File path: R/pkg/R/functions.R
##########
@@ -826,6 +826,55 @@ setMethod("xxhash64",
             column(jc)
           })
 
+#' @details
+#' \code{assert_true}: Returns null if the input column is true; throws an exception
+#' with the provided error message otherwise.
+#'
+#' @param errMsg (optional) The error message to be thrown.
+#'
+#' @rdname column_misc_functions
+#' @aliases assert_true assert_true,Column-method
+#' @examples
+#' \dontrun{
+#' tmp <- mutate(df, v1 = assert_true(df$vs < 2),
+#'                   v2 = assert_true(df$vs < 2, "custom error message"),
+#'                   v3 = assert_true(df$vs < 2, df$vs))
+#' head(tmp)}
+#' @note assert_true since 3.1.0
+setMethod("assert_true",
+          signature(x = "Column"),
+          function(x, errMsg = NULL) {
+            jc <- if (is.null(errMsg)) {
+              callJStatic("org.apache.spark.sql.functions", "assert_true", x@jc)
+            } else {
+              if (is.character(errMsg) && length(errMsg) == 1) {

Review comment:
       Shouldn't we throw an exception if `length(errMsg) != 1`? Just in case user does something like this?
   
   ```
   > assert_true(column("foo"), c("foo", "bar"))
   Error in invokeJava(isStatic = TRUE, className, methodName, ...) : 
     trying to get slot "jc" from an object of a basic class ("character") with no slots
   ```

##########
File path: R/pkg/R/functions.R
##########
@@ -826,6 +826,55 @@ setMethod("xxhash64",
             column(jc)
           })
 
+#' @details
+#' \code{assert_true}: Returns null if the input column is true; throws an exception
+#' with the provided error message otherwise.
+#'
+#' @param errMsg (optional) The error message to be thrown.
+#'
+#' @rdname column_misc_functions
+#' @aliases assert_true assert_true,Column-method
+#' @examples
+#' \dontrun{
+#' tmp <- mutate(df, v1 = assert_true(df$vs < 2),
+#'                   v2 = assert_true(df$vs < 2, "custom error message"),
+#'                   v3 = assert_true(df$vs < 2, df$vs))
+#' head(tmp)}
+#' @note assert_true since 3.1.0
+setMethod("assert_true",
+          signature(x = "Column"),
+          function(x, errMsg = NULL) {
+            jc <- if (is.null(errMsg)) {
+              callJStatic("org.apache.spark.sql.functions", "assert_true", x@jc)
+            } else {
+              if (is.character(errMsg) && length(errMsg) == 1) {

Review comment:
       Shouldn't we throw an exception if `length(errMsg) != 1`? Just in case user does something like this?
   
   ```R
   > assert_true(column("foo"), c("foo", "bar"))
   Error in invokeJava(isStatic = TRUE, className, methodName, ...) : 
     trying to get slot "jc" from an object of a basic class ("character") with no slots
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] zero323 commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
zero323 commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501535748



##########
File path: R/pkg/R/functions.R
##########
@@ -826,6 +826,55 @@ setMethod("xxhash64",
             column(jc)
           })
 
+#' @details
+#' \code{assert_true}: Returns null if the input column is true; throws an exception
+#' with the provided error message otherwise.
+#'
+#' @param errMsg (optional) The error message to be thrown.
+#'
+#' @rdname column_misc_functions
+#' @aliases assert_true assert_true,Column-method
+#' @examples
+#' \dontrun{
+#' tmp <- mutate(df, v1 = assert_true(df$vs < 2),
+#'                   v2 = assert_true(df$vs < 2, "custom error message"),
+#'                   v3 = assert_true(df$vs < 2, df$vs))
+#' head(tmp)}
+#' @note assert_true since 3.1.0
+setMethod("assert_true",
+          signature(x = "Column"),
+          function(x, errMsg = NULL) {
+            jc <- if (is.null(errMsg)) {
+              callJStatic("org.apache.spark.sql.functions", "assert_true", x@jc)
+            } else {
+              if (is.character(errMsg) && length(errMsg) == 1) {

Review comment:
       Shouldn't we throw an exception if `length(errMsg) != 1`? Just in case user does something like this?
   
   ```R
   > assert_true(column("foo"), c("foo", "bar"))
   Error in invokeJava(isStatic = TRUE, className, methodName, ...) : 
     trying to get slot "jc" from an object of a basic class ("character") with no slots
   ```
   
   i.e.
   
   ```R
              ...
               } else {
                 if (is.character(errMsg) {
                   stopifnot(length(errMsg) == 1)
              ...
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705269973






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704656516


   **[Test build #129465 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129465/testReport)** for PR 29947 at commit [`52e16ec`](https://github.com/apache/spark/commit/52e16ec95210634d0b44b9fbaae59ae090caa8b3).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705269973






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703904195






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705126289


   **[Test build #129526 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129526/testReport)** for PR 29947 at commit [`e5ad9e0`](https://github.com/apache/spark/commit/e5ad9e0cfa097c285ae5d484db08f261512abaea).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704500594


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34068/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703909483


   **[Test build #129423 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129423/testReport)** for PR 29947 at commit [`0ff60f4`](https://github.com/apache/spark/commit/0ff60f4bd16de40af275fe90e82a0d83afd605e6).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704465308


   **[Test build #129462 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129462/testReport)** for PR 29947 at commit [`5f9e5d5`](https://github.com/apache/spark/commit/5f9e5d56de8cbab77913617be86380c9b4c6903a).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703965926


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] zero323 commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
zero323 commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705474334


   @HyukjinKwon I am at work right now, with only my phone, but I'll open a PR once I am back home, unless it is resolved by then.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704548341


   **[Test build #129465 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129465/testReport)** for PR 29947 at commit [`52e16ec`](https://github.com/apache/spark/commit/52e16ec95210634d0b44b9fbaae59ae090caa8b3).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] maropu commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704602820


   okay, I'll check it in hours. cc: @HyukjinKwon @viirya @ueshin 


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704627459






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703930226


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34030/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704491419


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34068/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704508882






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703903223


   **[Test build #129422 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129422/testReport)** for PR 29947 at commit [`df1fc36`](https://github.com/apache/spark/commit/df1fc362e634449063114ed3eded141ac7264a0a).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703904177


   **[Test build #129422 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129422/testReport)** for PR 29947 at commit [`df1fc36`](https://github.com/apache/spark/commit/df1fc362e634449063114ed3eded141ac7264a0a).
    * This patch **fails Python style tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class DisableUnnecessaryBucketedScan(conf: SQLConf) extends Rule[SparkPlan] `
     * `abstract class JdbcConnectionProvider `


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704624585


   **[Test build #129461 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129461/testReport)** for PR 29947 at commit [`923ac46`](https://github.com/apache/spark/commit/923ac46ce2529b1b57811aa7cc2fa9299d65a7b6).
    * This patch **fails SparkR unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705312614


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34142/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703912704






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703912712


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129423/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703965853


   **[Test build #129426 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129426/testReport)** for PR 29947 at commit [`109af99`](https://github.com/apache/spark/commit/109af99cd6fb780d5ef2eca621b0032352d1fe61).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703967401


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] maropu commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501508474



##########
File path: python/pyspark/sql/functions.pyi
##########
@@ -137,6 +137,8 @@ def sha1(col: ColumnOrName) -> Column: ...
 def sha2(col: ColumnOrName, numBits: int) -> Column: ...
 def hash(*cols: ColumnOrName) -> Column: ...
 def xxhash64(*cols: ColumnOrName) -> Column: ...
+def assert_true(col: ColumnOrName, errMsg: Union[Column, str] = ...): ...

Review comment:
       @karenfeng Could you fix them above in followup?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704657117






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] maropu commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r500672056



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
##########
@@ -53,51 +53,80 @@ case class PrintToStderr(child: Expression) extends UnaryExpression {
 }
 
 /**
- * A function throws an exception if 'condition' is not true.
+ * Throw with the result of an expression (used for debugging).
  */
 @ExpressionDescription(
-  usage = "_FUNC_(expr) - Throws an exception if `expr` is not true.",
+  usage = "_FUNC_(expr) - Throws an exception with `expr`.",
   examples = """
     Examples:
-      > SELECT _FUNC_(0 < 1);
-       NULL
+      > SELECT _FUNC_('custom error message');
+       java.lang.RuntimeException
+       custom error message
   """,
-  since = "2.0.0")
-case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
-
-  override def nullable: Boolean = true
-
-  override def inputTypes: Seq[DataType] = Seq(BooleanType)
+  since = "3.1.0")
+case class RaiseError(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
 
+  override def foldable: Boolean = false
   override def dataType: DataType = NullType
+  override def inputTypes: Seq[AbstractDataType] = Seq(StringType)
 
-  override def prettyName: String = "assert_true"
+  override def prettyName: String = "raise_error"
 
-  private val errMsg = s"'${child.simpleString(SQLConf.get.maxToStringFields)}' is not true!"
-
-  override def eval(input: InternalRow) : Any = {
-    val v = child.eval(input)
-    if (v == null || java.lang.Boolean.FALSE.equals(v)) {
-      throw new RuntimeException(errMsg)
-    } else {
-      null
+  override def eval(input: InternalRow): Any = {
+    val value = child.eval(input)
+    if (value == null) {
+      throw new RuntimeException("null")

Review comment:
       nit: `RuntimeException("null")` instead of `RuntimeException()`?

##########
File path: sql/core/src/test/resources/sql-tests/inputs/misc-functions.sql
##########
@@ -8,3 +8,12 @@ select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2);
 select typeof(date '1986-05-23'),  typeof(timestamp '1986-05-23'), typeof(interval '23 days');
 select typeof(x'ABCD'), typeof('SPARK');
 select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark'));
+
+-- Spark-32793: Rewrite AssertTrue with RaiseError

Review comment:
       Could you add tests like this?
   ```
   scala> sql("select * from t").show()
   +---+
   |  v|
   +---+
   |  1|
   |  8|
   |  2|
   +---+
   
   scala> sql("select if(v > 5, raise_error('error found: ' || v), v + 1) from t").show()
   java.lang.RuntimeException: error found: 8
     at org.apache.spark.sql.catalyst.expressions.RaiseError.eval(misc.scala:80)
   ```
   That is because I think it is a common usecase to print out an error message with an invalid value.

##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
##########
@@ -332,7 +332,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper {
   }
 
   test("SPARK-17160: field names are properly escaped by AssertTrue") {
-    GenerateUnsafeProjection.generate(AssertTrue(Cast(Literal("\""), BooleanType)) :: Nil)
+    GenerateUnsafeProjection.generate(new AssertTrue(Cast(Literal("\""), BooleanType)).child :: Nil)

Review comment:
       nit: we don't need `new` now.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] maropu commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705280947


   LGTM, too.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704627463


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129462/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon closed pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
HyukjinKwon closed pull request #29947:
URL: https://github.com/apache/spark/pull/29947


   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705480612


   Sure, thanks!


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705244709


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129526/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703952523






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501561625



##########
File path: R/pkg/R/functions.R
##########
@@ -826,6 +826,55 @@ setMethod("xxhash64",
             column(jc)
           })
 
+#' @details
+#' \code{assert_true}: Returns null if the input column is true; throws an exception
+#' with the provided error message otherwise.
+#'
+#' @param errMsg (optional) The error message to be thrown.
+#'
+#' @rdname column_misc_functions
+#' @aliases assert_true assert_true,Column-method
+#' @examples
+#' \dontrun{
+#' tmp <- mutate(df, v1 = assert_true(df$vs < 2),
+#'                   v2 = assert_true(df$vs < 2, "custom error message"),
+#'                   v3 = assert_true(df$vs < 2, df$vs))
+#' head(tmp)}
+#' @note assert_true since 3.1.0
+setMethod("assert_true",
+          signature(x = "Column"),
+          function(x, errMsg = NULL) {
+            jc <- if (is.null(errMsg)) {
+              callJStatic("org.apache.spark.sql.functions", "assert_true", x@jc)
+            } else {
+              if (is.character(errMsg) && length(errMsg) == 1) {

Review comment:
       Yeah, more checks should be fine.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] zero323 commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
zero323 commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501518074



##########
File path: python/pyspark/sql/functions.pyi
##########
@@ -137,6 +137,8 @@ def sha1(col: ColumnOrName) -> Column: ...
 def sha2(col: ColumnOrName, numBits: int) -> Column: ...
 def hash(*cols: ColumnOrName) -> Column: ...
 def xxhash64(*cols: ColumnOrName) -> Column: ...
+def assert_true(col: ColumnOrName, errMsg: Union[Column, str] = ...): ...

Review comment:
       > * For `def raise_error` I'd use [`NoReturn`](https://docs.python.org/3/library/typing.html#typing.NoReturn):
   
   This might indicate intention here, though technically speaking it's still a `Column`, so
   
   ```python
   def raise_error(errMsg: Union[Column, str]) -> Column: ...
   ```
   
   is still correct (and literal one). Do you have any thoughts about it @HyukjinKwon?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] zero323 commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
zero323 commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705411778






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705148812


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34131/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703925683


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34029/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] zero323 commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
zero323 commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501502832



##########
File path: python/pyspark/sql/functions.pyi
##########
@@ -137,6 +137,8 @@ def sha1(col: ColumnOrName) -> Column: ...
 def sha2(col: ColumnOrName, numBits: int) -> Column: ...
 def hash(*cols: ColumnOrName) -> Column: ...
 def xxhash64(*cols: ColumnOrName) -> Column: ...
+def assert_true(col: ColumnOrName, errMsg: Union[Column, str] = ...): ...

Review comment:
       Two small notes (sorry for being late):
   
   - I think we should annotate return type for `assert_true` - it will type check because of implicit `Any`, but I think it is better to avoid such cases
   
      ```python
      def assert_true(col: ColumnOrName, errMsg: Union[Column, str] = ...) -> Column: ...
      ```
   
   - For `def raise_error` I'd use [`NoReturn`](https://docs.python.org/3/library/typing.html#typing.NoReturn):
   
       ```python
       from typing import NoReturn
   
       def raise_error(errMsg: Union[Column, str]) -> NoReturn: ...
       ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705126289


   **[Test build #129526 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129526/testReport)** for PR 29947 at commit [`e5ad9e0`](https://github.com/apache/spark/commit/e5ad9e0cfa097c285ae5d484db08f261512abaea).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705244702


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705170731


   **[Test build #129530 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129530/testReport)** for PR 29947 at commit [`e921f66`](https://github.com/apache/spark/commit/e921f66aca3829f50fddc90e66c168abd1f9ad11).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703912690


   **[Test build #129423 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129423/testReport)** for PR 29947 at commit [`0ff60f4`](https://github.com/apache/spark/commit/0ff60f4bd16de40af275fe90e82a0d83afd605e6).
    * This patch **fails R style tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705302055


   Merged to master.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704625565


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129461/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #29947:
URL: https://github.com/apache/spark/pull/29947#discussion_r501401567



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
##########
@@ -1659,7 +1660,6 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession {
       StructType(Seq(StructField("a", IntegerType, nullable = true))))
   }
 
-

Review comment:
       I would remove this line to prevent potential conflicts when we port it back or revert.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] karenfeng commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
karenfeng commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-704601172


   Thanks for the review @maropu! I addressed your comments - could you take another look? It looks like the documentation generation is failing, but I'm not sure how I impacted it.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [WIP][SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703903223


   **[Test build #129422 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129422/testReport)** for PR 29947 at commit [`df1fc36`](https://github.com/apache/spark/commit/df1fc362e634449063114ed3eded141ac7264a0a).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-705316748






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29947: [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29947:
URL: https://github.com/apache/spark/pull/29947#issuecomment-703936035


   **[Test build #129427 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129427/testReport)** for PR 29947 at commit [`c683d78`](https://github.com/apache/spark/commit/c683d78b517b3c9398fd53a3a74e567ded8e7a3d).


----------------------------------------------------------------
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:
users@infra.apache.org



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