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 2022/11/07 08:25:22 UTC

[GitHub] [spark] itholic commented on a diff in pull request #38123: [SPARK-40663][SQL] Migrate execution errors onto error classes: _LEGACY_ERROR_TEMP_2101-2125

itholic commented on code in PR #38123:
URL: https://github.com/apache/spark/pull/38123#discussion_r1015123703


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala:
##########
@@ -1156,114 +1156,180 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {
       cause = null)
   }
 
-  def notSupportNonPrimitiveTypeError(): Throwable = {
-    new RuntimeException("Not support non-primitive type now")
+  def notSupportNonPrimitiveTypeError(): SparkRuntimeException = {
+    new SparkRuntimeException(
+      errorClass = "_LEGACY_ERROR_TEMP_2101",
+      messageParameters = Map.empty)
   }
 
   def unsupportedTypeError(dataType: DataType): Throwable = {
-    new Exception(s"Unsupported type: ${dataType.catalogString}")
+    new SparkException(
+      errorClass = "_LEGACY_ERROR_TEMP_2102",
+      messageParameters = Map("catalogString" -> dataType.catalogString),
+      cause = null)
   }
 
   def useDictionaryEncodingWhenDictionaryOverflowError(): Throwable = {
-    new IllegalStateException(
-      "Dictionary encoding should not be used because of dictionary overflow.")
+    new SparkException(
+      errorClass = "_LEGACY_ERROR_TEMP_2103",
+      messageParameters = Map.empty,
+      cause = null)
   }
 
   def endOfIteratorError(): Throwable = {
-    new NoSuchElementException("End of the iterator")
+    new SparkException(
+      errorClass = "_LEGACY_ERROR_TEMP_2104",
+      messageParameters = Map.empty,
+      cause = null)
   }
 
   def cannotAllocateMemoryToGrowBytesToBytesMapError(): Throwable = {
-    new IOException("Could not allocate memory to grow BytesToBytesMap")
+    new SparkException(
+      errorClass = "_LEGACY_ERROR_TEMP_2105",
+      messageParameters = Map.empty,
+      cause = null)
   }
 
   def cannotAcquireMemoryToBuildLongHashedRelationError(size: Long, got: Long): Throwable = {
-    new SparkException(s"Can't acquire $size bytes memory to build hash relation, " +
-      s"got $got bytes")
+    new SparkException(
+      errorClass = "_LEGACY_ERROR_TEMP_2106",
+      messageParameters = Map("size" -> size.toString(), "got" -> got.toString()),
+      cause = null)
   }
 
   def cannotAcquireMemoryToBuildUnsafeHashedRelationError(): Throwable = {
-    new SparkOutOfMemoryError("There is not enough memory to build hash map")
+    new SparkOutOfMemoryError(
+      "_LEGACY_ERROR_TEMP_2107")
   }
 
-  def rowLargerThan256MUnsupportedError(): Throwable = {
-    new UnsupportedOperationException("Does not support row that is larger than 256M")
+  def rowLargerThan256MUnsupportedError(): SparkUnsupportedOperationException = {
+    new SparkUnsupportedOperationException(
+      errorClass = "_LEGACY_ERROR_TEMP_2108",
+      messageParameters = Map.empty)
   }
 
-  def cannotBuildHashedRelationWithUniqueKeysExceededError(): Throwable = {
-    new UnsupportedOperationException(
-      "Cannot build HashedRelation with more than 1/3 billions unique keys")
+  def cannotBuildHashedRelationWithUniqueKeysExceededError(): SparkUnsupportedOperationException = {
+    new SparkUnsupportedOperationException(
+      errorClass = "_LEGACY_ERROR_TEMP_2109",
+      messageParameters = Map.empty)
   }
 
-  def cannotBuildHashedRelationLargerThan8GError(): Throwable = {
-    new UnsupportedOperationException(
-      "Can not build a HashedRelation that is larger than 8G")
+  def cannotBuildHashedRelationLargerThan8GError(): SparkUnsupportedOperationException = {
+    new SparkUnsupportedOperationException(
+      errorClass = "_LEGACY_ERROR_TEMP_2110",
+      messageParameters = Map.empty)
   }
 
   def failedToPushRowIntoRowQueueError(rowQueue: String): Throwable = {
-    new SparkException(s"failed to push a row into $rowQueue")
+    new SparkException(
+      errorClass = "_LEGACY_ERROR_TEMP_2111",
+      messageParameters = Map("rowQueue" -> rowQueue),
+      cause = null)
   }
 
-  def unexpectedWindowFunctionFrameError(frame: String): Throwable = {
-    new RuntimeException(s"Unexpected window function frame $frame.")
+  def unexpectedWindowFunctionFrameError(frame: String): SparkRuntimeException = {
+    new SparkRuntimeException(
+      errorClass = "_LEGACY_ERROR_TEMP_2112",
+      messageParameters = Map("frame" -> frame))
   }
 
   def cannotParseStatisticAsPercentileError(
-      stats: String, e: NumberFormatException): Throwable = {
-    new IllegalArgumentException(s"Unable to parse $stats as a percentile", e)
+      stats: String, e: NumberFormatException): SparkIllegalArgumentException = {

Review Comment:
   Thanks for pointing out.
   
   Let me address it with https://github.com/apache/spark/pull/38104#discussion_r1014645923.



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