diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 63c54a71b904..a2690b9c2b0e 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -497,6 +497,12 @@ ], "sqlState" : "56000" }, + "CIRCULAR_CLASS_REFERENCE" : { + "message" : [ + "Cannot have circular references in class, but got the circular reference of class ." + ], + "sqlState" : "42602" + }, "CLASS_NOT_OVERRIDE_EXPECTED_METHOD" : { "message" : [ " must override either or ." @@ -7325,16 +7331,6 @@ "Exception when registering StreamingQueryListener." ] }, - "_LEGACY_ERROR_TEMP_2138" : { - "message" : [ - "Cannot have circular references in bean class, but got the circular reference of class ." - ] - }, - "_LEGACY_ERROR_TEMP_2139" : { - "message" : [ - "cannot have circular references in class, but got the circular reference of class ." - ] - }, "_LEGACY_ERROR_TEMP_2144" : { "message" : [ "Unable to find constructor for . This could happen if is an interface, or a trait without companion object constructor." diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala index 862067765447..fca3ea8fdb90 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala @@ -175,8 +175,8 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase { def cannotHaveCircularReferencesInBeanClassError( clazz: Class[_]): SparkUnsupportedOperationException = { new SparkUnsupportedOperationException( - errorClass = "_LEGACY_ERROR_TEMP_2138", - messageParameters = Map("clazz" -> clazz.toString)) + errorClass = "CIRCULAR_CLASS_REFERENCE", + messageParameters = Map("t" -> toSQLValue(clazz.toString))) } def cannotFindConstructorForTypeError(tpe: String): SparkUnsupportedOperationException = { @@ -187,8 +187,8 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase { def cannotHaveCircularReferencesInClassError(t: String): SparkUnsupportedOperationException = { new SparkUnsupportedOperationException( - errorClass = "_LEGACY_ERROR_TEMP_2139", - messageParameters = Map("t" -> t)) + errorClass = "CIRCULAR_CLASS_REFERENCE", + messageParameters = Map("t" -> toSQLValue(t))) } def cannotUseInvalidJavaIdentifierAsFieldNameError( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 45c34d9c7336..ca584d6b9ce8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1926,20 +1926,20 @@ class DatasetSuite extends QueryTest exception = intercept[SparkUnsupportedOperationException] { Seq(CircularReferenceClassA(null)).toDS() }, - condition = "_LEGACY_ERROR_TEMP_2139", - parameters = Map("t" -> "org.apache.spark.sql.CircularReferenceClassA")) + condition = "CIRCULAR_CLASS_REFERENCE", + parameters = Map("t" -> "'org.apache.spark.sql.CircularReferenceClassA'")) checkError( exception = intercept[SparkUnsupportedOperationException] { Seq(CircularReferenceClassC(null)).toDS() }, - condition = "_LEGACY_ERROR_TEMP_2139", - parameters = Map("t" -> "org.apache.spark.sql.CircularReferenceClassC")) + condition = "CIRCULAR_CLASS_REFERENCE", + parameters = Map("t" -> "'org.apache.spark.sql.CircularReferenceClassC'")) checkError( exception = intercept[SparkUnsupportedOperationException] { Seq(CircularReferenceClassD(null)).toDS() }, - condition = "_LEGACY_ERROR_TEMP_2139", - parameters = Map("t" -> "org.apache.spark.sql.CircularReferenceClassD")) + condition = "CIRCULAR_CLASS_REFERENCE", + parameters = Map("t" -> "'org.apache.spark.sql.CircularReferenceClassD'")) } test("SPARK-20125: option of map") {