You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "hannahkamundson (via GitHub)" <gi...@apache.org> on 2023/12/13 17:15:49 UTC

[PR] [WIP][SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

hannahkamundson opened a new pull request, #44336:
URL: https://github.com/apache/spark/pull/44336

   ### What changes were proposed in this pull request?
   I changed 2 `require`s to `SparkIllegalArgumentException`
   
   
   ### Why are the changes needed?
   All user facing exceptions should be `SparkException`s. The `require`s need to be changed to match this convention.
   
   
   ### Does this PR introduce _any_ user-facing change?
   Yes, instead of the `require` which would throw an `IllegalArgumentException`, this will now throw a `SparkIllegalArgumentException`.
   
   
   ### How was this patch tested?
   Unit tests
   
   
   ### Was this patch authored or co-authored using generative AI tooling?
   No
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "llkj1 (via GitHub)" <gi...@apache.org>.
llkj1 commented on PR #44336:
URL: https://github.com/apache/spark/pull/44336#issuecomment-1880016324

   感谢您与我们联系。(Thank you for contacting us.)
   ------------------------------------------------------------
   
   我们收到了您的请求,并会在一至两个工作日内给您回复。您的案例编号是 102195523219。
   
   想了解更多与开发相关主题的信息,请访问 开发者支持网页 (https://developer.apple.com/support/)。
                     
   Apple Developer Program Support
   
   
   We’ve received your support request and will get back to you in one to two business days. Your case number is 102195523219.
   
   For additional information on development-related topics, visit:
   https://developer.apple.com/support/
   
   Best regards, 
   
   Apple Developer Program Support
              
   
   ------------------------------------------------------------
   Copyright (c) 2024 Apple Inc. All rights reserved.
   
   Contact Us
   https://developer.apple.com/contact/
   
   Developer
   https://developer.apple.com/
   
   My Apple ID
   https://appleid.apple.com
   
   Privacy Policy
   https://www.apple.com/privacy/
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "hannahkamundson (via GitHub)" <gi...@apache.org>.
hannahkamundson commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1433134058


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala:
##########
@@ -1300,14 +1301,16 @@ trait ComplexTypeMergingExpression extends Expression {
   lazy val inputTypesForMerging: Seq[DataType] = children.map(_.dataType)
 
   def dataTypeCheck: Unit = {
-    require(
-      inputTypesForMerging.nonEmpty,
-      "The collection of input data types must not be empty.")
-    require(
-      TypeCoercion.haveSameType(inputTypesForMerging),
-      "All input types must be the same except nullable, containsNull, valueContainsNull flags. " +
-        s"The expression is: $this. " +
-        s"The input types found are\n\t${inputTypesForMerging.mkString("\n\t")}.")
+    SparkException.require(
+      requirement = inputTypesForMerging.nonEmpty,
+      errorClass = "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT.NO_INPUTS",

Review Comment:
   I'm pretty sure they are public facing because they are called form things like `Least`. I'll add tests



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "llkj1 (via GitHub)" <gi...@apache.org>.
llkj1 commented on PR #44336:
URL: https://github.com/apache/spark/pull/44336#issuecomment-1880018252

   感谢您与我们联系。(Thank you for contacting us.)
   ------------------------------------------------------------
   
   我们收到了您的请求,并会在一至两个工作日内给您回复。您的案例编号是 102195526981。
   
   想了解更多与开发相关主题的信息,请访问 开发者支持网页 (https://developer.apple.com/support/)。
                     
   Apple Developer Program Support
   
   
   We’ve received your support request and will get back to you in one to two business days. Your case number is 102195526981.
   
   For additional information on development-related topics, visit:
   https://developer.apple.com/support/
   
   Best regards, 
   
   Apple Developer Program Support
              
   
   ------------------------------------------------------------
   Copyright (c) 2024 Apple Inc. All rights reserved.
   
   Contact Us
   https://developer.apple.com/contact/
   
   Developer
   https://developer.apple.com/
   
   My Apple ID
   https://appleid.apple.com
   
   Privacy Policy
   https://www.apple.com/privacy/
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "llkj1 (via GitHub)" <gi...@apache.org>.
llkj1 commented on PR #44336:
URL: https://github.com/apache/spark/pull/44336#issuecomment-1880019229

   感谢您与我们联系。(Thank you for contacting us.)
   ------------------------------------------------------------
   
   我们收到了您的请求,并会在一至两个工作日内给您回复。您的案例编号是 102195529243。
   
   想了解更多与开发相关主题的信息,请访问 开发者支持网页 (https://developer.apple.com/support/)。
                     
   Apple Developer Program Support
   
   
   We’ve received your support request and will get back to you in one to two business days. Your case number is 102195529243.
   
   For additional information on development-related topics, visit:
   https://developer.apple.com/support/
   
   Best regards, 
   
   Apple Developer Program Support
              
   
   ------------------------------------------------------------
   Copyright (c) 2024 Apple Inc. All rights reserved.
   
   Contact Us
   https://developer.apple.com/contact/
   
   Developer
   https://developer.apple.com/
   
   My Apple ID
   https://appleid.apple.com
   
   Privacy Policy
   https://www.apple.com/privacy/
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "llkj1 (via GitHub)" <gi...@apache.org>.
llkj1 commented on PR #44336:
URL: https://github.com/apache/spark/pull/44336#issuecomment-1880018760

   感谢您与我们联系。(Thank you for contacting us.)
   ------------------------------------------------------------
   
   我们收到了您的请求,并会在一至两个工作日内给您回复。您的案例编号是 102195528265。
   
   想了解更多与开发相关主题的信息,请访问 开发者支持网页 (https://developer.apple.com/support/)。
                     
   Apple Developer Program Support
   
   
   We’ve received your support request and will get back to you in one to two business days. Your case number is 102195528265.
   
   For additional information on development-related topics, visit:
   https://developer.apple.com/support/
   
   Best regards, 
   
   Apple Developer Program Support
              
   
   ------------------------------------------------------------
   Copyright (c) 2024 Apple Inc. All rights reserved.
   
   Contact Us
   https://developer.apple.com/contact/
   
   Developer
   https://developer.apple.com/
   
   My Apple ID
   https://appleid.apple.com
   
   Privacy Policy
   https://www.apple.com/privacy/
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on PR #44336:
URL: https://github.com/apache/spark/pull/44336#issuecomment-1863956845

   @hannahkamundson Could you remove the unused imports, see the failed GitHub actions:
   ```
   [error] /home/runner/work/spark/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala:26:56: Unused import
   [error] Applicable -Wconf / @nowarn filters for this fatal warning: msg=<part of the message>, cat=unused-imports, site=org.apache.spark.sql.catalyst.expressions
   [error] import org.apache.spark.sql.catalyst.expressions.Cast.{toSQLId, toSQLType}
   [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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1433115102


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -500,6 +500,24 @@
     ],
     "sqlState" : "22004"
   },
+  "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT" : {
+    "message" : [
+      "Cannot process input data types for the expression: <expression>."
+    ],
+    "subClass" : {
+      "MISMATCHED_TYPES" : {
+        "message" : [
+          "All input types must be the same except nullable, containsNull, valueContainsNull flags, but found the input types <inputTypes>."
+        ]
+      },
+      "NO_INPUTS" : {
+        "message" : [
+          "The collection of input data types must not be empty."
+        ]
+      }
+    },
+    "sqlState" : "42K09"

Review Comment:
   @srielau WDYT of this state? Shouldn't we introduce a separate error class for require.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala:
##########
@@ -1300,14 +1301,16 @@ trait ComplexTypeMergingExpression extends Expression {
   lazy val inputTypesForMerging: Seq[DataType] = children.map(_.dataType)
 
   def dataTypeCheck: Unit = {
-    require(
-      inputTypesForMerging.nonEmpty,
-      "The collection of input data types must not be empty.")
-    require(
-      TypeCoercion.haveSameType(inputTypesForMerging),
-      "All input types must be the same except nullable, containsNull, valueContainsNull flags. " +
-        s"The expression is: $this. " +
-        s"The input types found are\n\t${inputTypesForMerging.mkString("\n\t")}.")
+    SparkException.require(
+      requirement = inputTypesForMerging.nonEmpty,
+      errorClass = "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT.NO_INPUTS",

Review Comment:
   @hannahkamundson Just in case, can users get the error using public API? If so, please, add a test. If not, we should think of converting them to internal errors.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "hannahkamundson (via GitHub)" <gi...@apache.org>.
hannahkamundson commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1461107744


##########
sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala:
##########
@@ -17,17 +17,17 @@
 
 package org.apache.spark.sql.errors
 
-import org.apache.spark.{SPARK_DOC_ROOT, SparkUnsupportedOperationException}
-import org.apache.spark.sql.{AnalysisException, ClassData, IntegratedUDFTestUtils, QueryTest, Row}
+import org.apache.spark.{SPARK_DOC_ROOT, SparkIllegalArgumentException, SparkUnsupportedOperationException}
+import org.apache.spark.sql._

Review Comment:
   Ya sorry I need to set up my IDE appropriately so it doesn't do this



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "hannahkamundson (via GitHub)" <gi...@apache.org>.
hannahkamundson commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1433134058


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala:
##########
@@ -1300,14 +1301,16 @@ trait ComplexTypeMergingExpression extends Expression {
   lazy val inputTypesForMerging: Seq[DataType] = children.map(_.dataType)
 
   def dataTypeCheck: Unit = {
-    require(
-      inputTypesForMerging.nonEmpty,
-      "The collection of input data types must not be empty.")
-    require(
-      TypeCoercion.haveSameType(inputTypesForMerging),
-      "All input types must be the same except nullable, containsNull, valueContainsNull flags. " +
-        s"The expression is: $this. " +
-        s"The input types found are\n\t${inputTypesForMerging.mkString("\n\t")}.")
+    SparkException.require(
+      requirement = inputTypesForMerging.nonEmpty,
+      errorClass = "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT.NO_INPUTS",

Review Comment:
   I'm pretty sure they are public facing because they are called from things like `Least`. I'll add tests
   (edit: spelling)



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "llkj1 (via GitHub)" <gi...@apache.org>.
llkj1 commented on PR #44336:
URL: https://github.com/apache/spark/pull/44336#issuecomment-1880017451

   感谢您与我们联系。(Thank you for contacting us.)
   ------------------------------------------------------------
   
   我们收到了您的请求,并会在一至两个工作日内给您回复。您的案例编号是 102195525426。
   
   想了解更多与开发相关主题的信息,请访问 开发者支持网页 (https://developer.apple.com/support/)。
                     
   Apple Developer Program Support
   
   
   We’ve received your support request and will get back to you in one to two business days. Your case number is 102195525426.
   
   For additional information on development-related topics, visit:
   https://developer.apple.com/support/
   
   Best regards, 
   
   Apple Developer Program Support
              
   
   ------------------------------------------------------------
   Copyright (c) 2024 Apple Inc. All rights reserved.
   
   Contact Us
   https://developer.apple.com/contact/
   
   Developer
   https://developer.apple.com/
   
   My Apple ID
   https://appleid.apple.com
   
   Privacy Policy
   https://www.apple.com/privacy/
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1433111027


##########
common/utils/src/main/scala/org/apache/spark/SparkException.scala:
##########
@@ -106,6 +106,21 @@ object SparkException {
       messageParameters = Map("message" -> msg),
       cause = cause)
   }
+
+  /**
+   * This is like the Scala require precondition, except it uses SparkException.

Review Comment:
   Precisely speaking, `SparkException` and `SparkIllegalArgumentException` are two different exception, even not sub-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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1431830689


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala:
##########
@@ -1300,14 +1300,16 @@ trait ComplexTypeMergingExpression extends Expression {
   lazy val inputTypesForMerging: Seq[DataType] = children.map(_.dataType)
 
   def dataTypeCheck: Unit = {
-    require(
-      inputTypesForMerging.nonEmpty,
-      "The collection of input data types must not be empty.")
-    require(
-      TypeCoercion.haveSameType(inputTypesForMerging),
-      "All input types must be the same except nullable, containsNull, valueContainsNull flags. " +
-        s"The expression is: $this. " +
-        s"The input types found are\n\t${inputTypesForMerging.mkString("\n\t")}.")
+    SparkException.require(
+      requirement = inputTypesForMerging.nonEmpty,
+      errorClass = "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT.NO_INPUTS",
+      messageParameters = Map("expression" -> this.toString))

Review Comment:
   Please, quote the expression:
   ```suggestion
         messageParameters = Map("expression" -> toSQLExpr(this)))
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala:
##########
@@ -1300,14 +1300,16 @@ trait ComplexTypeMergingExpression extends Expression {
   lazy val inputTypesForMerging: Seq[DataType] = children.map(_.dataType)
 
   def dataTypeCheck: Unit = {
-    require(
-      inputTypesForMerging.nonEmpty,
-      "The collection of input data types must not be empty.")
-    require(
-      TypeCoercion.haveSameType(inputTypesForMerging),
-      "All input types must be the same except nullable, containsNull, valueContainsNull flags. " +
-        s"The expression is: $this. " +
-        s"The input types found are\n\t${inputTypesForMerging.mkString("\n\t")}.")
+    SparkException.require(
+      requirement = inputTypesForMerging.nonEmpty,
+      errorClass = "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT.NO_INPUTS",
+      messageParameters = Map("expression" -> this.toString))
+    SparkException.require(
+      requirement = TypeCoercion.haveSameType(inputTypesForMerging),
+      errorClass = "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT.MISMATCHED_TYPES",
+      messageParameters = Map(
+        "expression" -> this.toString,
+        "inputTypes" -> inputTypesForMerging.mkString("\n\t")))

Review Comment:
   ```suggestion
           "inputTypes" -> inputTypesForMerging.map(toSQLType).mkString(", ")))
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala:
##########
@@ -1300,14 +1300,16 @@ trait ComplexTypeMergingExpression extends Expression {
   lazy val inputTypesForMerging: Seq[DataType] = children.map(_.dataType)
 
   def dataTypeCheck: Unit = {
-    require(
-      inputTypesForMerging.nonEmpty,
-      "The collection of input data types must not be empty.")
-    require(
-      TypeCoercion.haveSameType(inputTypesForMerging),
-      "All input types must be the same except nullable, containsNull, valueContainsNull flags. " +
-        s"The expression is: $this. " +
-        s"The input types found are\n\t${inputTypesForMerging.mkString("\n\t")}.")
+    SparkException.require(
+      requirement = inputTypesForMerging.nonEmpty,
+      errorClass = "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT.NO_INPUTS",
+      messageParameters = Map("expression" -> this.toString))
+    SparkException.require(
+      requirement = TypeCoercion.haveSameType(inputTypesForMerging),
+      errorClass = "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT.MISMATCHED_TYPES",
+      messageParameters = Map(
+        "expression" -> this.toString,

Review Comment:
   ```suggestion
           "expression" -> toSQLExpr(this),
   ```



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -500,6 +500,25 @@
     ],
     "sqlState" : "22004"
   },
+  "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT" : {
+    "message" : [
+      "Cannot process input data types for expression: <expression>."

Review Comment:
   ```suggestion
         "Cannot process input data types for the expression: <expression>."
   ```



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -500,6 +500,25 @@
     ],
     "sqlState" : "22004"
   },
+  "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT" : {
+    "message" : [
+      "Cannot process input data types for expression: <expression>."
+    ],
+    "subClass" : {
+      "MISMATCHED_TYPES" : {
+        "message" : [
+          "All input types must be the same except nullable, containsNull, valueContainsNull flags.",
+          "The input types found are <inputTypes>."

Review Comment:
   ```suggestion
             "All input types must be the same except nullable, containsNull, valueContainsNull flags, but found the input types <inputTypes>."
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "hannahkamundson (via GitHub)" <gi...@apache.org>.
hannahkamundson commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1425757933


##########
common/utils/src/main/scala/org/apache/spark/SparkException.scala:
##########
@@ -106,6 +106,20 @@ object SparkException {
       messageParameters = Map("message" -> msg),
       cause = cause)
   }
+
+  /**
+   * This is like the Scala require precondition, except it uses SparkException.
+   * @param requirement The requirement you want to check
+   * @param errorClass The error class to type if the requirement isn't passed
+   * @param messageParameters Message parameters to append to the message
+   */
+  def sparkRequire(requirement: Boolean,

Review Comment:
   Done!



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "llkj1 (via GitHub)" <gi...@apache.org>.
llkj1 commented on PR #44336:
URL: https://github.com/apache/spark/pull/44336#issuecomment-1880016835

   感谢您与我们联系。(Thank you for contacting us.)
   ------------------------------------------------------------
   
   我们收到了您的请求,并会在一至两个工作日内给您回复。您的案例编号是 102195524488。
   
   想了解更多与开发相关主题的信息,请访问 开发者支持网页 (https://developer.apple.com/support/)。
                     
   Apple Developer Program Support
   
   
   We’ve received your support request and will get back to you in one to two business days. Your case number is 102195524488.
   
   For additional information on development-related topics, visit:
   https://developer.apple.com/support/
   
   Best regards, 
   
   Apple Developer Program Support
              
   
   ------------------------------------------------------------
   Copyright (c) 2024 Apple Inc. All rights reserved.
   
   Contact Us
   https://developer.apple.com/contact/
   
   Developer
   https://developer.apple.com/
   
   My Apple ID
   https://appleid.apple.com
   
   Privacy Policy
   https://www.apple.com/privacy/
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1436132674


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala:
##########
@@ -1300,14 +1301,16 @@ trait ComplexTypeMergingExpression extends Expression {
   lazy val inputTypesForMerging: Seq[DataType] = children.map(_.dataType)
 
   def dataTypeCheck: Unit = {
-    require(
-      inputTypesForMerging.nonEmpty,
-      "The collection of input data types must not be empty.")
-    require(
-      TypeCoercion.haveSameType(inputTypesForMerging),
-      "All input types must be the same except nullable, containsNull, valueContainsNull flags. " +
-        s"The expression is: $this. " +
-        s"The input types found are\n\t${inputTypesForMerging.mkString("\n\t")}.")
+    SparkException.require(
+      requirement = inputTypesForMerging.nonEmpty,
+      errorClass = "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT.NO_INPUTS",

Review Comment:
   > I'll add tests
   
   Please, add end-to-end tests for the errors to `QueryCompilationErrorsSuite` using public API or SQL, and remove the tests from `ExpressionSuite`.
   
   FYI, we introduce error classes for user facing errors. In the case if an error is not visible to users, we should consider to convert it to an internal 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1461103218


##########
sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala:
##########
@@ -17,17 +17,17 @@
 
 package org.apache.spark.sql.errors
 
-import org.apache.spark.{SPARK_DOC_ROOT, SparkUnsupportedOperationException}
-import org.apache.spark.sql.{AnalysisException, ClassData, IntegratedUDFTestUtils, QueryTest, Row}
+import org.apache.spark.{SPARK_DOC_ROOT, SparkIllegalArgumentException, SparkUnsupportedOperationException}
+import org.apache.spark.sql._
 import org.apache.spark.sql.api.java.{UDF1, UDF2, UDF23Test}
-import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.catalyst.expressions.{Coalesce, Literal, UnsafeRow}
 import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog
 import org.apache.spark.sql.expressions.SparkUserDefinedFunction
-import org.apache.spark.sql.functions.{array, from_json, grouping, grouping_id, lit, struct, sum, udf}
+import org.apache.spark.sql.functions._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SharedSparkSession
-import org.apache.spark.sql.types.{BooleanType, IntegerType, MapType, StringType, StructField, StructType}
+import org.apache.spark.sql.types._

Review Comment:
   ditto



##########
sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala:
##########
@@ -17,17 +17,17 @@
 
 package org.apache.spark.sql.errors
 
-import org.apache.spark.{SPARK_DOC_ROOT, SparkUnsupportedOperationException}
-import org.apache.spark.sql.{AnalysisException, ClassData, IntegratedUDFTestUtils, QueryTest, Row}
+import org.apache.spark.{SPARK_DOC_ROOT, SparkIllegalArgumentException, SparkUnsupportedOperationException}
+import org.apache.spark.sql._
 import org.apache.spark.sql.api.java.{UDF1, UDF2, UDF23Test}
-import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.catalyst.expressions.{Coalesce, Literal, UnsafeRow}
 import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog
 import org.apache.spark.sql.expressions.SparkUserDefinedFunction
-import org.apache.spark.sql.functions.{array, from_json, grouping, grouping_id, lit, struct, sum, udf}
+import org.apache.spark.sql.functions._

Review Comment:
   ditto



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on PR #44336:
URL: https://github.com/apache/spark/pull/44336#issuecomment-1880015640

   @hannahkamundson Please, check the failed tests. It seems they are related to your changes:
   ```
   Map("expression" -> ""coalesce(1, a, a)"", "inputTypes" -> "["INT", "STRING", "STRING"]") did not equal Map("expression" -> "COALESCE", "inputTypes" -> "[INTEGER, STRING, STRING]") (SparkFunSuite.scala:362)
   2024-01-03T01:30:11.4609137Z [info]   Analysis:
   2024-01-03T01:30:11.4611048Z [info]   JavaCollectionWrappers$JMapWrapper("expression": ""coalesce(1, a, a)"" -> "COALESCE", "inputTypes": "["INT", "STRING", "STRING"]" -> "[INTEGER, STRING, STRING]")
   2024-01-03T01:30:11.4613088Z [info]   org.scalatest.exceptions.TestFailedException:
   ```


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1425839094


##########
common/utils/src/main/scala/org/apache/spark/SparkException.scala:
##########
@@ -106,6 +106,20 @@ object SparkException {
       messageParameters = Map("message" -> msg),
       cause = cause)
   }
+
+  /**
+   * This is like the Scala require precondition, except it uses SparkException.
+   * @param requirement The requirement you want to check
+   * @param errorClass The error class to type if the requirement isn't passed
+   * @param messageParameters Message parameters to append to the message
+   */
+  def checkArgs(requirement: Boolean,
+                errorClass: String,
+                messageParameters: Map[String, String]): Unit = {

Review Comment:
   ```suggestion
     def checkArgs(
         requirement: Boolean,
         errorClass: String,
         messageParameters: Map[String, String]): Unit = {
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1461103218


##########
sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala:
##########
@@ -17,17 +17,17 @@
 
 package org.apache.spark.sql.errors
 
-import org.apache.spark.{SPARK_DOC_ROOT, SparkUnsupportedOperationException}
-import org.apache.spark.sql.{AnalysisException, ClassData, IntegratedUDFTestUtils, QueryTest, Row}
+import org.apache.spark.{SPARK_DOC_ROOT, SparkIllegalArgumentException, SparkUnsupportedOperationException}
+import org.apache.spark.sql._
 import org.apache.spark.sql.api.java.{UDF1, UDF2, UDF23Test}
-import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.catalyst.expressions.{Coalesce, Literal, UnsafeRow}
 import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog
 import org.apache.spark.sql.expressions.SparkUserDefinedFunction
-import org.apache.spark.sql.functions.{array, from_json, grouping, grouping_id, lit, struct, sum, udf}
+import org.apache.spark.sql.functions._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SharedSparkSession
-import org.apache.spark.sql.types.{BooleanType, IntegerType, MapType, StringType, StructField, StructType}
+import org.apache.spark.sql.types._

Review Comment:
   ditto



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1461105228


##########
sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala:
##########
@@ -17,17 +17,17 @@
 
 package org.apache.spark.sql.errors
 
-import org.apache.spark.{SPARK_DOC_ROOT, SparkUnsupportedOperationException}
-import org.apache.spark.sql.{AnalysisException, ClassData, IntegratedUDFTestUtils, QueryTest, Row}
+import org.apache.spark.{SPARK_DOC_ROOT, SparkIllegalArgumentException, SparkUnsupportedOperationException}
+import org.apache.spark.sql._

Review Comment:
   but it is ok if there are ±6 imports.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "hannahkamundson (via GitHub)" <gi...@apache.org>.
hannahkamundson commented on PR #44336:
URL: https://github.com/apache/spark/pull/44336#issuecomment-1902758153

   Thanks for reviewing! I am making a follow up PR for that 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "hannahkamundson (via GitHub)" <gi...@apache.org>.
hannahkamundson commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1425846359


##########
common/utils/src/main/scala/org/apache/spark/SparkException.scala:
##########
@@ -106,6 +106,20 @@ object SparkException {
       messageParameters = Map("message" -> msg),
       cause = cause)
   }
+
+  /**
+   * This is like the Scala require precondition, except it uses SparkException.
+   * @param requirement The requirement you want to check
+   * @param errorClass The error class to type if the requirement isn't passed
+   * @param messageParameters Message parameters to append to the message
+   */
+  def checkArgs(requirement: Boolean,
+                errorClass: String,
+                messageParameters: Map[String, String]): Unit = {

Review Comment:
   Thanks! My IntelliJ prefers the first. Do you have specific settings set up to make it default to yours?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1425719892


##########
common/utils/src/main/scala/org/apache/spark/SparkException.scala:
##########
@@ -106,6 +106,20 @@ object SparkException {
       messageParameters = Map("message" -> msg),
       cause = cause)
   }
+
+  /**
+   * This is like the Scala require precondition, except it uses SparkException.
+   * @param requirement The requirement you want to check
+   * @param errorClass The error class to type if the requirement isn't passed
+   * @param messageParameters Message parameters to append to the message
+   */
+  def sparkRequire(requirement: Boolean,

Review Comment:
   I doubt slightly about the name. The `spark` prefix doesn't bring any additional meaning. How about `checkArgs`, for instance. also cc @srielau  



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "hannahkamundson (via GitHub)" <gi...@apache.org>.
hannahkamundson commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1433131387


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -500,6 +500,24 @@
     ],
     "sqlState" : "22004"
   },
+  "COMPLEX_EXPRESSION_UNSUPPORTED_INPUT" : {
+    "message" : [
+      "Cannot process input data types for the expression: <expression>."
+    ],
+    "subClass" : {
+      "MISMATCHED_TYPES" : {
+        "message" : [
+          "All input types must be the same except nullable, containsNull, valueContainsNull flags, but found the input types <inputTypes>."
+        ]
+      },
+      "NO_INPUTS" : {
+        "message" : [
+          "The collection of input data types must not be empty."
+        ]
+      }
+    },
+    "sqlState" : "42K09"

Review Comment:
   I'm not sure about the `sqlState`, but I don't think we should introduce a separate error class for `require` because there can be different `require` messages/errors depending on what was checked. This specific error class isn't for all `require`, it is just for one.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL][DOCS] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44336:
URL: https://github.com/apache/spark/pull/44336#discussion_r1461103077


##########
sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala:
##########
@@ -17,17 +17,17 @@
 
 package org.apache.spark.sql.errors
 
-import org.apache.spark.{SPARK_DOC_ROOT, SparkUnsupportedOperationException}
-import org.apache.spark.sql.{AnalysisException, ClassData, IntegratedUDFTestUtils, QueryTest, Row}
+import org.apache.spark.{SPARK_DOC_ROOT, SparkIllegalArgumentException, SparkUnsupportedOperationException}
+import org.apache.spark.sql._

Review Comment:
   Could you revert this, please. See https://github.com/databricks/scala-style-guide?tab=readme-ov-file#imports



##########
sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala:
##########
@@ -17,17 +17,17 @@
 
 package org.apache.spark.sql.errors
 
-import org.apache.spark.{SPARK_DOC_ROOT, SparkUnsupportedOperationException}
-import org.apache.spark.sql.{AnalysisException, ClassData, IntegratedUDFTestUtils, QueryTest, Row}
+import org.apache.spark.{SPARK_DOC_ROOT, SparkIllegalArgumentException, SparkUnsupportedOperationException}
+import org.apache.spark.sql._
 import org.apache.spark.sql.api.java.{UDF1, UDF2, UDF23Test}
-import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.catalyst.expressions.{Coalesce, Literal, UnsafeRow}
 import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog
 import org.apache.spark.sql.expressions.SparkUserDefinedFunction
-import org.apache.spark.sql.functions.{array, from_json, grouping, grouping_id, lit, struct, sum, udf}
+import org.apache.spark.sql.functions._

Review Comment:
   ditto



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on PR #44336:
URL: https://github.com/apache/spark/pull/44336#issuecomment-1902752730

   +1, LGTM. Merging to master.
   Thank you, @hannahkamundson and @HyukjinKwon for review.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk closed pull request #44336: [SPARK-42332][SQL] Changing the require to a SparkException in ComplexTypeMergingExpression
URL: https://github.com/apache/spark/pull/44336


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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