Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-42846][SQL] Remove error condition _LEGACY_ERROR_TEMP_2011 #48086

Closed
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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