aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorWenchen Fan <wenchen@databricks.com>2016-03-21 10:37:24 -0700
committerYin Huai <yhuai@databricks.com>2016-03-21 10:37:24 -0700
commit43ebf7a9cbd70d6af75e140a6fc91bf0ffc2b877 (patch)
tree2c4561fc312dd29156c1406b3777cbb8abbd48fe /sql
parent060a28c633e559376976561248bcb30c4739b76d (diff)
downloadspark-43ebf7a9cbd70d6af75e140a6fc91bf0ffc2b877.tar.gz
spark-43ebf7a9cbd70d6af75e140a6fc91bf0ffc2b877.tar.bz2
spark-43ebf7a9cbd70d6af75e140a6fc91bf0ffc2b877.zip
[SPARK-13456][SQL] fix creating encoders for case classes defined in Spark shell
## What changes were proposed in this pull request? case classes defined in REPL are wrapped by line classes, and we have a trick for scala 2.10 REPL to automatically register the wrapper classes to `OuterScope` so that we can use when create encoders. However, this trick doesn't work right after we upgrade to scala 2.11, and unfortunately the tests are only in scala 2.10, which makes this bug hidden until now. This PR moves the encoder tests to scala 2.11 `ReplSuite`, and fixes this bug by another approach(the previous trick can't port to scala 2.11 REPL): make `OuterScope` smarter that can detect classes defined in REPL and load the singleton of line wrapper classes automatically. ## How was this patch tested? the migrated encoder tests in `ReplSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #11410 from cloud-fan/repl.
Diffstat (limited to 'sql')
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala47
2 files changed, 47 insertions, 2 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index ccc65b4e52..ebb3a931da 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -571,7 +571,7 @@ class Analyzer(
if n.outerPointer.isEmpty &&
n.cls.isMemberClass &&
!Modifier.isStatic(n.cls.getModifiers) =>
- val outer = OuterScopes.outerScopes.get(n.cls.getDeclaringClass.getName)
+ val outer = OuterScopes.getOuterScope(n.cls)
if (outer == null) {
throw new AnalysisException(
s"Unable to generate an encoder for inner class `${n.cls.getName}` without " +
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala
index a753b187bc..c047e96463 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala
@@ -21,6 +21,8 @@ import java.util.concurrent.ConcurrentMap
import com.google.common.collect.MapMaker
+import org.apache.spark.util.Utils
+
object OuterScopes {
@transient
lazy val outerScopes: ConcurrentMap[String, AnyRef] =
@@ -28,7 +30,7 @@ object OuterScopes {
/**
* Adds a new outer scope to this context that can be used when instantiating an `inner class`
- * during deserialialization. Inner classes are created when a case class is defined in the
+ * during deserialization. Inner classes are created when a case class is defined in the
* Spark REPL and registering the outer scope that this class was defined in allows us to create
* new instances on the spark executors. In normal use, users should not need to call this
* function.
@@ -39,4 +41,47 @@ object OuterScopes {
def addOuterScope(outer: AnyRef): Unit = {
outerScopes.putIfAbsent(outer.getClass.getName, outer)
}
+
+ def getOuterScope(innerCls: Class[_]): AnyRef = {
+ assert(innerCls.isMemberClass)
+ val outerClassName = innerCls.getDeclaringClass.getName
+ val outer = outerScopes.get(outerClassName)
+ if (outer == null) {
+ outerClassName match {
+ // If the outer class is generated by REPL, users don't need to register it as it has
+ // only one instance and there is a way to retrieve it: get the `$read` object, call the
+ // `INSTANCE()` method to get the single instance of class `$read`. Then call `$iw()`
+ // method multiply times to get the single instance of the inner most `$iw` class.
+ case REPLClass(baseClassName) =>
+ val objClass = Utils.classForName(baseClassName + "$")
+ val objInstance = objClass.getField("MODULE$").get(null)
+ val baseInstance = objClass.getMethod("INSTANCE").invoke(objInstance)
+ val baseClass = Utils.classForName(baseClassName)
+
+ var getter = iwGetter(baseClass)
+ var obj = baseInstance
+ while (getter != null) {
+ obj = getter.invoke(obj)
+ getter = iwGetter(getter.getReturnType)
+ }
+
+ outerScopes.putIfAbsent(outerClassName, obj)
+ obj
+ case _ => null
+ }
+ } else {
+ outer
+ }
+ }
+
+ private def iwGetter(cls: Class[_]) = {
+ try {
+ cls.getMethod("$iw")
+ } catch {
+ case _: NoSuchMethodException => null
+ }
+ }
+
+ // The format of REPL generated wrapper class's name, e.g. `$line12.$read$$iw$$iw`
+ private[this] val REPLClass = """^(\$line(?:\d+)\.\$read)(?:\$\$iw)+$""".r
}