Skip to content

Commit

Permalink
[SPARK-42846][SQL] Remove error condition _LEGACY_ERROR_TEMP_2011
Browse files Browse the repository at this point in the history
### 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 apache#48086 from ivanjevtic-db/remove-legacy-error-temp-2011.

Authored-by: ivanjevtic-db <ivan.jevtic@databricks.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
  • Loading branch information
ivanjevtic-db authored and attilapiros committed Oct 4, 2024
1 parent a91e41f commit 5fddd26
Show file tree
Hide file tree
Showing 5 changed files with 7 additions and 24 deletions.
5 changes: 0 additions & 5 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -6656,11 +6656,6 @@
"Type <dataType> does not support ordered operations."
]
},
"_LEGACY_ERROR_TEMP_2011" : {
"message" : [
"Unexpected data type <dataType>."
]
},
"_LEGACY_ERROR_TEMP_2013" : {
"message" : [
"Negative values found in <frequencyExpression>"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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._

Expand Down Expand Up @@ -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)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand All @@ -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._

Expand Down Expand Up @@ -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)
}
Expand All @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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(
Expand Down

0 comments on commit 5fddd26

Please sign in to comment.