From 5fddd265a23226f7511e8dc9208e66aedcdbe0bf Mon Sep 17 00:00:00 2001 From: ivanjevtic-db Date: Thu, 12 Sep 2024 16:42:31 +0200 Subject: [PATCH] [SPARK-42846][SQL] Remove error condition _LEGACY_ERROR_TEMP_2011 ### What changes were proposed in this pull request? Removed error condition **_LEGACY_ERROR_TEMP_2011**, removed **dataTypeUnexpectedError**, **typeUnsupportedError**, and replaced them with **SparkException.internalError**. ### Why are the changes needed? It is impossible to trigger the error from user space. [Here](https://github.com/apache/spark/compare/master...ivanjevtic-db:spark:remove-legacy-error-temp-2011?expand=1#diff-688ac8011f7fb514154ff57cfb1278b15aec481d68c1a499c90f8a330d3a42a1L141) I changed dataTypeUnexpectedError to internalError, since _typeSoFar_ argument will always be either: - NullType if this is the first row or - Some type which is returned by the [_inferField_](https://github.com/apache/spark/compare/master...ivanjevtic-db:spark:remove-legacy-error-temp-2011?expand=1#diff-688ac8011f7fb514154ff57cfb1278b15aec481d68c1a499c90f8a330d3a42a1L125) function(which is a valid type). [Here](https://github.com/apache/spark/compare/master...ivanjevtic-db:spark:remove-legacy-error-temp-2011?expand=1#diff-e9a88a888c1543c718c24f25036307cb32348ca3f618a8fa19240bdc3c0ffaf4L553) I changed typeUnsupportedError to internalError, since: - in [this](https://github.com/apache/spark/compare/master...ivanjevtic-db:spark:remove-legacy-error-temp-2011?expand=1#diff-e9a88a888c1543c718c24f25036307cb32348ca3f618a8fa19240bdc3c0ffaf4L204) function call, the exception will be caught and - in [this](https://github.com/apache/spark/compare/master...ivanjevtic-db:spark:remove-legacy-error-temp-2011?expand=1#diff-e9a88a888c1543c718c24f25036307cb32348ca3f618a8fa19240bdc3c0ffaf4L367) function call, a valid _desiredType_ will always be passed. [Here](https://github.com/apache/spark/compare/master...ivanjevtic-db:spark:remove-legacy-error-temp-2011?expand=1#diff-d1fa4a2cbd66cff7d7d8a90d7ac70457a31e906cebb7d43a46a6036507fb4e7bL192) and [here](https://github.com/apache/spark/compare/master...ivanjevtic-db:spark:remove-legacy-error-temp-2011?expand=1#diff-d1fa4a2cbd66cff7d7d8a90d7ac70457a31e906cebb7d43a46a6036507fb4e7bL217) I changed dataTypeUnexpectedError to internalError, since there is a type signature [here](https://github.com/apache/spark/compare/master...ivanjevtic-db:spark:remove-legacy-error-temp-2011?expand=1#diff-d1fa4a2cbd66cff7d7d8a90d7ac70457a31e906cebb7d43a46a6036507fb4e7bL104) which prevents _child.dataType_ from being unexpected type. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Build passing. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48086 from ivanjevtic-db/remove-legacy-error-temp-2011. Authored-by: ivanjevtic-db Signed-off-by: Max Gekk --- .../src/main/resources/error/error-conditions.json | 5 ----- .../spark/sql/catalyst/csv/CSVInferSchema.scala | 4 ++-- .../aggregate/ApproximatePercentile.scala | 6 +++--- .../spark/sql/errors/QueryExecutionErrors.scala | 12 ------------ .../execution/datasources/PartitioningUtils.scala | 4 ++-- 5 files changed, 7 insertions(+), 24 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 29eda228c2daa..0ebeea9aed8d2 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -6656,11 +6656,6 @@ "Type does not support ordered operations." ] }, - "_LEGACY_ERROR_TEMP_2011" : { - "message" : [ - "Unexpected data type ." - ] - }, "_LEGACY_ERROR_TEMP_2013" : { "message" : [ "Negative values found in " diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala index 2c27da3cf6e15..5444ab6845867 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala @@ -21,12 +21,12 @@ import java.util.Locale import scala.util.control.Exception.allCatch +import org.apache.spark.SparkException import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter} import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT -import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -138,7 +138,7 @@ class CSVInferSchema(val options: CSVOptions) extends Serializable { case BooleanType => tryParseBoolean(field) case StringType => StringType case other: DataType => - throw QueryExecutionErrors.dataTypeUnexpectedError(other) + throw SparkException.internalError(s"Unexpected data type $other") } compatibleType(typeSoFar, typeElemInfer).getOrElse(StringType) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala index 4987e31b49911..8ad062ab0e2f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala @@ -21,6 +21,7 @@ import java.nio.ByteBuffer import com.google.common.primitives.{Doubles, Ints, Longs} +import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, TypeCheckSuccess} @@ -32,7 +33,6 @@ import org.apache.spark.sql.catalyst.types.PhysicalNumericType import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} import org.apache.spark.sql.catalyst.util.QuantileSummaries import org.apache.spark.sql.catalyst.util.QuantileSummaries.{defaultCompressThreshold, Stats} -import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.types._ import org.apache.spark.util.ArrayImplicits._ @@ -189,7 +189,7 @@ case class ApproximatePercentile( PhysicalNumericType.numeric(n) .toDouble(value.asInstanceOf[PhysicalNumericType#InternalType]) case other: DataType => - throw QueryExecutionErrors.dataTypeUnexpectedError(other) + throw SparkException.internalError(s"Unexpected data type $other") } buffer.add(doubleValue) } @@ -214,7 +214,7 @@ case class ApproximatePercentile( case DoubleType => doubleResult case _: DecimalType => doubleResult.map(Decimal(_)) case other: DataType => - throw QueryExecutionErrors.dataTypeUnexpectedError(other) + throw SparkException.internalError(s"Unexpected data type $other") } if (result.length == 0) { null diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index 0b37cf951a29b..2ab86a5c5f03f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -384,18 +384,6 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE s"The aggregate window function ${toSQLId(funcName)} does not support merging.") } - def dataTypeUnexpectedError(dataType: DataType): SparkUnsupportedOperationException = { - new SparkUnsupportedOperationException( - errorClass = "_LEGACY_ERROR_TEMP_2011", - messageParameters = Map("dataType" -> dataType.catalogString)) - } - - def typeUnsupportedError(dataType: DataType): SparkIllegalArgumentException = { - new SparkIllegalArgumentException( - errorClass = "_LEGACY_ERROR_TEMP_2011", - messageParameters = Map("dataType" -> dataType.toString())) - } - def negativeValueUnexpectedError( frequencyExpression : Expression): SparkIllegalArgumentException = { new SparkIllegalArgumentException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 676a2ab64d0a3..ffdca65151052 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -29,7 +29,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.fs.Path -import org.apache.spark.SparkRuntimeException +import org.apache.spark.{SparkException, SparkRuntimeException} import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec @@ -550,7 +550,7 @@ object PartitioningUtils extends SQLConfHelper { Cast(Literal(unescapePathName(value)), it).eval() case BinaryType => value.getBytes() case BooleanType => value.toBoolean - case dt => throw QueryExecutionErrors.typeUnsupportedError(dt) + case dt => throw SparkException.internalError(s"Unsupported partition type: $dt") } def validatePartitionColumn(