Skip to content

Commit 30e7433

Browse files
committed
[SPARK-11673][SQL] Remove the normal Project physical operator (and keep TungstenProject)
Also make full outer join being able to produce UnsafeRows. Author: Reynold Xin <[email protected]> Closes #9643 from rxin/SPARK-11673.
1 parent 14cf753 commit 30e7433

File tree

27 files changed

+80
-287
lines changed

27 files changed

+80
-287
lines changed

sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java

Lines changed: 0 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -170,13 +170,6 @@ public Iterator<UnsafeRow> sort(Iterator<UnsafeRow> inputIterator) throws IOExce
170170
return sort();
171171
}
172172

173-
/**
174-
* Return true if UnsafeExternalRowSorter can sort rows with the given schema, false otherwise.
175-
*/
176-
public static boolean supportsSchema(StructType schema) {
177-
return UnsafeProjection.canSupport(schema);
178-
}
179-
180173
private static final class RowComparator extends RecordComparator {
181174
private final Ordering<InternalRow> ordering;
182175
private final int numFields;

sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -17,13 +17,13 @@
1717

1818
package org.apache.spark.sql
1919

20+
import scala.reflect.ClassTag
21+
2022
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
2123
import org.apache.spark.sql.catalyst.expressions._
2224
import org.apache.spark.sql.types.{ObjectType, StructField, StructType}
2325
import org.apache.spark.util.Utils
2426

25-
import scala.reflect.ClassTag
26-
2727
/**
2828
* Used to convert a JVM object of type `T` to and from the internal Spark SQL representation.
2929
*
@@ -123,9 +123,9 @@ object Encoders {
123123

124124
new ExpressionEncoder[Any](
125125
schema,
126-
false,
126+
flat = false,
127127
extractExpressions,
128128
constructExpression,
129-
ClassTag.apply(cls))
129+
ClassTag(cls))
130130
}
131131
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -153,18 +153,18 @@ trait ScalaReflection {
153153
*/
154154
def constructorFor[T : TypeTag]: Expression = constructorFor(typeOf[T], None)
155155

156-
protected def constructorFor(
156+
private def constructorFor(
157157
tpe: `Type`,
158158
path: Option[Expression]): Expression = ScalaReflectionLock.synchronized {
159159

160160
/** Returns the current path with a sub-field extracted. */
161-
def addToPath(part: String) =
161+
def addToPath(part: String): Expression =
162162
path
163163
.map(p => UnresolvedExtractValue(p, expressions.Literal(part)))
164164
.getOrElse(UnresolvedAttribute(part))
165165

166166
/** Returns the current path with a field at ordinal extracted. */
167-
def addToPathOrdinal(ordinal: Int, dataType: DataType) =
167+
def addToPathOrdinal(ordinal: Int, dataType: DataType): Expression =
168168
path
169169
.map(p => GetStructField(p, StructField(s"_$ordinal", dataType), ordinal))
170170
.getOrElse(BoundReference(ordinal, dataType, false))

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -77,7 +77,7 @@ class EquivalentExpressions {
7777
* an empty collection if there are none.
7878
*/
7979
def getEquivalentExprs(e: Expression): Seq[Expression] = {
80-
equivalenceMap.get(Expr(e)).getOrElse(mutable.MutableList())
80+
equivalenceMap.getOrElse(Expr(e), mutable.MutableList())
8181
}
8282

8383
/**

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala

Lines changed: 0 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -102,16 +102,6 @@ abstract class UnsafeProjection extends Projection {
102102

103103
object UnsafeProjection {
104104

105-
/*
106-
* Returns whether UnsafeProjection can support given StructType, Array[DataType] or
107-
* Seq[Expression].
108-
*/
109-
def canSupport(schema: StructType): Boolean = canSupport(schema.fields.map(_.dataType))
110-
def canSupport(exprs: Seq[Expression]): Boolean = canSupport(exprs.map(_.dataType).toArray)
111-
private def canSupport(types: Array[DataType]): Boolean = {
112-
types.forall(GenerateUnsafeProjection.canSupport)
113-
}
114-
115105
/**
116106
* Returns an UnsafeProjection for given StructType.
117107
*/

sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala

Lines changed: 1 addition & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -57,10 +57,7 @@ case class Exchange(
5757
/**
5858
* Returns true iff we can support the data type, and we are not doing range partitioning.
5959
*/
60-
private lazy val tungstenMode: Boolean = {
61-
GenerateUnsafeProjection.canSupport(child.schema) &&
62-
!newPartitioning.isInstanceOf[RangePartitioning]
63-
}
60+
private lazy val tungstenMode: Boolean = !newPartitioning.isInstanceOf[RangePartitioning]
6461

6562
override def outputPartitioning: Partitioning = newPartitioning
6663

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -18,12 +18,10 @@
1818
package org.apache.spark.sql.execution
1919

2020
import org.apache.spark.SparkContext
21-
import org.apache.spark.annotation.Experimental
2221
import org.apache.spark.sql._
2322
import org.apache.spark.sql.catalyst.expressions._
2423
import org.apache.spark.sql.execution.datasources.DataSourceStrategy
2524

26-
@Experimental
2725
class SparkPlanner(val sqlContext: SQLContext) extends SparkStrategies {
2826
val sparkContext: SparkContext = sqlContext.sparkContext
2927

@@ -64,7 +62,7 @@ class SparkPlanner(val sqlContext: SQLContext) extends SparkStrategies {
6462

6563
val projectSet = AttributeSet(projectList.flatMap(_.references))
6664
val filterSet = AttributeSet(filterPredicates.flatMap(_.references))
67-
val filterCondition =
65+
val filterCondition: Option[Expression] =
6866
prunePushedDownFilters(filterPredicates).reduceLeftOption(catalyst.expressions.And)
6967

7068
// Right now we still use a projection even if the only evaluation is applying an alias
@@ -82,7 +80,7 @@ class SparkPlanner(val sqlContext: SQLContext) extends SparkStrategies {
8280
filterCondition.map(Filter(_, scan)).getOrElse(scan)
8381
} else {
8482
val scan = scanBuilder((projectSet ++ filterSet).toSeq)
85-
Project(projectList, filterCondition.map(Filter(_, scan)).getOrElse(scan))
83+
TungstenProject(projectList, filterCondition.map(Filter(_, scan)).getOrElse(scan))
8684
}
8785
}
8886
}

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala

Lines changed: 2 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -309,11 +309,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
309309
* if necessary.
310310
*/
311311
def getSortOperator(sortExprs: Seq[SortOrder], global: Boolean, child: SparkPlan): SparkPlan = {
312-
if (TungstenSort.supportsSchema(child.schema)) {
313-
execution.TungstenSort(sortExprs, global, child)
314-
} else {
315-
execution.Sort(sortExprs, global, child)
316-
}
312+
execution.TungstenSort(sortExprs, global, child)
317313
}
318314

319315
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
@@ -347,13 +343,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
347343
case logical.Sort(sortExprs, global, child) =>
348344
getSortOperator(sortExprs, global, planLater(child)):: Nil
349345
case logical.Project(projectList, child) =>
350-
// If unsafe mode is enabled and we support these data types in Unsafe, use the
351-
// Tungsten project. Otherwise, use the normal project.
352-
if (UnsafeProjection.canSupport(projectList) && UnsafeProjection.canSupport(child.schema)) {
353-
execution.TungstenProject(projectList, planLater(child)) :: Nil
354-
} else {
355-
execution.Project(projectList, planLater(child)) :: Nil
356-
}
346+
execution.TungstenProject(projectList, planLater(child)) :: Nil
357347
case logical.Filter(condition, child) =>
358348
execution.Filter(condition, planLater(child)) :: Nil
359349
case e @ logical.Expand(_, _, child) =>

sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -247,11 +247,7 @@ case class Window(
247247

248248
// Get all relevant projections.
249249
val result = createResultProjection(unboundExpressions)
250-
val grouping = if (child.outputsUnsafeRows) {
251-
UnsafeProjection.create(partitionSpec, child.output)
252-
} else {
253-
newProjection(partitionSpec, child.output)
254-
}
250+
val grouping = UnsafeProjection.create(partitionSpec, child.output)
255251

256252
// Manage the stream and the grouping.
257253
var nextRow: InternalRow = EmptyRow

sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -78,11 +78,9 @@ case class SortBasedAggregate(
7878
// so return an empty iterator.
7979
Iterator[InternalRow]()
8080
} else {
81-
val groupingKeyProjection = if (UnsafeProjection.canSupport(groupingExpressions)) {
81+
val groupingKeyProjection =
8282
UnsafeProjection.create(groupingExpressions, child.output)
83-
} else {
84-
newMutableProjection(groupingExpressions, child.output)()
85-
}
83+
8684
val outputIter = new SortBasedAggregationIterator(
8785
groupingKeyProjection,
8886
groupingExpressions.map(_.toAttribute),

0 commit comments

Comments
 (0)