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 @@ -76,6 +76,8 @@ class Analyzer(
ResolveGenerate ::
ResolveFunctions ::
ResolveAliases ::
ResolveWindowOrder ::
ResolveWindowFrame ::
ExtractWindowExpressions ::
GlobalAggregates ::
UnresolvedHavingClauseAttributes ::
Expand Down Expand Up @@ -525,6 +527,8 @@ class Analyzer(
case u @ UnresolvedFunction(name, children, isDistinct) =>
withPosition(u) {
registry.lookupFunction(name, children) match {
// DISTINCT is not meaningful in case of WindowFunctions.
case wf: WindowFunction2 => wf
// We get an aggregate function built based on AggregateFunction2 interface.
// So, we wrap it in AggregateExpression2.
case agg2: AggregateFunction2 => AggregateExpression2(agg2, Complete, isDistinct)
Expand Down Expand Up @@ -557,11 +561,18 @@ class Analyzer(
}

def containsAggregates(exprs: Seq[Expression]): Boolean = {
exprs.foreach(_.foreach {
case agg: AggregateExpression => return true
case _ =>
})
false
// Collect all Windowed Aggregate Expressions.
val blacklist = exprs.flatMap { expr =>
expr.collect {
case WindowExpression(ae: AggregateExpression, _) => ae
}
}.toSet

// Find the first Aggregate Expression that is not Windowed.
exprs.exists(_.collectFirst {
case ae: AggregateExpression if !blacklist.contains(ae) => ae
}.isDefined)

}
}

Expand Down Expand Up @@ -763,26 +774,38 @@ class Analyzer(

// Now, we extract regular expressions from expressionsWithWindowFunctions
// by using extractExpr.
val seenWindowAggregates = new ArrayBuffer[AggregateExpression]
val newExpressionsWithWindowFunctions = expressionsWithWindowFunctions.map {
_.transform {
// Extracts children expressions of a WindowFunction (input parameters of
// a WindowFunction).
case wf : WindowFunction =>
val newChildren = wf.children.map(extractExpr(_))
val newChildren = wf.children.map(extractExpr)
wf.withNewChildren(newChildren)

case wf : WindowFunction2 =>
val newChildren = wf.children.map(extractExpr)
wf.withNewChildren(newChildren)

// Extracts expressions from the partition spec and order spec.
case wsc @ WindowSpecDefinition(partitionSpec, orderSpec, _) =>
val newPartitionSpec = partitionSpec.map(extractExpr(_))
val newPartitionSpec = partitionSpec.map(extractExpr)
val newOrderSpec = orderSpec.map { so =>
val newChild = extractExpr(so.child)
so.copy(child = newChild)
}
wsc.copy(partitionSpec = newPartitionSpec, orderSpec = newOrderSpec)

// Extract Windowed AggregateExpression
case we @ WindowExpression(agg: AggregateExpression, spec: WindowSpecDefinition) =>
val newAggChildren = agg.children.map(extractExpr)
val newAgg = agg.withNewChildren(newAggChildren).asInstanceOf[AggregateExpression]
seenWindowAggregates += newAgg
WindowExpression(newAgg, spec)

// Extracts AggregateExpression. For example, for SUM(x) - Sum(y) OVER (...),
// we need to extract SUM(x).
case agg: AggregateExpression =>
case agg: AggregateExpression if !seenWindowAggregates.contains(agg) =>
val withName = Alias(agg, s"_w${extractedExprBuffer.length}")()
extractedExprBuffer += withName
withName.toAttribute
Expand Down Expand Up @@ -957,6 +980,42 @@ class Analyzer(
Project(p.output, newPlan.withNewChildren(newChild :: Nil))
}
}

/**
* Check and add proper window frames for all window functions.
*/
object ResolveWindowFrame extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case logical: LogicalPlan => logical transformExpressions {
case WindowExpression(wf: WindowFunction2,
WindowSpecDefinition(_, _, f: SpecifiedWindowFrame))
if wf.frame != UnspecifiedFrame && wf.frame != f =>
failAnalysis(s"Window Frame $f must match the required frame ${wf.frame}")
case WindowExpression(wf: WindowFunction2,
s @ WindowSpecDefinition(_, o, UnspecifiedFrame))
if wf.frame != UnspecifiedFrame =>
WindowExpression(wf, s.copy(frameSpecification = wf.frame))
case we @ WindowExpression(e, s @ WindowSpecDefinition(_, o, UnspecifiedFrame)) =>
val frame = SpecifiedWindowFrame.defaultWindowFrame(!o.isEmpty, true)
we.copy(windowSpec = s.copy(frameSpecification = frame))
}
}
}

/**
* Check and add order to [[AggregateWindowFunction]]s.
*/
object ResolveWindowOrder extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case logical: LogicalPlan => logical transformExpressions {
case WindowExpression(agg: AggregateWindowFunction, spec) if spec.orderSpec.isEmpty =>
failAnalysis(s"AggregateWindowFunction $agg window specification must be ordered")
case WindowExpression(rank: RankLike, spec) if spec.resolved =>
val order = spec.orderSpec.map(_.child)
WindowExpression(rank.withOrder(order), spec)
}
}
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -250,7 +250,17 @@ object FunctionRegistry {
expression[Sha1]("sha1"),
expression[Sha2]("sha2"),
expression[SparkPartitionID]("spark_partition_id"),
expression[InputFileName]("input_file_name")
expression[InputFileName]("input_file_name"),

// window functions
expression[Lead]("lead"),
expression[Lag]("lag"),
expression[RowNumber]("row_number"),
expression[CumeDist]("cume_dist"),
expression[NTile]("ntile"),
expression[Rank]("rank"),
expression[DenseRank]("dense_rank"),
expression[PercentRank]("percent_rank")
)

val builtin: FunctionRegistry = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.sql.catalyst.expressions

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp
import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateSafeProjection, GenerateUnsafeProjection}
import org.apache.spark.sql.types.{DataType, Decimal, StructType, _}
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
Expand All @@ -43,7 +44,10 @@ class InterpretedProjection(expressions: Seq[Expression]) extends Projection {
val outputArray = new Array[Any](exprArray.length)
var i = 0
while (i < exprArray.length) {
outputArray(i) = exprArray(i).eval(input)
exprArray(i) match {
case NoOp =>
case e => outputArray(i) = e.eval(input)
}
i += 1
}
new GenericInternalRow(outputArray)
Expand Down Expand Up @@ -79,7 +83,10 @@ case class InterpretedMutableProjection(expressions: Seq[Expression]) extends Mu
override def apply(input: InternalRow): InternalRow = {
var i = 0
while (i < exprArray.length) {
mutableRow(i) = exprArray(i).eval(input)
exprArray(i) match {
case NoOp =>
case e => mutableRow(i) = e.eval(input)
}
i += 1
}
mutableRow
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,108 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.catalyst.expressions.aggregate

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, CodeGenContext, CodegenFallback}
import org.apache.spark.sql.types.DataType
import org.apache.spark.util.collection.OpenHashSet


/** Reduce a set using an algebraic expression. */
case class ReduceSetAlgebraic(left: Expression, right: AlgebraicAggregate)
extends BinaryExpression with CodegenFallback {

override def dataType: DataType = right.dataType

private[this] val single = right.children.size == 1
private[this] val singleValueOrdinal = right.bufferSchema.length

// This might be taking reuse too far...
@transient private[this] lazy val buffer = {
val singleSize = if (single) 1 else 0
new GenericMutableRow(singleValueOrdinal + singleSize)
}

@transient private[this] lazy val initial =
InterpretedMutableProjection(right.initialValues).target(buffer)

@transient private[this] lazy val update = {
val schema = right.bufferAttributes ++ right.children.map { child =>
AttributeReference("child", child.dataType, child.nullable)()
}
new InterpretedMutableProjection(right.updateExpressions, schema).target(buffer)
}

@transient private[this] lazy val evaluate =
BindReferences.bindReference(right.evaluateExpression, right.bufferSchema.toAttributes)

@transient private[this] lazy val joinRow = new JoinedRow

override def eval(input: InternalRow): Any = {
val result = left.eval(input).asInstanceOf[OpenHashSet[Any]]
if (result != null) {
initial(EmptyRow)
val iterator = result.iterator
// Prevent branch during iteration.
if (single) {
while (iterator.hasNext) {
buffer.update(singleValueOrdinal, iterator.next)
update(buffer)
}
} else {
while (iterator.hasNext) {
joinRow(buffer, iterator.next.asInstanceOf[InternalRow])
update(joinRow)
}
}
evaluate.eval(buffer)
} else null
}
}
/** Reduce a set using an AggregateFunction2. */
case class ReduceSetAggregate(left: Expression, right: AggregateFunction2)
extends BinaryExpression with CodegenFallback {

right.withNewMutableBufferOffset(0)

override def dataType: DataType = right.dataType

private[this] val single = right.children.size == 1
@transient private[this] lazy val buffer = new GenericMutableRow(right.bufferSchema.size)
@transient private[this] lazy val singleValueInput = new GenericMutableRow(1)

override def eval(input: InternalRow): Any = {
val result = left.eval(input).asInstanceOf[OpenHashSet[Any]]
if (result != null) {
right.initialize(buffer)
val iterator = result.iterator
if (single) {
while (iterator.hasNext) {
singleValueInput.update(0, iterator.next())
right.update(buffer, singleValueInput)
}
} else {
while (iterator.hasNext) {
right.update(buffer, iterator.next().asInstanceOf[InternalRow])
}
}
right.eval(buffer)
} else null
}
}
Loading