|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | +package io.substrait.spark.expression |
| 18 | + |
| 19 | +import io.substrait.spark.expression.ToWindowFunction.fromSpark |
| 20 | + |
| 21 | +import org.apache.spark.sql.catalyst.expressions.{CurrentRow, Expression, FrameType, Literal, OffsetWindowFunction, RangeFrame, RowFrame, SpecifiedWindowFrame, UnboundedFollowing, UnboundedPreceding, UnspecifiedFrame, WindowExpression, WindowFrame, WindowSpecDefinition} |
| 22 | +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression |
| 23 | +import org.apache.spark.sql.types.{IntegerType, LongType} |
| 24 | + |
| 25 | +import io.substrait.`type`.Type |
| 26 | +import io.substrait.expression.{Expression => SExpression, ExpressionCreator, FunctionArg, WindowBound} |
| 27 | +import io.substrait.expression.Expression.WindowBoundsType |
| 28 | +import io.substrait.expression.WindowBound.{CURRENT_ROW, UNBOUNDED, WindowBoundVisitor} |
| 29 | +import io.substrait.extension.SimpleExtension |
| 30 | +import io.substrait.relation.ConsistentPartitionWindow.WindowRelFunctionInvocation |
| 31 | + |
| 32 | +import scala.collection.JavaConverters |
| 33 | + |
| 34 | +abstract class ToWindowFunction(functions: Seq[SimpleExtension.WindowFunctionVariant]) |
| 35 | + extends FunctionConverter[SimpleExtension.WindowFunctionVariant, WindowRelFunctionInvocation]( |
| 36 | + functions) { |
| 37 | + |
| 38 | + override def generateBinding( |
| 39 | + sparkExp: Expression, |
| 40 | + function: SimpleExtension.WindowFunctionVariant, |
| 41 | + arguments: Seq[FunctionArg], |
| 42 | + outputType: Type): WindowRelFunctionInvocation = { |
| 43 | + |
| 44 | + val (frameType, lower, upper) = sparkExp match { |
| 45 | + case WindowExpression(_: OffsetWindowFunction, _) => |
| 46 | + (WindowBoundsType.ROWS, UNBOUNDED, CURRENT_ROW) |
| 47 | + case WindowExpression( |
| 48 | + _, |
| 49 | + WindowSpecDefinition(_, _, SpecifiedWindowFrame(frameType, lower, upper))) => |
| 50 | + (fromSpark(frameType), fromSpark(lower), fromSpark(upper)) |
| 51 | + case WindowExpression(_, WindowSpecDefinition(_, orderSpec, UnspecifiedFrame)) => |
| 52 | + if (orderSpec.isEmpty) { |
| 53 | + (WindowBoundsType.ROWS, UNBOUNDED, UNBOUNDED) |
| 54 | + } else { |
| 55 | + (WindowBoundsType.RANGE, UNBOUNDED, CURRENT_ROW) |
| 56 | + } |
| 57 | + |
| 58 | + case _ => throw new UnsupportedOperationException(s"Unsupported window expression: $sparkExp") |
| 59 | + } |
| 60 | + |
| 61 | + ExpressionCreator.windowRelFunction( |
| 62 | + function, |
| 63 | + outputType, |
| 64 | + SExpression.AggregationPhase.INITIAL_TO_RESULT, // use defaults... |
| 65 | + SExpression.AggregationInvocation.ALL, // Spark doesn't define these |
| 66 | + frameType, |
| 67 | + lower, |
| 68 | + upper, |
| 69 | + JavaConverters.asJavaIterable(arguments) |
| 70 | + ) |
| 71 | + } |
| 72 | + |
| 73 | + def convert( |
| 74 | + expression: WindowExpression, |
| 75 | + operands: Seq[SExpression]): Option[WindowRelFunctionInvocation] = { |
| 76 | + val cls = expression.windowFunction match { |
| 77 | + case agg: AggregateExpression => agg.aggregateFunction.getClass |
| 78 | + case other => other.getClass |
| 79 | + } |
| 80 | + |
| 81 | + Option(signatures.get(cls)) |
| 82 | + .flatMap(m => m.attemptMatch(expression, operands)) |
| 83 | + } |
| 84 | + |
| 85 | + def apply( |
| 86 | + expression: WindowExpression, |
| 87 | + operands: Seq[SExpression]): WindowRelFunctionInvocation = { |
| 88 | + convert(expression, operands).getOrElse(throw new UnsupportedOperationException( |
| 89 | + s"Unable to find binding for call ${expression.windowFunction} -- $operands -- $expression")) |
| 90 | + } |
| 91 | +} |
| 92 | + |
| 93 | +object ToWindowFunction { |
| 94 | + def fromSpark(frameType: FrameType): WindowBoundsType = frameType match { |
| 95 | + case RowFrame => WindowBoundsType.ROWS |
| 96 | + case RangeFrame => WindowBoundsType.RANGE |
| 97 | + case other => throw new UnsupportedOperationException(s"Unsupported bounds type: $other.") |
| 98 | + } |
| 99 | + |
| 100 | + def fromSpark(bound: Expression): WindowBound = bound match { |
| 101 | + case UnboundedPreceding => WindowBound.UNBOUNDED |
| 102 | + case UnboundedFollowing => WindowBound.UNBOUNDED |
| 103 | + case CurrentRow => WindowBound.CURRENT_ROW |
| 104 | + case e: Literal => |
| 105 | + e.dataType match { |
| 106 | + case IntegerType | LongType => |
| 107 | + val offset = e.eval().asInstanceOf[Int] |
| 108 | + if (offset < 0) WindowBound.Preceding.of(-offset) |
| 109 | + else if (offset == 0) WindowBound.CURRENT_ROW |
| 110 | + else WindowBound.Following.of(offset) |
| 111 | + } |
| 112 | + case _ => throw new UnsupportedOperationException(s"Unexpected bound: $bound") |
| 113 | + } |
| 114 | + |
| 115 | + def toSparkFrame( |
| 116 | + boundsType: WindowBoundsType, |
| 117 | + lowerBound: WindowBound, |
| 118 | + upperBound: WindowBound): WindowFrame = { |
| 119 | + val frameType = boundsType match { |
| 120 | + case WindowBoundsType.ROWS => RowFrame |
| 121 | + case WindowBoundsType.RANGE => RangeFrame |
| 122 | + case WindowBoundsType.UNSPECIFIED => return UnspecifiedFrame |
| 123 | + } |
| 124 | + SpecifiedWindowFrame( |
| 125 | + frameType, |
| 126 | + toSparkBound(lowerBound, isLower = true), |
| 127 | + toSparkBound(upperBound, isLower = false)) |
| 128 | + } |
| 129 | + |
| 130 | + private def toSparkBound(bound: WindowBound, isLower: Boolean): Expression = { |
| 131 | + bound.accept(new WindowBoundVisitor[Expression, Exception] { |
| 132 | + |
| 133 | + override def visit(preceding: WindowBound.Preceding): Expression = |
| 134 | + Literal(-preceding.offset().intValue()) |
| 135 | + |
| 136 | + override def visit(following: WindowBound.Following): Expression = |
| 137 | + Literal(following.offset().intValue()) |
| 138 | + |
| 139 | + override def visit(currentRow: WindowBound.CurrentRow): Expression = CurrentRow |
| 140 | + |
| 141 | + override def visit(unbounded: WindowBound.Unbounded): Expression = |
| 142 | + if (isLower) UnboundedPreceding else UnboundedFollowing |
| 143 | + }) |
| 144 | + } |
| 145 | + |
| 146 | + def apply(functions: Seq[SimpleExtension.WindowFunctionVariant]): ToWindowFunction = { |
| 147 | + new ToWindowFunction(functions) { |
| 148 | + override def getSigs: Seq[Sig] = |
| 149 | + FunctionMappings.WINDOW_SIGS ++ FunctionMappings.AGGREGATE_SIGS |
| 150 | + } |
| 151 | + } |
| 152 | + |
| 153 | +} |
0 commit comments