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 2021/03/24 00:11:12 UTC

[GitHub] [spark] allisonwang-db commented on a change in pull request #31920: [SPARK-33604][SQL] Group exception messages in sql/execution

allisonwang-db commented on a change in pull request #31920:
URL: https://github.com/apache/spark/pull/31920#discussion_r600037091



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
##########
@@ -556,4 +559,179 @@ object QueryExecutionErrors {
       left: StructType, right: StructType, e: Throwable): Throwable = {
     new SparkException(s"Failed to merge incompatible schemas $left and $right", e)
   }
+
+  def unsupportedTypeForArgumentValuesError(values: Any): Throwable = {
+    new IllegalArgumentException(s"Unsupported type for argument values: $values")
+  }
+
+  def notSetExecutionIDError(): Throwable = {

Review comment:
       executionIDNotSetError

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
##########
@@ -556,4 +559,179 @@ object QueryExecutionErrors {
       left: StructType, right: StructType, e: Throwable): Throwable = {
     new SparkException(s"Failed to merge incompatible schemas $left and $right", e)
   }
+
+  def unsupportedTypeForArgumentValuesError(values: Any): Throwable = {
+    new IllegalArgumentException(s"Unsupported type for argument values: $values")
+  }
+
+  def notSetExecutionIDError(): Throwable = {
+    new IllegalStateException("Execution ID should be set")
+  }
+
+  def executeCanonicalizedPlanError(): Throwable = {
+    new IllegalStateException("A canonicalized plan is not supposed to be executed.")
+  }
+
+  def doExecuteBroadcastNotImplementedError(nodeName: String): Throwable = {
+    new UnsupportedOperationException(s"$nodeName does not implement doExecuteBroadcast")
+  }
+
+  def sparkPlanHasColumnSupportMismatchError(sparkPlan: Class[_], plan: String): Throwable = {
+    new IllegalStateException(s"Internal Error $sparkPlan has column support" +
+      s" mismatch:\n$plan")
+  }
+
+  def logicalOperatorNotReplacedByOptimizedOperatorError(

Review comment:
       logicalOperatorNotReplacedError

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
##########
@@ -556,4 +559,179 @@ object QueryExecutionErrors {
       left: StructType, right: StructType, e: Throwable): Throwable = {
     new SparkException(s"Failed to merge incompatible schemas $left and $right", e)
   }
+
+  def unsupportedTypeForArgumentValuesError(values: Any): Throwable = {
+    new IllegalArgumentException(s"Unsupported type for argument values: $values")
+  }
+
+  def notSetExecutionIDError(): Throwable = {
+    new IllegalStateException("Execution ID should be set")
+  }
+
+  def executeCanonicalizedPlanError(): Throwable = {
+    new IllegalStateException("A canonicalized plan is not supposed to be executed.")
+  }
+
+  def doExecuteBroadcastNotImplementedError(nodeName: String): Throwable = {
+    new UnsupportedOperationException(s"$nodeName does not implement doExecuteBroadcast")
+  }
+
+  def sparkPlanHasColumnSupportMismatchError(sparkPlan: Class[_], plan: String): Throwable = {
+    new IllegalStateException(s"Internal Error $sparkPlan has column support" +
+      s" mismatch:\n$plan")
+  }
+
+  def logicalOperatorNotReplacedByOptimizedOperatorError(
+      logicalOperator: String, replaceOperator: String): Throwable = {
+    new IllegalStateException(s"$logicalOperator should have been" +
+      s" replaced by $replaceOperator in the optimizer")
+  }
+
+  def ddlUnsupportedTemporarilyError(ddl: String): Throwable = {
+    new UnsupportedOperationException(s"$ddl is not supported temporarily.")
+  }
+
+  def operatingOnCanonicalizationPlanError(): Throwable = {
+    new IllegalStateException("operating on canonicalization plan")
+  }
+
+  def unexpectedShufflePartitionSpecError(p: String): Throwable = {
+    new IllegalStateException(s"unexpected $p")
+  }
+
+  def operatingOnCanonicalizedPlanError(): Throwable = {
+    new IllegalStateException("operating on canonicalized plan")
+  }
+
+  def wrongPlanForShuffleStageError(plan: String): Throwable = {
+    new IllegalStateException(s"wrong plan for shuffle stage:\n $plan")
+  }
+
+  def wrongPlanForBroadcastStageError(plan: String): Throwable = {
+    new IllegalStateException(s"wrong plan for broadcast stage:\n $plan")
+  }
+
+  def executeBroadcastTimeoutError(timeout: Long): Throwable = {
+    new SparkException(
+      s"""
+         |Could not execute broadcast in $timeout secs. You can increase the timeout
+         |for broadcasts via ${SQLConf.BROADCAST_TIMEOUT.key} or disable broadcast join
+         |by setting ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key} to -1
+       """.stripMargin.replaceAll("\n", " "))
+  }
+
+  def cannotCompareCostWithTargetCostError(cost: String): Throwable = {
+    new IllegalArgumentException(s"Could not compare cost with $cost")
+  }
+
+  def callMethodWithEmptyGroupingExpressionsError(): Throwable = {
+    new IllegalStateException(
+      "This method should not be called when groupingExpressions is not empty.")
+  }
+
+  def unsupportedDataTypeError(dt: DataType): Throwable = {
+    new UnsupportedOperationException(s"Unsupported data type: ${dt.catalogString}")
+  }
+
+  def notSupportTypeError(dataType: DataType): Throwable = {
+    new Exception(s"not support type: $dataType")
+  }
+
+  def notSupportNonPrimitiveTypeError(): Throwable = {
+    new RuntimeException("Not support non-primitive type now")
+  }
+
+  def unsupportedTypeError(dataType: DataType): Throwable = {
+    new Exception(s"Unsupported type: ${dataType.catalogString}")
+  }
+
+  def columnarInputUnsupportedError(): Throwable = {
+    new IllegalStateException("Columnar input is not supported")
+  }
+
+  def unsupportedTypeInEncodingError(operator: String): Throwable = {
+    new IllegalStateException(s"Not supported type in $operator.")
+  }
+
+  def useDictionaryEncodingWhenDictionaryOverflowError(): Throwable = {
+    new IllegalStateException(
+      "Dictionary encoding should not be used because of dictionary overflow.")
+  }
+
+  def hashJoinCannotTakeJoinTypeError(joinType: JoinType): Throwable = {
+    new IllegalArgumentException(s"HashJoin should not take $joinType as the JoinType")
+  }
+
+  def hashJoinCannotTakeJoinTypeWithBuildLeftError(joinType: JoinType): Throwable = {
+    new IllegalArgumentException(
+      s"HashJoin should not take $joinType as the JoinType with building left side")
+  }
+
+  def hashJoinCannotTakeJoinTypeWithBuildRightError(joinType: JoinType): Throwable = {
+    new IllegalArgumentException(
+      s"HashJoin should not take $joinType as the JoinType with building right side")
+  }
+
+  def endOfIteratorError(): Throwable = {
+    new NoSuchElementException("End of the iterator")
+  }
+
+  def cannotAllocateMemoryToGrowBytesToBytesMapError(): Throwable = {
+    new IOException("Could not allocate memory to grow BytesToBytesMap")
+  }
+
+  def cannotAcquireMemoryToBuildHashRelationError(size: Long, got: Long): Throwable = {
+    new SparkException(s"Can't acquire $size bytes memory to build hash relation, " +
+      s"got $got bytes")
+  }
+
+  def rowLargerThanLimitUnsupportedError(): Throwable = {
+    new UnsupportedOperationException("Does not support row that is larger than 256M")
+  }
+
+  def cannotBuildHashedRelationWithUniqueKeysExceededError(): Throwable = {
+    new UnsupportedOperationException(
+      "Cannot build HashedRelation with more than 1/3 billions unique keys")
+  }
+
+  def cannotBuildHashedRelationLargerThanLimitError(): Throwable = {

Review comment:
       cannotBuildHashedRelationLargerThan8GError

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
##########
@@ -303,4 +303,64 @@ object QueryParsingErrors {
     new ParseException(s"Found duplicate keys '$key'.", ctx)
   }
 
+  def formatForSetConfigurationUnExpectedError(ctx: SetConfigurationContext): Throwable = {
+    new ParseException(
+      s"""
+         |Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include
+         |special characters in key, or include semicolon in value, please use quotes,
+         |e.g., SET `ke y`=`v;alue`.
+       """.stripMargin.replaceAll("\n", " "), ctx)
+  }
+
+  def invalidPropertyKeyForSetQuotedConfigurationError(

Review comment:
       invalidPropertyKeyForSetConfigurationError

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
##########
@@ -303,4 +303,64 @@ object QueryParsingErrors {
     new ParseException(s"Found duplicate keys '$key'.", ctx)
   }
 
+  def formatForSetConfigurationUnExpectedError(ctx: SetConfigurationContext): Throwable = {
+    new ParseException(
+      s"""
+         |Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include
+         |special characters in key, or include semicolon in value, please use quotes,
+         |e.g., SET `ke y`=`v;alue`.
+       """.stripMargin.replaceAll("\n", " "), ctx)
+  }
+
+  def invalidPropertyKeyForSetQuotedConfigurationError(
+      keyCandidate: String, valueStr: String, ctx: SetQuotedConfigurationContext): Throwable = {
+    new ParseException(s"'$keyCandidate' is an invalid property key, please " +
+      s"use quotes, e.g. SET `$keyCandidate`=`$valueStr`", ctx)
+  }
+
+  def invalidPropertyValueForSetQuotedConfigurationError(
+      valueCandidate: String, keyStr: String, ctx: SetQuotedConfigurationContext): Throwable = {
+    new ParseException(s"'$valueCandidate' is an invalid property value, please " +
+      s"use quotes, e.g. SET `$keyStr`=`$valueCandidate`", ctx)
+  }
+
+  def formatForResetConfigurationUnExpectedError(ctx: ResetConfigurationContext): Throwable = {
+    new ParseException(
+      s"""
+         |Expected format is 'RESET' or 'RESET key'. If you want to include special characters
+         |in key, please use quotes, e.g., RESET `ke y`.
+       """.stripMargin.replaceAll("\n", " "), ctx)
+  }
+
+  def intervalValueOutOfRangeError(ctx: IntervalContext): Throwable = {
+    new ParseException("The interval value must be in the range of [-18, +18] hours" +
+      " with second precision", ctx)
+  }
+
+  def invalidTimeZoneDisplacementValueError(ctx: SetTimeZoneContext): Throwable = {
+    new ParseException("Invalid time zone displacement value", ctx)
+  }
+
+  def createTempTableNotSpecifyProviderError(ctx: CreateTableContext): Throwable = {
+    new ParseException("CREATE TEMPORARY TABLE without a provider is not allowed.", ctx)
+  }
+
+  def rowFormatNotUsedWithStoredAsError(ctx: CreateTableLikeContext): Throwable = {
+    new ParseException("'ROW FORMAT' must be used with 'STORED AS'", ctx)
+  }
+
+  def useDefinedRecordReaderOrWriterClassesError(ctx: ParserRuleContext): Throwable = {
+    new ParseException(
+      "Unsupported operation: Used defined record reader/writer classes.", ctx)
+  }
+
+  def directoryPathAndOptionPathBothSpecifiedError(ctx: InsertOverwriteDirContext): Throwable = {

Review comment:
       directoryPathAndOptionsPathBothSpecifiedError

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
##########
@@ -303,4 +303,64 @@ object QueryParsingErrors {
     new ParseException(s"Found duplicate keys '$key'.", ctx)
   }
 
+  def formatForSetConfigurationUnExpectedError(ctx: SetConfigurationContext): Throwable = {
+    new ParseException(
+      s"""
+         |Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include
+         |special characters in key, or include semicolon in value, please use quotes,
+         |e.g., SET `ke y`=`v;alue`.
+       """.stripMargin.replaceAll("\n", " "), ctx)
+  }
+
+  def invalidPropertyKeyForSetQuotedConfigurationError(
+      keyCandidate: String, valueStr: String, ctx: SetQuotedConfigurationContext): Throwable = {
+    new ParseException(s"'$keyCandidate' is an invalid property key, please " +
+      s"use quotes, e.g. SET `$keyCandidate`=`$valueStr`", ctx)
+  }
+
+  def invalidPropertyValueForSetQuotedConfigurationError(

Review comment:
       invalidPropertyValueForSetConfigurationError

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
##########
@@ -303,4 +303,64 @@ object QueryParsingErrors {
     new ParseException(s"Found duplicate keys '$key'.", ctx)
   }
 
+  def formatForSetConfigurationUnExpectedError(ctx: SetConfigurationContext): Throwable = {
+    new ParseException(
+      s"""
+         |Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include
+         |special characters in key, or include semicolon in value, please use quotes,
+         |e.g., SET `ke y`=`v;alue`.
+       """.stripMargin.replaceAll("\n", " "), ctx)
+  }
+
+  def invalidPropertyKeyForSetQuotedConfigurationError(
+      keyCandidate: String, valueStr: String, ctx: SetQuotedConfigurationContext): Throwable = {
+    new ParseException(s"'$keyCandidate' is an invalid property key, please " +
+      s"use quotes, e.g. SET `$keyCandidate`=`$valueStr`", ctx)
+  }
+
+  def invalidPropertyValueForSetQuotedConfigurationError(
+      valueCandidate: String, keyStr: String, ctx: SetQuotedConfigurationContext): Throwable = {
+    new ParseException(s"'$valueCandidate' is an invalid property value, please " +
+      s"use quotes, e.g. SET `$keyStr`=`$valueCandidate`", ctx)
+  }
+
+  def formatForResetConfigurationUnExpectedError(ctx: ResetConfigurationContext): Throwable = {

Review comment:
       unexpectedFormatForResetConfigurationError

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
##########
@@ -556,4 +559,179 @@ object QueryExecutionErrors {
       left: StructType, right: StructType, e: Throwable): Throwable = {
     new SparkException(s"Failed to merge incompatible schemas $left and $right", e)
   }
+
+  def unsupportedTypeForArgumentValuesError(values: Any): Throwable = {
+    new IllegalArgumentException(s"Unsupported type for argument values: $values")
+  }
+
+  def notSetExecutionIDError(): Throwable = {
+    new IllegalStateException("Execution ID should be set")
+  }
+
+  def executeCanonicalizedPlanError(): Throwable = {
+    new IllegalStateException("A canonicalized plan is not supposed to be executed.")
+  }
+
+  def doExecuteBroadcastNotImplementedError(nodeName: String): Throwable = {
+    new UnsupportedOperationException(s"$nodeName does not implement doExecuteBroadcast")
+  }
+
+  def sparkPlanHasColumnSupportMismatchError(sparkPlan: Class[_], plan: String): Throwable = {
+    new IllegalStateException(s"Internal Error $sparkPlan has column support" +
+      s" mismatch:\n$plan")
+  }
+
+  def logicalOperatorNotReplacedByOptimizedOperatorError(
+      logicalOperator: String, replaceOperator: String): Throwable = {
+    new IllegalStateException(s"$logicalOperator should have been" +
+      s" replaced by $replaceOperator in the optimizer")
+  }
+
+  def ddlUnsupportedTemporarilyError(ddl: String): Throwable = {
+    new UnsupportedOperationException(s"$ddl is not supported temporarily.")
+  }
+
+  def operatingOnCanonicalizationPlanError(): Throwable = {
+    new IllegalStateException("operating on canonicalization plan")
+  }
+
+  def unexpectedShufflePartitionSpecError(p: String): Throwable = {
+    new IllegalStateException(s"unexpected $p")
+  }
+
+  def operatingOnCanonicalizedPlanError(): Throwable = {
+    new IllegalStateException("operating on canonicalized plan")
+  }
+
+  def wrongPlanForShuffleStageError(plan: String): Throwable = {
+    new IllegalStateException(s"wrong plan for shuffle stage:\n $plan")
+  }
+
+  def wrongPlanForBroadcastStageError(plan: String): Throwable = {
+    new IllegalStateException(s"wrong plan for broadcast stage:\n $plan")
+  }
+
+  def executeBroadcastTimeoutError(timeout: Long): Throwable = {
+    new SparkException(
+      s"""
+         |Could not execute broadcast in $timeout secs. You can increase the timeout
+         |for broadcasts via ${SQLConf.BROADCAST_TIMEOUT.key} or disable broadcast join
+         |by setting ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key} to -1
+       """.stripMargin.replaceAll("\n", " "))
+  }
+
+  def cannotCompareCostWithTargetCostError(cost: String): Throwable = {
+    new IllegalArgumentException(s"Could not compare cost with $cost")
+  }
+
+  def callMethodWithEmptyGroupingExpressionsError(): Throwable = {
+    new IllegalStateException(
+      "This method should not be called when groupingExpressions is not empty.")
+  }
+
+  def unsupportedDataTypeError(dt: DataType): Throwable = {
+    new UnsupportedOperationException(s"Unsupported data type: ${dt.catalogString}")
+  }
+
+  def notSupportTypeError(dataType: DataType): Throwable = {
+    new Exception(s"not support type: $dataType")
+  }
+
+  def notSupportNonPrimitiveTypeError(): Throwable = {
+    new RuntimeException("Not support non-primitive type now")
+  }
+
+  def unsupportedTypeError(dataType: DataType): Throwable = {
+    new Exception(s"Unsupported type: ${dataType.catalogString}")
+  }
+
+  def columnarInputUnsupportedError(): Throwable = {
+    new IllegalStateException("Columnar input is not supported")
+  }
+
+  def unsupportedTypeInEncodingError(operator: String): Throwable = {
+    new IllegalStateException(s"Not supported type in $operator.")
+  }
+
+  def useDictionaryEncodingWhenDictionaryOverflowError(): Throwable = {
+    new IllegalStateException(
+      "Dictionary encoding should not be used because of dictionary overflow.")
+  }
+
+  def hashJoinCannotTakeJoinTypeError(joinType: JoinType): Throwable = {
+    new IllegalArgumentException(s"HashJoin should not take $joinType as the JoinType")
+  }
+
+  def hashJoinCannotTakeJoinTypeWithBuildLeftError(joinType: JoinType): Throwable = {
+    new IllegalArgumentException(
+      s"HashJoin should not take $joinType as the JoinType with building left side")
+  }
+
+  def hashJoinCannotTakeJoinTypeWithBuildRightError(joinType: JoinType): Throwable = {
+    new IllegalArgumentException(
+      s"HashJoin should not take $joinType as the JoinType with building right side")
+  }
+
+  def endOfIteratorError(): Throwable = {
+    new NoSuchElementException("End of the iterator")
+  }
+
+  def cannotAllocateMemoryToGrowBytesToBytesMapError(): Throwable = {
+    new IOException("Could not allocate memory to grow BytesToBytesMap")
+  }
+
+  def cannotAcquireMemoryToBuildHashRelationError(size: Long, got: Long): Throwable = {
+    new SparkException(s"Can't acquire $size bytes memory to build hash relation, " +
+      s"got $got bytes")
+  }
+
+  def rowLargerThanLimitUnsupportedError(): Throwable = {
+    new UnsupportedOperationException("Does not support row that is larger than 256M")
+  }
+
+  def cannotBuildHashedRelationWithUniqueKeysExceededError(): Throwable = {
+    new UnsupportedOperationException(
+      "Cannot build HashedRelation with more than 1/3 billions unique keys")
+  }
+
+  def cannotBuildHashedRelationLargerThanLimitError(): Throwable = {
+    new UnsupportedOperationException(
+      "Can not build a HashedRelation that is larger than 8G")
+  }
+
+  def cannotMergeSQLMetricWithAccumulatorV2Error(
+      metric: String, accumulatorV2: AccumulatorV2[Long, Long]): Throwable = {
+    new UnsupportedOperationException(
+      s"Cannot merge $metric with ${accumulatorV2.getClass.getName}")
+  }
+
+  def unexpectedMetricsTypeError(metricsType: String): Throwable = {
+    new IllegalStateException(s"unexpected metrics type: $metricsType")
+  }
+
+  def failedToPushRowIntoRowQueueError(rowQueue: String): Throwable = {
+    new SparkException(s"failed to push a row into $rowQueue")
+  }
+
+  def unexpectedWindowFunctionFrameError(frame: String): Throwable = {
+    new RuntimeException(s"Unexpected window function frame $frame.")
+  }
+
+  def cannotParseStatisticAsPercentileError(
+      stats: String, e: NumberFormatException): Throwable = {
+    new IllegalArgumentException(s"Unable to parse $stats as a percentile", e)
+  }
+
+  def statisticIsNotRecognisedStatisticError(stats: String): Throwable = {

Review comment:
       statisticNotRecognizedError

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
##########
@@ -556,4 +559,179 @@ object QueryExecutionErrors {
       left: StructType, right: StructType, e: Throwable): Throwable = {
     new SparkException(s"Failed to merge incompatible schemas $left and $right", e)
   }
+
+  def unsupportedTypeForArgumentValuesError(values: Any): Throwable = {
+    new IllegalArgumentException(s"Unsupported type for argument values: $values")
+  }
+
+  def notSetExecutionIDError(): Throwable = {
+    new IllegalStateException("Execution ID should be set")
+  }
+
+  def executeCanonicalizedPlanError(): Throwable = {
+    new IllegalStateException("A canonicalized plan is not supposed to be executed.")
+  }
+
+  def doExecuteBroadcastNotImplementedError(nodeName: String): Throwable = {
+    new UnsupportedOperationException(s"$nodeName does not implement doExecuteBroadcast")
+  }
+
+  def sparkPlanHasColumnSupportMismatchError(sparkPlan: Class[_], plan: String): Throwable = {
+    new IllegalStateException(s"Internal Error $sparkPlan has column support" +
+      s" mismatch:\n$plan")
+  }
+
+  def logicalOperatorNotReplacedByOptimizedOperatorError(
+      logicalOperator: String, replaceOperator: String): Throwable = {
+    new IllegalStateException(s"$logicalOperator should have been" +
+      s" replaced by $replaceOperator in the optimizer")
+  }
+
+  def ddlUnsupportedTemporarilyError(ddl: String): Throwable = {
+    new UnsupportedOperationException(s"$ddl is not supported temporarily.")
+  }
+
+  def operatingOnCanonicalizationPlanError(): Throwable = {
+    new IllegalStateException("operating on canonicalization plan")
+  }
+
+  def unexpectedShufflePartitionSpecError(p: String): Throwable = {
+    new IllegalStateException(s"unexpected $p")
+  }
+
+  def operatingOnCanonicalizedPlanError(): Throwable = {
+    new IllegalStateException("operating on canonicalized plan")
+  }
+
+  def wrongPlanForShuffleStageError(plan: String): Throwable = {
+    new IllegalStateException(s"wrong plan for shuffle stage:\n $plan")
+  }
+
+  def wrongPlanForBroadcastStageError(plan: String): Throwable = {
+    new IllegalStateException(s"wrong plan for broadcast stage:\n $plan")
+  }
+
+  def executeBroadcastTimeoutError(timeout: Long): Throwable = {
+    new SparkException(
+      s"""
+         |Could not execute broadcast in $timeout secs. You can increase the timeout
+         |for broadcasts via ${SQLConf.BROADCAST_TIMEOUT.key} or disable broadcast join
+         |by setting ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key} to -1
+       """.stripMargin.replaceAll("\n", " "))
+  }
+
+  def cannotCompareCostWithTargetCostError(cost: String): Throwable = {
+    new IllegalArgumentException(s"Could not compare cost with $cost")
+  }
+
+  def callMethodWithEmptyGroupingExpressionsError(): Throwable = {
+    new IllegalStateException(
+      "This method should not be called when groupingExpressions is not empty.")
+  }
+
+  def unsupportedDataTypeError(dt: DataType): Throwable = {
+    new UnsupportedOperationException(s"Unsupported data type: ${dt.catalogString}")
+  }
+
+  def notSupportTypeError(dataType: DataType): Throwable = {
+    new Exception(s"not support type: $dataType")
+  }
+
+  def notSupportNonPrimitiveTypeError(): Throwable = {
+    new RuntimeException("Not support non-primitive type now")
+  }
+
+  def unsupportedTypeError(dataType: DataType): Throwable = {
+    new Exception(s"Unsupported type: ${dataType.catalogString}")
+  }
+
+  def columnarInputUnsupportedError(): Throwable = {
+    new IllegalStateException("Columnar input is not supported")
+  }
+
+  def unsupportedTypeInEncodingError(operator: String): Throwable = {
+    new IllegalStateException(s"Not supported type in $operator.")
+  }
+
+  def useDictionaryEncodingWhenDictionaryOverflowError(): Throwable = {
+    new IllegalStateException(
+      "Dictionary encoding should not be used because of dictionary overflow.")
+  }
+
+  def hashJoinCannotTakeJoinTypeError(joinType: JoinType): Throwable = {
+    new IllegalArgumentException(s"HashJoin should not take $joinType as the JoinType")
+  }
+
+  def hashJoinCannotTakeJoinTypeWithBuildLeftError(joinType: JoinType): Throwable = {
+    new IllegalArgumentException(
+      s"HashJoin should not take $joinType as the JoinType with building left side")
+  }
+
+  def hashJoinCannotTakeJoinTypeWithBuildRightError(joinType: JoinType): Throwable = {
+    new IllegalArgumentException(
+      s"HashJoin should not take $joinType as the JoinType with building right side")
+  }
+
+  def endOfIteratorError(): Throwable = {
+    new NoSuchElementException("End of the iterator")
+  }
+
+  def cannotAllocateMemoryToGrowBytesToBytesMapError(): Throwable = {
+    new IOException("Could not allocate memory to grow BytesToBytesMap")
+  }
+
+  def cannotAcquireMemoryToBuildHashRelationError(size: Long, got: Long): Throwable = {
+    new SparkException(s"Can't acquire $size bytes memory to build hash relation, " +
+      s"got $got bytes")
+  }
+
+  def rowLargerThanLimitUnsupportedError(): Throwable = {

Review comment:
       rowLargerThan256MUnsupportedError

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
##########
@@ -303,4 +303,64 @@ object QueryParsingErrors {
     new ParseException(s"Found duplicate keys '$key'.", ctx)
   }
 
+  def formatForSetConfigurationUnExpectedError(ctx: SetConfigurationContext): Throwable = {
+    new ParseException(
+      s"""
+         |Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include
+         |special characters in key, or include semicolon in value, please use quotes,
+         |e.g., SET `ke y`=`v;alue`.
+       """.stripMargin.replaceAll("\n", " "), ctx)
+  }
+
+  def invalidPropertyKeyForSetQuotedConfigurationError(
+      keyCandidate: String, valueStr: String, ctx: SetQuotedConfigurationContext): Throwable = {
+    new ParseException(s"'$keyCandidate' is an invalid property key, please " +
+      s"use quotes, e.g. SET `$keyCandidate`=`$valueStr`", ctx)
+  }
+
+  def invalidPropertyValueForSetQuotedConfigurationError(
+      valueCandidate: String, keyStr: String, ctx: SetQuotedConfigurationContext): Throwable = {
+    new ParseException(s"'$valueCandidate' is an invalid property value, please " +
+      s"use quotes, e.g. SET `$keyStr`=`$valueCandidate`", ctx)
+  }
+
+  def formatForResetConfigurationUnExpectedError(ctx: ResetConfigurationContext): Throwable = {
+    new ParseException(
+      s"""
+         |Expected format is 'RESET' or 'RESET key'. If you want to include special characters
+         |in key, please use quotes, e.g., RESET `ke y`.
+       """.stripMargin.replaceAll("\n", " "), ctx)
+  }
+
+  def intervalValueOutOfRangeError(ctx: IntervalContext): Throwable = {
+    new ParseException("The interval value must be in the range of [-18, +18] hours" +
+      " with second precision", ctx)
+  }
+
+  def invalidTimeZoneDisplacementValueError(ctx: SetTimeZoneContext): Throwable = {
+    new ParseException("Invalid time zone displacement value", ctx)
+  }
+
+  def createTempTableNotSpecifyProviderError(ctx: CreateTableContext): Throwable = {
+    new ParseException("CREATE TEMPORARY TABLE without a provider is not allowed.", ctx)
+  }
+
+  def rowFormatNotUsedWithStoredAsError(ctx: CreateTableLikeContext): Throwable = {
+    new ParseException("'ROW FORMAT' must be used with 'STORED AS'", ctx)
+  }
+
+  def useDefinedRecordReaderOrWriterClassesError(ctx: ParserRuleContext): Throwable = {
+    new ParseException(
+      "Unsupported operation: Used defined record reader/writer classes.", ctx)
+  }
+
+  def directoryPathAndOptionPathBothSpecifiedError(ctx: InsertOverwriteDirContext): Throwable = {
+    new ParseException(
+      "Directory path and 'path' in OPTIONS should be specified one, but not both", ctx)
+  }
+
+  def illegalLocalFileSchemaError(ctx: InsertOverwriteDirContext): Throwable = {

Review comment:
       unsupportedLocalFileSchemeError

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
##########
@@ -303,4 +303,64 @@ object QueryParsingErrors {
     new ParseException(s"Found duplicate keys '$key'.", ctx)
   }
 
+  def formatForSetConfigurationUnExpectedError(ctx: SetConfigurationContext): Throwable = {

Review comment:
       unexpectedFomatForSetConfigurationError




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