From 9897413564ff27f0a311cc2cef6322422f3807ab Mon Sep 17 00:00:00 2001 From: guowei2 Date: Fri, 24 Oct 2014 17:55:47 +0800 Subject: [PATCH 01/14] window function --- .../sql/catalyst/analysis/Analyzer.scala | 12 +- .../sql/catalyst/analysis/unresolved.scala | 3 +- .../sql/catalyst/expressions/aggregates.scala | 13 +- .../plans/logical/basicOperators.scala | 13 + .../spark/sql/execution/SparkStrategies.scala | 2 + .../spark/sql/execution/WindowFunction.scala | 353 ++++++++++++++++++ .../org/apache/spark/sql/hive/HiveQl.scala | 231 +++++++++++- .../org/apache/spark/sql/hive/hiveUdfs.scala | 7 +- .../execution/HiveWindowFunctionSuite.scala | 312 ++++++++++++++++ 9 files changed, 928 insertions(+), 18 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 04639219a3650..f62387a4652d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -218,8 +218,16 @@ class Analyzer(catalog: Catalog, def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressions { - case u @ UnresolvedFunction(name, children) if u.childrenResolved => - registry.lookupFunction(name, children) + case u @ UnresolvedFunction(name, children, windowRange) if u.childrenResolved => { + val function = registry.lookupFunction(name, children) + if (windowRange != null) function match { + case agg: AggregateExpression => + function.asInstanceOf[AggregateExpression].windowRange = windowRange + case _ => logError(s"function " + name + " does not support window range") + } + function + } + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 77d84e1687e1b..12cc6e5c07689 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -63,7 +63,8 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo override def toString: String = s"'$name" } -case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression { +case class UnresolvedFunction(name: String, children: Seq[Expression], + windowRange: WindowRange = null) extends Expression { override def dataType = throw new UnresolvedException(this, "dataType") override def foldable = throw new UnresolvedException(this, "foldable") override def nullable = throw new UnresolvedException(this, "nullable") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 0cd90866e14a2..11136a3dcf18a 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -24,15 +24,22 @@ import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.util.collection.OpenHashSet -abstract class AggregateExpression extends Expression { +abstract class AggregateExpression extends Expression with Serializable { self: Product => + var windowRange: WindowRange = null /** * Creates a new instance that can be used to compute this aggregate expression for a group * of input rows/ */ def newInstance(): AggregateFunction + override def equals(other: Any): Boolean = { + if (super.equals(other)) { + this.windowRange == other.asInstanceOf[AggregateExpression].windowRange + } else false + } + /** * [[AggregateExpression.eval]] should never be invoked because [[AggregateExpression]]'s are * replaced with a physical aggregate operator at runtime. @@ -41,6 +48,8 @@ abstract class AggregateExpression extends Expression { throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } +case class WindowRange(windowType:String, preceding: Int, following: Int) + /** * Represents an aggregation that has been rewritten to be performed in two steps. * @@ -71,7 +80,7 @@ abstract class PartialAggregate extends AggregateExpression { * [[AggregateExpression]] with an algorithm that will be used to compute one specific result. */ abstract class AggregateFunction - extends AggregateExpression with Serializable with trees.LeafNode[Expression] { + extends AggregateExpression with trees.LeafNode[Expression] { self: Product => override type EvaluatedType = Any diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 64b8d45ebbf42..3bb7d23e5b739 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -143,6 +143,19 @@ case class Aggregate( override def output = aggregateExpressions.map(_.toAttribute) } +case class WindowFunction( + partitionExpressions: Seq[Expression], + functionExpressions: Seq[NamedExpression], + child: LogicalPlan) + extends UnaryNode { + + def references = + AttributeSet( + partitionExpressions.flatMap(_.references) ++ functionExpressions.flatMap(_.references)) + + override def output = functionExpressions.map(_.toAttribute) +} + case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { override def output = child.output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 1225d18857af2..883bea6d6caf7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -272,6 +272,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => execution.Aggregate(partial = false, group, agg, planLater(child)) :: Nil + case logical.WindowFunction(partition, function, child) => + execution.WindowFunction(partition, function, planLater(child)) :: Nil case logical.Sample(fraction, withReplacement, seed, child) => execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil case SparkLogicalPlan(alreadyPlanned) => alreadyPlanned :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala new file mode 100644 index 0000000000000..5f0ae0d3333a7 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala @@ -0,0 +1,353 @@ +/* + * 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.execution + +import java.util.HashMap + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.AllTuples +import org.apache.spark.sql.catalyst.plans.physical.ClusteredDistribution +import org.apache.spark.sql.catalyst.errors._ +import scala.collection.mutable.ArrayBuffer +import org.apache.spark.util.collection.CompactBuffer +import org.apache.spark.sql.catalyst.plans.physical.ClusteredDistribution +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection +import org.apache.spark.sql.catalyst.expressions.Alias +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.SortPartitions + + +/** + * :: DeveloperApi :: + * Groups input data by `partitionExpressions` and computes the `computeExpressions` for each + * group. + * @param partitionExpressions expressions that are evaluated to determine partition. + * @param functionExpressions expressions that are computed for each partition. + * @param child the input data source. + */ +@DeveloperApi +case class WindowFunction( + partitionExpressions: Seq[Expression], + functionExpressions: Seq[NamedExpression], + child: SparkPlan) + extends UnaryNode { + + override def requiredChildDistribution = + if (partitionExpressions == Nil) { + AllTuples :: Nil + } else { + ClusteredDistribution(partitionExpressions) :: Nil + } + + // HACK: Generators don't correctly preserve their output through serializations so we grab + // out child's output attributes statically here. + private[this] val childOutput = child.output + + override def output = functionExpressions.map(_.toAttribute) + + /** A list of functions that need to be computed for each partition. */ + private[this] val computeExpressions = new ArrayBuffer[AggregateExpression] + + private[this] val otherExpressions = new ArrayBuffer[NamedExpression] + + functionExpressions.foreach { sel => + sel.collect { + case func: AggregateExpression => computeExpressions += func + case other: NamedExpression if (!other.isInstanceOf[Alias]) => otherExpressions += other + } + } + + private[this] val functionAttributes = computeExpressions.map { func => + func -> AttributeReference(s"funcResult:$func", func.dataType, func.nullable)()} + + /** The schema of the result of all evaluations */ + private[this] val resultAttributes = + otherExpressions.map(_.toAttribute) ++ functionAttributes.map(_._2) + + private[this] val resultMap = + (otherExpressions.map { other => other -> other.toAttribute } ++ functionAttributes + ).toMap + + + private[this] val resultExpressions = functionExpressions.map { sel => + sel.transform { + case e: Expression if resultMap.contains(e) => resultMap(e) + } + } + + private[this] val sortExpressions = + if (child.isInstanceOf[SortPartitions]) { + child.asInstanceOf[SortPartitions].sortExpressions + } + else if (child.isInstanceOf[Sort]) { + child.asInstanceOf[Sort].sortOrder + } + else null + + /** Creates a new function buffer for a partition. */ + private[this] def newFunctionBuffer(): Array[AggregateFunction] = { + val buffer = new Array[AggregateFunction](computeExpressions.length) + var i = 0 + while (i < computeExpressions.length) { + val baseExpr = BindReferences.bindReference(computeExpressions(i), childOutput) + baseExpr.windowRange = computeExpressions(i).windowRange + buffer(i) = baseExpr.newInstance() + i += 1 + } + buffer + } + + private[this] def computeFunctions(rows: CompactBuffer[Row]): Array[Iterator[Any]] = { + val aggrFunctions = newFunctionBuffer() + val functionResults = new Array[Iterator[Any]](aggrFunctions.length) + var i = 0 + while (i < aggrFunctions.length) { + val aggrFunction = aggrFunctions(i) + val base = aggrFunction.base + if (base.windowRange == null) { + if (sortExpressions != null) { + if (aggrFunction.dataType.isInstanceOf[ArrayType]) { + rows.foreach(aggrFunction.update) + functionResults(i) = aggrFunction.eval(EmptyRow).asInstanceOf[Seq[Any]].iterator + } else { + functionResults(i) = rows.map(row => { + aggrFunction.update(row) + aggrFunction.eval(EmptyRow) + }).iterator + } + } else { + rows.foreach(aggrFunction.update) + functionResults(i) = aggrFunction.eval(EmptyRow) match { + case r: Seq[_] => r.iterator + case other => (0 to rows.size - 1).map(r => other).iterator + } + } + + } else { + functionResults(i) = + if (base.windowRange.windowType == "ROWS_RANGE") rowsWindowFunction(base, rows).iterator + else valueWindowFunction(base, rows).iterator + } + i += 1 + } + functionResults + } + + private[this] def rowsWindowFunction(base: AggregateExpression, + rows: CompactBuffer[Row]): CompactBuffer[Any] = { + + val rangeResults = new CompactBuffer[Any]() + var rowIndex = 0 + while (rowIndex < rows.size) { + + val windowRange = base.windowRange + var start = + if (windowRange.preceding == Int.MaxValue) 0 + else rowIndex - windowRange.preceding + if (start < 0) start = 0 + var end = + if (windowRange.following == Int.MaxValue) { + rows.size - 1 + } else { + rowIndex + windowRange.following + } + if (end > rows.size - 1) end = rows.size - 1 + + //new aggregate function + val aggr = base.newInstance() + (start to end).foreach(i => aggr.update(rows(i))) + + rangeResults += aggr.eval(EmptyRow) + rowIndex += 1 + } + rangeResults + } + + private[this] def valueWindowFunction(base: AggregateExpression, + rows: CompactBuffer[Row]): CompactBuffer[Any] = { + + val windowRange = base.windowRange + + // rande only support 1 order + val sortExpression = BindReferences.bindReference(sortExpressions.head, childOutput) + + val preceding = sortExpression.child.dataType match { + case IntegerType => Literal(windowRange.preceding) + case LongType => Literal(windowRange.preceding.toLong) + case DoubleType => Literal(windowRange.preceding.toDouble) + case FloatType => Literal(windowRange.preceding.toFloat) + case ShortType => Literal(windowRange.preceding.toShort) + case DecimalType => Literal(BigDecimal(windowRange.preceding)) + case _=> throw new Exception(s"not support dataType ") + } + val following = sortExpression.child.dataType match { + case IntegerType => Literal(windowRange.following) + case LongType => Literal(windowRange.following.toLong) + case DoubleType => Literal(windowRange.following.toDouble) + case FloatType => Literal(windowRange.following.toFloat) + case ShortType => Literal(windowRange.following.toShort) + case DecimalType => Literal(BigDecimal(windowRange.following)) + case _=> throw new Exception(s"not support dataType ") + } + + val rangeResults = new CompactBuffer[Any]() + var rowIndex = 0 + while (rowIndex < rows.size) { + val currentRow = rows(rowIndex) + val precedingExpr = + if (sortExpression.direction == Ascending) { + Literal(sortExpression.child.eval(currentRow)) - sortExpression.child <= preceding + } else { + sortExpression.child - Literal(sortExpression.child.eval(currentRow)) <= preceding + } + + + val followingExpr = + if (sortExpression.direction == Ascending) { + sortExpression.child - Literal(sortExpression.child.eval(currentRow)) <= following + } else { + Literal(sortExpression.child.eval(currentRow)) - sortExpression.child <= following + } + + var precedingIndex = 0 + var followingIndex = rows.size - 1 + if (sortExpression != null) { + + if (windowRange.preceding != Int.MaxValue) precedingIndex = rowIndex + while (precedingIndex > 0 && + precedingExpr.eval(rows(precedingIndex - 1)).asInstanceOf[Boolean]) { + precedingIndex -= 1 + } + + if (windowRange.following != Int.MaxValue) followingIndex = rowIndex + while (followingIndex < rows.size - 1 && + followingExpr.eval(rows(followingIndex + 1)).asInstanceOf[Boolean]) { + followingIndex += 1 + } + } + //new aggregate function + val aggr = base.newInstance() + (precedingIndex to followingIndex).foreach(i => aggr.update(rows(i))) + rangeResults += aggr.eval(EmptyRow) + rowIndex += 1 + } + rangeResults + } + + private[this] def getNextFunctionsRow( + functionsResult: Array[Iterator[Any]]): GenericMutableRow = { + val result = new GenericMutableRow(functionsResult.length) + var i = 0 + while (i < functionsResult.length) { + result(i) = functionsResult(i).next + i += 1 + } + result + } + + + override def execute() = attachTree(this, "execute") { + if (partitionExpressions.isEmpty) { + child.execute().mapPartitions { iter => + + val resultProjection = new InterpretedProjection(resultExpressions, resultAttributes) + + val otherProjection = new InterpretedMutableProjection(otherExpressions, childOutput) + val joinedRow = new JoinedRow + + val rows = new CompactBuffer[Row]() + while (iter.hasNext) { + rows += iter.next().copy() + } + new Iterator[Row] { + private[this] val functionsResult = computeFunctions(rows) + private[this] var currentRowIndex: Int = 0 + + override final def hasNext: Boolean = currentRowIndex < rows.size + + override final def next(): Row = { + + val otherResults = otherProjection(rows(currentRowIndex)).copy() + currentRowIndex += 1 + resultProjection(joinedRow(otherResults,getNextFunctionsRow(functionsResult))) + } + } + + } + } else { + child.execute().mapPartitions { iter => + val partitionTable = new HashMap[Row, CompactBuffer[Row]] + val partitionProjection = + new InterpretedMutableProjection(partitionExpressions, childOutput) + + var currentRow: Row = null + while (iter.hasNext) { + currentRow = iter.next() + val partitionKey = partitionProjection(currentRow).copy() + val existingMatchList = partitionTable.get(partitionKey) + val matchList = if (existingMatchList == null) { + val newMatchList = new CompactBuffer[Row]() + partitionTable.put(partitionKey, newMatchList) + newMatchList + } else { + existingMatchList + } + matchList += currentRow.copy() + } + + new Iterator[Row] { + private[this] val partitionTableIter = partitionTable.entrySet().iterator() + private[this] var currentpartition: CompactBuffer[Row] = _ + private[this] var functionsResult: Array[Iterator[Any]] = _ + private[this] var currentRowIndex: Int = -1 + + val resultProjection = new InterpretedProjection(resultExpressions, resultAttributes) + val otherProjection = new InterpretedMutableProjection(otherExpressions, childOutput) + val joinedRow = new JoinedRow + + override final def hasNext: Boolean = + (currentRowIndex != -1 && currentRowIndex < currentpartition.size) || + (partitionTableIter.hasNext && fetchNext()) + + override final def next(): Row = { + + val otherResults = otherProjection(currentpartition(currentRowIndex)).copy() + currentRowIndex += 1 + resultProjection(joinedRow(otherResults,getNextFunctionsRow(functionsResult))) + + } + + private final def fetchNext(): Boolean = { + + currentRowIndex = 0 + if (partitionTableIter.hasNext) { + currentpartition = partitionTableIter.next().getValue + functionsResult = computeFunctions(currentpartition) + true + } else false + } + } + + } + } + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index cd4e5a239ec66..e227a8459e7d8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -33,6 +33,9 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.types.decimal.Decimal +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable +import java.util.concurrent.ConcurrentHashMap /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -513,7 +516,7 @@ private[hive] object HiveQl { // Return one query for each insert clause. val queries = insertClauses.map { case Token("TOK_INSERT", singleInsert) => - val ( + var ( intoClause :: destClause :: selectClause :: @@ -526,7 +529,8 @@ private[hive] object HiveQl { clusterByClause :: distributeByClause :: limitClause :: - lateralViewClause :: Nil) = { + lateralViewClause :: + windowClause :: Nil) = { getClauses( Seq( "TOK_INSERT_INTO", @@ -541,10 +545,14 @@ private[hive] object HiveQl { "TOK_CLUSTERBY", "TOK_DISTRIBUTEBY", "TOK_LIMIT", - "TOK_LATERAL_VIEW"), + "TOK_LATERAL_VIEW", + "WINDOW"), singleInsert) } + initWindow + checkWindowDef(windowClause) + val relations = nodeToRelation(fromClause) val withWhere = whereClause.map { whereNode => val Seq(whereExpr) = whereNode.getChildren.toSeq @@ -607,11 +615,16 @@ private[hive] object HiveQl { // Not a transformation so must be either project or aggregation. val selectExpressions = nameExpressions(select.getChildren.flatMap(selExprNodeToExpr)) - groupByClause match { - case Some(groupBy) => - Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withLateralView) + checkWindowPartitions match { + case Some(partition) => + windowToPlan(partition, selectExpressions, withLateralView) case None => - Project(selectExpressions, withLateralView) + groupByClause match { + case Some(groupBy) => + Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withLateralView) + case None => + Project(selectExpressions, withLateralView) + } } } @@ -862,6 +875,198 @@ private[hive] object HiveQl { throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") } + // store the window def of current sql + //use thread id as key to avoid mistake when muti sqls parse at the same time + protected val windowDefMap = new ConcurrentHashMap[Long,Map[String, Seq[ASTNode]]]() + + // store the window spec of current sql + //use thread id as key to avoid mistake when muti sqls parse at the same time + protected val windowPartitionsMap = new ConcurrentHashMap[Long, ArrayBuffer[Node]]() + + protected def initWindow() = { + windowDefMap.put(Thread.currentThread().getId, Map[String, Seq[ASTNode]]()) + windowPartitionsMap.put(Thread.currentThread().getId, new ArrayBuffer[Node]()) + } + protected def checkWindowDef(windowClause: Option[Node]) = { + + var winDefs = windowDefMap.get(Thread.currentThread().getId) + + windowClause match { + case Some(window) => window.getChildren.foreach { + case Token("TOK_WINDOWDEF", Token(alias, Nil) :: Token("TOK_WINDOWSPEC", ws) :: Nil) => { + winDefs += alias -> ws + } + } + case None => //do nothing + } + + windowDefMap.put(Thread.currentThread().getId, winDefs) + } + + protected def translateWindowSpec(windowSpec: Seq[ASTNode]): Seq[ASTNode]= { + + windowSpec match { + case Token(alias, Nil) :: Nil => translateWindowSpec(getWindowSpec(alias)) + case Token(alias, Nil) :: range => { + val (partitionClause :: rowsRange :: valueRange :: Nil) = getClauses( + Seq( + "TOK_PARTITIONINGSPEC", + "TOK_WINDOWRANGE", + "TOK_WINDOWVALUES"), + translateWindowSpec(getWindowSpec(alias))) + partitionClause match { + case Some(partition) => partition.asInstanceOf[ASTNode] :: range + case None => range + } + } + case e => e + } + } + + protected def getWindowSpec(alias: String): Seq[ASTNode]= { + windowDefMap.get(Thread.currentThread().getId).getOrElse( + alias, sys.error("no window def for " + alias)) + } + + protected def addWindowPartitions(partition: Node) = { + + var winPartitions = windowPartitionsMap.get(Thread.currentThread().getId) + winPartitions += partition + windowPartitionsMap.put(Thread.currentThread().getId, winPartitions) + } + + protected def getWindowPartitions(): Seq[Node]= { + windowPartitionsMap.get(Thread.currentThread().getId).toSeq + } + + protected def checkWindowPartitions(): Option[Seq[ASTNode]] = { + + val partitionUnits = new ArrayBuffer[Seq[ASTNode]]() + + getWindowPartitions.map { + case Token("TOK_PARTITIONINGSPEC", partition) => Some(partition) + case _ => None + }.foreach { + case Some(partition) => { + if (partitionUnits.isEmpty) partitionUnits += partition + else { + //only add different window partitions + try { + partition zip partitionUnits.head foreach { + case (l,r) => l checkEquals r + } + } catch { + case re: RuntimeException => partitionUnits += partition + } + } + } + case None => //do nothing + } + + //check whether all window partitions are same, we just support same window partition now + if (partitionUnits.size == 0 && getWindowPartitions.size > 0) { + Some(Seq()) + } else if (partitionUnits.size == 1) { + Some(partitionUnits.head) + } else if (partitionUnits.size > 1) { + throw new NotImplementedError(s"not support muti different window specs now." + + " use subQuery instead.") + } else { + None + } + } + + protected def windowToPlan(partitionClause: Seq[ASTNode], + selectExpressions: Seq[NamedExpression], withLateralView: LogicalPlan): LogicalPlan = { + + val (orderByClause :: sortByClause :: distributeByClause :: clusterByClause :: Nil) = + getClauses( + Seq( + "TOK_ORDERBY", + "TOK_SORTBY", + "TOK_DISTRIBUTEBY", + "TOK_CLUSTERBY"), + partitionClause) + + val partitionExpr = (distributeByClause orElse clusterByClause) match { + case Some(partitionBy) => partitionBy.getChildren.map(nodeToExpr) + case None => Seq() + } + + val withWindowPartition = + (orderByClause, sortByClause, distributeByClause, clusterByClause) match { + case (Some(totalOrdering), None, None, None) => + Sort(totalOrdering.getChildren.map(nodeToSortOrder), withLateralView) + case (None, Some(perPartitionOrdering), None, None) => + SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), withLateralView) + case (None, None, Some(partitionExprs), None) => + Repartition(partitionExprs.getChildren.map(nodeToExpr), withLateralView) + case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => + SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), + Repartition(partitionExprs.getChildren.map(nodeToExpr), withLateralView)) + case (Some(perPartitionOrdering), None, Some(partitionExprs), None) => + SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), + Repartition(partitionExprs.getChildren.map(nodeToExpr), withLateralView)) + case (None, None, None, Some(clusterExprs)) => + SortPartitions(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)), + Repartition(clusterExprs.getChildren.map(nodeToExpr), withLateralView)) + case (None, None, None, None) => withLateralView + case _ => sys.error("Unsupported set of ordering / distribution clauses.") + } + + WindowFunction(partitionExpr, selectExpressions, withWindowPartition) + + } + + protected def checkWindowSpec(windowSpec: Seq[ASTNode]): WindowRange = { + + val (partitionClause :: rowsRange :: valueRange :: Nil) = getClauses( + Seq( + "TOK_PARTITIONINGSPEC", + "TOK_WINDOWRANGE", + "TOK_WINDOWVALUES"), + translateWindowSpec(windowSpec)) + + partitionClause match { + case Some(partition) => addWindowPartitions(partition) + case None => addWindowPartitions(null) + } + + (rowsRange orElse valueRange)match { + case Some(range) => { + val rangeSeq = range.getChildren.toIndexedSeq + + if (rangeSeq.size > 0) { + + val preceding = rangeSeq.get(0) match { + case Token("preceding", Token(name, Nil) :: Nil) => + if (name == "unbounded") Int.MaxValue + else name.toInt + case Token("current", Nil) => 0 + case _ => 0 + } + + val following = if (rangeSeq.size > 1) { + rangeSeq.get(1) match { + case Token("following", Token(name, Nil) :: Nil) => + if (name == "unbounded") Int.MaxValue + else name.toInt + case Token("current", Nil) => 0 + case _ => 0 + } + } else 0 + + if (rowsRange.isDefined) { + WindowRange("ROWS_RANGE", preceding, following) + } else { + WindowRange("VALUE_RANGE", preceding, following) + } + + } else null + } + case None => null + } + } protected val escapedIdentifier = "`([^`]+)`".r /** Strips backticks from ident if present */ @@ -1055,8 +1260,16 @@ private[hive] object HiveQl { Substring(nodeToExpr(string), nodeToExpr(pos), nodeToExpr(length)) /* UDFs - Must be last otherwise will preempt built in functions */ - case Token("TOK_FUNCTION", Token(name, Nil) :: args) => - UnresolvedFunction(name, args.map(nodeToExpr)) + case Token("TOK_FUNCTION", Token(name, Nil) :: args) => { + val exprs = new ArrayBuffer[Expression] + var windowRange: WindowRange = null + args.foreach { + case Token("TOK_WINDOWSPEC", winSpec) => windowRange = checkWindowSpec(winSpec) + case a: ASTNode => exprs += nodeToExpr(a) + } + UnresolvedFunction(name, exprs, windowRange) + } + case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: args) => UnresolvedFunction(name, Star(None) :: Nil) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index ed2e96df8ad77..5665f0e2d99da 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -186,7 +186,7 @@ private[hive] case class HiveGenericUdaf( children: Seq[Expression]) extends AggregateExpression with HiveInspectors { - type UDFType = AbstractGenericUDAFResolver + type UDFType = GenericUDAFResolver @transient protected lazy val resolver: AbstractGenericUDAFResolver = funcWrapper.createFunction() @@ -341,9 +341,8 @@ private[hive] case class HiveUdafFunction( private val returnInspector = function.init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors) - // Cast required to avoid type inference selecting a deprecated Hive API. - private val buffer = - function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] + // remove cast to support row_number,rank etc. + private val buffer = function.getNewAggregationBuffer override def eval(input: Row): Any = unwrap(function.evaluate(buffer), returnInspector) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala new file mode 100644 index 0000000000000..04e2073fc2f71 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala @@ -0,0 +1,312 @@ +/* + * 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.hive.execution + +import org.apache.spark.sql.hive.test.TestHive._ + + +class HiveWindowFunctionSuite extends HiveComparisonTest { + + override def beforeAll() { + sql("DROP TABLE IF EXISTS part").collect() + + sql(""" + |CREATE TABLE part( + | p_partkey INT, + | p_name STRING, + | p_mfgr STRING, + | p_brand STRING, + | p_type STRING, + | p_size INT, + | p_container STRING, + | p_retailprice DOUBLE, + | p_comment STRING + |) + """.stripMargin).collect() + + sql(""" + |LOAD DATA LOCAL INPATH '../data/files/part_tiny_without_duplicate.txt' + |OVERWRITE INTO TABLE part + """.stripMargin).collect() + } + + createQueryTest("1.testWindowing", + """ + |SELECT p_mfgr, p_name, p_size, + |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, + |sum(p_retailprice) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN + |unbounded preceding AND current row) AS s1 + |FROM part + """.stripMargin, false) + + createQueryTest("4.testCount", + """ + |SELECT p_mfgr, p_name, + |count(p_size) over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS cd + |FROM part + """.stripMargin, false) + + createQueryTest("5.testCountWithWindowingUDAF", + """ + |SELECT p_mfgr, p_name, + |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, + |count(p_size) over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS cd, + |p_retailprice, sum(p_retailprice) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows + |BETWEEN unbounded preceding AND current row) AS s1, + |p_size + |FROM part + """.stripMargin, false) + + createQueryTest("6.testCountInSubQ", + """ + |SELECT sub1.r, sub1.cd, sub1.s1 + |FROM (SELECT p_mfgr, p_name, + |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, + |count(p_size) over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS cd, + |p_retailprice, sum(p_retailprice) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows + |BETWEEN unbounded preceding AND current row) AS s1, + |p_size + |FROM part + |) sub1 + """.stripMargin, false) + createQueryTest("8.testMixedCaseAlias", + """ + |SELECT p_mfgr, p_name, p_size, + |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name, p_size desc) AS R + |FROM part + """.stripMargin, false) + + createQueryTest("9.testHavingWithWindowingNoGBY", + """ + |SELECT p_mfgr, p_name, p_size, + |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, + |sum(p_retailprice) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN + |unbounded preceding AND current row) AS s1 + |FROM part + """.stripMargin, false) + + createQueryTest("11.testFirstLast", + """ + |SELECT p_mfgr,p_name, p_size, + |sum(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN + |current row AND current row) AS s2, + |first_value(p_size) OVER w1 AS f, + |last_value(p_size) OVER w1 AS l + |FROM part + |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN + |2 preceding AND 2 following) + """.stripMargin, false) + + createQueryTest("12.testFirstLastWithWhere", + """ + |SELECT p_mfgr,p_name, p_size, + |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, + |sum(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN + |current row AND current row) AS s2, + |first_value(p_size) OVER w1 AS f, + |last_value(p_size) OVER w1 AS l + |FROM part + |where p_mfgr = 'Manufacturer#3' + |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN 2 preceding AND 2 following) + """.stripMargin, false) + + createQueryTest("13.testSumWindow", + """ + |SELECT p_mfgr,p_name, p_size, + |sum(p_size) OVER w1 AS s1, + |sum(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN + |current row AND current row) AS s2 + |FROM part + |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN + |2 preceding AND 2 following) + """.stripMargin, false) + + createQueryTest("14.testNoSortClause", + """ + |SELECT p_mfgr,p_name, p_size, + |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS r + |FROM part + |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN 2 preceding AND 2 following) + """.stripMargin, false) + + createQueryTest("18.testUDAFs", + """ + |SELECT p_mfgr,p_name, p_size, + |sum(p_retailprice) OVER w1 AS s, + |min(p_retailprice) OVER w1 AS mi, + |max(p_retailprice) OVER w1 AS ma, + |avg(p_retailprice) OVER w1 AS ag + |FROM part + |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_mfgr, p_name rows BETWEEN + |2 preceding AND 2 following) + """.stripMargin, false) + + createQueryTest("20.testSTATs", + """ + |SELECT p_mfgr,p_name, p_size, + |stddev(p_retailprice) OVER w1 AS sdev, + |stddev_pop(p_retailprice) OVER w1 AS sdev_pop, + |collect_set(p_size) OVER w1 AS uniq_size, + |variance(p_retailprice) OVER w1 AS var, + |corr(p_size, p_retailprice) OVER w1 AS cor, + |covar_pop(p_size, p_retailprice) OVER w1 AS covarp + |FROM part + |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_mfgr, p_name rows BETWEEN + |2 preceding AND 2 following) + """.stripMargin, false) + + createQueryTest("27.testMultipleRangeWindows", + """ + |SELECT p_mfgr,p_name, p_size, + |sum(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_size range BETWEEN + |10 preceding AND current row) AS s2, + |sum(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_size range BETWEEN + |current row AND 10 following ) AS s1 + |FROM part + |WINDOW w1 AS (rows BETWEEN 2 preceding AND 2 following) + """.stripMargin, false) + + createQueryTest("28.testPartOrderInUDAFInvoke", + """ + |SELECT p_mfgr, p_name, p_size, + |sum(p_size) OVER (PARTITION BY p_mfgr ORDER BY p_name rows BETWEEN + |2 preceding AND 2 following) AS s + |FROM part + """.stripMargin, false) + + createQueryTest("29.testPartOrderInWdwDef", + """ + |SELECT p_mfgr, p_name, p_size, + |sum(p_size) OVER w1 AS s + |FROM part + |WINDOW w1 AS (PARTITION BY p_mfgr ORDER BY p_name rows BETWEEN + |2 preceding AND 2 following) + """.stripMargin, false) + + createQueryTest("31.testWindowCrossReference", + """ + |SELECT p_mfgr, p_name, p_size, + |sum(p_size) OVER w1 AS s1, + |sum(p_size) OVER w2 AS s2 + |FROM part + |WINDOW w1 AS (PARTITION BY p_mfgr ORDER BY p_size range BETWEEN + |2 preceding AND 2 following), + |w2 AS w1 + """.stripMargin, false) + + createQueryTest("32.testWindowInheritance", + """ + |SELECT p_mfgr, p_name, p_size, + |sum(p_size) OVER w1 AS s1, + |sum(p_size) OVER w2 AS s2 + |FROM part + |WINDOW w1 AS (PARTITION BY p_mfgr ORDER BY p_size range BETWEEN + |2 preceding AND 2 following), + |w2 AS (w1 rows BETWEEN unbounded preceding AND current row) + """.stripMargin, false) + + createQueryTest("33.testWindowForwardReference", + """ + |SELECT p_mfgr, p_name, p_size, + |sum(p_size) OVER w1 AS s1, + |sum(p_size) OVER w2 AS s2, + |sum(p_size) OVER w3 AS s3 + |FROM part + |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_size range BETWEEN + |2 preceding AND 2 following), + |w2 AS w3, + |w3 AS (DISTRIBUTE BY p_mfgr SORT BY p_size range BETWEEN + |unbounded preceding AND current row) + """.stripMargin, false) + + createQueryTest("34.testWindowDefinitionPropagation", + """ + |SELECT p_mfgr, p_name, p_size, + |sum(p_size) OVER w1 AS s1, + |sum(p_size) OVER w2 AS s2, + |sum(p_size) OVER (w3 rows BETWEEN 2 preceding AND 2 following) AS s3 + |FROM part + |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_size range BETWEEN + |2 preceding AND 2 following), + |w2 AS w3, + |w3 AS (DISTRIBUTE BY p_mfgr SORT BY p_size range BETWEEN + |unbounded preceding AND current row) + """.stripMargin, false) + + createQueryTest("35.testDistinctWithWindowing", + """ + |SELECT DISTINCT p_mfgr, p_name, p_size, + |sum(p_size) OVER w1 AS s + |FROM part + |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN + |2 preceding AND 2 following) + """.stripMargin, false) + + createQueryTest("39.testUDFOnOrderCols", + """ + |SELECT p_mfgr, p_type, substr(p_type, 2) AS short_ptype, + |row_number() OVER (PARTITION BY p_mfgr ORDER BY substr(p_type, 2)) AS r + |FROM part + """.stripMargin, false) + + createQueryTest("40.testNoBetweenForRows", + """ + |SELECT p_mfgr, p_name, p_size, + |sum(p_retailprice) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows unbounded preceding) AS s1 + |FROM part + """.stripMargin, false) + + createQueryTest("41.testNoBetweenForRange", + """ + |SELECT p_mfgr, p_name, p_size, + |sum(p_retailprice) OVER (DISTRIBUTE BY p_mfgr SORT BY p_size range + |unbounded preceding) AS s1 + |FROM part + """.stripMargin, false) + + createQueryTest("42.testUnboundedFollowingForRows", + """ + |SELECT p_mfgr, p_name, p_size, + |sum(p_retailprice) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN + |current row AND unbounded following) AS s1 + |FROM part + """.stripMargin, false) + + createQueryTest("43.testUnboundedFollowingForRange", + """ + |SELECT p_mfgr, p_name, p_size, + |sum(p_retailprice) OVER (DISTRIBUTE BY p_mfgr SORT BY p_size range BETWEEN + |current row AND unbounded following) AS s1 + |FROM part + """.stripMargin, false) + + createQueryTest("44.testOverNoPartitionSingleAggregate", + """ + |SELECT p_name, p_retailprice, + |round(avg(p_retailprice) over(),2) + |FROM part + |ORDER BY p_name + """.stripMargin, false) + + createQueryTest("ntile", + """ + |SELECT p_name, ntile(4) OVER (PARTITION BY p_mfgr ORDER BY p_name) FROM part + """.stripMargin, false) + + +} From 7d7a703d5e7bf37e00d074cb8c04e2150f8fbeb4 Mon Sep 17 00:00:00 2001 From: guowei2 Date: Mon, 27 Oct 2014 13:29:35 +0800 Subject: [PATCH 02/14] window function --- .../files/part_tiny_without_duplicate.txt | 25 +++++++ ...ndowing-0-53287db3a565fc9e7a033adeaff23c0c | 25 +++++++ ...rstLast-0-774e42ff5ad9597bc58944c4eb07c389 | 25 +++++++ ...WithWhere-0-5368b010d938ca259761d4f8f594cb | 5 ++ ...mWindow-0-489dd1089dc50ea9560d5460b4941297 | 25 +++++++ ...tClause-0-32d1c6bfa5c9183fe15986dfd41f06c2 | 25 +++++++ ...stUDAFs-0-ff7dac6fba4549f2ff47d273f673fab6 | 25 +++++++ ...stSTATs-0-f9aa53aff9299759472df468cf7928a3 | 25 +++++++ ...Windows-0-f7c1d7801add3d16f5bf93df71f37d14 | 25 +++++++ ...FInvoke-0-58b599b7a13c0a33dd296fe163dc4e03 | 25 +++++++ ...nWdwDef-0-8dd305dfce19e934673cd341257f075e | 25 +++++++ ...ference-0-722c0216c87e556dc01b12ae452e61d5 | 25 +++++++ ...ritance-0-a4b587b733ed900335caf0b7fa4403ba | 25 +++++++ ...ference-0-a2789ff17b97ceb61d6f4575fc97c68b | 25 +++++++ ...agation-0-a46db570ae1d07579e1839a1fb5a68b6 | 25 +++++++ ...ndowing-0-7bd1c75ea286bf4db585db105b536cbd | 25 +++++++ ...derCols-0-8806b022acbc0a7cf586040a98e27efe | 25 +++++++ ...estCount-0-6ccae5fcb441ad90f14dff22de1aafc | 25 +++++++ ...ForRows-0-4eb434ad595e07bbfec506d5c11853e2 | 25 +++++++ ...orRange-0-ad44c5f2217c26d0b203ab36d4d7faaa | 25 +++++++ ...ForRows-0-6bb333833337b1ffe48f2b8277b64289 | 25 +++++++ ...orRange-0-add3531568fe3d3d8172f897ee2f2018 | 25 +++++++ ...gregate-0-2fb3d43169e2f20680e7477f0daaddf4 | 25 +++++++ ...ingUDAF-0-f22b5eae7b0255f15c480527036fd2b0 | 25 +++++++ ...tInSubQ-0-2e3e151f6ddc88c11149c72d884ba4bb | 25 +++++++ ...seAlias-0-1458fdd44340b7cf31a4c3980e53e643 | 25 +++++++ ...ngNoGBY-0-6b117eafdd9731cee1007ae9719888a1 | 25 +++++++ .../ntile-0-76735502d1f68740622f29da68b5a2e5 | 25 +++++++ .../execution/HiveWindowFunctionSuite.scala | 73 +++++++++---------- 29 files changed, 715 insertions(+), 38 deletions(-) create mode 100755 sql/hive/src/test/resources/data/files/part_tiny_without_duplicate.txt create mode 100644 sql/hive/src/test/resources/golden/1.testWindowing-0-53287db3a565fc9e7a033adeaff23c0c create mode 100644 sql/hive/src/test/resources/golden/11.testFirstLast-0-774e42ff5ad9597bc58944c4eb07c389 create mode 100644 sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-5368b010d938ca259761d4f8f594cb create mode 100644 sql/hive/src/test/resources/golden/13.testSumWindow-0-489dd1089dc50ea9560d5460b4941297 create mode 100644 sql/hive/src/test/resources/golden/14.testNoSortClause-0-32d1c6bfa5c9183fe15986dfd41f06c2 create mode 100644 sql/hive/src/test/resources/golden/18.testUDAFs-0-ff7dac6fba4549f2ff47d273f673fab6 create mode 100644 sql/hive/src/test/resources/golden/20.testSTATs-0-f9aa53aff9299759472df468cf7928a3 create mode 100644 sql/hive/src/test/resources/golden/27.testMultipleRangeWindows-0-f7c1d7801add3d16f5bf93df71f37d14 create mode 100644 sql/hive/src/test/resources/golden/28.testPartOrderInUDAFInvoke-0-58b599b7a13c0a33dd296fe163dc4e03 create mode 100644 sql/hive/src/test/resources/golden/29.testPartOrderInWdwDef-0-8dd305dfce19e934673cd341257f075e create mode 100644 sql/hive/src/test/resources/golden/31.testWindowCrossReference-0-722c0216c87e556dc01b12ae452e61d5 create mode 100644 sql/hive/src/test/resources/golden/32.testWindowInheritance-0-a4b587b733ed900335caf0b7fa4403ba create mode 100644 sql/hive/src/test/resources/golden/33.testWindowForwardReference-0-a2789ff17b97ceb61d6f4575fc97c68b create mode 100644 sql/hive/src/test/resources/golden/34.testWindowDefinitionPropagation-0-a46db570ae1d07579e1839a1fb5a68b6 create mode 100644 sql/hive/src/test/resources/golden/35.testDistinctWithWindowing-0-7bd1c75ea286bf4db585db105b536cbd create mode 100644 sql/hive/src/test/resources/golden/39.testUDFOnOrderCols-0-8806b022acbc0a7cf586040a98e27efe create mode 100644 sql/hive/src/test/resources/golden/4.testCount-0-6ccae5fcb441ad90f14dff22de1aafc create mode 100644 sql/hive/src/test/resources/golden/40.testNoBetweenForRows-0-4eb434ad595e07bbfec506d5c11853e2 create mode 100644 sql/hive/src/test/resources/golden/41.testNoBetweenForRange-0-ad44c5f2217c26d0b203ab36d4d7faaa create mode 100644 sql/hive/src/test/resources/golden/42.testUnboundedFollowingForRows-0-6bb333833337b1ffe48f2b8277b64289 create mode 100644 sql/hive/src/test/resources/golden/43.testUnboundedFollowingForRange-0-add3531568fe3d3d8172f897ee2f2018 create mode 100644 sql/hive/src/test/resources/golden/44.testOverNoPartitionSingleAggregate-0-2fb3d43169e2f20680e7477f0daaddf4 create mode 100644 sql/hive/src/test/resources/golden/5.testCountWithWindowingUDAF-0-f22b5eae7b0255f15c480527036fd2b0 create mode 100644 sql/hive/src/test/resources/golden/6.testCountInSubQ-0-2e3e151f6ddc88c11149c72d884ba4bb create mode 100644 sql/hive/src/test/resources/golden/8.testMixedCaseAlias-0-1458fdd44340b7cf31a4c3980e53e643 create mode 100644 sql/hive/src/test/resources/golden/9.testHavingWithWindowingNoGBY-0-6b117eafdd9731cee1007ae9719888a1 create mode 100644 sql/hive/src/test/resources/golden/ntile-0-76735502d1f68740622f29da68b5a2e5 diff --git a/sql/hive/src/test/resources/data/files/part_tiny_without_duplicate.txt b/sql/hive/src/test/resources/data/files/part_tiny_without_duplicate.txt new file mode 100755 index 0000000000000..23a2c94a87ecb --- /dev/null +++ b/sql/hive/src/test/resources/data/files/part_tiny_without_duplicate.txt @@ -0,0 +1,25 @@ +121152almond antique burnished rose metallicManufacturer#1Brand#14PROMO PLATED TIN2JUMBO BOX1173.15e pinto beans h +85768almond antique chartreuse lavender yellowManufacturer#1Brand#12LARGE BRUSHED STEEL34SM BAG1753.76refull +110592almond antique salmon chartreuse burlywoodManufacturer#1Brand#15PROMO BURNISHED NICKEL6JUMBO PKG1602.59 to the furiously +86428almond aquamarine burnished black steelManufacturer#1Brand#12STANDARD ANODIZED STEEL28WRAP BAG1414.42arefully +65667almond aquamarine pink moccasin thistleManufacturer#1Brand#12LARGE BURNISHED STEEL42JUMBO CASE1632.66e across the expr +105685almond antique violet chocolate turquoiseManufacturer#2Brand#22MEDIUM ANODIZED COPPER14MED CAN1690.68ly pending requ +191709almond antique violet turquoise frostedManufacturer#2Brand#22ECONOMY POLISHED STEEL40MED BOX1800.7 haggle +146985almond aquamarine midnight light salmonManufacturer#2Brand#23MEDIUM BURNISHED COPPER2SM CASE2031.98s cajole caref +132666almond aquamarine rose maroon antiqueManufacturer#2Brand#24SMALL POLISHED NICKEL25MED BOX1698.66even +195606almond aquamarine sandy cyan gainsboroManufacturer#2Brand#25STANDARD PLATED TIN18SM PKG1701.6ic de +90681almond antique chartreuse khaki whiteManufacturer#3Brand#31MEDIUM BURNISHED TIN17SM CASE1671.68are slyly after the sl +17273almond antique forest lavender goldenrodManufacturer#3Brand#35PROMO ANODIZED TIN14JUMBO CASE1190.27along the +112398almond antique metallic orange dimManufacturer#3Brand#32MEDIUM BURNISHED BRASS19JUMBO JAR1410.39ole car +40982almond antique misty red oliveManufacturer#3Brand#32ECONOMY PLATED COPPER1LG PKG1922.98c foxes can s +144293almond antique olive coral navajoManufacturer#3Brand#34STANDARD POLISHED STEEL45JUMBO CAN1337.29ag furiously about +49671almond antique gainsboro frosted violetManufacturer#4Brand#41SMALL BRUSHED BRASS10SM BOX1620.67ccounts run quick +48427almond antique violet mint lemonManufacturer#4Brand#42PROMO POLISHED STEEL39SM CASE1375.42hely ironic i +45261almond aquamarine floral ivory bisqueManufacturer#4Brand#42SMALL PLATED STEEL27WRAP CASE1206.26careful +17927almond aquamarine yellow dodger mintManufacturer#4Brand#41ECONOMY BRUSHED COPPER7SM PKG1844.92ites. eve +33357almond azure aquamarine papaya violetManufacturer#4Brand#41STANDARD ANODIZED TIN12WRAP CASE1290.35reful +192697almond antique blue firebrick mintManufacturer#5Brand#52MEDIUM BURNISHED TIN31LG DRUM1789.69ickly ir +42669almond antique medium spring khakiManufacturer#5Brand#51STANDARD BURNISHED TIN6MED CAN1611.66sits haggl +155733almond antique sky peru orangeManufacturer#5Brand#53SMALL PLATED BRASS2WRAP DRUM1788.73furiously. bra +15103almond aquamarine dodger light gainsboroManufacturer#5Brand#53ECONOMY BURNISHED STEEL46LG PACK1018.1packages hinder carefu +78486almond azure blanched chiffon midnightManufacturer#5Brand#52LARGE BRUSHED BRASS23MED BAG1464.48hely blith \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1.testWindowing-0-53287db3a565fc9e7a033adeaff23c0c b/sql/hive/src/test/resources/golden/1.testWindowing-0-53287db3a565fc9e7a033adeaff23c0c new file mode 100644 index 0000000000000..eef1d21739118 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1.testWindowing-0-53287db3a565fc9e7a033adeaff23c0c @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1173.15 +Manufacturer#1 almond antique chartreuse lavender yellow 34 2 2926.91 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 3 4529.5 +Manufacturer#1 almond aquamarine burnished black steel 28 4 5943.92 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 5 7576.58 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/11.testFirstLast-0-774e42ff5ad9597bc58944c4eb07c389 b/sql/hive/src/test/resources/golden/11.testFirstLast-0-774e42ff5ad9597bc58944c4eb07c389 new file mode 100644 index 0000000000000..67cc662325159 --- /dev/null +++ b/sql/hive/src/test/resources/golden/11.testFirstLast-0-774e42ff5ad9597bc58944c4eb07c389 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 2 6 +Manufacturer#1 almond antique chartreuse lavender yellow 34 34 2 28 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6 2 42 +Manufacturer#1 almond aquamarine burnished black steel 28 28 34 42 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 42 6 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 14 2 +Manufacturer#2 almond antique violet turquoise frosted 40 40 14 25 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 14 18 +Manufacturer#2 almond aquamarine rose maroon antique 25 25 40 18 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 18 2 18 +Manufacturer#3 almond antique chartreuse khaki white 17 17 17 19 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 17 1 +Manufacturer#3 almond antique metallic orange dim 19 19 17 45 +Manufacturer#3 almond antique misty red olive 1 1 14 45 +Manufacturer#3 almond antique olive coral navajo 45 45 19 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 10 10 27 +Manufacturer#4 almond antique violet mint lemon 39 39 10 7 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 10 12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 39 12 +Manufacturer#4 almond azure aquamarine papaya violet 12 12 27 12 +Manufacturer#5 almond antique blue firebrick mint 31 31 31 2 +Manufacturer#5 almond antique medium spring khaki 6 6 31 46 +Manufacturer#5 almond antique sky peru orange 2 2 31 23 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 6 23 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 2 23 diff --git a/sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-5368b010d938ca259761d4f8f594cb b/sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-5368b010d938ca259761d4f8f594cb new file mode 100644 index 0000000000000..09e30c7c57349 --- /dev/null +++ b/sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-5368b010d938ca259761d4f8f594cb @@ -0,0 +1,5 @@ +Manufacturer#3 almond antique chartreuse khaki white 17 1 17 17 19 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 14 17 1 +Manufacturer#3 almond antique metallic orange dim 19 3 19 17 45 +Manufacturer#3 almond antique misty red olive 1 4 1 14 45 +Manufacturer#3 almond antique olive coral navajo 45 5 45 19 45 diff --git a/sql/hive/src/test/resources/golden/13.testSumWindow-0-489dd1089dc50ea9560d5460b4941297 b/sql/hive/src/test/resources/golden/13.testSumWindow-0-489dd1089dc50ea9560d5460b4941297 new file mode 100644 index 0000000000000..4b62745f0ca86 --- /dev/null +++ b/sql/hive/src/test/resources/golden/13.testSumWindow-0-489dd1089dc50ea9560d5460b4941297 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 42 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 70 34 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 6 +Manufacturer#1 almond aquamarine burnished black steel 28 110 28 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 14 +Manufacturer#2 almond antique violet turquoise frosted 40 81 40 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 2 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 25 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 18 +Manufacturer#3 almond antique chartreuse khaki white 17 50 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 14 +Manufacturer#3 almond antique metallic orange dim 19 96 19 +Manufacturer#3 almond antique misty red olive 1 79 1 +Manufacturer#3 almond antique olive coral navajo 45 65 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 10 +Manufacturer#4 almond antique violet mint lemon 39 83 39 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 27 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 7 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 12 +Manufacturer#5 almond antique blue firebrick mint 31 39 31 +Manufacturer#5 almond antique medium spring khaki 6 85 6 +Manufacturer#5 almond antique sky peru orange 2 108 2 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 46 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 23 diff --git a/sql/hive/src/test/resources/golden/14.testNoSortClause-0-32d1c6bfa5c9183fe15986dfd41f06c2 b/sql/hive/src/test/resources/golden/14.testNoSortClause-0-32d1c6bfa5c9183fe15986dfd41f06c2 new file mode 100644 index 0000000000000..dcaab34edab14 --- /dev/null +++ b/sql/hive/src/test/resources/golden/14.testNoSortClause-0-32d1c6bfa5c9183fe15986dfd41f06c2 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 3 +Manufacturer#1 almond aquamarine burnished black steel 28 4 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 5 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 +Manufacturer#3 almond antique metallic orange dim 19 3 +Manufacturer#3 almond antique misty red olive 1 4 +Manufacturer#3 almond antique olive coral navajo 45 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 +Manufacturer#4 almond antique violet mint lemon 39 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 +Manufacturer#5 almond antique medium spring khaki 6 2 +Manufacturer#5 almond antique sky peru orange 2 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 diff --git a/sql/hive/src/test/resources/golden/18.testUDAFs-0-ff7dac6fba4549f2ff47d273f673fab6 b/sql/hive/src/test/resources/golden/18.testUDAFs-0-ff7dac6fba4549f2ff47d273f673fab6 new file mode 100644 index 0000000000000..d25b6d8cc0942 --- /dev/null +++ b/sql/hive/src/test/resources/golden/18.testUDAFs-0-ff7dac6fba4549f2ff47d273f673fab6 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 4529.5 1173.15 1753.76 1509.8333333333333 +Manufacturer#1 almond antique chartreuse lavender yellow 34 5943.92 1173.15 1753.76 1485.98 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 7576.58 1173.15 1753.76 1515.316 +Manufacturer#1 almond aquamarine burnished black steel 28 6403.43 1414.42 1753.76 1600.8575 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 4649.67 1414.42 1632.66 1549.89 +Manufacturer#2 almond antique violet chocolate turquoise 14 5523.360000000001 1690.68 2031.98 1841.1200000000001 +Manufacturer#2 almond antique violet turquoise frosted 40 7222.02 1690.68 2031.98 1805.505 +Manufacturer#2 almond aquamarine midnight light salmon 2 8923.62 1690.68 2031.98 1784.7240000000002 +Manufacturer#2 almond aquamarine rose maroon antique 25 7232.9400000000005 1698.66 2031.98 1808.2350000000001 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5432.24 1698.66 2031.98 1810.7466666666667 +Manufacturer#3 almond antique chartreuse khaki white 17 4272.34 1190.27 1671.68 1424.1133333333335 +Manufacturer#3 almond antique forest lavender goldenrod 14 6195.32 1190.27 1922.98 1548.83 +Manufacturer#3 almond antique metallic orange dim 19 7532.61 1190.27 1922.98 1506.522 +Manufacturer#3 almond antique misty red olive 1 5860.929999999999 1190.27 1922.98 1465.2324999999998 +Manufacturer#3 almond antique olive coral navajo 45 4670.66 1337.29 1922.98 1556.8866666666665 +Manufacturer#4 almond antique gainsboro frosted violet 10 4202.35 1206.26 1620.67 1400.7833333333335 +Manufacturer#4 almond antique violet mint lemon 39 6047.27 1206.26 1844.92 1511.8175 +Manufacturer#4 almond aquamarine floral ivory bisque 27 7337.620000000001 1206.26 1844.92 1467.5240000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 5716.950000000001 1206.26 1844.92 1429.2375000000002 +Manufacturer#4 almond azure aquamarine papaya violet 12 4341.530000000001 1206.26 1844.92 1447.176666666667 +Manufacturer#5 almond antique blue firebrick mint 31 5190.08 1611.66 1789.69 1730.0266666666666 +Manufacturer#5 almond antique medium spring khaki 6 6208.18 1018.1 1789.69 1552.045 +Manufacturer#5 almond antique sky peru orange 2 7672.66 1018.1 1789.69 1534.532 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 5882.970000000001 1018.1 1788.73 1470.7425000000003 +Manufacturer#5 almond azure blanched chiffon midnight 23 4271.3099999999995 1018.1 1788.73 1423.7699999999998 diff --git a/sql/hive/src/test/resources/golden/20.testSTATs-0-f9aa53aff9299759472df468cf7928a3 b/sql/hive/src/test/resources/golden/20.testSTATs-0-f9aa53aff9299759472df468cf7928a3 new file mode 100644 index 0000000000000..185cd4d550f15 --- /dev/null +++ b/sql/hive/src/test/resources/golden/20.testSTATs-0-f9aa53aff9299759472df468cf7928a3 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 245.94015862038924 245.94015862038924 [34,2,6] 60486.56162222222 0.7784579099243658 2725.560000000001 +Manufacturer#1 almond antique chartreuse lavender yellow 34 216.96051656925965 216.96051656925965 [34,2,6,28] 47071.86575 0.6017665496586917 1793.7100000000007 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 202.73109328368946 202.73109328368946 [34,2,6,42,28] 41099.896184 0.630785977101214 2009.9536000000007 +Manufacturer#1 almond aquamarine burnished black steel 28 121.6064517973862 121.6064517973862 [34,6,42,28] 14788.129118750014 0.2036684720435979 331.1337500000004 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 96.5751586416853 96.5751586416853 [6,42,28] 9326.761266666683 -1.4442181184933883E-4 -0.20666666666708502 +Manufacturer#2 almond antique violet chocolate turquoise 14 142.2363169751898 142.2363169751898 [2,40,14] 20231.169866666663 -0.49369526554523185 -1113.7466666666658 +Manufacturer#2 almond antique violet turquoise frosted 40 137.76306498840682 137.76306498840682 [2,25,40,14] 18978.662075 -0.5205630897335946 -1004.4812499999995 +Manufacturer#2 almond aquamarine midnight light salmon 2 130.03972279269132 130.03972279269132 [2,18,25,40,14] 16910.329504000005 -0.46908967495720255 -766.1791999999995 +Manufacturer#2 almond aquamarine rose maroon antique 25 135.55100986344584 135.55100986344584 [2,18,25,40] 18374.07627499999 -0.6091405874714462 -1128.1787499999987 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 156.44019460768044 156.44019460768044 [2,18,25] 24473.534488888927 -0.9571686373491608 -1441.4466666666676 +Manufacturer#3 almond antique chartreuse khaki white 17 196.7742266885805 196.7742266885805 [17,19,14] 38720.09628888887 0.5557168646224995 224.6944444444446 +Manufacturer#3 almond antique forest lavender goldenrod 14 275.14144189852607 275.14144189852607 [17,1,19,14] 75702.81305 -0.6720833036576083 -1296.9000000000003 +Manufacturer#3 almond antique metallic orange dim 19 260.23473614412046 260.23473614412046 [17,1,19,14,45] 67722.117896 -0.5703526513979519 -2129.0664 +Manufacturer#3 almond antique misty red olive 1 275.9139962356932 275.9139962356932 [1,19,14,45] 76128.53331875012 -0.577476899644802 -2547.7868749999993 +Manufacturer#3 almond antique olive coral navajo 45 260.5815918713796 260.5815918713796 [1,19,45] 67902.76602222225 -0.8710736366736884 -4099.731111111111 +Manufacturer#4 almond antique gainsboro frosted violet 10 170.13011889596618 170.13011889596618 [39,27,10] 28944.25735555559 -0.6656975320098423 -1347.4777777777779 +Manufacturer#4 almond antique violet mint lemon 39 242.26834609323197 242.26834609323197 [39,7,27,10] 58693.95151875002 -0.8051852719193339 -2537.328125 +Manufacturer#4 almond aquamarine floral ivory bisque 27 234.10001662537326 234.10001662537326 [39,7,27,10,12] 54802.817784000035 -0.6046935574240581 -1719.8079999999995 +Manufacturer#4 almond aquamarine yellow dodger mint 7 247.3342714197732 247.3342714197732 [39,7,27,12] 61174.24181875003 -0.5508665654707869 -1719.0368749999975 +Manufacturer#4 almond azure aquamarine papaya violet 12 283.3344330566893 283.3344330566893 [7,27,12] 80278.40095555557 -0.7755740084632333 -1867.4888888888881 +Manufacturer#5 almond antique blue firebrick mint 31 83.69879024746363 83.69879024746363 [2,6,31] 7005.487488888913 0.39004303087285047 418.9233333333353 +Manufacturer#5 almond antique medium spring khaki 6 316.68049612345885 316.68049612345885 [2,6,46,31] 100286.53662500004 -0.713612911776183 -4090.853749999999 +Manufacturer#5 almond antique sky peru orange 2 285.40506298242155 285.40506298242155 [2,23,6,46,31] 81456.04997600002 -0.712858514567818 -3297.2011999999986 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 285.43749038756283 285.43749038756283 [2,23,6,46] 81474.56091875004 -0.984128787153391 -4871.028125000002 +Manufacturer#5 almond azure blanched chiffon midnight 23 315.9225931564038 315.9225931564038 [2,23,46] 99807.08486666664 -0.9978877469246936 -5664.856666666666 diff --git a/sql/hive/src/test/resources/golden/27.testMultipleRangeWindows-0-f7c1d7801add3d16f5bf93df71f37d14 b/sql/hive/src/test/resources/golden/27.testMultipleRangeWindows-0-f7c1d7801add3d16f5bf93df71f37d14 new file mode 100644 index 0000000000000..cdb4a9b337503 --- /dev/null +++ b/sql/hive/src/test/resources/golden/27.testMultipleRangeWindows-0-f7c1d7801add3d16f5bf93df71f37d14 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 8 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 8 6 +Manufacturer#1 almond aquamarine burnished black steel 28 28 62 +Manufacturer#1 almond antique chartreuse lavender yellow 34 62 76 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 42 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 2 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 32 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 32 43 +Manufacturer#2 almond aquamarine rose maroon antique 25 43 25 +Manufacturer#2 almond antique violet turquoise frosted 40 40 40 +Manufacturer#3 almond antique misty red olive 1 1 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 50 +Manufacturer#3 almond antique chartreuse khaki white 17 31 36 +Manufacturer#3 almond antique metallic orange dim 19 50 19 +Manufacturer#3 almond antique olive coral navajo 45 45 45 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 29 +Manufacturer#4 almond antique gainsboro frosted violet 10 17 22 +Manufacturer#4 almond azure aquamarine papaya violet 12 29 12 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 27 +Manufacturer#4 almond antique violet mint lemon 39 39 39 +Manufacturer#5 almond antique sky peru orange 2 2 8 +Manufacturer#5 almond antique medium spring khaki 6 8 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 54 +Manufacturer#5 almond antique blue firebrick mint 31 54 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 46 diff --git a/sql/hive/src/test/resources/golden/28.testPartOrderInUDAFInvoke-0-58b599b7a13c0a33dd296fe163dc4e03 b/sql/hive/src/test/resources/golden/28.testPartOrderInUDAFInvoke-0-58b599b7a13c0a33dd296fe163dc4e03 new file mode 100644 index 0000000000000..f92eb135afe03 --- /dev/null +++ b/sql/hive/src/test/resources/golden/28.testPartOrderInUDAFInvoke-0-58b599b7a13c0a33dd296fe163dc4e03 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 42 +Manufacturer#1 almond antique chartreuse lavender yellow 34 70 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 +Manufacturer#1 almond aquamarine burnished black steel 28 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 +Manufacturer#2 almond antique violet turquoise frosted 40 81 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 +Manufacturer#3 almond antique chartreuse khaki white 17 50 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 +Manufacturer#3 almond antique metallic orange dim 19 96 +Manufacturer#3 almond antique misty red olive 1 79 +Manufacturer#3 almond antique olive coral navajo 45 65 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 +Manufacturer#4 almond antique violet mint lemon 39 83 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 +Manufacturer#5 almond antique blue firebrick mint 31 39 +Manufacturer#5 almond antique medium spring khaki 6 85 +Manufacturer#5 almond antique sky peru orange 2 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 diff --git a/sql/hive/src/test/resources/golden/29.testPartOrderInWdwDef-0-8dd305dfce19e934673cd341257f075e b/sql/hive/src/test/resources/golden/29.testPartOrderInWdwDef-0-8dd305dfce19e934673cd341257f075e new file mode 100644 index 0000000000000..f92eb135afe03 --- /dev/null +++ b/sql/hive/src/test/resources/golden/29.testPartOrderInWdwDef-0-8dd305dfce19e934673cd341257f075e @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 42 +Manufacturer#1 almond antique chartreuse lavender yellow 34 70 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 +Manufacturer#1 almond aquamarine burnished black steel 28 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 +Manufacturer#2 almond antique violet turquoise frosted 40 81 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 +Manufacturer#3 almond antique chartreuse khaki white 17 50 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 +Manufacturer#3 almond antique metallic orange dim 19 96 +Manufacturer#3 almond antique misty red olive 1 79 +Manufacturer#3 almond antique olive coral navajo 45 65 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 +Manufacturer#4 almond antique violet mint lemon 39 83 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 +Manufacturer#5 almond antique blue firebrick mint 31 39 +Manufacturer#5 almond antique medium spring khaki 6 85 +Manufacturer#5 almond antique sky peru orange 2 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 diff --git a/sql/hive/src/test/resources/golden/31.testWindowCrossReference-0-722c0216c87e556dc01b12ae452e61d5 b/sql/hive/src/test/resources/golden/31.testWindowCrossReference-0-722c0216c87e556dc01b12ae452e61d5 new file mode 100644 index 0000000000000..ca76646e2fd16 --- /dev/null +++ b/sql/hive/src/test/resources/golden/31.testWindowCrossReference-0-722c0216c87e556dc01b12ae452e61d5 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6 6 +Manufacturer#1 almond aquamarine burnished black steel 28 28 28 +Manufacturer#1 almond antique chartreuse lavender yellow 34 34 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 42 42 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 2 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 14 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 18 18 +Manufacturer#2 almond aquamarine rose maroon antique 25 25 25 +Manufacturer#2 almond antique violet turquoise frosted 40 40 40 +Manufacturer#3 almond antique misty red olive 1 1 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 14 +Manufacturer#3 almond antique chartreuse khaki white 17 36 36 +Manufacturer#3 almond antique metallic orange dim 19 36 36 +Manufacturer#3 almond antique olive coral navajo 45 45 45 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 7 +Manufacturer#4 almond antique gainsboro frosted violet 10 22 22 +Manufacturer#4 almond azure aquamarine papaya violet 12 22 22 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 27 +Manufacturer#4 almond antique violet mint lemon 39 39 39 +Manufacturer#5 almond antique sky peru orange 2 2 2 +Manufacturer#5 almond antique medium spring khaki 6 6 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 23 +Manufacturer#5 almond antique blue firebrick mint 31 31 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 46 diff --git a/sql/hive/src/test/resources/golden/32.testWindowInheritance-0-a4b587b733ed900335caf0b7fa4403ba b/sql/hive/src/test/resources/golden/32.testWindowInheritance-0-a4b587b733ed900335caf0b7fa4403ba new file mode 100644 index 0000000000000..b4f3c162e109a --- /dev/null +++ b/sql/hive/src/test/resources/golden/32.testWindowInheritance-0-a4b587b733ed900335caf0b7fa4403ba @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6 8 +Manufacturer#1 almond aquamarine burnished black steel 28 28 36 +Manufacturer#1 almond antique chartreuse lavender yellow 34 34 70 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 42 112 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 2 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 16 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 18 34 +Manufacturer#2 almond aquamarine rose maroon antique 25 25 59 +Manufacturer#2 almond antique violet turquoise frosted 40 40 99 +Manufacturer#3 almond antique misty red olive 1 1 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 15 +Manufacturer#3 almond antique chartreuse khaki white 17 36 32 +Manufacturer#3 almond antique metallic orange dim 19 36 51 +Manufacturer#3 almond antique olive coral navajo 45 45 96 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 7 +Manufacturer#4 almond antique gainsboro frosted violet 10 22 17 +Manufacturer#4 almond azure aquamarine papaya violet 12 22 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 56 +Manufacturer#4 almond antique violet mint lemon 39 39 95 +Manufacturer#5 almond antique sky peru orange 2 2 2 +Manufacturer#5 almond antique medium spring khaki 6 6 8 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 31 +Manufacturer#5 almond antique blue firebrick mint 31 31 62 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 108 diff --git a/sql/hive/src/test/resources/golden/33.testWindowForwardReference-0-a2789ff17b97ceb61d6f4575fc97c68b b/sql/hive/src/test/resources/golden/33.testWindowForwardReference-0-a2789ff17b97ceb61d6f4575fc97c68b new file mode 100644 index 0000000000000..b5cc706d1af26 --- /dev/null +++ b/sql/hive/src/test/resources/golden/33.testWindowForwardReference-0-a2789ff17b97ceb61d6f4575fc97c68b @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 2 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6 8 8 +Manufacturer#1 almond aquamarine burnished black steel 28 28 36 36 +Manufacturer#1 almond antique chartreuse lavender yellow 34 34 70 70 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 42 112 112 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 2 2 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 16 16 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 18 34 34 +Manufacturer#2 almond aquamarine rose maroon antique 25 25 59 59 +Manufacturer#2 almond antique violet turquoise frosted 40 40 99 99 +Manufacturer#3 almond antique misty red olive 1 1 1 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 15 15 +Manufacturer#3 almond antique chartreuse khaki white 17 36 32 32 +Manufacturer#3 almond antique metallic orange dim 19 36 51 51 +Manufacturer#3 almond antique olive coral navajo 45 45 96 96 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 7 7 +Manufacturer#4 almond antique gainsboro frosted violet 10 22 17 17 +Manufacturer#4 almond azure aquamarine papaya violet 12 22 29 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 56 56 +Manufacturer#4 almond antique violet mint lemon 39 39 95 95 +Manufacturer#5 almond antique sky peru orange 2 2 2 2 +Manufacturer#5 almond antique medium spring khaki 6 6 8 8 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 31 31 +Manufacturer#5 almond antique blue firebrick mint 31 31 62 62 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 108 108 diff --git a/sql/hive/src/test/resources/golden/34.testWindowDefinitionPropagation-0-a46db570ae1d07579e1839a1fb5a68b6 b/sql/hive/src/test/resources/golden/34.testWindowDefinitionPropagation-0-a46db570ae1d07579e1839a1fb5a68b6 new file mode 100644 index 0000000000000..00faf5bf02612 --- /dev/null +++ b/sql/hive/src/test/resources/golden/34.testWindowDefinitionPropagation-0-a46db570ae1d07579e1839a1fb5a68b6 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 2 36 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6 8 70 +Manufacturer#1 almond aquamarine burnished black steel 28 28 36 112 +Manufacturer#1 almond antique chartreuse lavender yellow 34 34 70 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 42 112 104 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 2 34 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 16 59 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 18 34 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 25 59 97 +Manufacturer#2 almond antique violet turquoise frosted 40 40 99 83 +Manufacturer#3 almond antique misty red olive 1 1 1 32 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 15 51 +Manufacturer#3 almond antique chartreuse khaki white 17 36 32 96 +Manufacturer#3 almond antique metallic orange dim 19 36 51 95 +Manufacturer#3 almond antique olive coral navajo 45 45 96 81 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 7 29 +Manufacturer#4 almond antique gainsboro frosted violet 10 22 17 56 +Manufacturer#4 almond azure aquamarine papaya violet 12 22 29 95 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 56 88 +Manufacturer#4 almond antique violet mint lemon 39 39 95 78 +Manufacturer#5 almond antique sky peru orange 2 2 2 31 +Manufacturer#5 almond antique medium spring khaki 6 6 8 62 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 31 108 +Manufacturer#5 almond antique blue firebrick mint 31 31 62 106 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 108 100 diff --git a/sql/hive/src/test/resources/golden/35.testDistinctWithWindowing-0-7bd1c75ea286bf4db585db105b536cbd b/sql/hive/src/test/resources/golden/35.testDistinctWithWindowing-0-7bd1c75ea286bf4db585db105b536cbd new file mode 100644 index 0000000000000..f92eb135afe03 --- /dev/null +++ b/sql/hive/src/test/resources/golden/35.testDistinctWithWindowing-0-7bd1c75ea286bf4db585db105b536cbd @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 42 +Manufacturer#1 almond antique chartreuse lavender yellow 34 70 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 +Manufacturer#1 almond aquamarine burnished black steel 28 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 +Manufacturer#2 almond antique violet turquoise frosted 40 81 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 +Manufacturer#3 almond antique chartreuse khaki white 17 50 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 +Manufacturer#3 almond antique metallic orange dim 19 96 +Manufacturer#3 almond antique misty red olive 1 79 +Manufacturer#3 almond antique olive coral navajo 45 65 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 +Manufacturer#4 almond antique violet mint lemon 39 83 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 +Manufacturer#5 almond antique blue firebrick mint 31 39 +Manufacturer#5 almond antique medium spring khaki 6 85 +Manufacturer#5 almond antique sky peru orange 2 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 diff --git a/sql/hive/src/test/resources/golden/39.testUDFOnOrderCols-0-8806b022acbc0a7cf586040a98e27efe b/sql/hive/src/test/resources/golden/39.testUDFOnOrderCols-0-8806b022acbc0a7cf586040a98e27efe new file mode 100644 index 0000000000000..48adb0d3c7c0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/39.testUDFOnOrderCols-0-8806b022acbc0a7cf586040a98e27efe @@ -0,0 +1,25 @@ +Manufacturer#1 LARGE BRUSHED STEEL ARGE BRUSHED STEEL 1 +Manufacturer#1 LARGE BURNISHED STEEL ARGE BURNISHED STEEL 2 +Manufacturer#1 PROMO BURNISHED NICKEL ROMO BURNISHED NICKEL 3 +Manufacturer#1 PROMO PLATED TIN ROMO PLATED TIN 4 +Manufacturer#1 STANDARD ANODIZED STEEL TANDARD ANODIZED STEEL 5 +Manufacturer#2 ECONOMY POLISHED STEEL CONOMY POLISHED STEEL 1 +Manufacturer#2 MEDIUM ANODIZED COPPER EDIUM ANODIZED COPPER 2 +Manufacturer#2 MEDIUM BURNISHED COPPER EDIUM BURNISHED COPPER 3 +Manufacturer#2 SMALL POLISHED NICKEL MALL POLISHED NICKEL 4 +Manufacturer#2 STANDARD PLATED TIN TANDARD PLATED TIN 5 +Manufacturer#3 ECONOMY PLATED COPPER CONOMY PLATED COPPER 1 +Manufacturer#3 MEDIUM BURNISHED BRASS EDIUM BURNISHED BRASS 2 +Manufacturer#3 MEDIUM BURNISHED TIN EDIUM BURNISHED TIN 3 +Manufacturer#3 PROMO ANODIZED TIN ROMO ANODIZED TIN 4 +Manufacturer#3 STANDARD POLISHED STEEL TANDARD POLISHED STEEL 5 +Manufacturer#4 ECONOMY BRUSHED COPPER CONOMY BRUSHED COPPER 1 +Manufacturer#4 SMALL BRUSHED BRASS MALL BRUSHED BRASS 2 +Manufacturer#4 SMALL PLATED STEEL MALL PLATED STEEL 3 +Manufacturer#4 PROMO POLISHED STEEL ROMO POLISHED STEEL 4 +Manufacturer#4 STANDARD ANODIZED TIN TANDARD ANODIZED TIN 5 +Manufacturer#5 LARGE BRUSHED BRASS ARGE BRUSHED BRASS 1 +Manufacturer#5 ECONOMY BURNISHED STEEL CONOMY BURNISHED STEEL 2 +Manufacturer#5 MEDIUM BURNISHED TIN EDIUM BURNISHED TIN 3 +Manufacturer#5 SMALL PLATED BRASS MALL PLATED BRASS 4 +Manufacturer#5 STANDARD BURNISHED TIN TANDARD BURNISHED TIN 5 diff --git a/sql/hive/src/test/resources/golden/4.testCount-0-6ccae5fcb441ad90f14dff22de1aafc b/sql/hive/src/test/resources/golden/4.testCount-0-6ccae5fcb441ad90f14dff22de1aafc new file mode 100644 index 0000000000000..d1a447cf9cd29 --- /dev/null +++ b/sql/hive/src/test/resources/golden/4.testCount-0-6ccae5fcb441ad90f14dff22de1aafc @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 1 +Manufacturer#1 almond antique chartreuse lavender yellow 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 3 +Manufacturer#1 almond aquamarine burnished black steel 4 +Manufacturer#1 almond aquamarine pink moccasin thistle 5 +Manufacturer#2 almond antique violet chocolate turquoise 1 +Manufacturer#2 almond antique violet turquoise frosted 2 +Manufacturer#2 almond aquamarine midnight light salmon 3 +Manufacturer#2 almond aquamarine rose maroon antique 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 +Manufacturer#3 almond antique chartreuse khaki white 1 +Manufacturer#3 almond antique forest lavender goldenrod 2 +Manufacturer#3 almond antique metallic orange dim 3 +Manufacturer#3 almond antique misty red olive 4 +Manufacturer#3 almond antique olive coral navajo 5 +Manufacturer#4 almond antique gainsboro frosted violet 1 +Manufacturer#4 almond antique violet mint lemon 2 +Manufacturer#4 almond aquamarine floral ivory bisque 3 +Manufacturer#4 almond aquamarine yellow dodger mint 4 +Manufacturer#4 almond azure aquamarine papaya violet 5 +Manufacturer#5 almond antique blue firebrick mint 1 +Manufacturer#5 almond antique medium spring khaki 2 +Manufacturer#5 almond antique sky peru orange 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 +Manufacturer#5 almond azure blanched chiffon midnight 5 diff --git a/sql/hive/src/test/resources/golden/40.testNoBetweenForRows-0-4eb434ad595e07bbfec506d5c11853e2 b/sql/hive/src/test/resources/golden/40.testNoBetweenForRows-0-4eb434ad595e07bbfec506d5c11853e2 new file mode 100644 index 0000000000000..84ca4de289e2c --- /dev/null +++ b/sql/hive/src/test/resources/golden/40.testNoBetweenForRows-0-4eb434ad595e07bbfec506d5c11853e2 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 +Manufacturer#1 almond antique chartreuse lavender yellow 34 2926.91 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4529.5 +Manufacturer#1 almond aquamarine burnished black steel 28 5943.92 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 7576.58 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 4272.34 +Manufacturer#3 almond antique misty red olive 1 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 7672.66 diff --git a/sql/hive/src/test/resources/golden/41.testNoBetweenForRange-0-ad44c5f2217c26d0b203ab36d4d7faaa b/sql/hive/src/test/resources/golden/41.testNoBetweenForRange-0-ad44c5f2217c26d0b203ab36d4d7faaa new file mode 100644 index 0000000000000..f8a04f09a84a4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/41.testNoBetweenForRange-0-ad44c5f2217c26d0b203ab36d4d7faaa @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 2775.74 +Manufacturer#1 almond aquamarine burnished black steel 28 4190.16 +Manufacturer#1 almond antique chartreuse lavender yellow 34 5943.92 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 7576.58 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 +Manufacturer#2 almond antique violet chocolate turquoise 14 3722.66 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5424.26 +Manufacturer#2 almond aquamarine rose maroon antique 25 7122.92 +Manufacturer#2 almond antique violet turquoise frosted 40 8923.62 +Manufacturer#3 almond antique misty red olive 1 1922.98 +Manufacturer#3 almond antique forest lavender goldenrod 14 3113.25 +Manufacturer#3 almond antique chartreuse khaki white 17 4784.93 +Manufacturer#3 almond antique metallic orange dim 19 6195.320000000001 +Manufacturer#3 almond antique olive coral navajo 45 7532.610000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 +Manufacturer#4 almond antique gainsboro frosted violet 10 3465.59 +Manufacturer#4 almond azure aquamarine papaya violet 12 4755.9400000000005 +Manufacturer#4 almond aquamarine floral ivory bisque 27 5962.200000000001 +Manufacturer#4 almond antique violet mint lemon 39 7337.620000000001 +Manufacturer#5 almond antique sky peru orange 2 1788.73 +Manufacturer#5 almond antique medium spring khaki 6 3400.3900000000003 +Manufacturer#5 almond azure blanched chiffon midnight 23 4864.870000000001 +Manufacturer#5 almond antique blue firebrick mint 31 6654.560000000001 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 7672.660000000002 diff --git a/sql/hive/src/test/resources/golden/42.testUnboundedFollowingForRows-0-6bb333833337b1ffe48f2b8277b64289 b/sql/hive/src/test/resources/golden/42.testUnboundedFollowingForRows-0-6bb333833337b1ffe48f2b8277b64289 new file mode 100644 index 0000000000000..8a18af7acc80d --- /dev/null +++ b/sql/hive/src/test/resources/golden/42.testUnboundedFollowingForRows-0-6bb333833337b1ffe48f2b8277b64289 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 7576.58 +Manufacturer#1 almond antique chartreuse lavender yellow 34 6403.43 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4649.67 +Manufacturer#1 almond aquamarine burnished black steel 28 3047.08 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 +Manufacturer#2 almond antique violet chocolate turquoise 14 8923.62 +Manufacturer#2 almond antique violet turquoise frosted 40 7232.9400000000005 +Manufacturer#2 almond aquamarine midnight light salmon 2 5432.24 +Manufacturer#2 almond aquamarine rose maroon antique 25 3400.26 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 +Manufacturer#3 almond antique chartreuse khaki white 17 7532.61 +Manufacturer#3 almond antique forest lavender goldenrod 14 5860.929999999999 +Manufacturer#3 almond antique metallic orange dim 19 4670.66 +Manufacturer#3 almond antique misty red olive 1 3260.27 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 +Manufacturer#4 almond antique gainsboro frosted violet 10 7337.620000000001 +Manufacturer#4 almond antique violet mint lemon 39 5716.950000000001 +Manufacturer#4 almond aquamarine floral ivory bisque 27 4341.530000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 3135.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 +Manufacturer#5 almond antique blue firebrick mint 31 7672.66 +Manufacturer#5 almond antique medium spring khaki 6 5882.970000000001 +Manufacturer#5 almond antique sky peru orange 2 4271.3099999999995 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 2482.58 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 diff --git a/sql/hive/src/test/resources/golden/43.testUnboundedFollowingForRange-0-add3531568fe3d3d8172f897ee2f2018 b/sql/hive/src/test/resources/golden/43.testUnboundedFollowingForRange-0-add3531568fe3d3d8172f897ee2f2018 new file mode 100644 index 0000000000000..3d2c0e9ace5f3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/43.testUnboundedFollowingForRange-0-add3531568fe3d3d8172f897ee2f2018 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 7576.58 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6403.43 +Manufacturer#1 almond aquamarine burnished black steel 28 4800.84 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3386.42 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 +Manufacturer#2 almond aquamarine midnight light salmon 2 8923.62 +Manufacturer#2 almond antique violet chocolate turquoise 14 6891.639999999999 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5200.96 +Manufacturer#2 almond aquamarine rose maroon antique 25 3499.36 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 +Manufacturer#3 almond antique misty red olive 1 7532.610000000001 +Manufacturer#3 almond antique forest lavender goldenrod 14 5609.63 +Manufacturer#3 almond antique chartreuse khaki white 17 4419.360000000001 +Manufacturer#3 almond antique metallic orange dim 19 2747.6800000000003 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7337.620000000001 +Manufacturer#4 almond antique gainsboro frosted violet 10 5492.7 +Manufacturer#4 almond azure aquamarine papaya violet 12 3872.0299999999997 +Manufacturer#4 almond aquamarine floral ivory bisque 27 2581.6800000000003 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 +Manufacturer#5 almond antique sky peru orange 2 7672.660000000002 +Manufacturer#5 almond antique medium spring khaki 6 5883.93 +Manufacturer#5 almond azure blanched chiffon midnight 23 4272.27 +Manufacturer#5 almond antique blue firebrick mint 31 2807.79 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 diff --git a/sql/hive/src/test/resources/golden/44.testOverNoPartitionSingleAggregate-0-2fb3d43169e2f20680e7477f0daaddf4 b/sql/hive/src/test/resources/golden/44.testOverNoPartitionSingleAggregate-0-2fb3d43169e2f20680e7477f0daaddf4 new file mode 100644 index 0000000000000..f4d20506d88cd --- /dev/null +++ b/sql/hive/src/test/resources/golden/44.testOverNoPartitionSingleAggregate-0-2fb3d43169e2f20680e7477f0daaddf4 @@ -0,0 +1,25 @@ +almond antique blue firebrick mint 1789.69 1561.72 +almond antique burnished rose metallic 1173.15 1561.72 +almond antique chartreuse khaki white 1671.68 1561.72 +almond antique chartreuse lavender yellow 1753.76 1561.72 +almond antique forest lavender goldenrod 1190.27 1561.72 +almond antique gainsboro frosted violet 1620.67 1561.72 +almond antique medium spring khaki 1611.66 1561.72 +almond antique metallic orange dim 1410.39 1561.72 +almond antique misty red olive 1922.98 1561.72 +almond antique olive coral navajo 1337.29 1561.72 +almond antique salmon chartreuse burlywood 1602.59 1561.72 +almond antique sky peru orange 1788.73 1561.72 +almond antique violet chocolate turquoise 1690.68 1561.72 +almond antique violet mint lemon 1375.42 1561.72 +almond antique violet turquoise frosted 1800.7 1561.72 +almond aquamarine burnished black steel 1414.42 1561.72 +almond aquamarine dodger light gainsboro 1018.1 1561.72 +almond aquamarine floral ivory bisque 1206.26 1561.72 +almond aquamarine midnight light salmon 2031.98 1561.72 +almond aquamarine pink moccasin thistle 1632.66 1561.72 +almond aquamarine rose maroon antique 1698.66 1561.72 +almond aquamarine sandy cyan gainsboro 1701.6 1561.72 +almond aquamarine yellow dodger mint 1844.92 1561.72 +almond azure aquamarine papaya violet 1290.35 1561.72 +almond azure blanched chiffon midnight 1464.48 1561.72 diff --git a/sql/hive/src/test/resources/golden/5.testCountWithWindowingUDAF-0-f22b5eae7b0255f15c480527036fd2b0 b/sql/hive/src/test/resources/golden/5.testCountWithWindowingUDAF-0-f22b5eae7b0255f15c480527036fd2b0 new file mode 100644 index 0000000000000..88bdc39bbd160 --- /dev/null +++ b/sql/hive/src/test/resources/golden/5.testCountWithWindowingUDAF-0-f22b5eae7b0255f15c480527036fd2b0 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 1 1 1173.15 1173.15 2 +Manufacturer#1 almond antique chartreuse lavender yellow 2 2 1753.76 2926.91 34 +Manufacturer#1 almond antique salmon chartreuse burlywood 3 3 1602.59 4529.5 6 +Manufacturer#1 almond aquamarine burnished black steel 4 4 1414.42 5943.92 28 +Manufacturer#1 almond aquamarine pink moccasin thistle 5 5 1632.66 7576.58 42 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 1690.68 1690.68 14 +Manufacturer#2 almond antique violet turquoise frosted 2 2 1800.7 3491.38 40 +Manufacturer#2 almond aquamarine midnight light salmon 3 3 2031.98 5523.360000000001 2 +Manufacturer#2 almond aquamarine rose maroon antique 4 4 1698.66 7222.02 25 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 5 1701.6 8923.62 18 +Manufacturer#3 almond antique chartreuse khaki white 1 1 1671.68 1671.68 17 +Manufacturer#3 almond antique forest lavender goldenrod 2 2 1190.27 2861.95 14 +Manufacturer#3 almond antique metallic orange dim 3 3 1410.39 4272.34 19 +Manufacturer#3 almond antique misty red olive 4 4 1922.98 6195.32 1 +Manufacturer#3 almond antique olive coral navajo 5 5 1337.29 7532.61 45 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 1620.67 1620.67 10 +Manufacturer#4 almond antique violet mint lemon 2 2 1375.42 2996.09 39 +Manufacturer#4 almond aquamarine floral ivory bisque 3 3 1206.26 4202.35 27 +Manufacturer#4 almond aquamarine yellow dodger mint 4 4 1844.92 6047.27 7 +Manufacturer#4 almond azure aquamarine papaya violet 5 5 1290.35 7337.620000000001 12 +Manufacturer#5 almond antique blue firebrick mint 1 1 1789.69 1789.69 31 +Manufacturer#5 almond antique medium spring khaki 2 2 1611.66 3401.3500000000004 6 +Manufacturer#5 almond antique sky peru orange 3 3 1788.73 5190.08 2 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 4 1018.1 6208.18 46 +Manufacturer#5 almond azure blanched chiffon midnight 5 5 1464.48 7672.66 23 diff --git a/sql/hive/src/test/resources/golden/6.testCountInSubQ-0-2e3e151f6ddc88c11149c72d884ba4bb b/sql/hive/src/test/resources/golden/6.testCountInSubQ-0-2e3e151f6ddc88c11149c72d884ba4bb new file mode 100644 index 0000000000000..43e401977fd9f --- /dev/null +++ b/sql/hive/src/test/resources/golden/6.testCountInSubQ-0-2e3e151f6ddc88c11149c72d884ba4bb @@ -0,0 +1,25 @@ +1 1 1173.15 +2 2 2926.91 +3 3 4529.5 +4 4 5943.92 +5 5 7576.58 +1 1 1690.68 +2 2 3491.38 +3 3 5523.360000000001 +4 4 7222.02 +5 5 8923.62 +1 1 1671.68 +2 2 2861.95 +3 3 4272.34 +4 4 6195.32 +5 5 7532.61 +1 1 1620.67 +2 2 2996.09 +3 3 4202.35 +4 4 6047.27 +5 5 7337.620000000001 +1 1 1789.69 +2 2 3401.3500000000004 +3 3 5190.08 +4 4 6208.18 +5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/8.testMixedCaseAlias-0-1458fdd44340b7cf31a4c3980e53e643 b/sql/hive/src/test/resources/golden/8.testMixedCaseAlias-0-1458fdd44340b7cf31a4c3980e53e643 new file mode 100644 index 0000000000000..dcaab34edab14 --- /dev/null +++ b/sql/hive/src/test/resources/golden/8.testMixedCaseAlias-0-1458fdd44340b7cf31a4c3980e53e643 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 3 +Manufacturer#1 almond aquamarine burnished black steel 28 4 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 5 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 +Manufacturer#3 almond antique metallic orange dim 19 3 +Manufacturer#3 almond antique misty red olive 1 4 +Manufacturer#3 almond antique olive coral navajo 45 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 +Manufacturer#4 almond antique violet mint lemon 39 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 +Manufacturer#5 almond antique medium spring khaki 6 2 +Manufacturer#5 almond antique sky peru orange 2 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 diff --git a/sql/hive/src/test/resources/golden/9.testHavingWithWindowingNoGBY-0-6b117eafdd9731cee1007ae9719888a1 b/sql/hive/src/test/resources/golden/9.testHavingWithWindowingNoGBY-0-6b117eafdd9731cee1007ae9719888a1 new file mode 100644 index 0000000000000..eef1d21739118 --- /dev/null +++ b/sql/hive/src/test/resources/golden/9.testHavingWithWindowingNoGBY-0-6b117eafdd9731cee1007ae9719888a1 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1173.15 +Manufacturer#1 almond antique chartreuse lavender yellow 34 2 2926.91 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 3 4529.5 +Manufacturer#1 almond aquamarine burnished black steel 28 4 5943.92 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 5 7576.58 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/ntile-0-76735502d1f68740622f29da68b5a2e5 b/sql/hive/src/test/resources/golden/ntile-0-76735502d1f68740622f29da68b5a2e5 new file mode 100644 index 0000000000000..5cbeceff48b9a --- /dev/null +++ b/sql/hive/src/test/resources/golden/ntile-0-76735502d1f68740622f29da68b5a2e5 @@ -0,0 +1,25 @@ +almond antique burnished rose metallic 1 +almond antique salmon chartreuse burlywood 1 +almond aquamarine burnished black steel 2 +almond antique chartreuse lavender yellow 3 +almond aquamarine pink moccasin thistle 4 +almond aquamarine midnight light salmon 1 +almond antique violet chocolate turquoise 1 +almond aquamarine sandy cyan gainsboro 2 +almond aquamarine rose maroon antique 3 +almond antique violet turquoise frosted 4 +almond antique misty red olive 1 +almond antique forest lavender goldenrod 1 +almond antique chartreuse khaki white 2 +almond antique metallic orange dim 3 +almond antique olive coral navajo 4 +almond aquamarine yellow dodger mint 1 +almond antique gainsboro frosted violet 1 +almond azure aquamarine papaya violet 2 +almond aquamarine floral ivory bisque 3 +almond antique violet mint lemon 4 +almond antique sky peru orange 1 +almond antique medium spring khaki 1 +almond azure blanched chiffon midnight 2 +almond antique blue firebrick mint 3 +almond aquamarine dodger light gainsboro 4 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala index 04e2073fc2f71..0a44eaf6226e5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala @@ -1,10 +1,10 @@ /* * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 + * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -17,32 +17,31 @@ package org.apache.spark.sql.hive.execution +import org.apache.spark.sql.hive._ +import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.{Row, SchemaRDD} - -class HiveWindowFunctionSuite extends HiveComparisonTest { +class HiveWindowFunctionSuit extends HiveComparisonTest { override def beforeAll() { sql("DROP TABLE IF EXISTS part").collect() sql(""" - |CREATE TABLE part( - | p_partkey INT, - | p_name STRING, - | p_mfgr STRING, - | p_brand STRING, - | p_type STRING, - | p_size INT, - | p_container STRING, - | p_retailprice DOUBLE, - | p_comment STRING - |) - """.stripMargin).collect() - - sql(""" - |LOAD DATA LOCAL INPATH '../data/files/part_tiny_without_duplicate.txt' - |OVERWRITE INTO TABLE part - """.stripMargin).collect() + |CREATE TABLE part( + | p_partkey INT, + | p_name STRING, + | p_mfgr STRING, + | p_brand STRING, + | p_type STRING, + | p_size INT, + | p_container STRING, + | p_retailprice DOUBLE, + | p_comment STRING + |) + """.stripMargin).collect() + + sql("LOAD DATA LOCAL INPATH '../data/files/part_tiny_without_duplicate.txt' OVERWRITE INTO TABLE part").collect() } createQueryTest("1.testWindowing", @@ -108,7 +107,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |first_value(p_size) OVER w1 AS f, |last_value(p_size) OVER w1 AS l |FROM part - |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN + |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN |2 preceding AND 2 following) """.stripMargin, false) @@ -122,7 +121,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |last_value(p_size) OVER w1 AS l |FROM part |where p_mfgr = 'Manufacturer#3' - |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN 2 preceding AND 2 following) + |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN 2 preceding AND 2 following) """.stripMargin, false) createQueryTest("13.testSumWindow", @@ -132,7 +131,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |sum(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN |current row AND current row) AS s2 |FROM part - |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN + |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN |2 preceding AND 2 following) """.stripMargin, false) @@ -141,7 +140,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |SELECT p_mfgr,p_name, p_size, |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS r |FROM part - |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN 2 preceding AND 2 following) + |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN 2 preceding AND 2 following) """.stripMargin, false) createQueryTest("18.testUDAFs", @@ -152,7 +151,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |max(p_retailprice) OVER w1 AS ma, |avg(p_retailprice) OVER w1 AS ag |FROM part - |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_mfgr, p_name rows BETWEEN + |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_mfgr, p_name rows BETWEEN |2 preceding AND 2 following) """.stripMargin, false) @@ -166,7 +165,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |corr(p_size, p_retailprice) OVER w1 AS cor, |covar_pop(p_size, p_retailprice) OVER w1 AS covarp |FROM part - |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_mfgr, p_name rows BETWEEN + |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_mfgr, p_name rows BETWEEN |2 preceding AND 2 following) """.stripMargin, false) @@ -178,7 +177,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |sum(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_size range BETWEEN |current row AND 10 following ) AS s1 |FROM part - |WINDOW w1 AS (rows BETWEEN 2 preceding AND 2 following) + |window w1 AS (rows BETWEEN 2 preceding AND 2 following) """.stripMargin, false) createQueryTest("28.testPartOrderInUDAFInvoke", @@ -194,7 +193,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |SELECT p_mfgr, p_name, p_size, |sum(p_size) OVER w1 AS s |FROM part - |WINDOW w1 AS (PARTITION BY p_mfgr ORDER BY p_name rows BETWEEN + |window w1 AS (PARTITION BY p_mfgr ORDER BY p_name rows BETWEEN |2 preceding AND 2 following) """.stripMargin, false) @@ -204,9 +203,8 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |sum(p_size) OVER w1 AS s1, |sum(p_size) OVER w2 AS s2 |FROM part - |WINDOW w1 AS (PARTITION BY p_mfgr ORDER BY p_size range BETWEEN - |2 preceding AND 2 following), - |w2 AS w1 + |window w1 AS (PARTITION BY p_mfgr ORDER BY p_size range BETWEEN 2 preceding AND 2 following), + | w2 AS w1 """.stripMargin, false) createQueryTest("32.testWindowInheritance", @@ -215,9 +213,8 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |sum(p_size) OVER w1 AS s1, |sum(p_size) OVER w2 AS s2 |FROM part - |WINDOW w1 AS (PARTITION BY p_mfgr ORDER BY p_size range BETWEEN - |2 preceding AND 2 following), - |w2 AS (w1 rows BETWEEN unbounded preceding AND current row) + |window w1 AS (PARTITION BY p_mfgr ORDER BY p_size range BETWEEN 2 preceding AND 2 following), + | w2 AS (w1 rows BETWEEN unbounded preceding AND current row) """.stripMargin, false) createQueryTest("33.testWindowForwardReference", @@ -227,7 +224,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |sum(p_size) OVER w2 AS s2, |sum(p_size) OVER w3 AS s3 |FROM part - |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_size range BETWEEN + |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_size range BETWEEN |2 preceding AND 2 following), |w2 AS w3, |w3 AS (DISTRIBUTE BY p_mfgr SORT BY p_size range BETWEEN @@ -241,7 +238,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |sum(p_size) OVER w2 AS s2, |sum(p_size) OVER (w3 rows BETWEEN 2 preceding AND 2 following) AS s3 |FROM part - |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_size range BETWEEN + |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_size range BETWEEN |2 preceding AND 2 following), |w2 AS w3, |w3 AS (DISTRIBUTE BY p_mfgr SORT BY p_size range BETWEEN @@ -253,7 +250,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |SELECT DISTINCT p_mfgr, p_name, p_size, |sum(p_size) OVER w1 AS s |FROM part - |WINDOW w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN + |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN |2 preceding AND 2 following) """.stripMargin, false) From 1999e07a23c18808738e4e3b14b64c1db108eda2 Mon Sep 17 00:00:00 2001 From: guowei2 Date: Mon, 27 Oct 2014 14:03:17 +0800 Subject: [PATCH 03/14] window function --- .../spark/sql/hive/execution/HiveWindowFunctionSuite.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala index 0a44eaf6226e5..bdcf31c043311 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala @@ -41,6 +41,8 @@ class HiveWindowFunctionSuit extends HiveComparisonTest { |) """.stripMargin).collect() + //remove duplicate data in part_tiny.txt for hive bug + // https://issues.apache.org/jira/browse/HIVE-8569 sql("LOAD DATA LOCAL INPATH '../data/files/part_tiny_without_duplicate.txt' OVERWRITE INTO TABLE part").collect() } @@ -302,7 +304,7 @@ class HiveWindowFunctionSuit extends HiveComparisonTest { createQueryTest("ntile", """ - |SELECT p_name, ntile(4) OVER (PARTITION BY p_mfgr ORDER BY p_name) FROM part + |SELECT p_name, ntile(4) OVER (PARTITION BY p_mfgr ORDER BY p_size) FROM part """.stripMargin, false) From 76bfd4b8b1137426b0dbcde5d56cefb0c98cfab5 Mon Sep 17 00:00:00 2001 From: guowei2 Date: Mon, 27 Oct 2014 22:16:22 +0800 Subject: [PATCH 04/14] window function --- .../apache/spark/sql/catalyst/expressions/aggregates.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 11136a3dcf18a..85382b5a7d7cd 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -34,10 +34,9 @@ abstract class AggregateExpression extends Expression with Serializable { */ def newInstance(): AggregateFunction - override def equals(other: Any): Boolean = { - if (super.equals(other)) { - this.windowRange == other.asInstanceOf[AggregateExpression].windowRange - } else false + override def canEqual(other: Any): Boolean = other match { + case that: AggregateExpression => this.windowRange == that.windowRange + case _ => false } /** From 88c5789d9f6989d0fedcbdd129de097152e2d8eb Mon Sep 17 00:00:00 2001 From: guowei2 Date: Tue, 28 Oct 2014 12:00:57 +0800 Subject: [PATCH 05/14] window function --- .../src/main/scala/org/apache/spark/sql/hive/HiveContext.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 6008e46fefa86..ee65f9d647957 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -412,6 +412,7 @@ object HiveContext { }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "NULL" case (d: Date, DateType) => new DateWritable(d).toString + case (o, ArrayType(typ, _)) => toHiveStructString(o, typ) case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") case (decimal: BigDecimal, DecimalType()) => // Hive strips trailing zeros so use its toString From 828199a48c619d03b4ec524dbdfe9c043baa5e14 Mon Sep 17 00:00:00 2001 From: guowei2 Date: Wed, 29 Oct 2014 15:49:01 +0800 Subject: [PATCH 06/14] fix problems after rebase --- .../spark/sql/catalyst/plans/logical/basicOperators.scala | 4 ---- .../spark/sql/hive/execution/HiveWindowFunctionSuite.scala | 7 +++++-- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 3bb7d23e5b739..58a38b45455ac 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -149,10 +149,6 @@ case class WindowFunction( child: LogicalPlan) extends UnaryNode { - def references = - AttributeSet( - partitionExpressions.flatMap(_.references) ++ functionExpressions.flatMap(_.references)) - override def output = functionExpressions.map(_.toAttribute) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala index bdcf31c043311..3deacfa0948ca 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.{Row, SchemaRDD} -class HiveWindowFunctionSuit extends HiveComparisonTest { +class HiveWindowFunctionSuite extends HiveComparisonTest { override def beforeAll() { sql("DROP TABLE IF EXISTS part").collect() @@ -43,7 +43,10 @@ class HiveWindowFunctionSuit extends HiveComparisonTest { //remove duplicate data in part_tiny.txt for hive bug // https://issues.apache.org/jira/browse/HIVE-8569 - sql("LOAD DATA LOCAL INPATH '../data/files/part_tiny_without_duplicate.txt' OVERWRITE INTO TABLE part").collect() + sql(s""" + |LOAD DATA LOCAL INPATH '${getHiveFile("data/files/part_tiny_without_duplicate.txt")}' + |OVERWRITE INTO TABLE part + """.stripMargin).collect() } createQueryTest("1.testWindowing", From 03bd77d5533f76484d7589e0296283b58f2d0688 Mon Sep 17 00:00:00 2001 From: guowei2 Date: Thu, 30 Oct 2014 18:12:42 +0800 Subject: [PATCH 07/14] change test suite and golden files --- .../apache/spark/sql/execution/WindowFunction.scala | 4 ++-- .../scala/org/apache/spark/sql/hive/HiveQl.scala | 12 ++++++------ ...testFirstLast-0-c12e5632668c2dccc3199e702b35ef4f} | 0 ...LastWithWhere-0-57f204d198c816534e61fd57b698a23f} | 0 ...testSumWindow-0-950eeb4542e08104e132cbd77a04b009} | 0 ...tNoSortClause-0-a7970bafdc1b1a2dba571d19136205bc} | 0 ... 18.testUDAFs-0-c25be3f2fe68ee897520a5f08444e2b4} | 0 ...> 20.testSTATs-0-dcb29e85268a6041b182839c5efa668} | 0 ...eRangeWindows-0-4ed0d472ff076b53b51f0054530875c2} | 0 ...OrderInWdwDef-0-c587d6c514e9c3831dbe65c21ad0919c} | 0 ...rossReference-0-8dd68cf98d1669c1a84d435e90eaa6a9} | 0 ...owInheritance-0-3e639abc3a4e79e1ac0c80f4b4395472} | 0 ...wardReference-0-a16b01f1d330f74046ae506342b33d36} | 0 ...onPropagation-0-cc8fb4a48938aa777681f77c650bc7f4} | 0 ...WithWindowing-0-39ad24885629d3deaf1fb52a0a2c7f77} | 0 .../sql/hive/execution/HiveWindowFunctionSuite.scala | 2 +- 16 files changed, 9 insertions(+), 9 deletions(-) rename sql/hive/src/test/resources/golden/{11.testFirstLast-0-774e42ff5ad9597bc58944c4eb07c389 => 11.testFirstLast-0-c12e5632668c2dccc3199e702b35ef4f} (100%) rename sql/hive/src/test/resources/golden/{12.testFirstLastWithWhere-0-5368b010d938ca259761d4f8f594cb => 12.testFirstLastWithWhere-0-57f204d198c816534e61fd57b698a23f} (100%) rename sql/hive/src/test/resources/golden/{13.testSumWindow-0-489dd1089dc50ea9560d5460b4941297 => 13.testSumWindow-0-950eeb4542e08104e132cbd77a04b009} (100%) rename sql/hive/src/test/resources/golden/{14.testNoSortClause-0-32d1c6bfa5c9183fe15986dfd41f06c2 => 14.testNoSortClause-0-a7970bafdc1b1a2dba571d19136205bc} (100%) rename sql/hive/src/test/resources/golden/{18.testUDAFs-0-ff7dac6fba4549f2ff47d273f673fab6 => 18.testUDAFs-0-c25be3f2fe68ee897520a5f08444e2b4} (100%) rename sql/hive/src/test/resources/golden/{20.testSTATs-0-f9aa53aff9299759472df468cf7928a3 => 20.testSTATs-0-dcb29e85268a6041b182839c5efa668} (100%) rename sql/hive/src/test/resources/golden/{27.testMultipleRangeWindows-0-f7c1d7801add3d16f5bf93df71f37d14 => 27.testMultipleRangeWindows-0-4ed0d472ff076b53b51f0054530875c2} (100%) rename sql/hive/src/test/resources/golden/{29.testPartOrderInWdwDef-0-8dd305dfce19e934673cd341257f075e => 29.testPartOrderInWdwDef-0-c587d6c514e9c3831dbe65c21ad0919c} (100%) rename sql/hive/src/test/resources/golden/{31.testWindowCrossReference-0-722c0216c87e556dc01b12ae452e61d5 => 31.testWindowCrossReference-0-8dd68cf98d1669c1a84d435e90eaa6a9} (100%) rename sql/hive/src/test/resources/golden/{32.testWindowInheritance-0-a4b587b733ed900335caf0b7fa4403ba => 32.testWindowInheritance-0-3e639abc3a4e79e1ac0c80f4b4395472} (100%) rename sql/hive/src/test/resources/golden/{33.testWindowForwardReference-0-a2789ff17b97ceb61d6f4575fc97c68b => 33.testWindowForwardReference-0-a16b01f1d330f74046ae506342b33d36} (100%) rename sql/hive/src/test/resources/golden/{34.testWindowDefinitionPropagation-0-a46db570ae1d07579e1839a1fb5a68b6 => 34.testWindowDefinitionPropagation-0-cc8fb4a48938aa777681f77c650bc7f4} (100%) rename sql/hive/src/test/resources/golden/{35.testDistinctWithWindowing-0-7bd1c75ea286bf4db585db105b536cbd => 35.testDistinctWithWindowing-0-39ad24885629d3deaf1fb52a0a2c7f77} (100%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala index 5f0ae0d3333a7..248fac40ee0e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala @@ -172,7 +172,7 @@ case class WindowFunction( } if (end > rows.size - 1) end = rows.size - 1 - //new aggregate function + // new aggregate function val aggr = base.newInstance() (start to end).foreach(i => aggr.update(rows(i))) @@ -244,7 +244,7 @@ case class WindowFunction( followingIndex += 1 } } - //new aggregate function + // new aggregate function val aggr = base.newInstance() (precedingIndex to followingIndex).foreach(i => aggr.update(rows(i))) rangeResults += aggr.eval(EmptyRow) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index e227a8459e7d8..24e6c395b0681 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -876,11 +876,11 @@ private[hive] object HiveQl { } // store the window def of current sql - //use thread id as key to avoid mistake when muti sqls parse at the same time + // use thread id as key to avoid mistake when muti sqls parse at the same time protected val windowDefMap = new ConcurrentHashMap[Long,Map[String, Seq[ASTNode]]]() // store the window spec of current sql - //use thread id as key to avoid mistake when muti sqls parse at the same time + // use thread id as key to avoid mistake when muti sqls parse at the same time protected val windowPartitionsMap = new ConcurrentHashMap[Long, ArrayBuffer[Node]]() protected def initWindow() = { @@ -897,7 +897,7 @@ private[hive] object HiveQl { winDefs += alias -> ws } } - case None => //do nothing + case None => // do nothing } windowDefMap.put(Thread.currentThread().getId, winDefs) @@ -950,7 +950,7 @@ private[hive] object HiveQl { case Some(partition) => { if (partitionUnits.isEmpty) partitionUnits += partition else { - //only add different window partitions + // only add different window partitions try { partition zip partitionUnits.head foreach { case (l,r) => l checkEquals r @@ -960,10 +960,10 @@ private[hive] object HiveQl { } } } - case None => //do nothing + case None => // do nothing } - //check whether all window partitions are same, we just support same window partition now + // check whether all window partitions are same, we just support same window partition now if (partitionUnits.size == 0 && getWindowPartitions.size > 0) { Some(Seq()) } else if (partitionUnits.size == 1) { diff --git a/sql/hive/src/test/resources/golden/11.testFirstLast-0-774e42ff5ad9597bc58944c4eb07c389 b/sql/hive/src/test/resources/golden/11.testFirstLast-0-c12e5632668c2dccc3199e702b35ef4f similarity index 100% rename from sql/hive/src/test/resources/golden/11.testFirstLast-0-774e42ff5ad9597bc58944c4eb07c389 rename to sql/hive/src/test/resources/golden/11.testFirstLast-0-c12e5632668c2dccc3199e702b35ef4f diff --git a/sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-5368b010d938ca259761d4f8f594cb b/sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-57f204d198c816534e61fd57b698a23f similarity index 100% rename from sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-5368b010d938ca259761d4f8f594cb rename to sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-57f204d198c816534e61fd57b698a23f diff --git a/sql/hive/src/test/resources/golden/13.testSumWindow-0-489dd1089dc50ea9560d5460b4941297 b/sql/hive/src/test/resources/golden/13.testSumWindow-0-950eeb4542e08104e132cbd77a04b009 similarity index 100% rename from sql/hive/src/test/resources/golden/13.testSumWindow-0-489dd1089dc50ea9560d5460b4941297 rename to sql/hive/src/test/resources/golden/13.testSumWindow-0-950eeb4542e08104e132cbd77a04b009 diff --git a/sql/hive/src/test/resources/golden/14.testNoSortClause-0-32d1c6bfa5c9183fe15986dfd41f06c2 b/sql/hive/src/test/resources/golden/14.testNoSortClause-0-a7970bafdc1b1a2dba571d19136205bc similarity index 100% rename from sql/hive/src/test/resources/golden/14.testNoSortClause-0-32d1c6bfa5c9183fe15986dfd41f06c2 rename to sql/hive/src/test/resources/golden/14.testNoSortClause-0-a7970bafdc1b1a2dba571d19136205bc diff --git a/sql/hive/src/test/resources/golden/18.testUDAFs-0-ff7dac6fba4549f2ff47d273f673fab6 b/sql/hive/src/test/resources/golden/18.testUDAFs-0-c25be3f2fe68ee897520a5f08444e2b4 similarity index 100% rename from sql/hive/src/test/resources/golden/18.testUDAFs-0-ff7dac6fba4549f2ff47d273f673fab6 rename to sql/hive/src/test/resources/golden/18.testUDAFs-0-c25be3f2fe68ee897520a5f08444e2b4 diff --git a/sql/hive/src/test/resources/golden/20.testSTATs-0-f9aa53aff9299759472df468cf7928a3 b/sql/hive/src/test/resources/golden/20.testSTATs-0-dcb29e85268a6041b182839c5efa668 similarity index 100% rename from sql/hive/src/test/resources/golden/20.testSTATs-0-f9aa53aff9299759472df468cf7928a3 rename to sql/hive/src/test/resources/golden/20.testSTATs-0-dcb29e85268a6041b182839c5efa668 diff --git a/sql/hive/src/test/resources/golden/27.testMultipleRangeWindows-0-f7c1d7801add3d16f5bf93df71f37d14 b/sql/hive/src/test/resources/golden/27.testMultipleRangeWindows-0-4ed0d472ff076b53b51f0054530875c2 similarity index 100% rename from sql/hive/src/test/resources/golden/27.testMultipleRangeWindows-0-f7c1d7801add3d16f5bf93df71f37d14 rename to sql/hive/src/test/resources/golden/27.testMultipleRangeWindows-0-4ed0d472ff076b53b51f0054530875c2 diff --git a/sql/hive/src/test/resources/golden/29.testPartOrderInWdwDef-0-8dd305dfce19e934673cd341257f075e b/sql/hive/src/test/resources/golden/29.testPartOrderInWdwDef-0-c587d6c514e9c3831dbe65c21ad0919c similarity index 100% rename from sql/hive/src/test/resources/golden/29.testPartOrderInWdwDef-0-8dd305dfce19e934673cd341257f075e rename to sql/hive/src/test/resources/golden/29.testPartOrderInWdwDef-0-c587d6c514e9c3831dbe65c21ad0919c diff --git a/sql/hive/src/test/resources/golden/31.testWindowCrossReference-0-722c0216c87e556dc01b12ae452e61d5 b/sql/hive/src/test/resources/golden/31.testWindowCrossReference-0-8dd68cf98d1669c1a84d435e90eaa6a9 similarity index 100% rename from sql/hive/src/test/resources/golden/31.testWindowCrossReference-0-722c0216c87e556dc01b12ae452e61d5 rename to sql/hive/src/test/resources/golden/31.testWindowCrossReference-0-8dd68cf98d1669c1a84d435e90eaa6a9 diff --git a/sql/hive/src/test/resources/golden/32.testWindowInheritance-0-a4b587b733ed900335caf0b7fa4403ba b/sql/hive/src/test/resources/golden/32.testWindowInheritance-0-3e639abc3a4e79e1ac0c80f4b4395472 similarity index 100% rename from sql/hive/src/test/resources/golden/32.testWindowInheritance-0-a4b587b733ed900335caf0b7fa4403ba rename to sql/hive/src/test/resources/golden/32.testWindowInheritance-0-3e639abc3a4e79e1ac0c80f4b4395472 diff --git a/sql/hive/src/test/resources/golden/33.testWindowForwardReference-0-a2789ff17b97ceb61d6f4575fc97c68b b/sql/hive/src/test/resources/golden/33.testWindowForwardReference-0-a16b01f1d330f74046ae506342b33d36 similarity index 100% rename from sql/hive/src/test/resources/golden/33.testWindowForwardReference-0-a2789ff17b97ceb61d6f4575fc97c68b rename to sql/hive/src/test/resources/golden/33.testWindowForwardReference-0-a16b01f1d330f74046ae506342b33d36 diff --git a/sql/hive/src/test/resources/golden/34.testWindowDefinitionPropagation-0-a46db570ae1d07579e1839a1fb5a68b6 b/sql/hive/src/test/resources/golden/34.testWindowDefinitionPropagation-0-cc8fb4a48938aa777681f77c650bc7f4 similarity index 100% rename from sql/hive/src/test/resources/golden/34.testWindowDefinitionPropagation-0-a46db570ae1d07579e1839a1fb5a68b6 rename to sql/hive/src/test/resources/golden/34.testWindowDefinitionPropagation-0-cc8fb4a48938aa777681f77c650bc7f4 diff --git a/sql/hive/src/test/resources/golden/35.testDistinctWithWindowing-0-7bd1c75ea286bf4db585db105b536cbd b/sql/hive/src/test/resources/golden/35.testDistinctWithWindowing-0-39ad24885629d3deaf1fb52a0a2c7f77 similarity index 100% rename from sql/hive/src/test/resources/golden/35.testDistinctWithWindowing-0-7bd1c75ea286bf4db585db105b536cbd rename to sql/hive/src/test/resources/golden/35.testDistinctWithWindowing-0-39ad24885629d3deaf1fb52a0a2c7f77 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala index 3deacfa0948ca..bda8ae73d04b8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala @@ -41,7 +41,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |) """.stripMargin).collect() - //remove duplicate data in part_tiny.txt for hive bug + // remove duplicate data in part_tiny.txt for hive bug // https://issues.apache.org/jira/browse/HIVE-8569 sql(s""" |LOAD DATA LOCAL INPATH '${getHiveFile("data/files/part_tiny_without_duplicate.txt")}' From d06baeba2dc859f860c8fd43c292275837b3e0e6 Mon Sep 17 00:00:00 2001 From: guowei2 Date: Wed, 5 Nov 2014 11:01:33 +0800 Subject: [PATCH 08/14] add constant objectinspector support for udafs, such as last_value(col, false) --- .../sql/catalyst/analysis/Analyzer.scala | 8 ++-- .../sql/catalyst/analysis/unresolved.scala | 2 +- .../sql/catalyst/expressions/aggregates.scala | 6 +-- .../spark/sql/execution/WindowFunction.scala | 46 +++++++++---------- .../org/apache/spark/sql/hive/HiveQl.scala | 30 ++++++------ .../org/apache/spark/sql/hive/hiveUdfs.scala | 8 +++- ...stLast-0-16766ca1836b01697b19591661fee2ef} | 0 ...thWhere-0-7073024e84e63b5e64ca0e054789fb5} | 0 .../execution/HiveWindowFunctionSuite.scala | 4 +- 9 files changed, 54 insertions(+), 50 deletions(-) rename sql/hive/src/test/resources/golden/{11.testFirstLast-0-c12e5632668c2dccc3199e702b35ef4f => 11.testFirstLast-0-16766ca1836b01697b19591661fee2ef} (100%) rename sql/hive/src/test/resources/golden/{12.testFirstLastWithWhere-0-57f204d198c816534e61fd57b698a23f => 12.testFirstLastWithWhere-0-7073024e84e63b5e64ca0e054789fb5} (100%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index f62387a4652d3..d1fc1d24c3c3c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -218,12 +218,12 @@ class Analyzer(catalog: Catalog, def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressions { - case u @ UnresolvedFunction(name, children, windowRange) if u.childrenResolved => { + case u @ UnresolvedFunction(name, children, windowFrame) if u.childrenResolved => { val function = registry.lookupFunction(name, children) - if (windowRange != null) function match { + if (windowFrame != null) function match { case agg: AggregateExpression => - function.asInstanceOf[AggregateExpression].windowRange = windowRange - case _ => logError(s"function " + name + " does not support window range") + function.asInstanceOf[AggregateExpression].windowFrame = windowFrame + case _ => logError(s"function " + name + " does not support window frame") } function } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 12cc6e5c07689..17010743bb6fa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -64,7 +64,7 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo } case class UnresolvedFunction(name: String, children: Seq[Expression], - windowRange: WindowRange = null) extends Expression { + windowFrame: WindowFrame = null) extends Expression { override def dataType = throw new UnresolvedException(this, "dataType") override def foldable = throw new UnresolvedException(this, "foldable") override def nullable = throw new UnresolvedException(this, "nullable") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 85382b5a7d7cd..04c75674fe851 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.collection.OpenHashSet abstract class AggregateExpression extends Expression with Serializable { self: Product => - var windowRange: WindowRange = null + var windowFrame: WindowFrame = null /** * Creates a new instance that can be used to compute this aggregate expression for a group * of input rows/ @@ -35,7 +35,7 @@ abstract class AggregateExpression extends Expression with Serializable { def newInstance(): AggregateFunction override def canEqual(other: Any): Boolean = other match { - case that: AggregateExpression => this.windowRange == that.windowRange + case that: AggregateExpression => this.windowFrame == that.windowFrame case _ => false } @@ -47,7 +47,7 @@ abstract class AggregateExpression extends Expression with Serializable { throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } -case class WindowRange(windowType:String, preceding: Int, following: Int) +case class WindowFrame(frameType:String, preceding: Int, following: Int) /** * Represents an aggregation that has been rewritten to be performed in two steps. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala index 248fac40ee0e7..e3e9cb9404fa2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala @@ -109,7 +109,7 @@ case class WindowFunction( var i = 0 while (i < computeExpressions.length) { val baseExpr = BindReferences.bindReference(computeExpressions(i), childOutput) - baseExpr.windowRange = computeExpressions(i).windowRange + baseExpr.windowFrame = computeExpressions(i).windowFrame buffer(i) = baseExpr.newInstance() i += 1 } @@ -123,7 +123,7 @@ case class WindowFunction( while (i < aggrFunctions.length) { val aggrFunction = aggrFunctions(i) val base = aggrFunction.base - if (base.windowRange == null) { + if (base.windowFrame == null) { if (sortExpressions != null) { if (aggrFunction.dataType.isInstanceOf[ArrayType]) { rows.foreach(aggrFunction.update) @@ -144,7 +144,7 @@ case class WindowFunction( } else { functionResults(i) = - if (base.windowRange.windowType == "ROWS_RANGE") rowsWindowFunction(base, rows).iterator + if (base.windowFrame.frameType == "ROWS_FRAME") rowsWindowFunction(base, rows).iterator else valueWindowFunction(base, rows).iterator } i += 1 @@ -159,16 +159,16 @@ case class WindowFunction( var rowIndex = 0 while (rowIndex < rows.size) { - val windowRange = base.windowRange + val windowFrame = base.windowFrame var start = - if (windowRange.preceding == Int.MaxValue) 0 - else rowIndex - windowRange.preceding + if (windowFrame.preceding == Int.MaxValue) 0 + else rowIndex - windowFrame.preceding if (start < 0) start = 0 var end = - if (windowRange.following == Int.MaxValue) { + if (windowFrame.following == Int.MaxValue) { rows.size - 1 } else { - rowIndex + windowRange.following + rowIndex + windowFrame.following } if (end > rows.size - 1) end = rows.size - 1 @@ -185,27 +185,27 @@ case class WindowFunction( private[this] def valueWindowFunction(base: AggregateExpression, rows: CompactBuffer[Row]): CompactBuffer[Any] = { - val windowRange = base.windowRange + val windowFrame = base.windowFrame // rande only support 1 order val sortExpression = BindReferences.bindReference(sortExpressions.head, childOutput) val preceding = sortExpression.child.dataType match { - case IntegerType => Literal(windowRange.preceding) - case LongType => Literal(windowRange.preceding.toLong) - case DoubleType => Literal(windowRange.preceding.toDouble) - case FloatType => Literal(windowRange.preceding.toFloat) - case ShortType => Literal(windowRange.preceding.toShort) - case DecimalType => Literal(BigDecimal(windowRange.preceding)) + case IntegerType => Literal(windowFrame.preceding) + case LongType => Literal(windowFrame.preceding.toLong) + case DoubleType => Literal(windowFrame.preceding.toDouble) + case FloatType => Literal(windowFrame.preceding.toFloat) + case ShortType => Literal(windowFrame.preceding.toShort) + case DecimalType => Literal(BigDecimal(windowFrame.preceding)) case _=> throw new Exception(s"not support dataType ") } val following = sortExpression.child.dataType match { - case IntegerType => Literal(windowRange.following) - case LongType => Literal(windowRange.following.toLong) - case DoubleType => Literal(windowRange.following.toDouble) - case FloatType => Literal(windowRange.following.toFloat) - case ShortType => Literal(windowRange.following.toShort) - case DecimalType => Literal(BigDecimal(windowRange.following)) + case IntegerType => Literal(windowFrame.following) + case LongType => Literal(windowFrame.following.toLong) + case DoubleType => Literal(windowFrame.following.toDouble) + case FloatType => Literal(windowFrame.following.toFloat) + case ShortType => Literal(windowFrame.following.toShort) + case DecimalType => Literal(BigDecimal(windowFrame.following)) case _=> throw new Exception(s"not support dataType ") } @@ -232,13 +232,13 @@ case class WindowFunction( var followingIndex = rows.size - 1 if (sortExpression != null) { - if (windowRange.preceding != Int.MaxValue) precedingIndex = rowIndex + if (windowFrame.preceding != Int.MaxValue) precedingIndex = rowIndex while (precedingIndex > 0 && precedingExpr.eval(rows(precedingIndex - 1)).asInstanceOf[Boolean]) { precedingIndex -= 1 } - if (windowRange.following != Int.MaxValue) followingIndex = rowIndex + if (windowFrame.following != Int.MaxValue) followingIndex = rowIndex while (followingIndex < rows.size - 1 && followingExpr.eval(rows(followingIndex + 1)).asInstanceOf[Boolean]) { followingIndex += 1 diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 24e6c395b0681..64a958be6ca6a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -907,16 +907,16 @@ private[hive] object HiveQl { windowSpec match { case Token(alias, Nil) :: Nil => translateWindowSpec(getWindowSpec(alias)) - case Token(alias, Nil) :: range => { - val (partitionClause :: rowsRange :: valueRange :: Nil) = getClauses( + case Token(alias, Nil) :: frame => { + val (partitionClause :: rowsFrame :: valueFrame :: Nil) = getClauses( Seq( "TOK_PARTITIONINGSPEC", "TOK_WINDOWRANGE", "TOK_WINDOWVALUES"), translateWindowSpec(getWindowSpec(alias))) partitionClause match { - case Some(partition) => partition.asInstanceOf[ASTNode] :: range - case None => range + case Some(partition) => partition.asInstanceOf[ASTNode] :: frame + case None => frame } } case e => e @@ -1018,9 +1018,9 @@ private[hive] object HiveQl { } - protected def checkWindowSpec(windowSpec: Seq[ASTNode]): WindowRange = { + protected def checkWindowSpec(windowSpec: Seq[ASTNode]): WindowFrame = { - val (partitionClause :: rowsRange :: valueRange :: Nil) = getClauses( + val (partitionClause :: rowsFrame :: valueFrame :: Nil) = getClauses( Seq( "TOK_PARTITIONINGSPEC", "TOK_WINDOWRANGE", @@ -1032,9 +1032,9 @@ private[hive] object HiveQl { case None => addWindowPartitions(null) } - (rowsRange orElse valueRange)match { - case Some(range) => { - val rangeSeq = range.getChildren.toIndexedSeq + (rowsFrame orElse valueFrame)match { + case Some(frame) => { + val rangeSeq = frame.getChildren.toIndexedSeq if (rangeSeq.size > 0) { @@ -1056,10 +1056,10 @@ private[hive] object HiveQl { } } else 0 - if (rowsRange.isDefined) { - WindowRange("ROWS_RANGE", preceding, following) + if (rowsFrame.isDefined) { + WindowFrame("ROWS_FRAME", preceding, following) } else { - WindowRange("VALUE_RANGE", preceding, following) + WindowFrame("VALUE_FRAME", preceding, following) } } else null @@ -1262,12 +1262,12 @@ private[hive] object HiveQl { /* UDFs - Must be last otherwise will preempt built in functions */ case Token("TOK_FUNCTION", Token(name, Nil) :: args) => { val exprs = new ArrayBuffer[Expression] - var windowRange: WindowRange = null + var windowFrame: WindowFrame = null args.foreach { - case Token("TOK_WINDOWSPEC", winSpec) => windowRange = checkWindowSpec(winSpec) + case Token("TOK_WINDOWSPEC", winSpec) => windowFrame = checkWindowSpec(winSpec) case a: ASTNode => exprs += nodeToExpr(a) } - UnresolvedFunction(name, exprs, windowRange) + UnresolvedFunction(name, exprs, windowFrame) } case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: args) => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 5665f0e2d99da..9e0a1436a4ba4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -335,7 +335,7 @@ private[hive] case class HiveUdafFunction( funcWrapper.createFunction[AbstractGenericUDAFResolver]() } - private val inspectors = exprs.map(_.dataType).map(toInspector).toArray + private val inspectors = exprs.map(toInspector).toArray private val function = resolver.getEvaluator(exprs.map(_.dataType.toTypeInfo).toArray) @@ -350,7 +350,11 @@ private[hive] case class HiveUdafFunction( val inputProjection = new InterpretedProjection(exprs) def update(input: Row): Unit = { - val inputs = inputProjection(input).asInstanceOf[Seq[AnyRef]].toArray + var i = -1 + val inputs = inputProjection(input).map(data => { + i += 1 + wrap(data, inspectors(i)) + }).toArray function.iterate(buffer, inputs) } } diff --git a/sql/hive/src/test/resources/golden/11.testFirstLast-0-c12e5632668c2dccc3199e702b35ef4f b/sql/hive/src/test/resources/golden/11.testFirstLast-0-16766ca1836b01697b19591661fee2ef similarity index 100% rename from sql/hive/src/test/resources/golden/11.testFirstLast-0-c12e5632668c2dccc3199e702b35ef4f rename to sql/hive/src/test/resources/golden/11.testFirstLast-0-16766ca1836b01697b19591661fee2ef diff --git a/sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-57f204d198c816534e61fd57b698a23f b/sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-7073024e84e63b5e64ca0e054789fb5 similarity index 100% rename from sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-57f204d198c816534e61fd57b698a23f rename to sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-7073024e84e63b5e64ca0e054789fb5 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala index bda8ae73d04b8..128901e82db5c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala @@ -110,7 +110,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |sum(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN |current row AND current row) AS s2, |first_value(p_size) OVER w1 AS f, - |last_value(p_size) OVER w1 AS l + |last_value(p_size, false) OVER w1 AS l |FROM part |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN |2 preceding AND 2 following) @@ -123,7 +123,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |sum(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN |current row AND current row) AS s2, |first_value(p_size) OVER w1 AS f, - |last_value(p_size) OVER w1 AS l + |last_value(p_size, false) OVER w1 AS l |FROM part |where p_mfgr = 'Manufacturer#3' |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN 2 preceding AND 2 following) From 173016c08770fd2aa6ee15c3f194c2282bd46e68 Mon Sep 17 00:00:00 2001 From: guowei2 Date: Wed, 26 Nov 2014 14:58:47 +0800 Subject: [PATCH 09/14] fix window function to support multi-different window partitions --- .../sql/catalyst/analysis/Analyzer.scala | 35 ++- .../sql/catalyst/analysis/unresolved.scala | 3 +- .../expressions/WindowAttribute.scala | 43 ++++ .../sql/catalyst/expressions/aggregates.scala | 6 +- .../plans/logical/basicOperators.scala | 5 +- .../spark/sql/execution/SparkStrategies.scala | 4 +- .../spark/sql/execution/WindowFunction.scala | 55 ++--- .../org/apache/spark/sql/hive/HiveQl.scala | 216 ++++++++---------- ...dowing-0-533469d61b3042b5a28021d7dc057069} | 0 ...hWhere-0-76c5a15c9378ec392de9039ce5884fd4} | 0 ...Clause-0-f58aed307dbc7320ee8e5c8d9d07ab96} | 0 ...essions-0-a574028e9f6e74e102d041a699f36b56 | 25 ++ ...Windows-0-965d427fb9310c1e338e8fd0049fb4f8 | 25 ++ ...stDISTs-0-73455a57a1c689137d9c7ca7eb7b8053 | 25 ++ ...ecRules-0-40198efe73963d6626d7157b71c9004c | 25 ++ ...usForms-0-653052092fd9992c868d5d1551ebc77a | 25 ++ ...sForms2-0-a8b8fcdfb6e4b29f8e6c1a3f4dd595d1 | 25 ++ ...tCount-0-ef3bf1647dcb18d1f8e6107eab349807} | 0 ...gregate-0-7776245f6859b7cba06d06f28ae577d} | 0 ...ngUDAF-0-e703f883d590a9f3a897f5ae7c7b9cfb} | 0 ...InSubQ-0-f6912910364e497344dea061fdecf258} | 0 ...eAlias-0-ac7c6f442dde6d5913e89a7881f35df8} | 0 ...gNoGBY-0-a04876c3478b374c339c33cc78a34150} | 0 .../execution/HiveWindowFunctionSuite.scala | 86 +++++-- 24 files changed, 419 insertions(+), 184 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WindowAttribute.scala rename sql/hive/src/test/resources/golden/{1.testWindowing-0-53287db3a565fc9e7a033adeaff23c0c => 1.testWindowing-0-533469d61b3042b5a28021d7dc057069} (100%) rename sql/hive/src/test/resources/golden/{12.testFirstLastWithWhere-0-7073024e84e63b5e64ca0e054789fb5 => 12.testFirstLastWithWhere-0-76c5a15c9378ec392de9039ce5884fd4} (100%) rename sql/hive/src/test/resources/golden/{14.testNoSortClause-0-a7970bafdc1b1a2dba571d19136205bc => 14.testNoSortClause-0-f58aed307dbc7320ee8e5c8d9d07ab96} (100%) create mode 100644 sql/hive/src/test/resources/golden/15.testExpressions-0-a574028e9f6e74e102d041a699f36b56 create mode 100644 sql/hive/src/test/resources/golden/16.testMultipleWindows-0-965d427fb9310c1e338e8fd0049fb4f8 create mode 100644 sql/hive/src/test/resources/golden/21.testDISTs-0-73455a57a1c689137d9c7ca7eb7b8053 create mode 100644 sql/hive/src/test/resources/golden/30.testDefaultPartitioningSpecRules-0-40198efe73963d6626d7157b71c9004c create mode 100644 sql/hive/src/test/resources/golden/37.testPartitioningVariousForms-0-653052092fd9992c868d5d1551ebc77a create mode 100644 sql/hive/src/test/resources/golden/38.testPartitioningVariousForms2-0-a8b8fcdfb6e4b29f8e6c1a3f4dd595d1 rename sql/hive/src/test/resources/golden/{4.testCount-0-6ccae5fcb441ad90f14dff22de1aafc => 4.testCount-0-ef3bf1647dcb18d1f8e6107eab349807} (100%) rename sql/hive/src/test/resources/golden/{44.testOverNoPartitionSingleAggregate-0-2fb3d43169e2f20680e7477f0daaddf4 => 44.testOverNoPartitionSingleAggregate-0-7776245f6859b7cba06d06f28ae577d} (100%) rename sql/hive/src/test/resources/golden/{5.testCountWithWindowingUDAF-0-f22b5eae7b0255f15c480527036fd2b0 => 5.testCountWithWindowingUDAF-0-e703f883d590a9f3a897f5ae7c7b9cfb} (100%) rename sql/hive/src/test/resources/golden/{6.testCountInSubQ-0-2e3e151f6ddc88c11149c72d884ba4bb => 6.testCountInSubQ-0-f6912910364e497344dea061fdecf258} (100%) rename sql/hive/src/test/resources/golden/{8.testMixedCaseAlias-0-1458fdd44340b7cf31a4c3980e53e643 => 8.testMixedCaseAlias-0-ac7c6f442dde6d5913e89a7881f35df8} (100%) rename sql/hive/src/test/resources/golden/{9.testHavingWithWindowingNoGBY-0-6b117eafdd9731cee1007ae9719888a1 => 9.testHavingWithWindowingNoGBY-0-a04876c3478b374c339c33cc78a34150} (100%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d1fc1d24c3c3c..d13bee39d80c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -60,6 +60,7 @@ class Analyzer(catalog: Catalog, ImplicitGenerate :: StarExpansion :: ResolveFunctions :: + WindowAttributes :: GlobalAggregates :: UnresolvedHavingClauseAttributes :: TrimGroupingAliases :: @@ -218,16 +219,30 @@ class Analyzer(catalog: Catalog, def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressions { - case u @ UnresolvedFunction(name, children, windowFrame) if u.childrenResolved => { - val function = registry.lookupFunction(name, children) - if (windowFrame != null) function match { - case agg: AggregateExpression => - function.asInstanceOf[AggregateExpression].windowFrame = windowFrame - case _ => logError(s"function " + name + " does not support window frame") - } - function - } + case u @ UnresolvedFunction(name, children) if u.childrenResolved => + registry.lookupFunction(name, children) + } + } + } + /** + * translate WindowAttribute in SelectExpression to attribute + */ + object WindowAttributes extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: WindowFunction => + q transformExpressions { + case u @ WindowAttribute(children, name, windowSpec) + // set window spec in AggregateExpression for execution and GlobalAggregates check + if (children.isInstanceOf[AggregateExpression]) => { + children.asInstanceOf[AggregateExpression].windowSpec = windowSpec + u + } + } + case q: LogicalPlan => + q transformExpressions { + // translate WindowAttribute in SelectExpression to attribute + case u @ WindowAttribute(children, name, windowSpec) => u.toAttribute } } } @@ -243,7 +258,7 @@ class Analyzer(catalog: Catalog, def containsAggregates(exprs: Seq[Expression]): Boolean = { exprs.foreach(_.foreach { - case agg: AggregateExpression => return true + case agg: AggregateExpression if (agg.windowSpec == null) => return true case _ => }) false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 17010743bb6fa..77d84e1687e1b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -63,8 +63,7 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo override def toString: String = s"'$name" } -case class UnresolvedFunction(name: String, children: Seq[Expression], - windowFrame: WindowFrame = null) extends Expression { +case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression { override def dataType = throw new UnresolvedException(this, "dataType") override def foldable = throw new UnresolvedException(this, "foldable") override def nullable = throw new UnresolvedException(this, "nullable") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WindowAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WindowAttribute.scala new file mode 100644 index 0000000000000..63cb9fa432932 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WindowAttribute.scala @@ -0,0 +1,43 @@ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute + + +/** + * + * @param child the computation being performed + * @param windowSpec the window spec definition + * @param exprId A globally unique id used to check if an [[AttributeReference]] refers to this + * alias. Auto-assigned if left blank. + */ +case class WindowAttribute(child: Expression, name: String, windowSpec: WindowSpec) + (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) + extends NamedExpression with trees.UnaryNode[Expression] { + + override type EvaluatedType = Any + + override def eval(input: Row) = child.eval(input) + + override def dataType = child.dataType + override def nullable = child.nullable + + override def toAttribute = { + if (resolved) { + AttributeReference(name, child.dataType, child.nullable)(exprId, qualifiers) + } else { + UnresolvedAttribute(name) + } + } + + override def toString: String = s"$child $windowSpec AS $name#${exprId.id}$typeSuffix" + + override protected final def otherCopyArgs = exprId :: qualifiers :: Nil + +} + +case class WindowSpec(windowPartition: WindowPartition, windowFrame: WindowFrame) + +case class WindowPartition(partitionBy: Seq[Expression], sortBy: Seq[SortOrder]) + +case class WindowFrame(frameType:String, preceding: Int, following: Int) \ No newline at end of file diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 04c75674fe851..50e2ec48a4114 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.collection.OpenHashSet abstract class AggregateExpression extends Expression with Serializable { self: Product => - var windowFrame: WindowFrame = null + var windowSpec: WindowSpec = null /** * Creates a new instance that can be used to compute this aggregate expression for a group * of input rows/ @@ -35,7 +35,7 @@ abstract class AggregateExpression extends Expression with Serializable { def newInstance(): AggregateFunction override def canEqual(other: Any): Boolean = other match { - case that: AggregateExpression => this.windowFrame == that.windowFrame + case that: AggregateExpression => this.windowSpec == that.windowSpec case _ => false } @@ -47,8 +47,6 @@ abstract class AggregateExpression extends Expression with Serializable { throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } -case class WindowFrame(frameType:String, preceding: Int, following: Int) - /** * Represents an aggregation that has been rewritten to be performed in two steps. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 58a38b45455ac..645bf917361f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -145,11 +145,12 @@ case class Aggregate( case class WindowFunction( partitionExpressions: Seq[Expression], - functionExpressions: Seq[NamedExpression], + computeExpressions: Seq[WindowAttribute], + otherExpressions: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { - override def output = functionExpressions.map(_.toAttribute) + override def output = (computeExpressions ++ otherExpressions).map(_.toAttribute) } case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 883bea6d6caf7..3fc953e5d1e0b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -272,8 +272,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => execution.Aggregate(partial = false, group, agg, planLater(child)) :: Nil - case logical.WindowFunction(partition, function, child) => - execution.WindowFunction(partition, function, planLater(child)) :: Nil + case logical.WindowFunction(partition, compute, other, child) => + execution.WindowFunction(partition, compute, other, planLater(child)) :: Nil case logical.Sample(fraction, withReplacement, seed, child) => execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil case SparkLogicalPlan(alreadyPlanned) => alreadyPlanned :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala index e3e9cb9404fa2..0399cd757477d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala @@ -41,13 +41,15 @@ import org.apache.spark.sql.catalyst.plans.logical.SortPartitions * Groups input data by `partitionExpressions` and computes the `computeExpressions` for each * group. * @param partitionExpressions expressions that are evaluated to determine partition. - * @param functionExpressions expressions that are computed for each partition. + * @param windowAttributes windowAttributes that are computed now for each partition. + * @param otherExpressions otherExpressions that are expressions except windowAttributes. * @param child the input data source. */ @DeveloperApi case class WindowFunction( partitionExpressions: Seq[Expression], - functionExpressions: Seq[NamedExpression], + windowAttributes: Seq[WindowAttribute], + otherExpressions: Seq[NamedExpression], child: SparkPlan) extends UnaryNode { @@ -62,43 +64,31 @@ case class WindowFunction( // out child's output attributes statically here. private[this] val childOutput = child.output - override def output = functionExpressions.map(_.toAttribute) + private[this] val computeExpressions = + windowAttributes.map(_.child.asInstanceOf[AggregateExpression]) - /** A list of functions that need to be computed for each partition. */ - private[this] val computeExpressions = new ArrayBuffer[AggregateExpression] + override def output = (windowAttributes ++ otherExpressions).map(_.toAttribute) - private[this] val otherExpressions = new ArrayBuffer[NamedExpression] - - functionExpressions.foreach { sel => - sel.collect { - case func: AggregateExpression => computeExpressions += func - case other: NamedExpression if (!other.isInstanceOf[Alias]) => otherExpressions += other - } - } - - private[this] val functionAttributes = computeExpressions.map { func => + private[this] val computeAttributes = computeExpressions.map { func => func -> AttributeReference(s"funcResult:$func", func.dataType, func.nullable)()} + private[this] val otherAttributes = otherExpressions.map(_.toAttribute) + /** The schema of the result of all evaluations */ - private[this] val resultAttributes = - otherExpressions.map(_.toAttribute) ++ functionAttributes.map(_._2) + private[this] val resultAttributes = otherAttributes ++ computeAttributes.map(_._2) private[this] val resultMap = - (otherExpressions.map { other => other -> other.toAttribute } ++ functionAttributes + (otherExpressions.map { other => other -> other.toAttribute } ++ computeAttributes ).toMap - - private[this] val resultExpressions = functionExpressions.map { sel => + private[this] val resultExpressions = (windowAttributes ++ otherExpressions).map { sel => sel.transform { case e: Expression if resultMap.contains(e) => resultMap(e) } } private[this] val sortExpressions = - if (child.isInstanceOf[SortPartitions]) { - child.asInstanceOf[SortPartitions].sortExpressions - } - else if (child.isInstanceOf[Sort]) { + if (child.isInstanceOf[Sort]) { child.asInstanceOf[Sort].sortOrder } else null @@ -109,7 +99,7 @@ case class WindowFunction( var i = 0 while (i < computeExpressions.length) { val baseExpr = BindReferences.bindReference(computeExpressions(i), childOutput) - baseExpr.windowFrame = computeExpressions(i).windowFrame + baseExpr.windowSpec = computeExpressions(i).windowSpec buffer(i) = baseExpr.newInstance() i += 1 } @@ -123,7 +113,8 @@ case class WindowFunction( while (i < aggrFunctions.length) { val aggrFunction = aggrFunctions(i) val base = aggrFunction.base - if (base.windowFrame == null) { + val windowSpec = base.windowSpec + if (windowSpec.windowFrame == null) { if (sortExpressions != null) { if (aggrFunction.dataType.isInstanceOf[ArrayType]) { rows.foreach(aggrFunction.update) @@ -144,7 +135,9 @@ case class WindowFunction( } else { functionResults(i) = - if (base.windowFrame.frameType == "ROWS_FRAME") rowsWindowFunction(base, rows).iterator + if (windowSpec.windowFrame.frameType == "ROWS_FRAME") { + rowsWindowFunction(base, rows).iterator + } else valueWindowFunction(base, rows).iterator } i += 1 @@ -159,7 +152,7 @@ case class WindowFunction( var rowIndex = 0 while (rowIndex < rows.size) { - val windowFrame = base.windowFrame + val windowFrame = base.windowSpec.windowFrame var start = if (windowFrame.preceding == Int.MaxValue) 0 else rowIndex - windowFrame.preceding @@ -185,7 +178,7 @@ case class WindowFunction( private[this] def valueWindowFunction(base: AggregateExpression, rows: CompactBuffer[Row]): CompactBuffer[Any] = { - val windowFrame = base.windowFrame + val windowFrame = base.windowSpec.windowFrame // rande only support 1 order val sortExpression = BindReferences.bindReference(sortExpressions.head, childOutput) @@ -271,7 +264,7 @@ case class WindowFunction( val resultProjection = new InterpretedProjection(resultExpressions, resultAttributes) - val otherProjection = new InterpretedMutableProjection(otherExpressions, childOutput) + val otherProjection = new InterpretedMutableProjection(otherAttributes, childOutput) val joinedRow = new JoinedRow val rows = new CompactBuffer[Row]() @@ -321,7 +314,7 @@ case class WindowFunction( private[this] var currentRowIndex: Int = -1 val resultProjection = new InterpretedProjection(resultExpressions, resultAttributes) - val otherProjection = new InterpretedMutableProjection(otherExpressions, childOutput) + val otherProjection = new InterpretedMutableProjection(otherAttributes, childOutput) val joinedRow = new JoinedRow override final def hasNext: Boolean = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 64a958be6ca6a..7dd3bc5ab6a40 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -516,7 +516,7 @@ private[hive] object HiveQl { // Return one query for each insert clause. val queries = insertClauses.map { case Token("TOK_INSERT", singleInsert) => - var ( + val ( intoClause :: destClause :: selectClause :: @@ -550,8 +550,7 @@ private[hive] object HiveQl { singleInsert) } - initWindow - checkWindowDef(windowClause) + initWindowDef(windowClause) val relations = nodeToRelation(fromClause) val withWhere = whereClause.map { whereNode => @@ -615,16 +614,14 @@ private[hive] object HiveQl { // Not a transformation so must be either project or aggregation. val selectExpressions = nameExpressions(select.getChildren.flatMap(selExprNodeToExpr)) - checkWindowPartitions match { - case Some(partition) => - windowToPlan(partition, selectExpressions, withLateralView) - case None => - groupByClause match { - case Some(groupBy) => - Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withLateralView) - case None => - Project(selectExpressions, withLateralView) - } + //not support both window and group by yet + groupByClause match { + case Some(groupBy) => + Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withLateralView) + case None => { + val withWindowView = windowToPlan(selectExpressions, withLateralView) + Project(selectExpressions, withWindowView) + } } } @@ -876,20 +873,15 @@ private[hive] object HiveQl { } // store the window def of current sql - // use thread id as key to avoid mistake when muti sqls parse at the same time - protected val windowDefMap = new ConcurrentHashMap[Long,Map[String, Seq[ASTNode]]]() + // use thread id as key to avoid mistake when multi-sqls parse at the same time + protected val windowDefMap = new ConcurrentHashMap[Long, Map[String, Seq[ASTNode]]]() - // store the window spec of current sql - // use thread id as key to avoid mistake when muti sqls parse at the same time - protected val windowPartitionsMap = new ConcurrentHashMap[Long, ArrayBuffer[Node]]() + // store the window spec number of current sql + protected val windowNoMap = new ConcurrentHashMap[Long, Int]() - protected def initWindow() = { - windowDefMap.put(Thread.currentThread().getId, Map[String, Seq[ASTNode]]()) - windowPartitionsMap.put(Thread.currentThread().getId, new ArrayBuffer[Node]()) - } - protected def checkWindowDef(windowClause: Option[Node]) = { + protected def initWindowDef(windowClause: Option[Node]) = { - var winDefs = windowDefMap.get(Thread.currentThread().getId) + var winDefs = Map[String, Seq[ASTNode]]() windowClause match { case Some(window) => window.getChildren.foreach { @@ -901,8 +893,15 @@ private[hive] object HiveQl { } windowDefMap.put(Thread.currentThread().getId, winDefs) + windowNoMap.put(Thread.currentThread().getId, 0) } + protected def nextWindowNo: Int = { + var no = windowNoMap.get(Thread.currentThread().getId) + no += 1 + windowNoMap.put(Thread.currentThread().getId, no) + no + } protected def translateWindowSpec(windowSpec: Seq[ASTNode]): Seq[ASTNode]= { windowSpec match { @@ -928,97 +927,59 @@ private[hive] object HiveQl { alias, sys.error("no window def for " + alias)) } - protected def addWindowPartitions(partition: Node) = { + protected def windowToPlan(selectExpressions: Seq[NamedExpression], + withLateralView: LogicalPlan): LogicalPlan = { - var winPartitions = windowPartitionsMap.get(Thread.currentThread().getId) - winPartitions += partition - windowPartitionsMap.put(Thread.currentThread().getId, winPartitions) - } + val windowAttributes = new ArrayBuffer[WindowAttribute] + val attrExpressions = new ArrayBuffer[NamedExpression] + selectExpressions.foreach { sel => + sel.collect { + case win: WindowAttribute => windowAttributes += win + case attr: UnresolvedAttribute => attrExpressions += attr + } + } - protected def getWindowPartitions(): Seq[Node]= { - windowPartitionsMap.get(Thread.currentThread().getId).toSeq - } + // Multi-step of WindowFunction will be needed, + // if there're multi-windowAttributes with different window partitions, + var currentPlan = withLateralView + while (!windowAttributes.isEmpty) { - protected def checkWindowPartitions(): Option[Seq[ASTNode]] = { + var currentPartition: WindowPartition = null + val computeExpressions = new ArrayBuffer[WindowAttribute] + val otherExpressions = attrExpressions.clone() - val partitionUnits = new ArrayBuffer[Seq[ASTNode]]() + windowAttributes.foreach(expr => { + val windowPartition = expr.windowSpec.windowPartition - getWindowPartitions.map { - case Token("TOK_PARTITIONINGSPEC", partition) => Some(partition) - case _ => None - }.foreach { - case Some(partition) => { - if (partitionUnits.isEmpty) partitionUnits += partition - else { - // only add different window partitions - try { - partition zip partitionUnits.head foreach { - case (l,r) => l checkEquals r - } - } catch { - case re: RuntimeException => partitionUnits += partition - } + if (currentPartition == null) { + computeExpressions += expr + currentPartition = windowPartition + } else if (currentPartition == windowPartition) { + computeExpressions += expr } - } - case None => // do nothing - } - - // check whether all window partitions are same, we just support same window partition now - if (partitionUnits.size == 0 && getWindowPartitions.size > 0) { - Some(Seq()) - } else if (partitionUnits.size == 1) { - Some(partitionUnits.head) - } else if (partitionUnits.size > 1) { - throw new NotImplementedError(s"not support muti different window specs now." + - " use subQuery instead.") - } else { - None - } - } + }) - protected def windowToPlan(partitionClause: Seq[ASTNode], - selectExpressions: Seq[NamedExpression], withLateralView: LogicalPlan): LogicalPlan = { - - val (orderByClause :: sortByClause :: distributeByClause :: clusterByClause :: Nil) = - getClauses( - Seq( - "TOK_ORDERBY", - "TOK_SORTBY", - "TOK_DISTRIBUTEBY", - "TOK_CLUSTERBY"), - partitionClause) - - val partitionExpr = (distributeByClause orElse clusterByClause) match { - case Some(partitionBy) => partitionBy.getChildren.map(nodeToExpr) - case None => Seq() - } + val partitionExpr = currentPartition.partitionBy + val sortExpr = currentPartition.sortBy - val withWindowPartition = - (orderByClause, sortByClause, distributeByClause, clusterByClause) match { - case (Some(totalOrdering), None, None, None) => - Sort(totalOrdering.getChildren.map(nodeToSortOrder), withLateralView) - case (None, Some(perPartitionOrdering), None, None) => - SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), withLateralView) - case (None, None, Some(partitionExprs), None) => - Repartition(partitionExprs.getChildren.map(nodeToExpr), withLateralView) - case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => - SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), - Repartition(partitionExprs.getChildren.map(nodeToExpr), withLateralView)) - case (Some(perPartitionOrdering), None, Some(partitionExprs), None) => - SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), - Repartition(partitionExprs.getChildren.map(nodeToExpr), withLateralView)) - case (None, None, None, Some(clusterExprs)) => - SortPartitions(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)), - Repartition(clusterExprs.getChildren.map(nodeToExpr), withLateralView)) - case (None, None, None, None) => withLateralView - case _ => sys.error("Unsupported set of ordering / distribution clauses.") - } + val withWindowPartition = + if (partitionExpr.size > 0) { + if (sortExpr.size > 0) SortPartitions(sortExpr, Repartition(partitionExpr, currentPlan)) + else Repartition(partitionExpr, currentPlan) + } else { + if (sortExpr.size > 0) Sort(sortExpr, currentPlan) + else currentPlan + } - WindowFunction(partitionExpr, selectExpressions, withWindowPartition) + currentPlan = WindowFunction(partitionExpr, computeExpressions.toSeq, otherExpressions, withWindowPartition) + attrExpressions ++= computeExpressions.map(_.toAttribute) + windowAttributes --= computeExpressions + } + currentPlan } - protected def checkWindowSpec(windowSpec: Seq[ASTNode]): WindowFrame = { + protected def checkWindowSpec(windowSpec: Seq[ASTNode]): WindowSpec = { val (partitionClause :: rowsFrame :: valueFrame :: Nil) = getClauses( Seq( @@ -1027,12 +988,33 @@ private[hive] object HiveQl { "TOK_WINDOWVALUES"), translateWindowSpec(windowSpec)) - partitionClause match { - case Some(partition) => addWindowPartitions(partition) - case None => addWindowPartitions(null) + val wp = partitionClause match { + case Some(partition) => { + val (orderByClause :: sortByClause :: distributeByClause :: clusterByClause :: Nil) = + getClauses( + Seq( + "TOK_ORDERBY", + "TOK_SORTBY", + "TOK_DISTRIBUTEBY", + "TOK_CLUSTERBY"), + partition.getChildren.toIndexedSeq.asInstanceOf[Seq[ASTNode]]) + + val partitionExpr = (distributeByClause orElse clusterByClause) match { + case Some(partitionBy) => partitionBy.getChildren.map(nodeToExpr) + case None => Seq() + } + + val sortByExpr = (orderByClause orElse sortByClause) match { + case Some(sortBy) => sortBy.getChildren.map(nodeToSortOrder) + case None => Seq() + } + WindowPartition(partitionExpr, sortByExpr) + } + case None => WindowPartition(Seq(), Seq()) } - (rowsFrame orElse valueFrame)match { + + (rowsFrame orElse valueFrame) match { case Some(frame) => { val rangeSeq = frame.getChildren.toIndexedSeq @@ -1057,14 +1039,14 @@ private[hive] object HiveQl { } else 0 if (rowsFrame.isDefined) { - WindowFrame("ROWS_FRAME", preceding, following) + WindowSpec(wp, WindowFrame("ROWS_FRAME", preceding, following)) } else { - WindowFrame("VALUE_FRAME", preceding, following) + WindowSpec(wp, WindowFrame("VALUE_FRAME", preceding, following)) } - } else null + } else WindowSpec(wp, null) } - case None => null + case None => WindowSpec(wp, null) } } @@ -1261,13 +1243,15 @@ private[hive] object HiveQl { /* UDFs - Must be last otherwise will preempt built in functions */ case Token("TOK_FUNCTION", Token(name, Nil) :: args) => { - val exprs = new ArrayBuffer[Expression] - var windowFrame: WindowFrame = null + val exprs = new ArrayBuffer[Node] + var windowSpec: WindowSpec = null args.foreach { - case Token("TOK_WINDOWSPEC", winSpec) => windowFrame = checkWindowSpec(winSpec) - case a: ASTNode => exprs += nodeToExpr(a) + case Token("TOK_WINDOWSPEC", winSpec) => windowSpec = checkWindowSpec(winSpec) + case a: ASTNode => exprs += a } - UnresolvedFunction(name, exprs, windowFrame) + if (windowSpec == null) UnresolvedFunction(name, exprs.map(nodeToExpr)) + else WindowAttribute(UnresolvedFunction(name, exprs.map(nodeToExpr)), + s"w_" + nextWindowNo, windowSpec)() } case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: args) => diff --git a/sql/hive/src/test/resources/golden/1.testWindowing-0-53287db3a565fc9e7a033adeaff23c0c b/sql/hive/src/test/resources/golden/1.testWindowing-0-533469d61b3042b5a28021d7dc057069 similarity index 100% rename from sql/hive/src/test/resources/golden/1.testWindowing-0-53287db3a565fc9e7a033adeaff23c0c rename to sql/hive/src/test/resources/golden/1.testWindowing-0-533469d61b3042b5a28021d7dc057069 diff --git a/sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-7073024e84e63b5e64ca0e054789fb5 b/sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-76c5a15c9378ec392de9039ce5884fd4 similarity index 100% rename from sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-7073024e84e63b5e64ca0e054789fb5 rename to sql/hive/src/test/resources/golden/12.testFirstLastWithWhere-0-76c5a15c9378ec392de9039ce5884fd4 diff --git a/sql/hive/src/test/resources/golden/14.testNoSortClause-0-a7970bafdc1b1a2dba571d19136205bc b/sql/hive/src/test/resources/golden/14.testNoSortClause-0-f58aed307dbc7320ee8e5c8d9d07ab96 similarity index 100% rename from sql/hive/src/test/resources/golden/14.testNoSortClause-0-a7970bafdc1b1a2dba571d19136205bc rename to sql/hive/src/test/resources/golden/14.testNoSortClause-0-f58aed307dbc7320ee8e5c8d9d07ab96 diff --git a/sql/hive/src/test/resources/golden/15.testExpressions-0-a574028e9f6e74e102d041a699f36b56 b/sql/hive/src/test/resources/golden/15.testExpressions-0-a574028e9f6e74e102d041a699f36b56 new file mode 100644 index 0000000000000..697f4fac91824 --- /dev/null +++ b/sql/hive/src/test/resources/golden/15.testExpressions-0-a574028e9f6e74e102d041a699f36b56 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2.0 0.0 2 2 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1 2 18.0 16.0 2 34 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 2 3 14.0 14.236104336041748 2 6 2 +Manufacturer#1 almond aquamarine burnished black steel 28 2 4 17.5 13.738631664034086 2 28 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 3 5 22.4 15.717506163510802 2 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 14.0 0.0 4 14 14 +Manufacturer#2 almond antique violet turquoise frosted 40 1 2 27.0 13.0 4 40 14 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 3 18.666666666666668 15.86050300449376 4 2 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 2 4 20.25 14.00669482783144 4 25 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 3 5 19.8 12.560254774486067 4 18 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 17.0 0.0 2 17 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 1 2 15.5 1.5 2 14 17 +Manufacturer#3 almond antique metallic orange dim 19 2 3 16.666666666666668 2.0548046676563256 2 19 17 +Manufacturer#3 almond antique misty red olive 1 2 4 12.75 7.013380069552769 2 1 14 +Manufacturer#3 almond antique olive coral navajo 45 3 5 19.2 14.344336861632886 2 45 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 10.0 0.0 0 10 10 +Manufacturer#4 almond antique violet mint lemon 39 1 2 24.5 14.5 0 39 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 2 3 25.333333333333332 11.897712198383164 0 27 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 2 4 20.75 13.007209539328564 0 7 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 3 5 19.0 12.149074038789951 0 12 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 31.0 0.0 1 31 31 +Manufacturer#5 almond antique medium spring khaki 6 1 2 18.5 12.5 1 6 31 +Manufacturer#5 almond antique sky peru orange 2 2 3 13.0 12.832251036613439 1 2 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 2 4 21.25 18.102140757380052 1 46 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 3 5 21.6 16.206171663906314 1 23 2 diff --git a/sql/hive/src/test/resources/golden/16.testMultipleWindows-0-965d427fb9310c1e338e8fd0049fb4f8 b/sql/hive/src/test/resources/golden/16.testMultipleWindows-0-965d427fb9310c1e338e8fd0049fb4f8 new file mode 100644 index 0000000000000..31847f075da18 --- /dev/null +++ b/sql/hive/src/test/resources/golden/16.testMultipleWindows-0-965d427fb9310c1e338e8fd0049fb4f8 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 34 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 8 2 +Manufacturer#1 almond aquamarine burnished black steel 28 28 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 14 +Manufacturer#2 almond antique violet turquoise frosted 40 40 14 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 25 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 32 2 +Manufacturer#3 almond antique chartreuse khaki white 17 31 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 17 +Manufacturer#3 almond antique metallic orange dim 19 50 17 +Manufacturer#3 almond antique misty red olive 1 1 14 +Manufacturer#3 almond antique olive coral navajo 45 45 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 17 10 +Manufacturer#4 almond antique violet mint lemon 39 39 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 29 27 +Manufacturer#5 almond antique blue firebrick mint 31 31 31 +Manufacturer#5 almond antique medium spring khaki 6 8 31 +Manufacturer#5 almond antique sky peru orange 2 2 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 2 diff --git a/sql/hive/src/test/resources/golden/21.testDISTs-0-73455a57a1c689137d9c7ca7eb7b8053 b/sql/hive/src/test/resources/golden/21.testDISTs-0-73455a57a1c689137d9c7ca7eb7b8053 new file mode 100644 index 0000000000000..fffbd3702c136 --- /dev/null +++ b/sql/hive/src/test/resources/golden/21.testDISTs-0-73455a57a1c689137d9c7ca7eb7b8053 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 [{"x":1173.15,"y":1.0},{"x":1602.59,"y":1.0},{"x":1753.76,"y":1.0}] 110592.0 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 [{"x":1173.15,"y":1.0},{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1753.76,"y":1.0}] 98510.0 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 [{"x":1173.15,"y":1.0},{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0},{"x":1753.76,"y":1.0}] 86428.0 3 +Manufacturer#1 almond aquamarine burnished black steel 28 [{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0},{"x":1753.76,"y":1.0}] 86098.0 4 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 [{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0}] 86428.0 5 +Manufacturer#2 almond antique violet chocolate turquoise 14 [{"x":1690.68,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 1 +Manufacturer#2 almond antique violet turquoise frosted 40 [{"x":1690.68,"y":1.0},{"x":1698.66,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 139825.5 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 [{"x":1690.68,"y":1.0},{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 [{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 169347.0 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 [{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 5 +Manufacturer#3 almond antique chartreuse khaki white 17 [{"x":1190.27,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0}] 90681.0 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 [{"x":1190.27,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0},{"x":1922.98,"y":1.0}] 65831.5 2 +Manufacturer#3 almond antique metallic orange dim 19 [{"x":1190.27,"y":1.0},{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0},{"x":1922.98,"y":1.0}] 90681.0 3 +Manufacturer#3 almond antique misty red olive 1 [{"x":1190.27,"y":1.0},{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1922.98,"y":1.0}] 76690.0 4 +Manufacturer#3 almond antique olive coral navajo 45 [{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1922.98,"y":1.0}] 112398.0 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 [{"x":1206.26,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0}] 48427.0 1 +Manufacturer#4 almond antique violet mint lemon 39 [{"x":1206.26,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0},{"x":1844.92,"y":1.0}] 46844.0 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0},{"x":1844.92,"y":1.0}] 45261.0 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1375.42,"y":1.0},{"x":1844.92,"y":1.0}] 39309.0 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1844.92,"y":1.0}] 33357.0 5 +Manufacturer#5 almond antique blue firebrick mint 31 [{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 155733.0 1 +Manufacturer#5 almond antique medium spring khaki 6 [{"x":1018.1,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 99201.0 2 +Manufacturer#5 almond antique sky peru orange 2 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 78486.0 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0}] 60577.5 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1788.73,"y":1.0}] 78486.0 5 diff --git a/sql/hive/src/test/resources/golden/30.testDefaultPartitioningSpecRules-0-40198efe73963d6626d7157b71c9004c b/sql/hive/src/test/resources/golden/30.testDefaultPartitioningSpecRules-0-40198efe73963d6626d7157b71c9004c new file mode 100644 index 0000000000000..7595384c1874a --- /dev/null +++ b/sql/hive/src/test/resources/golden/30.testDefaultPartitioningSpecRules-0-40198efe73963d6626d7157b71c9004c @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 42 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 70 36 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 42 +Manufacturer#1 almond aquamarine burnished black steel 28 110 70 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 112 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 14 +Manufacturer#2 almond antique violet turquoise frosted 40 81 54 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 56 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 81 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 99 +Manufacturer#3 almond antique chartreuse khaki white 17 50 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 31 +Manufacturer#3 almond antique metallic orange dim 19 96 50 +Manufacturer#3 almond antique misty red olive 1 79 51 +Manufacturer#3 almond antique olive coral navajo 45 65 96 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 10 +Manufacturer#4 almond antique violet mint lemon 39 83 49 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 76 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 83 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 95 +Manufacturer#5 almond antique blue firebrick mint 31 39 31 +Manufacturer#5 almond antique medium spring khaki 6 85 37 +Manufacturer#5 almond antique sky peru orange 2 108 39 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 85 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 108 diff --git a/sql/hive/src/test/resources/golden/37.testPartitioningVariousForms-0-653052092fd9992c868d5d1551ebc77a b/sql/hive/src/test/resources/golden/37.testPartitioningVariousForms-0-653052092fd9992c868d5d1551ebc77a new file mode 100644 index 0000000000000..725a98a18947a --- /dev/null +++ b/sql/hive/src/test/resources/golden/37.testPartitioningVariousForms-0-653052092fd9992c868d5d1551ebc77a @@ -0,0 +1,25 @@ +Manufacturer#1 7576.58 1173.15 1753.76 1515.32 5 +Manufacturer#1 7576.58 1173.15 1753.76 1515.32 5 +Manufacturer#1 7576.58 1173.15 1753.76 1515.32 5 +Manufacturer#1 7576.58 1173.15 1753.76 1515.32 5 +Manufacturer#1 7576.58 1173.15 1753.76 1515.32 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 diff --git a/sql/hive/src/test/resources/golden/38.testPartitioningVariousForms2-0-a8b8fcdfb6e4b29f8e6c1a3f4dd595d1 b/sql/hive/src/test/resources/golden/38.testPartitioningVariousForms2-0-a8b8fcdfb6e4b29f8e6c1a3f4dd595d1 new file mode 100644 index 0000000000000..5b463d16ae69d --- /dev/null +++ b/sql/hive/src/test/resources/golden/38.testPartitioningVariousForms2-0-a8b8fcdfb6e4b29f8e6c1a3f4dd595d1 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1173.15 1173.15 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 1753.76 1753.76 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 1602.59 1602.59 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 1414.42 1414.42 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 1632.66 1632.66 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1690.68 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 1800.7 1800.7 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 2031.98 2031.98 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 1698.66 1698.66 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 1701.6 1701.6 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1671.68 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 1190.27 1190.27 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 1410.39 1410.39 +Manufacturer#3 almond antique misty red olive 1 1922.98 1922.98 1922.98 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 1337.29 1337.29 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1620.67 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 1375.42 1375.42 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 1206.26 1206.26 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 1844.92 1844.92 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 1290.35 1290.35 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1789.69 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 1611.66 1611.66 +Manufacturer#5 almond antique sky peru orange 2 1788.73 1788.73 1788.73 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 1018.1 1018.1 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 1464.48 1464.48 diff --git a/sql/hive/src/test/resources/golden/4.testCount-0-6ccae5fcb441ad90f14dff22de1aafc b/sql/hive/src/test/resources/golden/4.testCount-0-ef3bf1647dcb18d1f8e6107eab349807 similarity index 100% rename from sql/hive/src/test/resources/golden/4.testCount-0-6ccae5fcb441ad90f14dff22de1aafc rename to sql/hive/src/test/resources/golden/4.testCount-0-ef3bf1647dcb18d1f8e6107eab349807 diff --git a/sql/hive/src/test/resources/golden/44.testOverNoPartitionSingleAggregate-0-2fb3d43169e2f20680e7477f0daaddf4 b/sql/hive/src/test/resources/golden/44.testOverNoPartitionSingleAggregate-0-7776245f6859b7cba06d06f28ae577d similarity index 100% rename from sql/hive/src/test/resources/golden/44.testOverNoPartitionSingleAggregate-0-2fb3d43169e2f20680e7477f0daaddf4 rename to sql/hive/src/test/resources/golden/44.testOverNoPartitionSingleAggregate-0-7776245f6859b7cba06d06f28ae577d diff --git a/sql/hive/src/test/resources/golden/5.testCountWithWindowingUDAF-0-f22b5eae7b0255f15c480527036fd2b0 b/sql/hive/src/test/resources/golden/5.testCountWithWindowingUDAF-0-e703f883d590a9f3a897f5ae7c7b9cfb similarity index 100% rename from sql/hive/src/test/resources/golden/5.testCountWithWindowingUDAF-0-f22b5eae7b0255f15c480527036fd2b0 rename to sql/hive/src/test/resources/golden/5.testCountWithWindowingUDAF-0-e703f883d590a9f3a897f5ae7c7b9cfb diff --git a/sql/hive/src/test/resources/golden/6.testCountInSubQ-0-2e3e151f6ddc88c11149c72d884ba4bb b/sql/hive/src/test/resources/golden/6.testCountInSubQ-0-f6912910364e497344dea061fdecf258 similarity index 100% rename from sql/hive/src/test/resources/golden/6.testCountInSubQ-0-2e3e151f6ddc88c11149c72d884ba4bb rename to sql/hive/src/test/resources/golden/6.testCountInSubQ-0-f6912910364e497344dea061fdecf258 diff --git a/sql/hive/src/test/resources/golden/8.testMixedCaseAlias-0-1458fdd44340b7cf31a4c3980e53e643 b/sql/hive/src/test/resources/golden/8.testMixedCaseAlias-0-ac7c6f442dde6d5913e89a7881f35df8 similarity index 100% rename from sql/hive/src/test/resources/golden/8.testMixedCaseAlias-0-1458fdd44340b7cf31a4c3980e53e643 rename to sql/hive/src/test/resources/golden/8.testMixedCaseAlias-0-ac7c6f442dde6d5913e89a7881f35df8 diff --git a/sql/hive/src/test/resources/golden/9.testHavingWithWindowingNoGBY-0-6b117eafdd9731cee1007ae9719888a1 b/sql/hive/src/test/resources/golden/9.testHavingWithWindowingNoGBY-0-a04876c3478b374c339c33cc78a34150 similarity index 100% rename from sql/hive/src/test/resources/golden/9.testHavingWithWindowingNoGBY-0-6b117eafdd9731cee1007ae9719888a1 rename to sql/hive/src/test/resources/golden/9.testHavingWithWindowingNoGBY-0-a04876c3478b374c339c33cc78a34150 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala index 128901e82db5c..a0c0610f3b02d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala @@ -52,7 +52,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { createQueryTest("1.testWindowing", """ |SELECT p_mfgr, p_name, p_size, - |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, + |row_number() OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, |sum(p_retailprice) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN |unbounded preceding AND current row) AS s1 |FROM part @@ -61,15 +61,15 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { createQueryTest("4.testCount", """ |SELECT p_mfgr, p_name, - |count(p_size) over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS cd + |count(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS cd |FROM part """.stripMargin, false) createQueryTest("5.testCountWithWindowingUDAF", """ |SELECT p_mfgr, p_name, - |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, - |count(p_size) over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS cd, + |row_number() OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, + |count(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS cd, |p_retailprice, sum(p_retailprice) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows |BETWEEN unbounded preceding AND current row) AS s1, |p_size @@ -80,8 +80,8 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { """ |SELECT sub1.r, sub1.cd, sub1.s1 |FROM (SELECT p_mfgr, p_name, - |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, - |count(p_size) over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS cd, + |row_number() OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, + |count(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS cd, |p_retailprice, sum(p_retailprice) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows |BETWEEN unbounded preceding AND current row) AS s1, |p_size @@ -91,14 +91,14 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { createQueryTest("8.testMixedCaseAlias", """ |SELECT p_mfgr, p_name, p_size, - |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name, p_size desc) AS R + |row_number() OVER (DISTRIBUTE BY p_mfgr SORT BY p_name, p_size desc) AS R |FROM part """.stripMargin, false) createQueryTest("9.testHavingWithWindowingNoGBY", """ |SELECT p_mfgr, p_name, p_size, - |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, + |row_number() OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, |sum(p_retailprice) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN |unbounded preceding AND current row) AS s1 |FROM part @@ -119,7 +119,7 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { createQueryTest("12.testFirstLastWithWhere", """ |SELECT p_mfgr,p_name, p_size, - |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, + |row_number() OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS r, |sum(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN |current row AND current row) AS s2, |first_value(p_size) OVER w1 AS f, @@ -143,11 +143,37 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { createQueryTest("14.testNoSortClause", """ |SELECT p_mfgr,p_name, p_size, - |row_number() over(DISTRIBUTE BY p_mfgr SORT BY p_name) AS r + |row_number() OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS r |FROM part |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN 2 preceding AND 2 following) """.stripMargin, false) + createQueryTest("15.testExpressions", + """ + |SELECT p_mfgr,p_name, p_size, + |ntile(3) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS nt, + |count(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS ca, + |avg(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS avg, + |stddev(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS st, + |first_value(p_size % 5) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS fv, + |last_value(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS lv, + |first_value(p_size) OVER w1 AS fvW1 + |FROM part + |window w1 as + |(DISTRIBUTE BY p_mfgr SORT BY p_mfgr, p_name rows BETWEEN 2 preceding AND 2 following) + """.stripMargin, false) + + createQueryTest("16.testMultipleWindows", + """ + |SELECT p_mfgr,p_name, p_size, + |sum(p_size) OVER (DISTRIBUTE BY p_mfgr SORT BY p_size + |range BETWEEN 5 preceding AND current row) AS s2, + |first_value(p_size) OVER w1 AS fv1 + |FROM part + |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_mfgr, p_name + |rows BETWEEN 2 preceding AND 2 following) + """.stripMargin, false) + createQueryTest("18.testUDAFs", """ |SELECT p_mfgr,p_name, p_size, @@ -174,6 +200,17 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |2 preceding AND 2 following) """.stripMargin, false) + createQueryTest("21.testDISTs", + """ + |SELECT p_mfgr,p_name, p_size, + |histogram_numeric(p_retailprice, 5) OVER w1 AS hist, + |percentile(p_partkey, 0.5) OVER w1 AS per, + |row_number() OVER (DISTRIBUTE BY p_mfgr SORT BY p_name) AS rn + |FROM part + |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_mfgr, p_name + |rows BETWEEN 2 preceding AND 2 following) + """.stripMargin, false) + createQueryTest("27.testMultipleRangeWindows", """ |SELECT p_mfgr,p_name, p_size, @@ -202,6 +239,16 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |2 preceding AND 2 following) """.stripMargin, false) + createQueryTest("30.testDefaultPartitioningSpecRules", + """ + |SELECT p_mfgr, p_name, p_size, + |sum(p_size) OVER w1 AS s, + |sum(p_size) OVER w2 AS s2 + |FROM part + |window w1 AS (DISTRIBUTE BY p_mfgr SORT BY p_name rows BETWEEN 2 preceding AND 2 following), + | w2 AS (PARTITION BY p_mfgr ORDER BY p_name) + """.stripMargin, false) + createQueryTest("31.testWindowCrossReference", """ |SELECT p_mfgr, p_name, p_size, @@ -259,6 +306,17 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { |2 preceding AND 2 following) """.stripMargin, false) + createQueryTest("38.testPartitioningVariousForms2", + """ + |SELECT p_mfgr, p_name, p_size, + |sum(p_retailprice) OVER (PARTITION BY p_mfgr, p_name ORDER BY p_mfgr, p_name + |rows BETWEEN unbounded preceding AND current row) AS s1, + |min(p_retailprice) OVER (DISTRIBUTE BY p_mfgr, p_name SORT BY p_mfgr, p_name + |rows BETWEEN unbounded preceding AND current row) AS s2, + |max(p_retailprice) OVER (PARTITION BY p_mfgr, p_name ORDER BY p_name) AS s3 + |FROM part + """.stripMargin, false) + createQueryTest("39.testUDFOnOrderCols", """ |SELECT p_mfgr, p_type, substr(p_type, 2) AS short_ptype, @@ -300,15 +358,9 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { createQueryTest("44.testOverNoPartitionSingleAggregate", """ |SELECT p_name, p_retailprice, - |round(avg(p_retailprice) over(),2) + |round(avg(p_retailprice) OVER (),2) |FROM part |ORDER BY p_name """.stripMargin, false) - createQueryTest("ntile", - """ - |SELECT p_name, ntile(4) OVER (PARTITION BY p_mfgr ORDER BY p_size) FROM part - """.stripMargin, false) - - } From ab21933e64b3ee7afdcbb622bec935a34fe0785c Mon Sep 17 00:00:00 2001 From: guowei2 Date: Thu, 27 Nov 2014 16:40:26 +0800 Subject: [PATCH 10/14] fix DecimalType bug after rebase --- .../expressions/WindowAttribute.scala | 19 ++++++++++++++++++- .../spark/sql/execution/WindowFunction.scala | 4 ++-- .../org/apache/spark/sql/hive/HiveQl.scala | 5 +++-- 3 files changed, 23 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WindowAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WindowAttribute.scala index 63cb9fa432932..93272837639f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WindowAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WindowAttribute.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.apache.spark.sql.catalyst.trees @@ -40,4 +57,4 @@ case class WindowSpec(windowPartition: WindowPartition, windowFrame: WindowFrame case class WindowPartition(partitionBy: Seq[Expression], sortBy: Seq[SortOrder]) -case class WindowFrame(frameType:String, preceding: Int, following: Int) \ No newline at end of file +case class WindowFrame(frameType:String, preceding: Int, following: Int) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala index 0399cd757477d..3a97adf9e2997 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala @@ -189,7 +189,7 @@ case class WindowFunction( case DoubleType => Literal(windowFrame.preceding.toDouble) case FloatType => Literal(windowFrame.preceding.toFloat) case ShortType => Literal(windowFrame.preceding.toShort) - case DecimalType => Literal(BigDecimal(windowFrame.preceding)) + case DecimalType() => Literal(BigDecimal(windowFrame.preceding)) case _=> throw new Exception(s"not support dataType ") } val following = sortExpression.child.dataType match { @@ -198,7 +198,7 @@ case class WindowFunction( case DoubleType => Literal(windowFrame.following.toDouble) case FloatType => Literal(windowFrame.following.toFloat) case ShortType => Literal(windowFrame.following.toShort) - case DecimalType => Literal(BigDecimal(windowFrame.following)) + case DecimalType() => Literal(BigDecimal(windowFrame.following)) case _=> throw new Exception(s"not support dataType ") } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 7dd3bc5ab6a40..d8cdc231839eb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -971,8 +971,9 @@ private[hive] object HiveQl { else currentPlan } - currentPlan = WindowFunction(partitionExpr, computeExpressions.toSeq, otherExpressions, withWindowPartition) - attrExpressions ++= computeExpressions.map(_.toAttribute) + currentPlan = WindowFunction( + partitionExpr, computeExpressions.toSeq, otherExpressions, withWindowPartition) + attrExpressions ++= computeExpressions windowAttributes --= computeExpressions } From 66ef7a6d449f6ec1e644d2e73118d8be1cb56cde Mon Sep 17 00:00:00 2001 From: guowei2 Date: Fri, 28 Nov 2014 17:51:16 +0800 Subject: [PATCH 11/14] fix bug about attribute reference --- .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index d8cdc231839eb..61b8e0db96efc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -935,7 +935,8 @@ private[hive] object HiveQl { selectExpressions.foreach { sel => sel.collect { case win: WindowAttribute => windowAttributes += win - case attr: UnresolvedAttribute => attrExpressions += attr + case attr: UnresolvedAttribute => + if (!attrExpressions.contains(attr)) attrExpressions += attr } } @@ -971,6 +972,11 @@ private[hive] object HiveQl { else currentPlan } + (partitionExpr ++ sortExpr.map(_.child)).collect { + case attr: UnresolvedAttribute => + if (!otherExpressions.contains(attr)) otherExpressions += attr + } + currentPlan = WindowFunction( partitionExpr, computeExpressions.toSeq, otherExpressions, withWindowPartition) attrExpressions ++= computeExpressions From dc87d8d08c33644e61f6355ed07baf720b0e9ef9 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 4 Dec 2014 14:34:32 +0800 Subject: [PATCH 12/14] WIP: refactoring window functions support --- .../sql/catalyst/analysis/Analyzer.scala | 10 +- .../expressions/WindowAttribute.scala | 12 +- .../spark/sql/execution/WindowFunction.scala | 86 +++--- .../org/apache/spark/sql/hive/HiveQl.scala | 279 +++++++----------- 4 files changed, 154 insertions(+), 233 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d13bee39d80c6..7b15c457bef00 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -232,12 +232,10 @@ class Analyzer(catalog: Catalog, def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: WindowFunction => q transformExpressions { - case u @ WindowAttribute(children, name, windowSpec) + case u @ WindowAttribute(child: AggregateExpression, name, windowSpec) => // set window spec in AggregateExpression for execution and GlobalAggregates check - if (children.isInstanceOf[AggregateExpression]) => { - children.asInstanceOf[AggregateExpression].windowSpec = windowSpec - u - } + child.windowSpec = windowSpec + u } case q: LogicalPlan => q transformExpressions { @@ -258,7 +256,7 @@ class Analyzer(catalog: Catalog, def containsAggregates(exprs: Seq[Expression]): Boolean = { exprs.foreach(_.foreach { - case agg: AggregateExpression if (agg.windowSpec == null) => return true + case agg: AggregateExpression if agg.windowSpec == null => return true case _ => }) false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WindowAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WindowAttribute.scala index 93272837639f8..8d503fa82ccd8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WindowAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WindowAttribute.scala @@ -28,7 +28,8 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute * @param exprId A globally unique id used to check if an [[AttributeReference]] refers to this * alias. Auto-assigned if left blank. */ -case class WindowAttribute(child: Expression, name: String, windowSpec: WindowSpec) +case class WindowAttribute + (child: Expression, name: String, windowSpec: WindowSpec) (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends NamedExpression with trees.UnaryNode[Expression] { @@ -50,11 +51,14 @@ case class WindowAttribute(child: Expression, name: String, windowSpec: WindowSp override def toString: String = s"$child $windowSpec AS $name#${exprId.id}$typeSuffix" override protected final def otherCopyArgs = exprId :: qualifiers :: Nil - } -case class WindowSpec(windowPartition: WindowPartition, windowFrame: WindowFrame) +case class WindowSpec(windowPartition: WindowPartition, windowFrame: Option[WindowFrame]) case class WindowPartition(partitionBy: Seq[Expression], sortBy: Seq[SortOrder]) -case class WindowFrame(frameType:String, preceding: Int, following: Int) +sealed trait FrameType +case object ValueFrame extends FrameType +case object RowsFrame extends FrameType + +case class WindowFrame(frameType: FrameType, preceding: Int, following: Int) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala index 3a97adf9e2997..ef0f1ead78cdd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowFunction.scala @@ -20,20 +20,12 @@ package org.apache.spark.sql.execution import java.util.HashMap import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.physical.AllTuples -import org.apache.spark.sql.catalyst.plans.physical.ClusteredDistribution +import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.errors._ -import scala.collection.mutable.ArrayBuffer -import org.apache.spark.util.collection.CompactBuffer -import org.apache.spark.sql.catalyst.plans.physical.ClusteredDistribution -import org.apache.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection -import org.apache.spark.sql.catalyst.expressions.Alias +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, InterpretedMutableProjection, _} +import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution} import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.SortPartitions +import org.apache.spark.util.collection.CompactBuffer /** @@ -47,10 +39,10 @@ import org.apache.spark.sql.catalyst.plans.logical.SortPartitions */ @DeveloperApi case class WindowFunction( - partitionExpressions: Seq[Expression], - windowAttributes: Seq[WindowAttribute], - otherExpressions: Seq[NamedExpression], - child: SparkPlan) + partitionExpressions: Seq[Expression], + windowAttributes: Seq[WindowAttribute], + otherExpressions: Seq[NamedExpression], + child: SparkPlan) extends UnaryNode { override def requiredChildDistribution = @@ -78,8 +70,7 @@ case class WindowFunction( private[this] val resultAttributes = otherAttributes ++ computeAttributes.map(_._2) private[this] val resultMap = - (otherExpressions.map { other => other -> other.toAttribute } ++ computeAttributes - ).toMap + (otherExpressions.map { other => other -> other.toAttribute } ++ computeAttributes).toMap private[this] val resultExpressions = (windowAttributes ++ otherExpressions).map { sel => sel.transform { @@ -87,11 +78,10 @@ case class WindowFunction( } } - private[this] val sortExpressions = - if (child.isInstanceOf[Sort]) { - child.asInstanceOf[Sort].sortOrder - } - else null + private[this] val sortExpressions = child match { + case Sort(sortOrder, _, _) => sortOrder + case _ => null + } /** Creates a new function buffer for a partition. */ private[this] def newFunctionBuffer(): Array[AggregateFunction] = { @@ -114,16 +104,23 @@ case class WindowFunction( val aggrFunction = aggrFunctions(i) val base = aggrFunction.base val windowSpec = base.windowSpec - if (windowSpec.windowFrame == null) { + + windowSpec.windowFrame.map { frame => + functionResults(i) = frame.frameType match { + case RowsFrame => rowsWindowFunction(base, rows).iterator + case ValueFrame => valueWindowFunction(base, rows).iterator + } + }.getOrElse { if (sortExpressions != null) { - if (aggrFunction.dataType.isInstanceOf[ArrayType]) { - rows.foreach(aggrFunction.update) - functionResults(i) = aggrFunction.eval(EmptyRow).asInstanceOf[Seq[Any]].iterator - } else { - functionResults(i) = rows.map(row => { - aggrFunction.update(row) - aggrFunction.eval(EmptyRow) - }).iterator + aggrFunction.dataType match { + case _: ArrayType => + rows.foreach(aggrFunction.update) + functionResults(i) = aggrFunction.eval(EmptyRow).asInstanceOf[Seq[Any]].iterator + case _ => + functionResults(i) = rows.map { row => + aggrFunction.update(row) + aggrFunction.eval(EmptyRow) + }.iterator } } else { rows.foreach(aggrFunction.update) @@ -132,13 +129,6 @@ case class WindowFunction( case other => (0 to rows.size - 1).map(r => other).iterator } } - - } else { - functionResults(i) = - if (windowSpec.windowFrame.frameType == "ROWS_FRAME") { - rowsWindowFunction(base, rows).iterator - } - else valueWindowFunction(base, rows).iterator } i += 1 } @@ -151,8 +141,7 @@ case class WindowFunction( val rangeResults = new CompactBuffer[Any]() var rowIndex = 0 while (rowIndex < rows.size) { - - val windowFrame = base.windowSpec.windowFrame + val windowFrame = base.windowSpec.windowFrame.get var start = if (windowFrame.preceding == Int.MaxValue) 0 else rowIndex - windowFrame.preceding @@ -178,9 +167,9 @@ case class WindowFunction( private[this] def valueWindowFunction(base: AggregateExpression, rows: CompactBuffer[Row]): CompactBuffer[Any] = { - val windowFrame = base.windowSpec.windowFrame + val windowFrame = base.windowSpec.windowFrame.get - // rande only support 1 order + // range only support 1 order val sortExpression = BindReferences.bindReference(sortExpressions.head, childOutput) val preceding = sortExpression.child.dataType match { @@ -206,25 +195,24 @@ case class WindowFunction( var rowIndex = 0 while (rowIndex < rows.size) { val currentRow = rows(rowIndex) + val eval = sortExpression.child.eval(currentRow) val precedingExpr = if (sortExpression.direction == Ascending) { - Literal(sortExpression.child.eval(currentRow)) - sortExpression.child <= preceding + Literal(eval) - sortExpression.child <= preceding } else { - sortExpression.child - Literal(sortExpression.child.eval(currentRow)) <= preceding + sortExpression.child - Literal(eval) <= preceding } - val followingExpr = if (sortExpression.direction == Ascending) { - sortExpression.child - Literal(sortExpression.child.eval(currentRow)) <= following + sortExpression.child - Literal(eval) <= following } else { - Literal(sortExpression.child.eval(currentRow)) - sortExpression.child <= following + Literal(eval) - sortExpression.child <= following } var precedingIndex = 0 var followingIndex = rows.size - 1 if (sortExpression != null) { - if (windowFrame.preceding != Int.MaxValue) precedingIndex = rowIndex while (precedingIndex > 0 && precedingExpr.eval(rows(precedingIndex - 1)).asInstanceOf[Boolean]) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 61b8e0db96efc..bb089cf8c464d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.hive import java.sql.Date +import java.util.concurrent.atomic.AtomicInteger + import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.lib.Node @@ -34,8 +36,6 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.types.decimal.Decimal import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable -import java.util.concurrent.ConcurrentHashMap /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -550,7 +550,7 @@ private[hive] object HiveQl { singleInsert) } - initWindowDef(windowClause) + collectWindowDefs(windowClause) val relations = nodeToRelation(fromClause) val withWhere = whereClause.map { whereNode => @@ -614,15 +614,10 @@ private[hive] object HiveQl { // Not a transformation so must be either project or aggregation. val selectExpressions = nameExpressions(select.getChildren.flatMap(selExprNodeToExpr)) - //not support both window and group by yet - groupByClause match { - case Some(groupBy) => - Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withLateralView) - case None => { - val withWindowView = windowToPlan(selectExpressions, withLateralView) - Project(selectExpressions, withWindowView) - } - } + // Window with group by clause is not supported yet + groupByClause + .map(g => Aggregate(g.getChildren.map(nodeToExpr), selectExpressions, withLateralView)) + .getOrElse(Project(selectExpressions, windowToPlan(selectExpressions, withLateralView))) } val withDistinct = @@ -872,189 +867,129 @@ private[hive] object HiveQl { throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") } - // store the window def of current sql - // use thread id as key to avoid mistake when multi-sqls parse at the same time - protected val windowDefMap = new ConcurrentHashMap[Long, Map[String, Seq[ASTNode]]]() - - // store the window spec number of current sql - protected val windowNoMap = new ConcurrentHashMap[Long, Int]() - - protected def initWindowDef(windowClause: Option[Node]) = { - - var winDefs = Map[String, Seq[ASTNode]]() + protected val windowDefs = new ThreadLocal[Map[String, Seq[ASTNode]]] { + override def initialValue() = Map.empty[String, Seq[ASTNode]] + } - windowClause match { - case Some(window) => window.getChildren.foreach { - case Token("TOK_WINDOWDEF", Token(alias, Nil) :: Token("TOK_WINDOWSPEC", ws) :: Nil) => { - winDefs += alias -> ws - } - } - case None => // do nothing - } + protected val nextWindowSpecId: AtomicInteger = new AtomicInteger(0) - windowDefMap.put(Thread.currentThread().getId, winDefs) - windowNoMap.put(Thread.currentThread().getId, 0) - } + protected def collectWindowDefs(windowClause: Option[Node]) = { + val definitions = windowClause.toSeq.flatMap(_.getChildren.toSeq).collect { + case Token("TOK_WINDOWDEF", Token(alias, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => + alias -> spec + }.toMap - protected def nextWindowNo: Int = { - var no = windowNoMap.get(Thread.currentThread().getId) - no += 1 - windowNoMap.put(Thread.currentThread().getId, no) - no + windowDefs.set(definitions) } - protected def translateWindowSpec(windowSpec: Seq[ASTNode]): Seq[ASTNode]= { + protected def substituteWindowSpec(windowSpec: Seq[ASTNode]): Seq[ASTNode] = { windowSpec match { - case Token(alias, Nil) :: Nil => translateWindowSpec(getWindowSpec(alias)) - case Token(alias, Nil) :: frame => { - val (partitionClause :: rowsFrame :: valueFrame :: Nil) = getClauses( + case Token(alias, Nil) :: Nil => + substituteWindowSpec(getWindowSpec(alias)) + + case Token(alias, Nil) :: frame => + val (partitionClause :: _ /* range frame */ :: _ /* value frame */ :: Nil) = getClauses( Seq( "TOK_PARTITIONINGSPEC", "TOK_WINDOWRANGE", "TOK_WINDOWVALUES"), - translateWindowSpec(getWindowSpec(alias))) - partitionClause match { - case Some(partition) => partition.asInstanceOf[ASTNode] :: frame - case None => frame - } - } - case e => e - } - } - - protected def getWindowSpec(alias: String): Seq[ASTNode]= { - windowDefMap.get(Thread.currentThread().getId).getOrElse( - alias, sys.error("no window def for " + alias)) - } + substituteWindowSpec(getWindowSpec(alias))) - protected def windowToPlan(selectExpressions: Seq[NamedExpression], - withLateralView: LogicalPlan): LogicalPlan = { + partitionClause + .map(_.asInstanceOf[ASTNode] :: frame) + .getOrElse(frame) - val windowAttributes = new ArrayBuffer[WindowAttribute] - val attrExpressions = new ArrayBuffer[NamedExpression] - selectExpressions.foreach { sel => - sel.collect { - case win: WindowAttribute => windowAttributes += win - case attr: UnresolvedAttribute => - if (!attrExpressions.contains(attr)) attrExpressions += attr - } + case e => + e } + } - // Multi-step of WindowFunction will be needed, - // if there're multi-windowAttributes with different window partitions, - var currentPlan = withLateralView - while (!windowAttributes.isEmpty) { - - var currentPartition: WindowPartition = null - val computeExpressions = new ArrayBuffer[WindowAttribute] - val otherExpressions = attrExpressions.clone() - - windowAttributes.foreach(expr => { - val windowPartition = expr.windowSpec.windowPartition + protected def getWindowSpec(alias: String): Seq[ASTNode] = { + windowDefs.get().getOrElse(alias, sys.error(s"No window named $alias found.")) + } - if (currentPartition == null) { - computeExpressions += expr - currentPartition = windowPartition - } else if (currentPartition == windowPartition) { - computeExpressions += expr - } - }) + protected def windowToPlan( + selectExpressions: Seq[NamedExpression], + withLateralView: LogicalPlan): LogicalPlan = { + val windowExpressions = selectExpressions.flatMap(_.collect { case a: WindowAttribute => a }) + val attributes = selectExpressions.flatMap(_.collect { + case a: UnresolvedAttribute => a: NamedExpression + }) + + val windowPartitions = windowExpressions.map(_.windowSpec.windowPartition).distinct + val (restWindowExpressions, _, withWindow) = + windowPartitions.foldLeft((windowExpressions, attributes, withLateralView)) { + case ((expressions, propagatedAttrs, plan), part @ WindowPartition(partitionBy, sortBy)) => + val (computeExpressions, restWindowExpressions) = + expressions.partition(_.windowSpec.windowPartition == part) + + val withWindowPartition = (partitionBy, sortBy) match { + case (Nil, Nil) => plan + case (Nil, s) => Sort(s, plan) + case (p, Nil) => Repartition(p, plan) + case (p, s) => SortPartitions(s, Repartition(p, plan)) + } - val partitionExpr = currentPartition.partitionBy - val sortExpr = currentPartition.sortBy + val otherExpressions = (propagatedAttrs ++ (partitionBy ++ sortBy.map(_.child)).collect { + case a: UnresolvedAttribute => a + }).distinct - val withWindowPartition = - if (partitionExpr.size > 0) { - if (sortExpr.size > 0) SortPartitions(sortExpr, Repartition(partitionExpr, currentPlan)) - else Repartition(partitionExpr, currentPlan) - } else { - if (sortExpr.size > 0) Sort(sortExpr, currentPlan) - else currentPlan - } - - (partitionExpr ++ sortExpr.map(_.child)).collect { - case attr: UnresolvedAttribute => - if (!otherExpressions.contains(attr)) otherExpressions += attr + (restWindowExpressions, propagatedAttrs ++ computeExpressions, + WindowFunction(partitionBy, computeExpressions, otherExpressions, withWindowPartition)) } - currentPlan = WindowFunction( - partitionExpr, computeExpressions.toSeq, otherExpressions, withWindowPartition) - attrExpressions ++= computeExpressions - windowAttributes --= computeExpressions + assert(restWindowExpressions.isEmpty) - } - currentPlan + withWindow } - protected def checkWindowSpec(windowSpec: Seq[ASTNode]): WindowSpec = { - + protected def parseWindowSpec(windowSpec: Seq[ASTNode]): WindowSpec = { val (partitionClause :: rowsFrame :: valueFrame :: Nil) = getClauses( Seq( "TOK_PARTITIONINGSPEC", "TOK_WINDOWRANGE", "TOK_WINDOWVALUES"), - translateWindowSpec(windowSpec)) + substituteWindowSpec(windowSpec)) - val wp = partitionClause match { - case Some(partition) => { - val (orderByClause :: sortByClause :: distributeByClause :: clusterByClause :: Nil) = - getClauses( - Seq( - "TOK_ORDERBY", - "TOK_SORTBY", - "TOK_DISTRIBUTEBY", - "TOK_CLUSTERBY"), - partition.getChildren.toIndexedSeq.asInstanceOf[Seq[ASTNode]]) - - val partitionExpr = (distributeByClause orElse clusterByClause) match { - case Some(partitionBy) => partitionBy.getChildren.map(nodeToExpr) - case None => Seq() - } - - val sortByExpr = (orderByClause orElse sortByClause) match { - case Some(sortBy) => sortBy.getChildren.map(nodeToSortOrder) - case None => Seq() - } - WindowPartition(partitionExpr, sortByExpr) + val windowPartition = partitionClause.map { partition => + val (orderByClause :: sortByClause :: distributeByClause :: clusterByClause :: Nil) = + getClauses( + Seq( + "TOK_ORDERBY", + "TOK_SORTBY", + "TOK_DISTRIBUTEBY", + "TOK_CLUSTERBY"), + partition.getChildren.toSeq.asInstanceOf[Seq[ASTNode]]) + + val partitionBy = distributeByClause.orElse(clusterByClause).toSeq + val sortBy = orderByClause.orElse(sortByClause).toSeq + + WindowPartition( + partitionBy.flatMap(_.getChildren.map(nodeToExpr)), + sortBy.flatMap(_.getChildren.map(nodeToSortOrder))) + }.getOrElse(WindowPartition(Nil, Nil)) + + val maybeWindowFrame = rowsFrame.orElse(valueFrame).flatMap { frame => + val ranges = frame.getChildren.toList + val frameType = rowsFrame.map(_ => RowsFrame).getOrElse(ValueFrame) + + def nodeToBound(node: Node) = node match { + case Token("preceding" | "following", Token(count, Nil) :: Nil) => + if (count == "unbounded") Int.MaxValue else count.toInt + case _ => 0 } - case None => WindowPartition(Seq(), Seq()) - } - - - (rowsFrame orElse valueFrame) match { - case Some(frame) => { - val rangeSeq = frame.getChildren.toIndexedSeq - - if (rangeSeq.size > 0) { - - val preceding = rangeSeq.get(0) match { - case Token("preceding", Token(name, Nil) :: Nil) => - if (name == "unbounded") Int.MaxValue - else name.toInt - case Token("current", Nil) => 0 - case _ => 0 - } - val following = if (rangeSeq.size > 1) { - rangeSeq.get(1) match { - case Token("following", Token(name, Nil) :: Nil) => - if (name == "unbounded") Int.MaxValue - else name.toInt - case Token("current", Nil) => 0 - case _ => 0 - } - } else 0 - - if (rowsFrame.isDefined) { - WindowSpec(wp, WindowFrame("ROWS_FRAME", preceding, following)) - } else { - WindowSpec(wp, WindowFrame("VALUE_FRAME", preceding, following)) - } - - } else WindowSpec(wp, null) + ranges match { + case precedingNode :: followingNode :: _ => + Some(WindowFrame(frameType, nodeToBound(precedingNode), nodeToBound(followingNode))) + case precedingNode :: Nil => + Some(WindowFrame(frameType, nodeToBound(precedingNode), 0)) + case Nil => + None } - case None => WindowSpec(wp, null) } + + WindowSpec(windowPartition, maybeWindowFrame) } protected val escapedIdentifier = "`([^`]+)`".r @@ -1249,17 +1184,13 @@ private[hive] object HiveQl { Substring(nodeToExpr(string), nodeToExpr(pos), nodeToExpr(length)) /* UDFs - Must be last otherwise will preempt built in functions */ - case Token("TOK_FUNCTION", Token(name, Nil) :: args) => { - val exprs = new ArrayBuffer[Node] - var windowSpec: WindowSpec = null - args.foreach { - case Token("TOK_WINDOWSPEC", winSpec) => windowSpec = checkWindowSpec(winSpec) - case a: ASTNode => exprs += a - } - if (windowSpec == null) UnresolvedFunction(name, exprs.map(nodeToExpr)) - else WindowAttribute(UnresolvedFunction(name, exprs.map(nodeToExpr)), - s"w_" + nextWindowNo, windowSpec)() - } + case Token("TOK_FUNCTION", Token(name, Nil) :: tail) => + val (specNodes, argNodes) = tail.partition(_.getText == "TOK_WINDOWSPEC") + val maybeWindowSpec = specNodes.collectFirst { case Token(_, spec) => parseWindowSpec(spec) } + val function = UnresolvedFunction(name, argNodes.map(nodeToExpr)) + maybeWindowSpec + .map(WindowAttribute(function, s"w_${nextWindowSpecId.getAndIncrement}", _)()) + .getOrElse(function) case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: args) => UnresolvedFunction(name, Star(None) :: Nil) From 2da61753590fe00ecf46219f387d70d48c6dd32a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 15 Dec 2014 14:42:57 +0800 Subject: [PATCH 13/14] Removed trailing spaces from query string in HiveWindowFunctionSuite --- ...ningVariousForms2-0-e3e5b0b7ab5e8395320886ef4711a0ee} | 0 .../sql/hive/execution/HiveWindowFunctionSuite.scala | 9 +++------ 2 files changed, 3 insertions(+), 6 deletions(-) rename sql/hive/src/test/resources/golden/{38.testPartitioningVariousForms2-0-a8b8fcdfb6e4b29f8e6c1a3f4dd595d1 => 38.testPartitioningVariousForms2-0-e3e5b0b7ab5e8395320886ef4711a0ee} (100%) diff --git a/sql/hive/src/test/resources/golden/38.testPartitioningVariousForms2-0-a8b8fcdfb6e4b29f8e6c1a3f4dd595d1 b/sql/hive/src/test/resources/golden/38.testPartitioningVariousForms2-0-e3e5b0b7ab5e8395320886ef4711a0ee similarity index 100% rename from sql/hive/src/test/resources/golden/38.testPartitioningVariousForms2-0-a8b8fcdfb6e4b29f8e6c1a3f4dd595d1 rename to sql/hive/src/test/resources/golden/38.testPartitioningVariousForms2-0-e3e5b0b7ab5e8395320886ef4711a0ee diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala index a0c0610f3b02d..576bcf622fe90 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionSuite.scala @@ -17,10 +17,7 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive._ -import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -import org.apache.spark.sql.{Row, SchemaRDD} class HiveWindowFunctionSuite extends HiveComparisonTest { @@ -309,14 +306,14 @@ class HiveWindowFunctionSuite extends HiveComparisonTest { createQueryTest("38.testPartitioningVariousForms2", """ |SELECT p_mfgr, p_name, p_size, - |sum(p_retailprice) OVER (PARTITION BY p_mfgr, p_name ORDER BY p_mfgr, p_name + |sum(p_retailprice) OVER (PARTITION BY p_mfgr, p_name ORDER BY p_mfgr, p_name |rows BETWEEN unbounded preceding AND current row) AS s1, - |min(p_retailprice) OVER (DISTRIBUTE BY p_mfgr, p_name SORT BY p_mfgr, p_name + |min(p_retailprice) OVER (DISTRIBUTE BY p_mfgr, p_name SORT BY p_mfgr, p_name |rows BETWEEN unbounded preceding AND current row) AS s2, |max(p_retailprice) OVER (PARTITION BY p_mfgr, p_name ORDER BY p_name) AS s3 |FROM part """.stripMargin, false) - + createQueryTest("39.testUDFOnOrderCols", """ |SELECT p_mfgr, p_type, substr(p_type, 2) AS short_ptype, From 922a8b9bfe0278577378c3cd9fc13cb9998b6e0f Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 16 Dec 2014 01:18:42 +0800 Subject: [PATCH 14/14] Fixes COUNT with window spec --- .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index bb089cf8c464d..f63fe90c0b029 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -1054,6 +1054,10 @@ private[hive] object HiveQl { /* Aggregate Functions */ case Token("TOK_FUNCTION", Token(AVG(), Nil) :: arg :: Nil) => Average(nodeToExpr(arg)) case Token("TOK_FUNCTION", Token(COUNT(), Nil) :: arg :: Nil) => Count(nodeToExpr(arg)) + case Token("TOK_FUNCTION", + Token(COUNT(), Nil) :: arg :: Token("TOK_WINDOWSPEC", spec) :: Nil) => + val count = Count(nodeToExpr(arg)) + WindowAttribute(count, s"w_${nextWindowSpecId.getAndIncrement}", parseWindowSpec(spec))() case Token("TOK_FUNCTIONSTAR", Token(COUNT(), Nil) :: Nil) => Count(Literal(1)) case Token("TOK_FUNCTIONDI", Token(COUNT(), Nil) :: args) => CountDistinct(args.map(nodeToExpr)) case Token("TOK_FUNCTION", Token(SUM(), Nil) :: arg :: Nil) => Sum(nodeToExpr(arg))