You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by gu...@apache.org on 2018/03/12 13:13:32 UTC

spark git commit: [SPARK-23462][SQL] improve missing field error message in `StructType`

Repository: spark
Updated Branches:
  refs/heads/master b6f837c9d -> b304e07e0


[SPARK-23462][SQL] improve missing field error message in `StructType`

## What changes were proposed in this pull request?

The error message ```s"""Field "$name" does not exist."""``` is thrown when looking up an unknown field in StructType. In the error message, we should also contain the information about which columns/fields exist in this struct.

## How was this patch tested?

Added new unit tests.

Note: I created a new `StructTypeSuite.scala` as I couldn't find an existing suite that's suitable to place these tests. I may be missing something so feel free to propose new locations.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Xiayun Sun <xi...@gmail.com>

Closes #20649 from xysun/SPARK-23462.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b304e07e
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b304e07e
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b304e07e

Branch: refs/heads/master
Commit: b304e07e0671faf96530f9d8f49c55a83b07fa15
Parents: b6f837c
Author: Xiayun Sun <xi...@gmail.com>
Authored: Mon Mar 12 22:13:28 2018 +0900
Committer: hyukjinkwon <gu...@gmail.com>
Committed: Mon Mar 12 22:13:28 2018 +0900

----------------------------------------------------------------------
 .../org/apache/spark/sql/types/StructType.scala | 11 ++++--
 .../spark/sql/types/StructTypeSuite.scala       | 40 ++++++++++++++++++++
 2 files changed, 48 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/b304e07e/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
index d5011c3..362676b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
@@ -271,7 +271,9 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru
    */
   def apply(name: String): StructField = {
     nameToField.getOrElse(name,
-      throw new IllegalArgumentException(s"""Field "$name" does not exist."""))
+      throw new IllegalArgumentException(
+        s"""Field "$name" does not exist.
+           |Available fields: ${fieldNames.mkString(", ")}""".stripMargin))
   }
 
   /**
@@ -284,7 +286,8 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru
     val nonExistFields = names -- fieldNamesSet
     if (nonExistFields.nonEmpty) {
       throw new IllegalArgumentException(
-        s"Field ${nonExistFields.mkString(",")} does not exist.")
+        s"""Nonexistent field(s): ${nonExistFields.mkString(", ")}.
+           |Available fields: ${fieldNames.mkString(", ")}""".stripMargin)
     }
     // Preserve the original order of fields.
     StructType(fields.filter(f => names.contains(f.name)))
@@ -297,7 +300,9 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru
    */
   def fieldIndex(name: String): Int = {
     nameToIndex.getOrElse(name,
-      throw new IllegalArgumentException(s"""Field "$name" does not exist."""))
+      throw new IllegalArgumentException(
+        s"""Field "$name" does not exist.
+           |Available fields: ${fieldNames.mkString(", ")}""".stripMargin))
   }
 
   private[sql] def getFieldIndex(name: String): Option[Int] = {

http://git-wip-us.apache.org/repos/asf/spark/blob/b304e07e/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala
new file mode 100644
index 0000000..c6ca8bb
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.types
+
+import org.apache.spark.SparkFunSuite
+
+class StructTypeSuite extends SparkFunSuite {
+
+  val s = StructType.fromDDL("a INT, b STRING")
+
+  test("lookup a single missing field should output existing fields") {
+    val e = intercept[IllegalArgumentException](s("c")).getMessage
+    assert(e.contains("Available fields: a, b"))
+  }
+
+  test("lookup a set of missing fields should output existing fields") {
+    val e = intercept[IllegalArgumentException](s(Set("a", "c"))).getMessage
+    assert(e.contains("Available fields: a, b"))
+  }
+
+  test("lookup fieldIndex for missing field should output existing fields") {
+    val e = intercept[IllegalArgumentException](s.fieldIndex("c")).getMessage
+    assert(e.contains("Available fields: a, b"))
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org