Repository: spark
Updated Branches:
  refs/heads/master d1c1fbc34 -> 6323e4bd7


[SPARK-15732][SQL] better error message when use java reserved keyword as field 
name

## What changes were proposed in this pull request?

When users create a case class and use java reserved keyword as field name, 
spark sql will generate illegal java code and throw exception at runtime.

This PR checks the field names when building the encoder, and if illegal field 
names are used, throw exception immediately with a good error message.

## How was this patch tested?

new test in DatasetSuite

Author: Wenchen Fan <wenc...@databricks.com>

Closes #13485 from cloud-fan/java.


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

Branch: refs/heads/master
Commit: 6323e4bd763eafe23c619a89552c00e1244e4690
Parents: d1c1fbc
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Thu Jun 2 18:13:04 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Jun 2 18:13:04 2016 -0700

----------------------------------------------------------------------
 .../org/apache/spark/sql/catalyst/ScalaReflection.scala  | 11 +++++++++++
 .../test/scala/org/apache/spark/sql/DatasetSuite.scala   | 10 ++++++++++
 2 files changed, 21 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/6323e4bd/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
index 052cc48..4750861 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
@@ -582,6 +582,11 @@ object ScalaReflection extends ScalaReflection {
         case t if definedByConstructorParams(t) =>
           val params = getConstructorParameters(t)
           val nonNullOutput = CreateNamedStruct(params.flatMap { case 
(fieldName, fieldType) =>
+            if (javaKeywords.contains(fieldName)) {
+              throw new UnsupportedOperationException(s"`$fieldName` is a 
reserved keyword and " +
+                "cannot be used as field name\n" + 
walkedTypePath.mkString("\n"))
+            }
+
             val fieldValue = Invoke(inputObject, fieldName, 
dataTypeFor(fieldType))
             val clsName = getClassNameFromType(fieldType)
             val newPath = s"""- field (class: "$clsName", name: 
"$fieldName")""" +: walkedTypePath
@@ -720,6 +725,12 @@ object ScalaReflection extends ScalaReflection {
     tpe <:< localTypeOf[Product] || tpe <:< 
localTypeOf[DefinedByConstructorParams]
   }
 
+  private val javaKeywords = Set("abstract", "assert", "boolean", "break", 
"byte", "case", "catch",
+    "char", "class", "const", "continue", "default", "do", "double", "else", 
"extends", "false",
+    "final", "finally", "float", "for", "goto", "if", "implements", "import", 
"instanceof", "int",
+    "interface", "long", "native", "new", "null", "package", "private", 
"protected", "public",
+    "return", "short", "static", "strictfp", "super", "switch", 
"synchronized", "this", "throw",
+    "throws", "transient", "true", "try", "void", "volatile", "while")
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/6323e4bd/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
----------------------------------------------------------------------
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 a3881ff..df8f4b0 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
@@ -786,6 +786,14 @@ class DatasetSuite extends QueryTest with SharedSQLContext 
{
     val result = joined.collect().toSet
     assert(result == Set(ClassData("a", 1) -> null, ClassData("b", 2) -> 
ClassData("x", 2)))
   }
+
+  test("better error message when use java reserved keyword as field name") {
+    val e = intercept[UnsupportedOperationException] {
+      Seq(InvalidInJava(1)).toDS()
+    }
+    assert(e.getMessage.contains(
+      "`abstract` is a reserved keyword and cannot be used as field name"))
+  }
 }
 
 case class Generic[T](id: T, value: Double)
@@ -809,6 +817,8 @@ case class ClassNullableData(a: String, b: Integer)
 case class NestedStruct(f: ClassData)
 case class DeepNestedStruct(f: NestedStruct)
 
+case class InvalidInJava(`abstract`: Int)
+
 /**
  * A class used to test serialization using encoders. This class throws 
exceptions when using
  * Java serialization -- so the only way it can be "serialized" is through our 
encoders.


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

Reply via email to