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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution

import org.apache.commons.lang.StringUtils

import org.apache.spark.Partition
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{AnalysisException, Row, SparkSession, SQLContext}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier}
Expand All @@ -27,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics}
import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning}
import org.apache.spark.sql.execution.datasources.HadoopFsRelation
import org.apache.spark.sql.execution.datasources.{FilePartition, HadoopFsRelation}
import org.apache.spark.sql.execution.datasources.parquet.{DefaultSource => ParquetSource}
import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.sql.internal.SQLConf
Expand Down Expand Up @@ -147,6 +148,7 @@ private[sql] case class RowDataSourceScanExec(
rdd: RDD[InternalRow],
@transient relation: BaseRelation,
override val outputPartitioning: Partitioning,
override val outputOrdering: Seq[SortOrder],
override val metadata: Map[String, String],
override val metastoreTableIdentifier: Option[TableIdentifier])
extends DataSourceScanExec with CodegenSupport {
Expand Down Expand Up @@ -221,6 +223,7 @@ private[sql] case class BatchedDataSourceScanExec(
rdd: RDD[InternalRow],
@transient relation: BaseRelation,
override val outputPartitioning: Partitioning,
override val outputOrdering: Seq[SortOrder],
override val metadata: Map[String, String],
override val metastoreTableIdentifier: Option[TableIdentifier])
extends DataSourceScanExec with CodegenSupport {
Expand Down Expand Up @@ -338,37 +341,66 @@ private[sql] object DataSourceScanExec {
rdd: RDD[InternalRow],
relation: BaseRelation,
metadata: Map[String, String] = Map.empty,
metastoreTableIdentifier: Option[TableIdentifier] = None): DataSourceScanExec = {
val outputPartitioning = {
val bucketSpec = relation match {
// TODO: this should be closer to bucket planning.
case r: HadoopFsRelation
if r.sparkSession.sessionState.conf.bucketingEnabled => r.bucketSpec
case _ => None
}
metastoreTableIdentifier: Option[TableIdentifier] = None,
partitions: Seq[Partition] = Seq()): DataSourceScanExec = {
val (outputPartitioning, outputOrdering) = {

def toAttribute(colName: String): Attribute = output.find(_.name == colName).getOrElse {
throw new AnalysisException(s"bucket column $colName not found in existing columns " +
s"(${output.map(_.name).mkString(", ")})")
}

val (bucketSpec, partitionColumns) = relation match {
// TODO: this should be closer to bucket planning.
case r: HadoopFsRelation if r.sparkSession.sessionState.conf.bucketingEnabled =>
(r.bucketSpec, r.partitionSchema.fieldNames.toSeq.map(toAttribute))
case _ => (None, Seq())
}

bucketSpec.map { spec =>
val numBuckets = spec.numBuckets
val bucketColumns = spec.bucketColumnNames.map(toAttribute)
HashPartitioning(bucketColumns, numBuckets)

val singleFilePartitions = partitions.forall {
case f: FilePartition => f.files.length <= 1
case _ => false
}

// In case of bucketing, its possible to have multiple files belonging to the
// same bucket in a given table partition. Each of these files are locally sorted
// but those files combined together are not globally sorted. Given that,
// the RDD partition will not be sorted even if the table has sort columns set.
val sortOrder = if (singleFilePartitions) {
spec.sortColumnNames.map(c => SortOrder(toAttribute(c), Ascending))
} else {
Nil
}
(HashPartitioning(bucketColumns ++ partitionColumns, numBuckets), sortOrder)
}.getOrElse {
UnknownPartitioning(0)
(UnknownPartitioning(0), Nil)
}
}

relation match {
case r: HadoopFsRelation
if r.fileFormat.supportBatch(r.sparkSession, StructType.fromAttributes(output)) =>
BatchedDataSourceScanExec(
output, rdd, relation, outputPartitioning, metadata, metastoreTableIdentifier)
output,
rdd,
relation,
outputPartitioning,
outputOrdering,
metadata,
metastoreTableIdentifier)
case _ =>
RowDataSourceScanExec(
output, rdd, relation, outputPartitioning, metadata, metastoreTableIdentifier)
output,
rdd,
relation,
outputPartitioning,
outputOrdering,
metadata,
metastoreTableIdentifier)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -210,7 +210,8 @@ private[sql] object FileSourceStrategy extends Strategy with Logging {
plannedPartitions),
files,
meta,
table)
table,
plannedPartitions)

val afterScanFilter = afterScanFilters.toSeq.reduceOption(expressions.And)
val withFilter = afterScanFilter.map(execution.FilterExec(_, scan)).getOrElse(scan)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -236,7 +236,12 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] {
children = children.zip(requiredChildOrderings).map { case (child, requiredOrdering) =>
if (requiredOrdering.nonEmpty) {
// If child.outputOrdering is [a, b] and requiredOrdering is [a], we do not need to sort.
if (requiredOrdering != child.outputOrdering.take(requiredOrdering.length)) {
val orderingMatched = requiredOrdering.zip(child.outputOrdering).forall {
case (requiredOrder, childOutputOrder) =>
requiredOrder == childOutputOrder
}

if (orderingMatched) {
SortExec(requiredOrdering, global = false, child = child)
} else {
child
Expand Down