Skip to content

Commit ce4a5d2

Browse files
committed
Fixed test cases in SQL except ParquetIOSuite.
1 parent 66d5ef1 commit ce4a5d2

File tree

4 files changed

+18
-9
lines changed

4 files changed

+18
-9
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,7 @@ object NewRelationInstances extends Rule[LogicalPlan] {
4747
.toSet
4848

4949
plan transform {
50-
case l: MultiInstanceRelation if multiAppearance contains l => l.newInstance
50+
case l: MultiInstanceRelation if multiAppearance.contains(l) => l.newInstance()
5151
}
5252
}
5353
}

sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala

Lines changed: 15 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -17,26 +17,26 @@
1717

1818
package org.apache.spark.sql
1919

20-
import org.apache.spark.sql.catalyst.ScalaReflection
21-
2220
import scala.language.implicitConversions
2321
import scala.reflect.ClassTag
2422

2523
import com.fasterxml.jackson.core.JsonFactory
2624

2725
import org.apache.spark.rdd.RDD
2826
import org.apache.spark.storage.StorageLevel
27+
import org.apache.spark.sql.catalyst.ScalaReflection
2928
import org.apache.spark.sql.catalyst.expressions._
3029
import org.apache.spark.sql.catalyst.expressions.{Literal => LiteralExpr}
3130
import org.apache.spark.sql.catalyst.plans.{JoinType, Inner}
3231
import org.apache.spark.sql.catalyst.plans.logical._
32+
import org.apache.spark.sql.execution.LogicalRDD
3333
import org.apache.spark.sql.json.JsonRDD
3434
import org.apache.spark.sql.types.{NumericType, StructType}
3535

3636

3737
class DataFrame(
3838
val sqlContext: SQLContext,
39-
val logicalPlan: LogicalPlan,
39+
val baseLogicalPlan: LogicalPlan,
4040
operatorsEnabled: Boolean)
4141
extends DataFrameSpecificApi with RDDApi[Row] {
4242

@@ -46,7 +46,16 @@ class DataFrame(
4646
def this(sqlContext: SQLContext, plan: LogicalPlan) = this(sqlContext, plan, true)
4747

4848
@transient
49-
protected[sql] lazy val queryExecution = sqlContext.executePlan(logicalPlan)
49+
protected[sql] lazy val queryExecution = sqlContext.executePlan(baseLogicalPlan)
50+
51+
@transient protected[sql] val logicalPlan: LogicalPlan = baseLogicalPlan match {
52+
// For various commands (like DDL) and queries with side effects, we force query optimization to
53+
// happen right away to let these side effects take place eagerly.
54+
case _: Command | _: InsertIntoTable | _: CreateTableAsSelect[_] |_: WriteToFile =>
55+
LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext)
56+
case _ =>
57+
baseLogicalPlan
58+
}
5059

5160
private[this] implicit def toDataFrame(logicalPlan: LogicalPlan): DataFrame = {
5261
new DataFrame(sqlContext, logicalPlan, true)
@@ -123,8 +132,8 @@ class DataFrame(
123132
override def as(name: String): DataFrame = Subquery(name, logicalPlan)
124133

125134
@scala.annotation.varargs
126-
override def select(cols: Column*): DataFrame = {
127-
val exprs = cols.zipWithIndex.map {
135+
override def select(col: Column, cols: Column*): DataFrame = {
136+
val exprs = (col +: cols).zipWithIndex.map {
128137
case (Column(expr: NamedExpression), _) =>
129138
expr
130139
case (Column(expr: Expression), _) =>

sql/core/src/main/scala/org/apache/spark/sql/api.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -85,7 +85,7 @@ trait DataFrameSpecificApi {
8585
def apply(projection: Product): DataFrame
8686

8787
@scala.annotation.varargs
88-
def select(cols: Column*): DataFrame
88+
def select(col: Column, cols: Column*): DataFrame
8989

9090
/** Filtering */
9191
def apply(condition: Column): DataFrame = filter(condition)

sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -102,7 +102,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest {
102102
sparkContext
103103
.parallelize(0 to 1000)
104104
.map(i => Tuple1(i / 100.0))
105-
.select('_1 cast decimal)
105+
.select($"_1" cast decimal)
106106

107107
for ((precision, scale) <- Seq((5, 2), (1, 0), (1, 1), (18, 10), (18, 17))) {
108108
withTempPath { dir =>

0 commit comments

Comments
 (0)