You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2023/02/08 17:47:53 UTC
[spark] branch branch-3.4 updated: [SPARK-42314][SQL] Assign name to _LEGACY_ERROR_TEMP_2127
This is an automated email from the ASF dual-hosted git repository.
maxgekk pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/branch-3.4 by this push:
new c3584dc9d5d [SPARK-42314][SQL] Assign name to _LEGACY_ERROR_TEMP_2127
c3584dc9d5d is described below
commit c3584dc9d5de953d0185b2192b34940a1ff4fc64
Author: itholic <ha...@databricks.com>
AuthorDate: Wed Feb 8 22:47:10 2023 +0500
[SPARK-42314][SQL] Assign name to _LEGACY_ERROR_TEMP_2127
### What changes were proposed in this pull request?
This PR proposes to assign name to _LEGACY_ERROR_TEMP_2127, "DUPLICATED_MAP_KEY".
### Why are the changes needed?
We should assign proper name to _LEGACY_ERROR_TEMP_*
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
`./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*`
Closes #39890 from itholic/LEGACY_2127.
Lead-authored-by: itholic <ha...@databricks.com>
Co-authored-by: Haejoon Lee <44...@users.noreply.github.com>
Signed-off-by: Max Gekk <ma...@gmail.com>
(cherry picked from commit 4b50a46f1c6ba4ffe2c42f70a512879c28d11dcf)
Signed-off-by: Max Gekk <ma...@gmail.com>
---
core/src/main/resources/error/error-classes.json | 11 ++++---
.../spark/sql/errors/QueryExecutionErrors.scala | 5 ++-
.../expressions/CollectionExpressionsSuite.scala | 27 ++++++++++++----
.../catalyst/expressions/ComplexTypeSuite.scala | 26 +++++++++++----
.../expressions/HigherOrderFunctionsSuite.scala | 11 +++++--
.../catalyst/util/ArrayBasedMapBuilderSuite.scala | 37 +++++++++++++++++-----
6 files changed, 86 insertions(+), 31 deletions(-)
diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json
index bed9d0e180c..e95988a260a 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -506,6 +506,12 @@
],
"sqlState" : "22012"
},
+ "DUPLICATED_MAP_KEY" : {
+ "message" : [
+ "Duplicate map key <key> was found, please check the input data. If you want to remove the duplicated keys, you can set <mapKeyDedupPolicy> to \"LAST_WIN\" so that the key inserted at last takes precedence."
+ ],
+ "sqlState" : "23505"
+ },
"DUPLICATE_KEY" : {
"message" : [
"Found duplicate keys <keyColumn>."
@@ -4197,11 +4203,6 @@
"Unsuccessful attempt to build maps with <size> elements due to exceeding the map size limit <maxRoundedArrayLength>."
]
},
- "_LEGACY_ERROR_TEMP_2127" : {
- "message" : [
- "Duplicate map key <key> was found, please check the input data. If you want to remove the duplicated keys, you can set <mapKeyDedupPolicy> to <lastWin> so that the key inserted at last takes precedence."
- ]
- },
"_LEGACY_ERROR_TEMP_2128" : {
"message" : [
"The key array and value array of MapData must have the same length."
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 129d369085b..4abeeef0fc6 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
@@ -1330,11 +1330,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {
def duplicateMapKeyFoundError(key: Any): SparkRuntimeException = {
new SparkRuntimeException(
- errorClass = "_LEGACY_ERROR_TEMP_2127",
+ errorClass = "DUPLICATED_MAP_KEY",
messageParameters = Map(
"key" -> key.toString(),
- "mapKeyDedupPolicy" -> toSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key),
- "lastWin" -> toSQLConf(SQLConf.MapKeyDedupPolicy.LAST_WIN.toString())))
+ "mapKeyDedupPolicy" -> toSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key)))
}
def mapDataKeyArrayLengthDiffersFromValueArrayLengthError(): SparkRuntimeException = {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
index 9b97430594d..5917d84df1e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
@@ -169,8 +169,13 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
MapType(IntegerType, IntegerType, valueContainsNull = true))
val mNull = Literal.create(null, MapType(StringType, StringType))
- checkExceptionInExpression[RuntimeException](
- MapConcat(Seq(m0, m1)), "Duplicate map key")
+ checkErrorInExpression[SparkRuntimeException](
+ MapConcat(Seq(m0, m1)),
+ errorClass = "DUPLICATED_MAP_KEY",
+ parameters = Map(
+ "key" -> "a",
+ "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"")
+ )
withSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key -> SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) {
// overlapping maps should remove duplicated map keys w.r.t. last win policy.
checkEvaluation(MapConcat(Seq(m0, m1)), create_map("a" -> "4", "b" -> "2", "c" -> "3"))
@@ -324,8 +329,13 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
checkEvaluation(MapFromEntries(ai2), Map.empty)
checkEvaluation(MapFromEntries(ai3), null)
- checkExceptionInExpression[RuntimeException](
- MapFromEntries(ai4), "Duplicate map key")
+ checkErrorInExpression[SparkRuntimeException](
+ MapFromEntries(ai4),
+ errorClass = "DUPLICATED_MAP_KEY",
+ parameters = Map(
+ "key" -> "1",
+ "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"")
+ )
withSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key -> SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) {
// Duplicated map keys will be removed w.r.t. the last wins policy.
checkEvaluation(MapFromEntries(ai4), create_map(1 -> 20))
@@ -351,8 +361,13 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
checkEvaluation(MapFromEntries(as2), Map.empty)
checkEvaluation(MapFromEntries(as3), null)
- checkExceptionInExpression[RuntimeException](
- MapFromEntries(as4), "Duplicate map key")
+ checkErrorInExpression[SparkRuntimeException](
+ MapFromEntries(as4),
+ errorClass = "DUPLICATED_MAP_KEY",
+ parameters = Map(
+ "key" -> "a",
+ "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"")
+ )
withSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key -> SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) {
// Duplicated map keys will be removed w.r.t. the last wins policy.
checkEvaluation(MapFromEntries(as4), create_map("a" -> "bb"))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
index 8818fe645ad..a6de0b092cd 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
@@ -282,8 +282,13 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper {
CreateMap(interlace(strWithNull, intSeq.map(Literal(_)))),
"NULL_MAP_KEY")
- checkExceptionInExpression[RuntimeException](
- CreateMap(Seq(Literal(1), Literal(2), Literal(1), Literal(3))), "Duplicate map key")
+ checkErrorInExpression[SparkRuntimeException](
+ CreateMap(Seq(Literal(1), Literal(2), Literal(1), Literal(3))),
+ errorClass = "DUPLICATED_MAP_KEY",
+ parameters = Map(
+ "key" -> "1",
+ "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"")
+ )
withSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key -> SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) {
// Duplicated map keys will be removed w.r.t. the last wins policy.
checkEvaluation(
@@ -388,11 +393,15 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper {
MapFromArrays(intWithNullArray, strArray),
"NULL_MAP_KEY")
- checkExceptionInExpression[RuntimeException](
+ checkErrorInExpression[SparkRuntimeException](
MapFromArrays(
Literal.create(Seq(1, 1), ArrayType(IntegerType)),
Literal.create(Seq(2, 3), ArrayType(IntegerType))),
- "Duplicate map key")
+ errorClass = "DUPLICATED_MAP_KEY",
+ parameters = Map(
+ "key" -> "1",
+ "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"")
+ )
withSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key -> SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) {
// Duplicated map keys will be removed w.r.t. the last wins policy.
checkEvaluation(
@@ -512,8 +521,13 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper {
val m6 = Map("a" -> "1", "b" -> "2", "c" -> "3")
checkEvaluation(StringToMap(s6, NonFoldableLiteral("&"), NonFoldableLiteral("=")), m6)
- checkExceptionInExpression[RuntimeException](
- new StringToMap(Literal("a:1,b:2,a:3")), "Duplicate map key")
+ checkErrorInExpression[SparkRuntimeException](
+ new StringToMap(Literal("a:1,b:2,a:3")),
+ errorClass = "DUPLICATED_MAP_KEY",
+ parameters = Map(
+ "key" -> "a",
+ "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"")
+ )
withSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key -> SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) {
// Duplicated map keys will be removed w.r.t. the last wins policy.
checkEvaluation(
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala
index 383010427b1..c0670560656 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.expressions
-import org.apache.spark.{SparkException, SparkFunSuite}
+import org.apache.spark.{SparkException, SparkFunSuite, SparkRuntimeException}
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
import org.apache.spark.sql.catalyst.expressions.Cast._
@@ -469,8 +469,13 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper
transformKeys(transformKeys(ai0, plusOne), plusValue),
create_map(3 -> 1, 5 -> 2, 7 -> 3, 9 -> 4))
- checkExceptionInExpression[RuntimeException](
- transformKeys(ai0, modKey), "Duplicate map key")
+ checkErrorInExpression[SparkRuntimeException](
+ transformKeys(ai0, modKey),
+ errorClass = "DUPLICATED_MAP_KEY",
+ parameters = Map(
+ "key" -> "1",
+ "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"")
+ )
withSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key -> SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) {
// Duplicated map keys will be removed w.r.t. the last wins policy.
checkEvaluation(transformKeys(ai0, modKey), create_map(1 -> 4, 2 -> 2, 0 -> 3))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala
index c8dbecac249..5811f4cd4c8 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala
@@ -51,8 +51,13 @@ class ArrayBasedMapBuilderSuite extends SparkFunSuite with SQLHelper {
test("fail while duplicated keys detected") {
val builder = new ArrayBasedMapBuilder(IntegerType, IntegerType)
builder.put(1, 1)
- val e = intercept[RuntimeException](builder.put(1, 2))
- assert(e.getMessage.contains("Duplicate map key 1 was found"))
+ checkError(
+ exception = intercept[SparkRuntimeException](builder.put(1, 2)),
+ errorClass = "DUPLICATED_MAP_KEY",
+ parameters = Map(
+ "key" -> "1",
+ "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"")
+ )
}
test("remove duplicated keys with last wins policy") {
@@ -71,9 +76,15 @@ class ArrayBasedMapBuilderSuite extends SparkFunSuite with SQLHelper {
val builder = new ArrayBasedMapBuilder(BinaryType, IntegerType)
builder.put(Array(1.toByte), 1)
builder.put(Array(2.toByte), 2)
- val e = intercept[RuntimeException](builder.put(Array(1.toByte), 3))
// By default duplicated map key fails the query.
- assert(e.getMessage.contains("Duplicate map key"))
+ val arr = Array(1.toByte)
+ checkError(
+ exception = intercept[SparkRuntimeException](builder.put(arr, 3)),
+ errorClass = "DUPLICATED_MAP_KEY",
+ parameters = Map(
+ "key" -> arr.toString,
+ "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"")
+ )
withSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key -> SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) {
val builder = new ArrayBasedMapBuilder(BinaryType, IntegerType)
@@ -102,9 +113,14 @@ class ArrayBasedMapBuilderSuite extends SparkFunSuite with SQLHelper {
val builder = new ArrayBasedMapBuilder(new StructType().add("i", "int"), IntegerType)
builder.put(InternalRow(1), 1)
builder.put(InternalRow(2), 2)
- val e = intercept[RuntimeException](builder.put(unsafeRow, 3))
// By default duplicated map key fails the query.
- assert(e.getMessage.contains("Duplicate map key"))
+ checkError(
+ exception = intercept[SparkRuntimeException](builder.put(unsafeRow, 3)),
+ errorClass = "DUPLICATED_MAP_KEY",
+ parameters = Map(
+ "key" -> "[0,1]",
+ "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"")
+ )
withSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key -> SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) {
val builder = new ArrayBasedMapBuilder(new StructType().add("i", "int"), IntegerType)
@@ -131,9 +147,14 @@ class ArrayBasedMapBuilderSuite extends SparkFunSuite with SQLHelper {
val builder = new ArrayBasedMapBuilder(ArrayType(IntegerType), IntegerType)
builder.put(new GenericArrayData(Seq(1, 1)), 1)
builder.put(new GenericArrayData(Seq(2, 2)), 2)
- val e = intercept[RuntimeException](builder.put(unsafeArray, 3))
// By default duplicated map key fails the query.
- assert(e.getMessage.contains("Duplicate map key"))
+ checkError(
+ exception = intercept[SparkRuntimeException](builder.put(unsafeArray, 3)),
+ errorClass = "DUPLICATED_MAP_KEY",
+ parameters = Map(
+ "key" -> unsafeArray.toString,
+ "mapKeyDedupPolicy" -> "\"spark.sql.mapKeyDedupPolicy\"")
+ )
withSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key -> SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) {
val builder = new ArrayBasedMapBuilder(ArrayType(IntegerType), IntegerType)
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org