Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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,14 +19,14 @@ package org.apache.spark.sql.execution.datasources.v2

import scala.collection.mutable

import org.apache.spark.sql.catalyst.expressions.{Alias, AliasHelper, And, Attribute, AttributeReference, Cast, Divide, DivideDTInterval, DivideYMInterval, EqualTo, Expression, If, IntegerLiteral, Literal, NamedExpression, PredicateHelper, ProjectionOverSchema, SubqueryExpression}
import org.apache.spark.sql.catalyst.expressions.{Alias, AliasHelper, And, Attribute, AttributeReference, Cast, Divide, DivideDTInterval, DivideYMInterval, EqualTo, Expression, If, IntegerLiteral, Literal, NamedExpression, PredicateHelper, ProjectionOverSchema, SortOrder, SubqueryExpression}
import org.apache.spark.sql.catalyst.expressions.aggregate
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.optimizer.CollapseProject
import org.apache.spark.sql.catalyst.planning.ScanOperation
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LeafNode, Limit, LocalLimit, LogicalPlan, Project, Sample, Sort}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.connector.expressions.SortOrder
import org.apache.spark.sql.connector.expressions.{SortOrder => V2SortOrder}
import org.apache.spark.sql.connector.expressions.aggregate.{Aggregation, Avg, Count, GeneralAggregateFunc, Sum}
import org.apache.spark.sql.connector.expressions.filter.Predicate
import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, V1Scan}
Expand Down Expand Up @@ -374,9 +374,11 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit
sHolder.pushedLimit = Some(limit)
}
operation
case s @ Sort(order, _, operation @ ScanOperation(_, filter, sHolder: ScanBuilderHolder))
case s @ Sort(order, _, operation @ ScanOperation(project, filter, sHolder: ScanBuilderHolder))
if filter.isEmpty =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we should add one more condition CollapseProject.canCollapseExpressions(...)

val orders = DataSourceStrategy.translateSortOrders(order)
val aliasMap = getAliasMap(project)
val newOrder = order.map(replaceAlias(_, aliasMap)).asInstanceOf[Seq[SortOrder]]
val orders = DataSourceStrategy.translateSortOrders(newOrder)
if (orders.length == order.length) {
val topNPushed = PushDownUtils.pushTopN(sHolder.builder, orders.toArray, limit)
if (topNPushed) {
Expand Down Expand Up @@ -427,7 +429,7 @@ case class ScanBuilderHolder(
builder: ScanBuilder) extends LeafNode {
var pushedLimit: Option[Int] = None

var sortOrders: Seq[SortOrder] = Seq.empty[SortOrder]
var sortOrders: Seq[V2SortOrder] = Seq.empty[V2SortOrder]

var pushedSample: Option[TableSampleInfo] = None

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -267,6 +267,18 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
checkAnswer(df8, Seq(Row(2, "alex", 12000.00, 1200.0, false)))
}

test("simple scan with top N: order by with alias") {
val df1 = spark.read
.table("h2.test.employee")
.select($"NAME", $"SALARY".as("mySalary"))
.sort("mySalary")
.limit(1)
checkSortRemoved(df1)
checkPushedInfo(df1,
"PushedFilters: [], PushedTopN: ORDER BY [SALARY ASC NULLS FIRST] LIMIT 1, ")
checkAnswer(df1, Seq(Row("cathy", 9000.00)))
}

test("scan with filter push-down") {
val df = spark.table("h2.test.people").filter($"id" > 1)
checkFiltersRemoved(df)
Expand Down