Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -43,8 +43,8 @@ import org.apache.spark.util.Utils
import org.apache.spark.util.collection.BitSet

trait DataSourceScanExec extends LeafExecNode {
val relation: BaseRelation
val tableIdentifier: Option[TableIdentifier]
def relation: BaseRelation
def tableIdentifier: Option[TableIdentifier]

protected val nodeNamePrefix: String = ""

Expand Down Expand Up @@ -103,7 +103,7 @@ case class RowDataSourceScanExec(
handledFilters: Set[Filter],
rdd: RDD[InternalRow],
@transient relation: BaseRelation,
override val tableIdentifier: Option[TableIdentifier])
tableIdentifier: Option[TableIdentifier])
extends DataSourceScanExec with InputRDDCodegen {

def output: Seq[Attribute] = requiredColumnsIndex.map(fullOutput)
Expand Down Expand Up @@ -164,7 +164,7 @@ case class FileSourceScanExec(
partitionFilters: Seq[Expression],
optionalBucketSet: Option[BitSet],
dataFilters: Seq[Expression],
override val tableIdentifier: Option[TableIdentifier])
tableIdentifier: Option[TableIdentifier])
extends DataSourceScanExec {

// Note that some vals referring the file-based relation are lazy intentionally
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,13 +29,13 @@ import org.apache.spark.sql.types.{IntegralType, LongType}
trait HashJoin {
self: SparkPlan =>

val leftKeys: Seq[Expression]
val rightKeys: Seq[Expression]
val joinType: JoinType
val buildSide: BuildSide
val condition: Option[Expression]
val left: SparkPlan
val right: SparkPlan
def leftKeys: Seq[Expression]
def rightKeys: Seq[Expression]
def joinType: JoinType
def buildSide: BuildSide
def condition: Option[Expression]
def left: SparkPlan
def right: SparkPlan

override def simpleStringWithNodeId(): String = {
val opId = ExplainUtils.getOpId(this)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.python
import scala.collection.JavaConverters._

import org.apache.spark.TaskContext
import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
import org.apache.spark.api.python.ChainedPythonFunctions
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.execution.SparkPlan
Expand Down Expand Up @@ -61,7 +61,7 @@ private[spark] class BatchIterator[T](iter: Iterator[T], batchSize: Int)
*/
case class ArrowEvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan,
evalType: Int)
extends EvalPythonExec(udfs, resultAttrs, child) {
extends EvalPythonExec {

private val batchSize = conf.arrowMaxRecordsPerBatch
private val sessionLocalTimeZone = conf.sessionLocalTimeZone
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ import org.apache.spark.sql.types.{StructField, StructType}
* A physical plan that evaluates a [[PythonUDF]]
*/
case class BatchEvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan)
extends EvalPythonExec(udfs, resultAttrs, child) {
extends EvalPythonExec {

protected override def evaluate(
funcs: Seq[ChainedPythonFunctions],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,8 +57,9 @@ import org.apache.spark.util.Utils
* there should be always some rows buffered in the socket or Python process, so the pulling from
* RowQueue ALWAYS happened after pushing into it.
*/
abstract class EvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan)
extends UnaryExecNode {
trait EvalPythonExec extends UnaryExecNode {
def udfs: Seq[PythonUDF]
def resultAttrs: Seq[Attribute]

override def output: Seq[Attribute] = child.output ++ resultAttrs

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ case class WindowInPandasExec(
partitionSpec: Seq[Expression],
orderSpec: Seq[SortOrder],
child: SparkPlan)
extends WindowExecBase(windowExpression, partitionSpec, orderSpec, child) {
extends WindowExecBase {

override def output: Seq[Attribute] =
child.output ++ windowExpression.map(_.toAttribute)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ case class FlatMapGroupsWithStateExec(
outputMode: OutputMode,
timeoutConf: GroupStateTimeout,
batchTimestampMs: Option[Long],
override val eventTimeWatermark: Option[Long],
eventTimeWatermark: Option[Long],
child: SparkPlan
) extends UnaryExecNode with ObjectProducerExec with StateStoreWriter with WatermarkSupport {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ case class WindowExec(
partitionSpec: Seq[Expression],
orderSpec: Seq[SortOrder],
child: SparkPlan)
extends WindowExecBase(windowExpression, partitionSpec, orderSpec, child) {
extends WindowExecBase {

override def output: Seq[Attribute] =
child.output ++ windowExpression.map(_.toAttribute)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,10 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
import org.apache.spark.sql.types.{CalendarIntervalType, DateType, IntegerType, TimestampType}

abstract class WindowExecBase(
windowExpression: Seq[NamedExpression],
partitionSpec: Seq[Expression],
orderSpec: Seq[SortOrder],
child: SparkPlan) extends UnaryExecNode {
trait WindowExecBase extends UnaryExecNode {
def windowExpression: Seq[NamedExpression]
def partitionSpec: Seq[Expression]
def orderSpec: Seq[SortOrder]

/**
* Create the resulting projection.
Expand Down