diff options
4 files changed, 15 insertions, 4 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 327cf87f30..13aff760e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -76,7 +76,7 @@ private[sql] object DataFrame { */ // TODO: Improve documentation. @Experimental -trait DataFrame extends RDDApi[Row] { +trait DataFrame extends RDDApi[Row] with Serializable { val sqlContext: SQLContext diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 3863df5318..4c6e19cace 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -44,8 +44,8 @@ import org.apache.spark.sql.types.{NumericType, StructType} * Internal implementation of [[DataFrame]]. Users of the API should use [[DataFrame]] directly. */ private[sql] class DataFrameImpl protected[sql]( - override val sqlContext: SQLContext, - val queryExecution: SQLContext#QueryExecution) + @transient override val sqlContext: SQLContext, + @transient val queryExecution: SQLContext#QueryExecution) extends DataFrame { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala index 6bf21dd1bc..7bc7683576 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala @@ -167,6 +167,15 @@ object Dsl { ////////////////////////////////////////////////////////////////////////////////////////////// /** + * Returns the first column that is not null. + * {{{ + * df.select(coalesce(df("a"), df("b"))) + * }}} + */ + @scala.annotation.varargs + def coalesce(e: Column*): Column = Coalesce(e.map(_.expr)) + + /** * Unary minus, i.e. negate the expression. * {{{ * // Select the amount column and negates all values. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index ca5e62f295..8aae222acd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -129,9 +129,11 @@ class SQLContext(@transient val sparkContext: SparkContext) * A collection of methods that are considered experimental, but can be used to hook into * the query planner for advanced functionalities. */ + @transient val experimental: ExperimentalMethods = new ExperimentalMethods(this) /** Returns a [[DataFrame]] with no rows or columns. */ + @transient lazy val emptyDataFrame = DataFrame(this, NoRelation) /** @@ -178,7 +180,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * (Scala-specific) * Implicit methods available in Scala for converting common Scala objects into [[DataFrame]]s. */ - object implicits { + object implicits extends Serializable { // scalastyle:on /** Creates a DataFrame from an RDD of case classes or tuples. */ |