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 5e42b409dcc59645d9c96b2a8126b70c06ad923e..7b543b6c2aa42316084c36bd854648f01c32070e 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ @@ -61,6 +63,7 @@ class Analyzer( ResolveGenerate :: ImplicitGenerate :: ResolveFunctions :: + ExtractWindowExpressions :: GlobalAggregates :: UnresolvedHavingClauseAttributes :: TrimGroupingAliases :: @@ -529,6 +532,203 @@ class Analyzer( makeGeneratorOutput(p.generator, p.generatorOutput), p.child) } } + + /** + * Extracts [[WindowExpression]]s from the projectList of a [[Project]] operator and + * aggregateExpressions of an [[Aggregate]] operator and creates individual [[Window]] + * operators for every distinct [[WindowSpecDefinition]]. + * + * This rule handles three cases: + * - A [[Project]] having [[WindowExpression]]s in its projectList; + * - An [[Aggregate]] having [[WindowExpression]]s in its aggregateExpressions. + * - An [[Filter]]->[[Aggregate]] pattern representing GROUP BY with a HAVING + * clause and the [[Aggregate]] has [[WindowExpression]]s in its aggregateExpressions. + * Note: If there is a GROUP BY clause in the query, aggregations and corresponding + * filters (expressions in the HAVING clause) should be evaluated before any + * [[WindowExpression]]. If a query has SELECT DISTINCT, the DISTINCT part should be + * evaluated after all [[WindowExpression]]s. + * + * For every case, the transformation works as follows: + * 1. For a list of [[Expression]]s (a projectList or an aggregateExpressions), partitions + * it two lists of [[Expression]]s, one for all [[WindowExpression]]s and another for + * all regular expressions. + * 2. For all [[WindowExpression]]s, groups them based on their [[WindowSpecDefinition]]s. + * 3. For every distinct [[WindowSpecDefinition]], creates a [[Window]] operator and inserts + * it into the plan tree. + */ + object ExtractWindowExpressions extends Rule[LogicalPlan] { + def hasWindowFunction(projectList: Seq[NamedExpression]): Boolean = + projectList.exists(hasWindowFunction) + + def hasWindowFunction(expr: NamedExpression): Boolean = { + expr.find { + case window: WindowExpression => true + case _ => false + }.isDefined + } + + /** + * From a Seq of [[NamedExpression]]s, extract window expressions and + * other regular expressions. + */ + def extract( + expressions: Seq[NamedExpression]): (Seq[NamedExpression], Seq[NamedExpression]) = { + // First, we simple partition the input expressions to two part, one having + // WindowExpressions and another one without WindowExpressions. + val (windowExpressions, regularExpressions) = expressions.partition(hasWindowFunction) + + // Then, we need to extract those regular expressions used in the WindowExpression. + // For example, when we have col1 - Sum(col2 + col3) OVER (PARTITION BY col4 ORDER BY col5), + // we need to make sure that col1 to col5 are all projected from the child of the Window + // operator. + val extractedExprBuffer = new ArrayBuffer[NamedExpression]() + def extractExpr(expr: Expression): Expression = expr match { + case ne: NamedExpression => + // If a named expression is not in regularExpressions, add extract it and replace it + // with an AttributeReference. + val missingExpr = + AttributeSet(Seq(expr)) -- (regularExpressions ++ extractedExprBuffer) + if (missingExpr.nonEmpty) { + extractedExprBuffer += ne + } + ne.toAttribute + case e: Expression if e.foldable => + e // No need to create an attribute reference if it will be evaluated as a Literal. + case e: Expression => + // For other expressions, we extract it and replace it with an AttributeReference (with + // an interal column name, e.g. "_w0"). + val withName = Alias(e, s"_w${extractedExprBuffer.length}")() + extractedExprBuffer += withName + withName.toAttribute + } + + // Now, we extract expressions from windowExpressions by using extractExpr. + val newWindowExpressions = windowExpressions.map { + _.transform { + // Extracts children expressions of a WindowFunction (input parameters of + // a WindowFunction). + case wf : WindowFunction => + val newChildren = wf.children.map(extractExpr(_)) + wf.withNewChildren(newChildren) + + // Extracts expressions from the partition spec and order spec. + case wsc @ WindowSpecDefinition(partitionSpec, orderSpec, _) => + val newPartitionSpec = partitionSpec.map(extractExpr(_)) + val newOrderSpec = orderSpec.map { so => + val newChild = extractExpr(so.child) + so.copy(child = newChild) + } + wsc.copy(partitionSpec = newPartitionSpec, orderSpec = newOrderSpec) + + // Extracts AggregateExpression. For example, for SUM(x) - Sum(y) OVER (...), + // we need to extract SUM(x). + case agg: AggregateExpression => + val withName = Alias(agg, s"_w${extractedExprBuffer.length}")() + extractedExprBuffer += withName + withName.toAttribute + }.asInstanceOf[NamedExpression] + } + + (newWindowExpressions, regularExpressions ++ extractedExprBuffer) + } + + /** + * Adds operators for Window Expressions. Every Window operator handles a single Window Spec. + */ + def addWindow(windowExpressions: Seq[NamedExpression], child: LogicalPlan): LogicalPlan = { + // First, we group window expressions based on their Window Spec. + val groupedWindowExpression = windowExpressions.groupBy { expr => + val windowExpression = expr.find { + case window: WindowExpression => true + case other => false + }.map(_.asInstanceOf[WindowExpression].windowSpec) + windowExpression.getOrElse( + failAnalysis(s"$windowExpressions does not have any WindowExpression.")) + }.toSeq + + // For every Window Spec, we add a Window operator and set currentChild as the child of it. + var currentChild = child + var i = 0 + while (i < groupedWindowExpression.size) { + val (windowSpec, windowExpressions) = groupedWindowExpression(i) + // Set currentChild to the newly created Window operator. + currentChild = Window(currentChild.output, windowExpressions, windowSpec, currentChild) + + // Move to next WindowExpression. + i += 1 + } + + // We return the top operator. + currentChild + } + + // We have to use transformDown at here to make sure the rule of + // "Aggregate with Having clause" will be triggered. + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + // Lookup WindowSpecDefinitions. This rule works with unresolved children. + case WithWindowDefinition(windowDefinitions, child) => + child.transform { + case plan => plan.transformExpressions { + case UnresolvedWindowExpression(c, WindowSpecReference(windowName)) => + val errorMessage = + s"Window specification $windowName is not defined in the WINDOW clause." + val windowSpecDefinition = + windowDefinitions + .get(windowName) + .getOrElse(failAnalysis(errorMessage)) + WindowExpression(c, windowSpecDefinition) + } + } + + // Aggregate with Having clause. This rule works with an unresolved Aggregate because + // a resolved Aggregate will not have Window Functions. + case f @ Filter(condition, a @ Aggregate(groupingExprs, aggregateExprs, child)) + if child.resolved && + hasWindowFunction(aggregateExprs) && + !a.expressions.exists(!_.resolved) => + val (windowExpressions, aggregateExpressions) = extract(aggregateExprs) + // Create an Aggregate operator to evaluate aggregation functions. + val withAggregate = Aggregate(groupingExprs, aggregateExpressions, child) + // Add a Filter operator for conditions in the Having clause. + val withFilter = Filter(condition, withAggregate) + val withWindow = addWindow(windowExpressions, withFilter) + + // Finally, generate output columns according to the original projectList. + val finalProjectList = aggregateExprs.map (_.toAttribute) + Project(finalProjectList, withWindow) + + case p: LogicalPlan if !p.childrenResolved => p + + // Aggregate without Having clause. + case a @ Aggregate(groupingExprs, aggregateExprs, child) + if hasWindowFunction(aggregateExprs) && + !a.expressions.exists(!_.resolved) => + val (windowExpressions, aggregateExpressions) = extract(aggregateExprs) + // Create an Aggregate operator to evaluate aggregation functions. + val withAggregate = Aggregate(groupingExprs, aggregateExpressions, child) + // Add Window operators. + val withWindow = addWindow(windowExpressions, withAggregate) + + // Finally, generate output columns according to the original projectList. + val finalProjectList = aggregateExprs.map (_.toAttribute) + Project(finalProjectList, withWindow) + + // We only extract Window Expressions after all expressions of the Project + // have been resolved. + case p @ Project(projectList, child) + if hasWindowFunction(projectList) && !p.expressions.exists(!_.resolved) => + val (windowExpressions, regularExpressions) = extract(projectList) + // We add a project to get all needed expressions for window expressions from the child + // of the original Project operator. + val withProject = Project(regularExpressions, child) + // Add Window operators. + val withWindow = addWindow(windowExpressions, withProject) + + // Finally, generate output columns according to the original projectList. + val finalProjectList = projectList.map (_.toAttribute) + Project(finalProjectList, withWindow) + } + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 2381689e17525c4f75bf6c03780508e217239804..c8288c6767004a6f165a70a1b4ffe5b3748936c4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -70,6 +70,11 @@ trait CheckAnalysis { failAnalysis( s"invalid expression ${b.prettyString} " + s"between ${b.left.simpleString} and ${b.right.simpleString}") + + case w @ WindowExpression(windowFunction, windowSpec) if windowSpec.validate.nonEmpty => + // The window spec is not valid. + val reason = windowSpec.validate.get + failAnalysis(s"Window specification $windowSpec is not valid because $reason") } operator match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index c2866cd955409379501abfdf33c1eccb62a87b9d..8cae548279eb166c5884998eed90e5116fb0659f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -548,3 +548,97 @@ class JoinedRow5 extends Row { } } } + +/** + * JIT HACK: Replace with macros + */ +class JoinedRow6 extends Row { + private[this] var row1: Row = _ + private[this] var row2: Row = _ + + def this(left: Row, right: Row) = { + this() + row1 = left + row2 = right + } + + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ + def apply(r1: Row, r2: Row): Row = { + row1 = r1 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq + + override def length: Int = row1.length + row2.length + + override def apply(i: Int): Any = + if (i < row1.length) row1(i) else row2(i - row1.length) + + override def isNullAt(i: Int): Boolean = + if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length) + + override def getInt(i: Int): Int = + if (i < row1.length) row1.getInt(i) else row2.getInt(i - row1.length) + + override def getLong(i: Int): Long = + if (i < row1.length) row1.getLong(i) else row2.getLong(i - row1.length) + + override def getDouble(i: Int): Double = + if (i < row1.length) row1.getDouble(i) else row2.getDouble(i - row1.length) + + override def getBoolean(i: Int): Boolean = + if (i < row1.length) row1.getBoolean(i) else row2.getBoolean(i - row1.length) + + override def getShort(i: Int): Short = + if (i < row1.length) row1.getShort(i) else row2.getShort(i - row1.length) + + override def getByte(i: Int): Byte = + if (i < row1.length) row1.getByte(i) else row2.getByte(i - row1.length) + + override def getFloat(i: Int): Float = + if (i < row1.length) row1.getFloat(i) else row2.getFloat(i - row1.length) + + override def getString(i: Int): String = + if (i < row1.length) row1.getString(i) else row2.getString(i - row1.length) + + override def getAs[T](i: Int): T = + if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) + + override def copy(): Row = { + val totalSize = row1.length + row2.length + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString: String = { + // Make sure toString never throws NullPointerException. + if ((row1 eq null) && (row2 eq null)) { + "[ empty row ]" + } else if (row1 eq null) { + row2.mkString("[", ",", "]") + } else if (row2 eq null) { + row1.mkString("[", ",", "]") + } else { + mkString("[", ",", "]") + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala new file mode 100644 index 0000000000000000000000000000000000000000..099d67ca7fee3a3d0ae09ed0795c7715f726cef5 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -0,0 +1,340 @@ +/* + * 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.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types.{NumericType, DataType} + +/** + * The trait of the Window Specification (specified in the OVER clause or WINDOW clause) for + * Window Functions. + */ +sealed trait WindowSpec + +/** + * The specification for a window function. + * @param partitionSpec It defines the way that input rows are partitioned. + * @param orderSpec It defines the ordering of rows in a partition. + * @param frameSpecification It defines the window frame in a partition. + */ +case class WindowSpecDefinition( + partitionSpec: Seq[Expression], + orderSpec: Seq[SortOrder], + frameSpecification: WindowFrame) extends Expression with WindowSpec { + + def validate: Option[String] = frameSpecification match { + case UnspecifiedFrame => + Some("Found a UnspecifiedFrame. It should be converted to a SpecifiedWindowFrame " + + "during analysis. Please file a bug report.") + case frame: SpecifiedWindowFrame => frame.validate.orElse { + def checkValueBasedBoundaryForRangeFrame(): Option[String] = { + if (orderSpec.length > 1) { + // It is not allowed to have a value-based PRECEDING and FOLLOWING + // as the boundary of a Range Window Frame. + Some("This Range Window Frame only accepts at most one ORDER BY expression.") + } else if (orderSpec.nonEmpty && !orderSpec.head.dataType.isInstanceOf[NumericType]) { + Some("The data type of the expression in the ORDER BY clause should be a numeric type.") + } else { + None + } + } + + (frame.frameType, frame.frameStart, frame.frameEnd) match { + case (RangeFrame, vp: ValuePreceding, _) => checkValueBasedBoundaryForRangeFrame() + case (RangeFrame, vf: ValueFollowing, _) => checkValueBasedBoundaryForRangeFrame() + case (RangeFrame, _, vp: ValuePreceding) => checkValueBasedBoundaryForRangeFrame() + case (RangeFrame, _, vf: ValueFollowing) => checkValueBasedBoundaryForRangeFrame() + case (_, _, _) => None + } + } + } + + type EvaluatedType = Any + + override def children: Seq[Expression] = partitionSpec ++ orderSpec + + override lazy val resolved: Boolean = + childrenResolved && frameSpecification.isInstanceOf[SpecifiedWindowFrame] + + + override def toString: String = simpleString + + override def eval(input: Row): EvaluatedType = throw new UnsupportedOperationException + override def nullable: Boolean = true + override def foldable: Boolean = false + override def dataType: DataType = throw new UnsupportedOperationException +} + +/** + * A Window specification reference that refers to the [[WindowSpecDefinition]] defined + * under the name `name`. + */ +case class WindowSpecReference(name: String) extends WindowSpec + +/** + * The trait used to represent the type of a Window Frame. + */ +sealed trait FrameType + +/** + * RowFrame treats rows in a partition individually. When a [[ValuePreceding]] + * or a [[ValueFollowing]] is used as its [[FrameBoundary]], the value is considered + * as a physical offset. + * For example, `ROW BETWEEN 1 PRECEDING AND 1 FOLLOWING` represents a 3-row frame, + * from the row precedes the current row to the row follows the current row. + */ +case object RowFrame extends FrameType + +/** + * RangeFrame treats rows in a partition as groups of peers. + * All rows having the same `ORDER BY` ordering are considered as peers. + * When a [[ValuePreceding]] or a [[ValueFollowing]] is used as its [[FrameBoundary]], + * the value is considered as a logical offset. + * For example, assuming the value of the current row's `ORDER BY` expression `expr` is `v`, + * `RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING` represents a frame containing rows whose values + * `expr` are in the range of [v-1, v+1]. + * + * If `ORDER BY` clause is not defined, all rows in the partition is considered as peers + * of the current row. + */ +case object RangeFrame extends FrameType + +/** + * The trait used to represent the type of a Window Frame Boundary. + */ +sealed trait FrameBoundary { + def notFollows(other: FrameBoundary): Boolean +} + +/** UNBOUNDED PRECEDING boundary. */ +case object UnboundedPreceding extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => true + case vp: ValuePreceding => true + case CurrentRow => true + case vf: ValueFollowing => true + case UnboundedFollowing => true + } + + override def toString: String = "UNBOUNDED PRECEDING" +} + +/** <value> PRECEDING boundary. */ +case class ValuePreceding(value: Int) extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case ValuePreceding(anotherValue) => value >= anotherValue + case CurrentRow => true + case vf: ValueFollowing => true + case UnboundedFollowing => true + } + + override def toString: String = s"$value PRECEDING" +} + +/** CURRENT ROW boundary. */ +case object CurrentRow extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case vp: ValuePreceding => false + case CurrentRow => true + case vf: ValueFollowing => true + case UnboundedFollowing => true + } + + override def toString: String = "CURRENT ROW" +} + +/** <value> FOLLOWING boundary. */ +case class ValueFollowing(value: Int) extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case vp: ValuePreceding => false + case CurrentRow => false + case ValueFollowing(anotherValue) => value <= anotherValue + case UnboundedFollowing => true + } + + override def toString: String = s"$value FOLLOWING" +} + +/** UNBOUNDED FOLLOWING boundary. */ +case object UnboundedFollowing extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case vp: ValuePreceding => false + case CurrentRow => false + case vf: ValueFollowing => false + case UnboundedFollowing => true + } + + override def toString: String = "UNBOUNDED FOLLOWING" +} + +/** + * The trait used to represent the a Window Frame. + */ +sealed trait WindowFrame + +/** Used as a place holder when a frame specification is not defined. */ +case object UnspecifiedFrame extends WindowFrame + +/** A specified Window Frame. */ +case class SpecifiedWindowFrame( + frameType: FrameType, + frameStart: FrameBoundary, + frameEnd: FrameBoundary) extends WindowFrame { + + /** If this WindowFrame is valid or not. */ + def validate: Option[String] = (frameType, frameStart, frameEnd) match { + case (_, UnboundedFollowing, _) => + Some(s"$UnboundedFollowing is not allowed as the start of a Window Frame.") + case (_, _, UnboundedPreceding) => + Some(s"$UnboundedPreceding is not allowed as the end of a Window Frame.") + // case (RowFrame, start, end) => ??? RowFrame specific rule + // case (RangeFrame, start, end) => ??? RangeFrame specific rule + case (_, start, end) => + if (start.notFollows(end)) { + None + } else { + val reason = + s"The end of this Window Frame $end is smaller than the start of " + + s"this Window Frame $start." + Some(reason) + } + } + + override def toString: String = frameType match { + case RowFrame => s"ROWS BETWEEN $frameStart AND $frameEnd" + case RangeFrame => s"RANGE BETWEEN $frameStart AND $frameEnd" + } +} + +object SpecifiedWindowFrame { + /** + * + * @param hasOrderSpecification If the window spec has order by expressions. + * @param acceptWindowFrame If the window function accepts user-specified frame. + * @return + */ + def defaultWindowFrame( + hasOrderSpecification: Boolean, + acceptWindowFrame: Boolean): SpecifiedWindowFrame = { + if (hasOrderSpecification && acceptWindowFrame) { + // If order spec is defined and the window function supports user specified window frames, + // the default frame is RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW. + SpecifiedWindowFrame(RangeFrame, UnboundedPreceding, CurrentRow) + } else { + // Otherwise, the default frame is + // ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING. + SpecifiedWindowFrame(RowFrame, UnboundedPreceding, UnboundedFollowing) + } + } +} + +/** + * Every window function needs to maintain a output buffer for its output. + * It should expect that for a n-row window frame, it will be called n times + * to retrieve value corresponding with these n rows. + */ +trait WindowFunction extends Expression { + self: Product => + + def init(): Unit + + def reset(): Unit + + def prepareInputParameters(input: Row): AnyRef + + def update(input: AnyRef): Unit + + def batchUpdate(inputs: Array[AnyRef]): Unit + + def evaluate(): Unit + + def get(index: Int): Any + + def newInstance(): WindowFunction +} + +case class UnresolvedWindowFunction( + name: String, + children: Seq[Expression]) + extends Expression with WindowFunction { + + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def foldable: Boolean = throw new UnresolvedException(this, "foldable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override lazy val resolved = false + + override def init(): Unit = + throw new UnresolvedException(this, "init") + override def reset(): Unit = + throw new UnresolvedException(this, "reset") + override def prepareInputParameters(input: Row): AnyRef = + throw new UnresolvedException(this, "prepareInputParameters") + override def update(input: AnyRef): Unit = + throw new UnresolvedException(this, "update") + override def batchUpdate(inputs: Array[AnyRef]): Unit = + throw new UnresolvedException(this, "batchUpdate") + override def evaluate(): Unit = + throw new UnresolvedException(this, "evaluate") + override def get(index: Int): Any = + throw new UnresolvedException(this, "get") + // Unresolved functions are transient at compile time and don't get evaluated during execution. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + + override def toString: String = s"'$name(${children.mkString(",")})" + + override def newInstance(): WindowFunction = + throw new UnresolvedException(this, "newInstance") +} + +case class UnresolvedWindowExpression( + child: UnresolvedWindowFunction, + windowSpec: WindowSpecReference) extends UnaryExpression { + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def foldable: Boolean = throw new UnresolvedException(this, "foldable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override lazy val resolved = false + + // Unresolved functions are transient at compile time and don't get evaluated during execution. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") +} + +case class WindowExpression( + windowFunction: WindowFunction, + windowSpec: WindowSpecDefinition) extends Expression { + override type EvaluatedType = Any + + override def children: Seq[Expression] = + windowFunction :: windowSpec :: Nil + + override def eval(input: Row): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + + override def dataType: DataType = windowFunction.dataType + override def foldable: Boolean = windowFunction.foldable + override def nullable: Boolean = windowFunction.nullable + + override def toString: String = s"$windowFunction $windowSpec" +} 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 21208c8a5c281df31735bb095beb57ffacbf1209..ba0abb2df596c03d7ad8455a365965ebbd03ead3 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 @@ -25,13 +25,14 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend override def output: Seq[Attribute] = projectList.map(_.toAttribute) override lazy val resolved: Boolean = { - val containsAggregatesOrGenerators = projectList.exists ( _.collect { + val hasSpecialExpressions = projectList.exists ( _.collect { case agg: AggregateExpression => agg case generator: Generator => generator + case window: WindowExpression => window }.nonEmpty ) - !expressions.exists(!_.resolved) && childrenResolved && !containsAggregatesOrGenerators + !expressions.exists(!_.resolved) && childrenResolved && !hasSpecialExpressions } } @@ -170,6 +171,12 @@ case class With(child: LogicalPlan, cteRelations: Map[String, Subquery]) extends override def output: Seq[Attribute] = child.output } +case class WithWindowDefinition( + windowDefinitions: Map[String, WindowSpecDefinition], + child: LogicalPlan) extends UnaryNode { + override def output: Seq[Attribute] = child.output +} + case class WriteToFile( path: String, child: LogicalPlan) extends UnaryNode { @@ -195,9 +202,28 @@ case class Aggregate( child: LogicalPlan) extends UnaryNode { + override lazy val resolved: Boolean = { + val hasWindowExpressions = aggregateExpressions.exists ( _.collect { + case window: WindowExpression => window + }.nonEmpty + ) + + !expressions.exists(!_.resolved) && childrenResolved && !hasWindowExpressions + } + override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute) } +case class Window( + projectList: Seq[Attribute], + windowExpressions: Seq[NamedExpression], + windowSpec: WindowSpecDefinition, + child: LogicalPlan) extends UnaryNode { + + override def output: Seq[Attribute] = + (projectList ++ windowExpressions).map(_.toAttribute) +} + /** * Apply the all of the GroupExpressions to every input row, hence we will get * multiple output rows for a input row. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 97502ed3afe72af3c829a1ba1741e9b894b834c1..4b93f7d31b8089f9fa5643029f8985f6d0b0330c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -71,6 +71,15 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { this.eq(other) || this == other } + /** + * Find the first [[TreeNode]] that satisfies the condition specified by `f`. + * The condition is recursively applied to this node and all of its children (pre-order). + */ + def find(f: BaseType => Boolean): Option[BaseType] = f(this) match { + case true => Some(this) + case false => children.foldLeft(None: Option[BaseType]) { (l, r) => l.orElse(r.find(f)) } + } + /** * Runs the given function on this node and then recursively on [[children]]. * @param f the function to be applied to each node in the tree. @@ -151,6 +160,20 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { val remainingNewChildren = newChildren.toBuffer val remainingOldChildren = children.toBuffer val newArgs = productIterator.map { + // This rule is used to handle children is a input argument. + case s: Seq[_] => s.map { + case arg: TreeNode[_] if children contains arg => + val newChild = remainingNewChildren.remove(0) + val oldChild = remainingOldChildren.remove(0) + if (newChild fastEquals oldChild) { + oldChild + } else { + changed = true + newChild + } + case nonChild: AnyRef => nonChild + case null => null + } case arg: TreeNode[_] if children contains arg => val newChild = remainingNewChildren.remove(0) val oldChild = remainingOldChildren.remove(0) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 6b393327cc97afc70588491b766225eb395c9b87..786ddba403f2c28e33ed634421d88dce69784313 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{StringType, NullType} +import org.apache.spark.sql.types.{IntegerType, StringType, NullType} case class Dummy(optKey: Option[Expression]) extends Expression { def children: Seq[Expression] = optKey.toSeq @@ -129,5 +129,47 @@ class TreeNodeSuite extends FunSuite { assert(expected === actual) } + test("find") { + val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) + // Find the top node. + var actual: Option[Expression] = expression.find { + case add: Add => true + case other => false + } + var expected: Option[Expression] = + Some(Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4))))) + assert(expected === actual) + + // Find the first children. + actual = expression.find { + case Literal(1, IntegerType) => true + case other => false + } + expected = Some(Literal(1)) + assert(expected === actual) + // Find an internal node (Subtract). + actual = expression.find { + case sub: Subtract => true + case other => false + } + expected = Some(Subtract(Literal(3), Literal(4))) + assert(expected === actual) + + // Find a leaf node. + actual = expression.find { + case Literal(3, IntegerType) => true + case other => false + } + expected = Some(Literal(3)) + assert(expected === actual) + + // Find nothing. + actual = expression.find { + case Literal(100, IntegerType) => true + case other => false + } + expected = None + assert(expected === actual) + } } 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 326e8ce4ca524e47e76e6960842f83b7eb8756cc..56a4689eb58f0210a62a754ccd4e11599d8ae99b 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 @@ -303,6 +303,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Expand(projections, output, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => execution.Aggregate(partial = false, group, agg, planLater(child)) :: Nil + case logical.Window(projectList, windowExpressions, spec, child) => + execution.Window(projectList, windowExpressions, spec, planLater(child)) :: Nil case logical.Sample(lb, ub, withReplacement, seed, child) => execution.Sample(lb, ub, withReplacement, seed, planLater(child)) :: Nil case logical.LocalRelation(output, data) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala new file mode 100644 index 0000000000000000000000000000000000000000..217b559def5124aec1c2e3e03a68142af1db11f4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala @@ -0,0 +1,480 @@ +/* + * 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 + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, Distribution, ClusteredDistribution, Partitioning} +import org.apache.spark.util.collection.CompactBuffer + +/** + * :: DeveloperApi :: + * For every row, evaluates `windowExpression` containing Window Functions and attaches + * the results with other regular expressions (presented by `projectList`). + * Evert operator handles a single Window Specification, `windowSpec`. + */ +case class Window( + projectList: Seq[Attribute], + windowExpression: Seq[NamedExpression], + windowSpec: WindowSpecDefinition, + child: SparkPlan) + extends UnaryNode { + + override def output: Seq[Attribute] = + (projectList ++ windowExpression).map(_.toAttribute) + + override def requiredChildDistribution: Seq[Distribution] = + if (windowSpec.partitionSpec.isEmpty) { + // This operator will be very expensive. + AllTuples :: Nil + } else { + ClusteredDistribution(windowSpec.partitionSpec) :: Nil + } + + // Since window functions are adding columns to the input rows, the child's outputPartitioning + // is preserved. + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = { + // The required child ordering has two parts. + // The first part is the expressions in the partition specification. + // We add these expressions to the required ordering to make sure input rows are grouped + // based on the partition specification. So, we only need to process a single partition + // at a time. + // The second part is the expressions specified in the ORDER BY cluase. + // Basically, we first use sort to group rows based on partition specifications and then sort + // Rows in a group based on the order specification. + (windowSpec.partitionSpec.map(SortOrder(_, Ascending)) ++ windowSpec.orderSpec) :: Nil + } + + // Since window functions basically add columns to input rows, this operator + // will not change the ordering of input rows. + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + case class ComputedWindow( + unbound: WindowExpression, + windowFunction: WindowFunction, + resultAttribute: AttributeReference) + + // A list of window functions that need to be computed for each group. + private[this] val computedWindowExpressions = windowExpression.flatMap { window => + window.collect { + case w: WindowExpression => + ComputedWindow( + w, + BindReferences.bindReference(w.windowFunction, child.output), + AttributeReference(s"windowResult:$w", w.dataType, w.nullable)()) + } + }.toArray + + private[this] val windowFrame = + windowSpec.frameSpecification.asInstanceOf[SpecifiedWindowFrame] + + // Create window functions. + private[this] def windowFunctions(): Array[WindowFunction] = { + val functions = new Array[WindowFunction](computedWindowExpressions.length) + var i = 0 + while (i < computedWindowExpressions.length) { + functions(i) = computedWindowExpressions(i).windowFunction.newInstance() + functions(i).init() + i += 1 + } + functions + } + + // The schema of the result of all window function evaluations + private[this] val computedSchema = computedWindowExpressions.map(_.resultAttribute) + + private[this] val computedResultMap = + computedWindowExpressions.map { w => w.unbound -> w.resultAttribute }.toMap + + private[this] val windowExpressionResult = windowExpression.map { window => + window.transform { + case w: WindowExpression if computedResultMap.contains(w) => computedResultMap(w) + } + } + + def execute(): RDD[Row] = { + child.execute().mapPartitions { iter => + new Iterator[Row] { + + // Although input rows are grouped based on windowSpec.partitionSpec, we need to + // know when we have a new partition. + // This is to manually construct an ordering that can be used to compare rows. + // TODO: We may want to have a newOrdering that takes BoundReferences. + // So, we can take advantave of code gen. + private val partitionOrdering: Ordering[Row] = + RowOrdering.forSchema(windowSpec.partitionSpec.map(_.dataType)) + + // This is used to project expressions for the partition specification. + protected val partitionGenerator = + newMutableProjection(windowSpec.partitionSpec, child.output)() + + // This is ued to project expressions for the order specification. + protected val rowOrderGenerator = + newMutableProjection(windowSpec.orderSpec.map(_.child), child.output)() + + // The position of next output row in the inputRowBuffer. + var rowPosition: Int = 0 + // The number of buffered rows in the inputRowBuffer (the size of the current partition). + var partitionSize: Int = 0 + // The buffer used to buffer rows in a partition. + var inputRowBuffer: CompactBuffer[Row] = _ + // The partition key of the current partition. + var currentPartitionKey: Row = _ + // The partition key of next partition. + var nextPartitionKey: Row = _ + // The first row of next partition. + var firstRowInNextPartition: Row = _ + // Indicates if this partition is the last one in the iter. + var lastPartition: Boolean = false + + def createBoundaryEvaluator(): () => Unit = { + def findPhysicalBoundary( + boundary: FrameBoundary): () => Int = boundary match { + case UnboundedPreceding => () => 0 + case UnboundedFollowing => () => partitionSize - 1 + case CurrentRow => () => rowPosition + case ValuePreceding(value) => + () => + val newPosition = rowPosition - value + if (newPosition > 0) newPosition else 0 + case ValueFollowing(value) => + () => + val newPosition = rowPosition + value + if (newPosition < partitionSize) newPosition else partitionSize - 1 + } + + def findLogicalBoundary( + boundary: FrameBoundary, + searchDirection: Int, + evaluator: Expression, + joinedRow: JoinedRow): () => Int = boundary match { + case UnboundedPreceding => () => 0 + case UnboundedFollowing => () => partitionSize - 1 + case other => + () => { + // CurrentRow, ValuePreceding, or ValueFollowing. + var newPosition = rowPosition + searchDirection + var stopSearch = false + // rowOrderGenerator is a mutable projection. + // We need to make a copy of the returned by rowOrderGenerator since we will + // compare searched row with this currentOrderByValue. + val currentOrderByValue = rowOrderGenerator(inputRowBuffer(rowPosition)).copy() + while (newPosition >= 0 && newPosition < partitionSize && !stopSearch) { + val r = rowOrderGenerator(inputRowBuffer(newPosition)) + stopSearch = + !(evaluator.eval(joinedRow(currentOrderByValue, r)).asInstanceOf[Boolean]) + if (!stopSearch) { + newPosition += searchDirection + } + } + newPosition -= searchDirection + + if (newPosition < 0) { + 0 + } else if (newPosition >= partitionSize) { + partitionSize - 1 + } else { + newPosition + } + } + } + + windowFrame.frameType match { + case RowFrame => + val findStart = findPhysicalBoundary(windowFrame.frameStart) + val findEnd = findPhysicalBoundary(windowFrame.frameEnd) + () => { + frameStart = findStart() + frameEnd = findEnd() + } + case RangeFrame => + val joinedRowForBoundaryEvaluation: JoinedRow = new JoinedRow() + val orderByExpr = windowSpec.orderSpec.head + val currentRowExpr = + BoundReference(0, orderByExpr.dataType, orderByExpr.nullable) + val examedRowExpr = + BoundReference(1, orderByExpr.dataType, orderByExpr.nullable) + val differenceExpr = Abs(Subtract(currentRowExpr, examedRowExpr)) + + val frameStartEvaluator = windowFrame.frameStart match { + case CurrentRow => EqualTo(currentRowExpr, examedRowExpr) + case ValuePreceding(value) => + LessThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case ValueFollowing(value) => + GreaterThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case o => Literal(true) // This is just a dummy expression, we will not use it. + } + + val frameEndEvaluator = windowFrame.frameEnd match { + case CurrentRow => EqualTo(currentRowExpr, examedRowExpr) + case ValuePreceding(value) => + GreaterThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case ValueFollowing(value) => + LessThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case o => Literal(true) // This is just a dummy expression, we will not use it. + } + + val findStart = + findLogicalBoundary( + boundary = windowFrame.frameStart, + searchDirection = -1, + evaluator = frameStartEvaluator, + joinedRow = joinedRowForBoundaryEvaluation) + val findEnd = + findLogicalBoundary( + boundary = windowFrame.frameEnd, + searchDirection = 1, + evaluator = frameEndEvaluator, + joinedRow = joinedRowForBoundaryEvaluation) + () => { + frameStart = findStart() + frameEnd = findEnd() + } + } + } + + val boundaryEvaluator = createBoundaryEvaluator() + // Indicates if we the specified window frame requires us to maintain a sliding frame + // (e.g. RANGES BETWEEN 1 PRECEDING AND CURRENT ROW) or the window frame + // is the entire partition (e.g. ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING). + val requireUpdateFrame: Boolean = { + def requireUpdateBoundary(boundary: FrameBoundary): Boolean = boundary match { + case UnboundedPreceding => false + case UnboundedFollowing => false + case _ => true + } + + requireUpdateBoundary(windowFrame.frameStart) || + requireUpdateBoundary(windowFrame.frameEnd) + } + // The start position of the current frame in the partition. + var frameStart: Int = 0 + // The end position of the current frame in the partition. + var frameEnd: Int = -1 + // Window functions. + val functions: Array[WindowFunction] = windowFunctions() + // Buffers used to store input parameters for window functions. Because we may need to + // maintain a sliding frame, we use this buffer to avoid evaluate the parameters from + // the same row multiple times. + val windowFunctionParameterBuffers: Array[util.LinkedList[AnyRef]] = + functions.map(_ => new util.LinkedList[AnyRef]()) + + // The projection used to generate the final result rows of this operator. + private[this] val resultProjection = + newMutableProjection( + projectList ++ windowExpressionResult, + projectList ++ computedSchema)() + + // The row used to hold results of window functions. + private[this] val windowExpressionResultRow = + new GenericMutableRow(computedSchema.length) + + private[this] val joinedRow = new JoinedRow6 + + // Initialize this iterator. + initialize() + + private def initialize(): Unit = { + if (iter.hasNext) { + val currentRow = iter.next().copy() + // partitionGenerator is a mutable projection. Since we need to track nextPartitionKey, + // we are making a copy of the returned partitionKey at here. + nextPartitionKey = partitionGenerator(currentRow).copy() + firstRowInNextPartition = currentRow + fetchNextPartition() + } else { + // The iter is an empty one. So, we set all of the following variables + // to make sure hasNext will return false. + lastPartition = true + rowPosition = 0 + partitionSize = 0 + } + } + + // Indicates if we will have new output row. + override final def hasNext: Boolean = { + !lastPartition || (rowPosition < partitionSize) + } + + override final def next(): Row = { + if (hasNext) { + if (rowPosition == partitionSize) { + // All rows of this buffer have been consumed. + // We will move to next partition. + fetchNextPartition() + } + // Get the input row for the current output row. + val inputRow = inputRowBuffer(rowPosition) + // Get all results of the window functions for this output row. + var i = 0 + while (i < functions.length) { + windowExpressionResultRow.update(i, functions(i).get(rowPosition)) + i += 1 + } + + // Construct the output row. + val outputRow = resultProjection(joinedRow(inputRow, windowExpressionResultRow)) + // We will move to the next one. + rowPosition += 1 + if (requireUpdateFrame && rowPosition < partitionSize) { + // If we need to maintain a sliding frame and + // we will still work on this partition when next is called next time, do the update. + updateFrame() + } + + // Return the output row. + outputRow + } else { + // no more result + throw new NoSuchElementException + } + } + + // Fetch the next partition. + private def fetchNextPartition(): Unit = { + // Create a new buffer for input rows. + inputRowBuffer = new CompactBuffer[Row]() + // We already have the first row for this partition + // (recorded in firstRowInNextPartition). Add it back. + inputRowBuffer += firstRowInNextPartition + // Set the current partition key. + currentPartitionKey = nextPartitionKey + // Now, we will start to find all rows belonging to this partition. + // Create a variable to track if we see the next partition. + var findNextPartition = false + // The search will stop when we see the next partition or there is no + // input row left in the iter. + while (iter.hasNext && !findNextPartition) { + // Make a copy of the input row since we will put it in the buffer. + val currentRow = iter.next().copy() + // Get the partition key based on the partition specification. + // For the below compare method, we do not need to make a copy of partitionKey. + val partitionKey = partitionGenerator(currentRow) + // Check if the current row belongs the current input row. + val comparing = partitionOrdering.compare(currentPartitionKey, partitionKey) + if (comparing == 0) { + // This row is still in the current partition. + inputRowBuffer += currentRow + } else { + // The current input row is in a different partition. + findNextPartition = true + // partitionGenerator is a mutable projection. + // Since we need to track nextPartitionKey and we determine that it should be set + // as partitionKey, we are making a copy of the partitionKey at here. + nextPartitionKey = partitionKey.copy() + firstRowInNextPartition = currentRow + } + } + + // We have not seen a new partition. It means that there is no new row in the + // iter. The current partition is the last partition of the iter. + if (!findNextPartition) { + lastPartition = true + } + + // We have got all rows for the current partition. + // Set rowPosition to 0 (the next output row will be based on the first + // input row of this partition). + rowPosition = 0 + // The size of this partition. + partitionSize = inputRowBuffer.size + // Reset all parameter buffers of window functions. + var i = 0 + while (i < windowFunctionParameterBuffers.length) { + windowFunctionParameterBuffers(i).clear() + i += 1 + } + frameStart = 0 + frameEnd = -1 + // Create the first window frame for this partition. + // If we do not need to maintain a sliding frame, this frame will + // have the entire partition. + updateFrame() + } + + /** The function used to maintain the sliding frame. */ + private def updateFrame(): Unit = { + // Based on the difference between the new frame and old frame, + // updates the buffers holding input parameters of window functions. + // We will start to prepare input parameters starting from the row + // indicated by offset in the input row buffer. + def updateWindowFunctionParameterBuffers( + numToRemove: Int, + numToAdd: Int, + offset: Int): Unit = { + // First, remove unneeded entries from the head of every buffer. + var i = 0 + while (i < numToRemove) { + var j = 0 + while (j < windowFunctionParameterBuffers.length) { + windowFunctionParameterBuffers(j).remove() + j += 1 + } + i += 1 + } + // Then, add needed entries to the tail of every buffer. + i = 0 + while (i < numToAdd) { + var j = 0 + while (j < windowFunctionParameterBuffers.length) { + // Ask the function to prepare the input parameters. + val parameters = functions(j).prepareInputParameters(inputRowBuffer(i + offset)) + windowFunctionParameterBuffers(j).add(parameters) + j += 1 + } + i += 1 + } + } + + // Record the current frame start point and end point before + // we update them. + val previousFrameStart = frameStart + val previousFrameEnd = frameEnd + boundaryEvaluator() + updateWindowFunctionParameterBuffers( + frameStart - previousFrameStart, + frameEnd - previousFrameEnd, + previousFrameEnd + 1) + // Evaluate the current frame. + evaluateCurrentFrame() + } + + /** Evaluate the current window frame. */ + private def evaluateCurrentFrame(): Unit = { + var i = 0 + while (i < functions.length) { + // Reset the state of the window function. + functions(i).reset() + // Get all buffered input parameters based on rows of this window frame. + val inputParameters = windowFunctionParameterBuffers(i).toArray() + // Send these input parameters to the window function. + functions(i).batchUpdate(inputParameters) + // Ask the function to evaluate based on this window frame. + functions(i).evaluate() + i += 1 + } + } + } + } + } +} diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 81ee48ef4152f65be63e74fe332ffdff18b33a59..5e411c2fdba9df21d89db2c24dca026430928622 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -185,7 +185,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Hive does not support buckets. ".*bucket.*", - // No window support yet + // We have our own tests based on these query files. ".*window.*", // Fails in hive with authorization errors. 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 1d8d0b5c322ad50ffd831e1ede49ef6edb66e304..f25723e53f07c3aabce36ef4b8c16c2dfac9842d 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 @@ -252,6 +252,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.CreateTables :: catalog.PreInsertionCasts :: ExtractPythonUdfs :: + ResolveHiveWindowFunction :: sources.PreInsertCastAndRename :: Nil } 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 63a8c05f775b8790805e159c9a107be90acaecf4..8a0686a2d81c9c81a358b7f2b6a2a33c1c0cd584 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 @@ -421,16 +421,16 @@ private[hive] object HiveQl { } /** - * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) - * is equivalent to + * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) + * is equivalent to * SELECT MAX(value) FROM src GROUP BY k1, k2 UNION SELECT MAX(value) FROM src GROUP BY k2 * Check the following link for details. - * + * https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C+Grouping+and+Rollup * * The bitmask denotes the grouping expressions validity for a grouping set, * the bitmask also be called as grouping id (`GROUPING__ID`, the virtual column in Hive) - * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of + * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of * GROUPING SETS (k1, k2) and (k2) should be 3 and 2 respectively. */ protected def extractGroupingSet(children: Seq[ASTNode]): (Seq[Expression], Seq[Int]) = { @@ -444,7 +444,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val bitmasks: Seq[Int] = setASTs.map(set => set match { case Token("TOK_GROUPING_SETS_EXPRESSION", null) => 0 - case Token("TOK_GROUPING_SETS_EXPRESSION", children) => + case Token("TOK_GROUPING_SETS_EXPRESSION", children) => children.foldLeft(0)((bitmap, col) => { val colString = col.asInstanceOf[ASTNode].toStringTree() require(keyMap.contains(colString), s"$colString doens't show up in the GROUP BY list") @@ -613,7 +613,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C clusterByClause :: distributeByClause :: limitClause :: - lateralViewClause :: Nil) = { + lateralViewClause :: + windowClause :: Nil) = { getClauses( Seq( "TOK_INSERT_INTO", @@ -631,15 +632,16 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C "TOK_CLUSTERBY", "TOK_DISTRIBUTEBY", "TOK_LIMIT", - "TOK_LATERAL_VIEW"), + "TOK_LATERAL_VIEW", + "WINDOW"), singleInsert) } - + val relations = fromClause match { case Some(f) => nodeToRelation(f) case None => OneRowRelation } - + val withWhere = whereClause.map { whereNode => val Seq(whereExpr) = whereNode.getChildren.toSeq Filter(nodeToExpr(whereExpr), relations) @@ -691,7 +693,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val serdeProps = propsClause.map { case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) => (name, value) - } + } (Nil, serdeClass, serdeProps) case Nil => (Nil, "", Nil) @@ -736,7 +738,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C // The projection of the query can either be a normal projection, an aggregation // (if there is a group by) or a script transformation. val withProject: LogicalPlan = transformation.getOrElse { - val selectExpressions = + val selectExpressions = nameExpressions(select.getChildren.flatMap(selExprNodeToExpr).toSeq) Seq( groupByClause.map(e => e match { @@ -764,31 +766,34 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C Some(Project(selectExpressions, withLateralView))).flatten.head } - val withDistinct = - if (selectDistinctClause.isDefined) Distinct(withProject) else withProject - + // Handle HAVING clause. val withHaving = havingClause.map { h => val havingExpr = h.getChildren.toSeq match { case Seq(hexpr) => nodeToExpr(hexpr) } // Note that we added a cast to boolean. If the expression itself is already boolean, // the optimizer will get rid of the unnecessary cast. - Filter(Cast(havingExpr, BooleanType), withDistinct) - }.getOrElse(withDistinct) + Filter(Cast(havingExpr, BooleanType), withProject) + }.getOrElse(withProject) + + // Handle SELECT DISTINCT + val withDistinct = + if (selectDistinctClause.isDefined) Distinct(withHaving) else withHaving + // Handle ORDER BY, SORT BY, DISTRIBETU BY, and CLUSTER BY clause. val withSort = (orderByClause, sortByClause, distributeByClause, clusterByClause) match { case (Some(totalOrdering), None, None, None) => - Sort(totalOrdering.getChildren.map(nodeToSortOrder), true, withHaving) + Sort(totalOrdering.getChildren.map(nodeToSortOrder), true, withDistinct) case (None, Some(perPartitionOrdering), None, None) => - Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, withHaving) + Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, withDistinct) case (None, None, Some(partitionExprs), None) => - RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withHaving) + RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withDistinct) case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, - RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withHaving)) + RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withDistinct)) case (None, None, None, Some(clusterExprs)) => Sort(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)), false, - RepartitionByExpression(clusterExprs.getChildren.map(nodeToExpr), withHaving)) - case (None, None, None, None) => withHaving + RepartitionByExpression(clusterExprs.getChildren.map(nodeToExpr), withDistinct)) + case (None, None, None, None) => withDistinct case _ => sys.error("Unsupported set of ordering / distribution clauses.") } @@ -797,6 +802,27 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C .map(Limit(_, withSort)) .getOrElse(withSort) + // Collect all window specifications defined in the WINDOW clause. + val windowDefinitions = windowClause.map(_.getChildren.toSeq.collect { + case Token("TOK_WINDOWDEF", + Token(windowName, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => + windowName -> nodesToWindowSpecification(spec) + }.toMap) + // Handle cases like + // window w1 as (partition by p_mfgr order by p_name + // range between 2 preceding and 2 following), + // w2 as w1 + val resolvedCrossReference = windowDefinitions.map { + windowDefMap => windowDefMap.map { + case (windowName, WindowSpecReference(other)) => + (windowName, windowDefMap(other).asInstanceOf[WindowSpecDefinition]) + case o => o.asInstanceOf[(String, WindowSpecDefinition)] + } + } + + val withWindowDefinitions = + resolvedCrossReference.map(WithWindowDefinition(_, withLimit)).getOrElse(withLimit) + // TOK_INSERT_INTO means to add files to the table. // TOK_DESTINATION means to overwrite the table. val resultDestination = @@ -804,7 +830,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val overwrite = intoClause.isEmpty nodeToDest( resultDestination, - withLimit, + withWindowDefinitions, overwrite) } @@ -1053,7 +1079,6 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") } - protected val escapedIdentifier = "`([^`]+)`".r /** Strips backticks from ident if present */ protected def cleanIdentifier(ident: String): String = ident match { @@ -1250,6 +1275,25 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C Substring(nodeToExpr(string), nodeToExpr(pos), nodeToExpr(length)) case Token("TOK_FUNCTION", Token(COALESCE(), Nil) :: list) => Coalesce(list.map(nodeToExpr)) + /* Window Functions */ + case Token("TOK_FUNCTION", Token(name, Nil) +: args :+ Token("TOK_WINDOWSPEC", spec)) => + val function = UnresolvedWindowFunction(name, args.map(nodeToExpr)) + nodesToWindowSpecification(spec) match { + case reference: WindowSpecReference => + UnresolvedWindowExpression(function, reference) + case definition: WindowSpecDefinition => + WindowExpression(function, definition) + } + case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => + // Safe to use Literal(1)? + val function = UnresolvedWindowFunction(name, Literal(1) :: Nil) + nodesToWindowSpecification(spec) match { + case reference: WindowSpecReference => + UnresolvedWindowExpression(function, reference) + case definition: WindowSpecDefinition => + WindowExpression(function, definition) + } + /* UDFs - Must be last otherwise will preempt built in functions */ case Token("TOK_FUNCTION", Token(name, Nil) :: args) => UnresolvedFunction(name, args.map(nodeToExpr)) @@ -1312,6 +1356,89 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C """.stripMargin) } + def nodesToWindowSpecification(nodes: Seq[ASTNode]): WindowSpec = nodes match { + case Token(windowName, Nil) :: Nil => + // Refer to a window spec defined in the window clause. + WindowSpecReference(windowName) + case Nil => + // OVER() + WindowSpecDefinition( + partitionSpec = Nil, + orderSpec = Nil, + frameSpecification = UnspecifiedFrame) + case spec => + val (partitionClause :: rowFrame :: rangeFrame :: Nil) = + getClauses( + Seq( + "TOK_PARTITIONINGSPEC", + "TOK_WINDOWRANGE", + "TOK_WINDOWVALUES"), + spec) + + // Handle Partition By and Order By. + val (partitionSpec, orderSpec) = partitionClause.map { partitionAndOrdering => + val (partitionByClause :: orderByClause :: sortByClause :: clusterByClause :: Nil) = + getClauses( + Seq("TOK_DISTRIBUTEBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_CLUSTERBY"), + partitionAndOrdering.getChildren.toSeq.asInstanceOf[Seq[ASTNode]]) + + (partitionByClause, orderByClause.orElse(sortByClause), clusterByClause) match { + case (Some(partitionByExpr), Some(orderByExpr), None) => + (partitionByExpr.getChildren.map(nodeToExpr), + orderByExpr.getChildren.map(nodeToSortOrder)) + case (Some(partitionByExpr), None, None) => + (partitionByExpr.getChildren.map(nodeToExpr), Nil) + case (None, Some(orderByExpr), None) => + (Nil, orderByExpr.getChildren.map(nodeToSortOrder)) + case (None, None, Some(clusterByExpr)) => + val expressions = clusterByExpr.getChildren.map(nodeToExpr) + (expressions, expressions.map(SortOrder(_, Ascending))) + case _ => + throw new NotImplementedError( + s"""No parse rules for Node ${partitionAndOrdering.getName} + """.stripMargin) + } + }.getOrElse { + (Nil, Nil) + } + + // Handle Window Frame + val windowFrame = + if (rowFrame.isEmpty && rangeFrame.isEmpty) { + UnspecifiedFrame + } else { + val frameType = rowFrame.map(_ => RowFrame).getOrElse(RangeFrame) + def nodeToBoundary(node: Node): FrameBoundary = node match { + case Token("preceding", Token(count, Nil) :: Nil) => + if (count == "unbounded") UnboundedPreceding else ValuePreceding(count.toInt) + case Token("following", Token(count, Nil) :: Nil) => + if (count == "unbounded") UnboundedFollowing else ValueFollowing(count.toInt) + case Token("current", Nil) => CurrentRow + case _ => + throw new NotImplementedError( + s"""No parse rules for the Window Frame Boundary based on Node ${node.getName} + """.stripMargin) + } + + rowFrame.orElse(rangeFrame).map { frame => + frame.getChildren.toList match { + case precedingNode :: followingNode :: Nil => + SpecifiedWindowFrame( + frameType, + nodeToBoundary(precedingNode), + nodeToBoundary(followingNode)) + case precedingNode :: Nil => + SpecifiedWindowFrame(frameType, nodeToBoundary(precedingNode), CurrentRow) + case _ => + throw new NotImplementedError( + s"""No parse rules for the Window Frame based on Node ${frame.getName} + """.stripMargin) + } + }.getOrElse(sys.error(s"If you see this, please file a bug report with your query.")) + } + + WindowSpecDefinition(partitionSpec, orderSpec, windowFrame) + } val explode = "(?i)explode".r def nodesToGenerator(nodes: Seq[Node]): (Generator, Seq[String]) = { 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 4b6f0ad75f54f9a8e5597f40a173d250cd938ad5..fd0b6f058595dca5bbe7c9c318652e43a1aaef74 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 @@ -17,27 +17,27 @@ package org.apache.spark.sql.hive +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper +import org.apache.spark.sql.AnalysisException import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ConstantObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory -import org.apache.hadoop.hive.ql.exec.{UDF, UDAF} -import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} +import org.apache.hadoop.hive.ql.exec._ import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ import org.apache.spark.Logging import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{Generate, Project, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.types._ -import org.apache.spark.sql.catalyst.analysis.MultiAlias -import org.apache.spark.sql.catalyst.errors.TreeNodeException /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -189,6 +189,219 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr } } +/** + * Resolves [[UnresolvedWindowFunction]] to [[HiveWindowFunction]]. + */ +private[spark] object ResolveHiveWindowFunction extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + case p: LogicalPlan if !p.childrenResolved => p + + // We are resolving WindowExpressions at here. When we get here, we have already + // replaced those WindowSpecReferences. + case p: LogicalPlan => + p transformExpressions { + case WindowExpression( + UnresolvedWindowFunction(name, children), + windowSpec: WindowSpecDefinition) => + // First, let's find the window function info. + val windowFunctionInfo: WindowFunctionInfo = + Option(FunctionRegistry.getWindowFunctionInfo(name.toLowerCase)).getOrElse( + throw new AnalysisException(s"Couldn't find window function $name")) + + // Get the class of this function. + // In Hive 0.12, there is no windowFunctionInfo.getFunctionClass. So, we use + // windowFunctionInfo.getfInfo().getFunctionClass for both Hive 0.13 and Hive 0.13.1. + val functionClass = windowFunctionInfo.getfInfo().getFunctionClass + val newChildren = + // Rank(), DENSE_RANK(), CUME_DIST(), and PERCENT_RANK() do not take explicit + // input parameters and requires implicit parameters, which + // are expressions in Order By clause. + if (classOf[GenericUDAFRank].isAssignableFrom(functionClass)) { + if (children.nonEmpty) { + throw new AnalysisException(s"$name does not take input parameters.") + } + windowSpec.orderSpec.map(_.child) + } else { + children + } + + // If the class is UDAF, we need to use UDAFBridge. + val isUDAFBridgeRequired = + if (classOf[UDAF].isAssignableFrom(functionClass)) { + true + } else { + false + } + + // Create the HiveWindowFunction. For the meaning of isPivotResult, see the doc of + // HiveWindowFunction. + val windowFunction = + HiveWindowFunction( + new HiveFunctionWrapper(functionClass.getName), + windowFunctionInfo.isPivotResult, + isUDAFBridgeRequired, + newChildren) + + // Second, check if the specified window function can accept window definition. + windowSpec.frameSpecification match { + case frame: SpecifiedWindowFrame if !windowFunctionInfo.isSupportsWindow => + // This Hive window function does not support user-speficied window frame. + throw new AnalysisException( + s"Window function $name does not take a frame specification.") + case frame: SpecifiedWindowFrame if windowFunctionInfo.isSupportsWindow && + windowFunctionInfo.isPivotResult => + // These two should not be true at the same time when a window frame is defined. + // If so, throw an exception. + throw new AnalysisException(s"Could not handle Hive window function $name because " + + s"it supports both a user specified window frame and pivot result.") + case _ => // OK + } + // Resolve those UnspecifiedWindowFrame because the physical Window operator still needs + // a window frame specification to work. + val newWindowSpec = windowSpec.frameSpecification match { + case UnspecifiedFrame => + val newWindowFrame = + SpecifiedWindowFrame.defaultWindowFrame( + windowSpec.orderSpec.nonEmpty, + windowFunctionInfo.isSupportsWindow) + WindowSpecDefinition(windowSpec.partitionSpec, windowSpec.orderSpec, newWindowFrame) + case _ => windowSpec + } + + // Finally, we create a WindowExpression with the resolved window function and + // specified window spec. + WindowExpression(windowFunction, newWindowSpec) + } + } +} + +/** + * A [[WindowFunction]] implementation wrapping Hive's window function. + * @param funcWrapper The wrapper for the Hive Window Function. + * @param pivotResult If it is true, the Hive function will return a list of values representing + * the values of the added columns. Otherwise, a single value is returned for + * current row. + * @param isUDAFBridgeRequired If it is true, the function returned by functionWrapper's + * createFunction is UDAF, we need to use GenericUDAFBridge to wrap + * it as a GenericUDAFResolver2. + * @param children Input parameters. + */ +private[hive] case class HiveWindowFunction( + funcWrapper: HiveFunctionWrapper, + pivotResult: Boolean, + isUDAFBridgeRequired: Boolean, + children: Seq[Expression]) extends WindowFunction + with HiveInspectors { + + // Hive window functions are based on GenericUDAFResolver2. + type UDFType = GenericUDAFResolver2 + + @transient + protected lazy val resolver: GenericUDAFResolver2 = + if (isUDAFBridgeRequired) { + new GenericUDAFBridge(funcWrapper.createFunction[UDAF]()) + } else { + funcWrapper.createFunction[GenericUDAFResolver2]() + } + + @transient + protected lazy val inputInspectors = children.map(toInspector).toArray + + // The GenericUDAFEvaluator used to evaluate the window function. + @transient + protected lazy val evaluator: GenericUDAFEvaluator = { + val parameterInfo = new SimpleGenericUDAFParameterInfo(inputInspectors, false, false) + resolver.getEvaluator(parameterInfo) + } + + // The object inspector of values returned from the Hive window function. + @transient + protected lazy val returnInspector = { + evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inputInspectors) + } + + def dataType: DataType = + if (!pivotResult) { + inspectorToDataType(returnInspector) + } else { + // If pivotResult is true, we should take the element type out as the data type of this + // function. + inspectorToDataType(returnInspector) match { + case ArrayType(dt, _) => dt + case _ => + sys.error( + s"error resolve the data type of window function ${funcWrapper.functionClassName}") + } + } + + def nullable: Boolean = true + + override type EvaluatedType = Any + + override def eval(input: Row): Any = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + + @transient + lazy val inputProjection = new InterpretedProjection(children) + + @transient + private var hiveEvaluatorBuffer: AggregationBuffer = _ + // Output buffer. + private var outputBuffer: Any = _ + + override def init(): Unit = { + evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inputInspectors) + } + + // Reset the hiveEvaluatorBuffer and outputPosition + override def reset(): Unit = { + // We create a new aggregation buffer to workaround the bug in GenericUDAFRowNumber. + // Basically, GenericUDAFRowNumberEvaluator.reset calls RowNumberBuffer.init. + // However, RowNumberBuffer.init does not really reset this buffer. + hiveEvaluatorBuffer = evaluator.getNewAggregationBuffer + evaluator.reset(hiveEvaluatorBuffer) + } + + override def prepareInputParameters(input: Row): AnyRef = { + wrap(inputProjection(input), inputInspectors, new Array[AnyRef](children.length)) + } + // Add input parameters for a single row. + override def update(input: AnyRef): Unit = { + evaluator.iterate(hiveEvaluatorBuffer, input.asInstanceOf[Array[AnyRef]]) + } + + override def batchUpdate(inputs: Array[AnyRef]): Unit = { + var i = 0 + while (i < inputs.length) { + evaluator.iterate(hiveEvaluatorBuffer, inputs(i).asInstanceOf[Array[AnyRef]]) + i += 1 + } + } + + override def evaluate(): Unit = { + outputBuffer = unwrap(evaluator.evaluate(hiveEvaluatorBuffer), returnInspector) + } + + override def get(index: Int): Any = { + if (!pivotResult) { + // if pivotResult is false, we will get a single value for all rows in the frame. + outputBuffer + } else { + // if pivotResult is true, we will get a Seq having the same size with the size + // of the window frame. At here, we will return the result at the position of + // index in the output buffer. + outputBuffer.asInstanceOf[Seq[Any]].get(index) + } + } + + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } + + override def newInstance: WindowFunction = + new HiveWindowFunction(funcWrapper, pivotResult, isUDAFBridgeRequired, children) +} + private[hive] case class HiveGenericUdaf( funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) extends AggregateExpression diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 1. testWindowing-0-327a8cd39fe30255ff492ee86f660522 b/sql/hive/src/test/resources/golden/windowing.q -- 1. testWindowing-0-327a8cd39fe30255ff492ee86f660522 new file mode 100644 index 0000000000000000000000000000000000000000..850c41c8115d6f4ee3c712324d68fc2a9b5ac3c7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 1. testWindowing-0-327a8cd39fe30255ff492ee86f660522 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 10. testHavingWithWindowingCondRankNoGBY-0-fef4bf638d52a9a601845347010602fd b/sql/hive/src/test/resources/golden/windowing.q -- 10. testHavingWithWindowingCondRankNoGBY-0-fef4bf638d52a9a601845347010602fd new file mode 100644 index 0000000000000000000000000000000000000000..850c41c8115d6f4ee3c712324d68fc2a9b5ac3c7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 10. testHavingWithWindowingCondRankNoGBY-0-fef4bf638d52a9a601845347010602fd @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 11. testFirstLast-0-86bb9c97d92fdcd941bcb5143513e2e6 b/sql/hive/src/test/resources/golden/windowing.q -- 11. testFirstLast-0-86bb9c97d92fdcd941bcb5143513e2e6 new file mode 100644 index 0000000000000000000000000000000000000000..921679cdcf569bcabc8a6d767f3548a77e72cd66 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 11. testFirstLast-0-86bb9c97d92fdcd941bcb5143513e2e6 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 2 34 +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/windowing.q -- 12. testFirstLastWithWhere-0-84345a9f685ba63b87caa4bb16b122b5 b/sql/hive/src/test/resources/golden/windowing.q -- 12. testFirstLastWithWhere-0-84345a9f685ba63b87caa4bb16b122b5 new file mode 100644 index 0000000000000000000000000000000000000000..09e30c7c57349e41c0c4f99b9d4a8141c9cc5aca --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 12. testFirstLastWithWhere-0-84345a9f685ba63b87caa4bb16b122b5 @@ -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/windowing.q -- 13. testSumWindow-0-6cfc8840d3a4469b0fe11d63182cb59f b/sql/hive/src/test/resources/golden/windowing.q -- 13. testSumWindow-0-6cfc8840d3a4469b0fe11d63182cb59f new file mode 100644 index 0000000000000000000000000000000000000000..01ee88ff233029c665ef24ac310f09ec4b27e163 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 13. testSumWindow-0-6cfc8840d3a4469b0fe11d63182cb59f @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 2 +Manufacturer#1 almond antique burnished rose metallic 2 44 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 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/windowing.q -- 14. testNoSortClause-0-88d96a526d3cae6ed8168c5b228974d1 b/sql/hive/src/test/resources/golden/windowing.q -- 14. testNoSortClause-0-88d96a526d3cae6ed8168c5b228974d1 new file mode 100644 index 0000000000000000000000000000000000000000..c78eb640c9c2744724a97c63a30dfd687de69cec --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 14. testNoSortClause-0-88d96a526d3cae6ed8168c5b228974d1 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 +Manufacturer#3 almond antique metallic orange dim 19 3 3 +Manufacturer#3 almond antique misty red olive 1 4 4 +Manufacturer#3 almond antique olive coral navajo 45 5 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 +Manufacturer#4 almond antique violet mint lemon 39 2 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 +Manufacturer#5 almond antique medium spring khaki 6 2 2 +Manufacturer#5 almond antique sky peru orange 2 3 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 15. testExpressions-0-11f6c13cf2710ce7054654cca136e73e b/sql/hive/src/test/resources/golden/windowing.q -- 15. testExpressions-0-11f6c13cf2710ce7054654cca136e73e new file mode 100644 index 0000000000000000000000000000000000000000..050138ccf04ce2c34ed96daf1fdbf9349c925048 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 15. testExpressions-0-11f6c13cf2710ce7054654cca136e73e @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 0.0 1 2 2.0 0.0 2 2 2 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 0.0 1 2 2.0 0.0 2 2 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 0.5 0.4 2 3 12.666666666666666 15.084944665313014 2 34 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 0.6666666666666666 0.6 2 4 11.0 13.379088160259652 2 6 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 0.8333333333333334 0.8 3 5 14.4 13.763720427268202 2 28 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 1.0 1.0 3 6 19.0 16.237815945091466 2 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 0.2 0.0 1 1 14.0 0.0 4 14 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 0.4 0.25 1 2 27.0 13.0 4 40 14 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 0.6 0.5 2 3 18.666666666666668 15.86050300449376 4 2 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 0.8 0.75 2 4 20.25 14.00669482783144 4 25 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 1.0 1.0 3 5 19.8 12.560254774486067 4 18 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 0.2 0.0 1 1 17.0 0.0 2 17 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 0.4 0.25 1 2 15.5 1.5 2 14 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 0.6 0.5 2 3 16.666666666666668 2.0548046676563256 2 19 17 +Manufacturer#3 almond antique misty red olive 1 4 4 0.8 0.75 2 4 12.75 7.013380069552769 2 1 14 +Manufacturer#3 almond antique olive coral navajo 45 5 5 1.0 1.0 3 5 19.2 14.344336861632886 2 45 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 0.2 0.0 1 1 10.0 0.0 0 10 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 0.4 0.25 1 2 24.5 14.5 0 39 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 0.6 0.5 2 3 25.333333333333332 11.897712198383164 0 27 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 0.8 0.75 2 4 20.75 13.007209539328564 0 7 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 1.0 1.0 3 5 19.0 12.149074038789951 0 12 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 0.2 0.0 1 1 31.0 0.0 1 31 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 0.4 0.25 1 2 18.5 12.5 1 6 31 +Manufacturer#5 almond antique sky peru orange 2 3 3 0.6 0.5 2 3 13.0 12.832251036613439 1 2 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 0.8 0.75 2 4 21.25 18.102140757380052 1 46 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 1.0 1.0 3 5 21.6 16.206171663906314 1 23 2 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 16. testMultipleWindows-0-efd1476255eeb1b1961149144f574b7a b/sql/hive/src/test/resources/golden/windowing.q -- 16. testMultipleWindows-0-efd1476255eeb1b1961149144f574b7a new file mode 100644 index 0000000000000000000000000000000000000000..c10888852b504389a0acf97e19363885e62acf40 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 16. testMultipleWindows-0-efd1476255eeb1b1961149144f574b7a @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 4 4 2 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 4 4 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 0.5 38 34 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 0.6666666666666666 44 10 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 0.8333333333333334 72 28 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 1.0 114 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 0.2 14 14 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 0.4 54 40 14 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 0.6 56 2 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 0.8 81 25 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 1.0 99 32 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 0.2 17 31 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 0.4 31 14 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 0.6 50 50 17 +Manufacturer#3 almond antique misty red olive 1 4 4 0.8 51 1 14 +Manufacturer#3 almond antique olive coral navajo 45 5 5 1.0 96 45 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 0.2 10 17 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 0.4 49 39 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 0.6 76 27 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 0.8 83 7 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 1.0 95 29 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 0.2 31 31 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 0.4 37 8 31 +Manufacturer#5 almond antique sky peru orange 2 3 3 0.6 39 2 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 0.8 85 46 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 1.0 108 23 2 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 17. testCountStar-0-1b1fc185c8fddf68e58e92f29052ab2d b/sql/hive/src/test/resources/golden/windowing.q -- 17. testCountStar-0-1b1fc185c8fddf68e58e92f29052ab2d new file mode 100644 index 0000000000000000000000000000000000000000..b1309a497d68ec2669e811c36c5b5773da329dcd --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 17. testCountStar-0-1b1fc185c8fddf68e58e92f29052ab2d @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 2 2 +Manufacturer#1 almond antique burnished rose metallic 2 2 2 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 3 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 4 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 5 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 6 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 14 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 17 +Manufacturer#3 almond antique misty red olive 1 4 4 14 +Manufacturer#3 almond antique olive coral navajo 45 5 5 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 31 +Manufacturer#5 almond antique sky peru orange 2 3 3 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 2 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 18. testUDAFs-0-6974e5959e41a661e09db18547fef58a b/sql/hive/src/test/resources/golden/windowing.q -- 18. testUDAFs-0-6974e5959e41a661e09db18547fef58a new file mode 100644 index 0000000000000000000000000000000000000000..52d2ee8d0cd3f3e44cedc93a41cccb90ec9fcfd4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 18. testUDAFs-0-6974e5959e41a661e09db18547fef58a @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 4100.06 1173.15 1753.76 1366.6866666666667 +Manufacturer#1 almond antique burnished rose metallic 2 5702.650000000001 1173.15 1753.76 1425.6625000000001 +Manufacturer#1 almond antique chartreuse lavender yellow 34 7117.070000000001 1173.15 1753.76 1423.4140000000002 +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/windowing.q -- 19. testUDAFsWithGBY-0-67d15ee5915ac64a738fd4b60d75eb35 b/sql/hive/src/test/resources/golden/windowing.q -- 19. testUDAFsWithGBY-0-67d15ee5915ac64a738fd4b60d75eb35 new file mode 100644 index 0000000000000000000000000000000000000000..6461642d34a21171a3f10673a3aa4755374ec7f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 19. testUDAFsWithGBY-0-67d15ee5915ac64a738fd4b60d75eb35 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 4529.5 1173.15 1173.15 1509.8333333333333 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 5943.92 1753.76 1753.76 1485.98 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 7576.58 1602.59 1602.59 1515.316 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 6403.43 1414.42 1414.42 1600.8575 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 4649.67 1632.66 1632.66 1549.89 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 5523.360000000001 1690.68 1690.68 1841.1200000000001 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 7222.02 1800.7 1800.7 1805.505 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 8923.62 2031.98 2031.98 1784.7240000000002 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 7232.9400000000005 1698.66 1698.66 1808.2350000000001 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5432.24 1701.6 1701.6 1810.7466666666667 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 4272.34 1671.68 1671.68 1424.1133333333335 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 6195.32 1190.27 1190.27 1548.83 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 7532.61 1410.39 1410.39 1506.522 +Manufacturer#3 almond antique misty red olive 1 1922.98 5860.929999999999 1922.98 1922.98 1465.2324999999998 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 4670.66 1337.29 1337.29 1556.8866666666665 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 4202.35 1620.67 1620.67 1400.7833333333335 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 6047.27 1375.42 1375.42 1511.8175 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 7337.620000000001 1206.26 1206.26 1467.5240000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 5716.950000000001 1844.92 1844.92 1429.2375000000002 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 4341.530000000001 1290.35 1290.35 1447.176666666667 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 5190.08 1789.69 1789.69 1730.0266666666666 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 6208.18 1611.66 1611.66 1552.045 +Manufacturer#5 almond antique sky peru orange 2 1788.73 7672.66 1788.73 1788.73 1534.532 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 5882.970000000001 1018.1 1018.1 1470.7425000000003 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 4271.3099999999995 1464.48 1464.48 1423.7699999999998 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 2. testGroupByWithPartitioning-0-cb5618b1e626f3a9d4a030b508b5d251 b/sql/hive/src/test/resources/golden/windowing.q -- 2. testGroupByWithPartitioning-0-cb5618b1e626f3a9d4a030b508b5d251 new file mode 100644 index 0000000000000000000000000000000000000000..2c30e652aa26d6baac06f06ebab3fd7ac12f010e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 2. testGroupByWithPartitioning-0-cb5618b1e626f3a9d4a030b508b5d251 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 b/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 new file mode 100644 index 0000000000000000000000000000000000000000..1f7e8a5d67036678c557542be33dc808198bdbba --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 273.70217881648074 273.70217881648074 [34,2] 74912.8826888888 1.0 4128.782222222221 +Manufacturer#1 almond antique burnished rose metallic 2 258.10677784349235 258.10677784349235 [34,2,6] 66619.10876874991 0.811328754177887 2801.7074999999995 +Manufacturer#1 almond antique chartreuse lavender yellow 34 230.90151585470358 230.90151585470358 [34,2,6,28] 53315.51002399992 0.695639377397664 2210.7864 +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/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad b/sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad new file mode 100644 index 0000000000000000000000000000000000000000..e7c39f454fb37cd3f2ee48b5fb044b7ff3797f03 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 [{"x":1173.15,"y":2.0},{"x":1753.76,"y":1.0}] 121152.0 1 +Manufacturer#1 almond antique burnished rose metallic 2 [{"x":1173.15,"y":2.0},{"x":1602.59,"y":1.0},{"x":1753.76,"y":1.0}] 115872.0 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 [{"x":1173.15,"y":2.0},{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1753.76,"y":1.0}] 110592.0 3 +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 4 +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 5 +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 6 +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/windowing.q -- 24. testLateralViews-0-dea06072f0a64fe4537fae854944ed5a b/sql/hive/src/test/resources/golden/windowing.q -- 24. testLateralViews-0-dea06072f0a64fe4537fae854944ed5a new file mode 100644 index 0000000000000000000000000000000000000000..dc83c9fffe9323b69d5e0c03bc59113687fcf71e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 24. testLateralViews-0-dea06072f0a64fe4537fae854944ed5a @@ -0,0 +1,78 @@ +Manufacturer#1 almond antique burnished rose metallic 1 2 2 +Manufacturer#1 almond antique burnished rose metallic 1 2 4 +Manufacturer#1 almond antique burnished rose metallic 2 2 6 +Manufacturer#1 almond antique burnished rose metallic 2 2 6 +Manufacturer#1 almond antique burnished rose metallic 3 2 6 +Manufacturer#1 almond antique burnished rose metallic 3 2 6 +Manufacturer#1 almond antique salmon chartreuse burlywood 1 6 10 +Manufacturer#1 almond antique salmon chartreuse burlywood 2 6 14 +Manufacturer#1 almond antique salmon chartreuse burlywood 3 6 18 +Manufacturer#1 almond aquamarine burnished black steel 1 28 40 +Manufacturer#1 almond aquamarine burnished black steel 2 28 62 +Manufacturer#1 almond aquamarine burnished black steel 3 28 84 +Manufacturer#1 almond antique chartreuse lavender yellow 1 34 90 +Manufacturer#1 almond antique chartreuse lavender yellow 2 34 96 +Manufacturer#1 almond antique chartreuse lavender yellow 3 34 102 +Manufacturer#1 almond aquamarine pink moccasin thistle 1 42 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 2 42 118 +Manufacturer#1 almond aquamarine pink moccasin thistle 3 42 126 +Manufacturer#2 almond aquamarine midnight light salmon 1 2 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 4 +Manufacturer#2 almond aquamarine midnight light salmon 3 2 6 +Manufacturer#2 almond antique violet chocolate turquoise 1 14 18 +Manufacturer#2 almond antique violet chocolate turquoise 2 14 30 +Manufacturer#2 almond antique violet chocolate turquoise 3 14 42 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 1 18 46 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 2 18 50 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 3 18 54 +Manufacturer#2 almond aquamarine rose maroon antique 1 25 61 +Manufacturer#2 almond aquamarine rose maroon antique 2 25 68 +Manufacturer#2 almond aquamarine rose maroon antique 3 25 75 +Manufacturer#2 almond antique violet turquoise frosted 1 40 90 +Manufacturer#2 almond antique violet turquoise frosted 2 40 105 +Manufacturer#2 almond antique violet turquoise frosted 3 40 120 +Manufacturer#3 almond antique misty red olive 1 1 1 +Manufacturer#3 almond antique misty red olive 2 1 2 +Manufacturer#3 almond antique misty red olive 3 1 3 +Manufacturer#3 almond antique forest lavender goldenrod 1 14 16 +Manufacturer#3 almond antique forest lavender goldenrod 2 14 29 +Manufacturer#3 almond antique forest lavender goldenrod 3 14 42 +Manufacturer#3 almond antique chartreuse khaki white 1 17 45 +Manufacturer#3 almond antique chartreuse khaki white 2 17 48 +Manufacturer#3 almond antique chartreuse khaki white 3 17 51 +Manufacturer#3 almond antique metallic orange dim 1 19 53 +Manufacturer#3 almond antique metallic orange dim 2 19 55 +Manufacturer#3 almond antique metallic orange dim 3 19 57 +Manufacturer#3 almond antique olive coral navajo 1 45 83 +Manufacturer#3 almond antique olive coral navajo 2 45 109 +Manufacturer#3 almond antique olive coral navajo 3 45 135 +Manufacturer#4 almond aquamarine yellow dodger mint 1 7 7 +Manufacturer#4 almond aquamarine yellow dodger mint 2 7 14 +Manufacturer#4 almond aquamarine yellow dodger mint 3 7 21 +Manufacturer#4 almond antique gainsboro frosted violet 1 10 24 +Manufacturer#4 almond antique gainsboro frosted violet 2 10 27 +Manufacturer#4 almond antique gainsboro frosted violet 3 10 30 +Manufacturer#4 almond azure aquamarine papaya violet 1 12 32 +Manufacturer#4 almond azure aquamarine papaya violet 2 12 34 +Manufacturer#4 almond azure aquamarine papaya violet 3 12 36 +Manufacturer#4 almond aquamarine floral ivory bisque 1 27 51 +Manufacturer#4 almond aquamarine floral ivory bisque 2 27 66 +Manufacturer#4 almond aquamarine floral ivory bisque 3 27 81 +Manufacturer#4 almond antique violet mint lemon 1 39 93 +Manufacturer#4 almond antique violet mint lemon 2 39 105 +Manufacturer#4 almond antique violet mint lemon 3 39 117 +Manufacturer#5 almond antique sky peru orange 1 2 2 +Manufacturer#5 almond antique sky peru orange 2 2 4 +Manufacturer#5 almond antique sky peru orange 3 2 6 +Manufacturer#5 almond antique medium spring khaki 1 6 10 +Manufacturer#5 almond antique medium spring khaki 2 6 14 +Manufacturer#5 almond antique medium spring khaki 3 6 18 +Manufacturer#5 almond azure blanched chiffon midnight 1 23 35 +Manufacturer#5 almond azure blanched chiffon midnight 2 23 52 +Manufacturer#5 almond azure blanched chiffon midnight 3 23 69 +Manufacturer#5 almond antique blue firebrick mint 1 31 77 +Manufacturer#5 almond antique blue firebrick mint 2 31 85 +Manufacturer#5 almond antique blue firebrick mint 3 31 93 +Manufacturer#5 almond aquamarine dodger light gainsboro 1 46 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 2 46 123 +Manufacturer#5 almond aquamarine dodger light gainsboro 3 46 138 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 26. testGroupByHavingWithSWQAndAlias-0-b996a664b06e5741c08079d5c38241bc b/sql/hive/src/test/resources/golden/windowing.q -- 26. testGroupByHavingWithSWQAndAlias-0-b996a664b06e5741c08079d5c38241bc new file mode 100644 index 0000000000000000000000000000000000000000..2c30e652aa26d6baac06f06ebab3fd7ac12f010e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 26. testGroupByHavingWithSWQAndAlias-0-b996a664b06e5741c08079d5c38241bc @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 27. testMultipleRangeWindows-0-227e080e337d734dd88ff814b3b412e4 b/sql/hive/src/test/resources/golden/windowing.q -- 27. testMultipleRangeWindows-0-227e080e337d734dd88ff814b3b412e4 new file mode 100644 index 0000000000000000000000000000000000000000..b2a91ba727a726f08ba54c463613def82663fa9a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 27. testMultipleRangeWindows-0-227e080e337d734dd88ff814b3b412e4 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 4 10 +Manufacturer#1 almond antique burnished rose metallic 2 4 10 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 10 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/windowing.q -- 28. testPartOrderInUDAFInvoke-0-25912ae7d18c91cc09e17e57968fb5db b/sql/hive/src/test/resources/golden/windowing.q -- 28. testPartOrderInUDAFInvoke-0-25912ae7d18c91cc09e17e57968fb5db new file mode 100644 index 0000000000000000000000000000000000000000..5bcb0fa941d65a0b26da9af9628d211eeecf5749 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 28. testPartOrderInUDAFInvoke-0-25912ae7d18c91cc09e17e57968fb5db @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 +Manufacturer#1 almond antique burnished rose metallic 2 44 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 +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/windowing.q -- 29. testPartOrderInWdwDef-0-88945892370ccbc1125a927a3d55342a b/sql/hive/src/test/resources/golden/windowing.q -- 29. testPartOrderInWdwDef-0-88945892370ccbc1125a927a3d55342a new file mode 100644 index 0000000000000000000000000000000000000000..5bcb0fa941d65a0b26da9af9628d211eeecf5749 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 29. testPartOrderInWdwDef-0-88945892370ccbc1125a927a3d55342a @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 +Manufacturer#1 almond antique burnished rose metallic 2 44 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 +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/windowing.q -- 3. testGroupByHavingWithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100 b/sql/hive/src/test/resources/golden/windowing.q -- 3. testGroupByHavingWithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100 new file mode 100644 index 0000000000000000000000000000000000000000..2c30e652aa26d6baac06f06ebab3fd7ac12f010e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 3. testGroupByHavingWithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 30. testDefaultPartitioningSpecRules-0-fa80b09c99e3c1487de48ea71a88dada b/sql/hive/src/test/resources/golden/windowing.q -- 30. testDefaultPartitioningSpecRules-0-fa80b09c99e3c1487de48ea71a88dada new file mode 100644 index 0000000000000000000000000000000000000000..698a44349d2a65cef205e553e199ae310995b0ea --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 30. testDefaultPartitioningSpecRules-0-fa80b09c99e3c1487de48ea71a88dada @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 4 +Manufacturer#1 almond antique burnished rose metallic 2 44 4 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 38 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 44 +Manufacturer#1 almond aquamarine burnished black steel 28 110 72 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 114 +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/windowing.q -- 36. testRankWithPartitioning-0-45ccbaf0ee083858f7661c66b11d4768 b/sql/hive/src/test/resources/golden/windowing.q -- 36. testRankWithPartitioning-0-45ccbaf0ee083858f7661c66b11d4768 new file mode 100644 index 0000000000000000000000000000000000000000..e35257d98382ee79f5a22375f8ee3be3266144e9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 36. testRankWithPartitioning-0-45ccbaf0ee083858f7661c66b11d4768 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 +Manufacturer#1 almond aquamarine burnished black steel 28 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 +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/windowing.q -- 37. testPartitioningVariousForms-0-3436e50214f9afdec84334e10faa931a b/sql/hive/src/test/resources/golden/windowing.q -- 37. testPartitioningVariousForms-0-3436e50214f9afdec84334e10faa931a new file mode 100644 index 0000000000000000000000000000000000000000..9c0ca6c7a00ba7e54b8f9a47d3fd332950d2bac1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 37. testPartitioningVariousForms-0-3436e50214f9afdec84334e10faa931a @@ -0,0 +1,26 @@ +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +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/windowing.q -- 38. testPartitioningVariousForms2-0-cba9d84a6b1bb5e36595338d4602377e b/sql/hive/src/test/resources/golden/windowing.q -- 38. testPartitioningVariousForms2-0-cba9d84a6b1bb5e36595338d4602377e new file mode 100644 index 0000000000000000000000000000000000000000..fc27df2f2b648574739571f20a558eb06d6d876f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 38. testPartitioningVariousForms2-0-cba9d84a6b1bb5e36595338d4602377e @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1173.15 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 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/windowing.q -- 39. testUDFOnOrderCols-0-7647562850dd367ef1e6c63117805423 b/sql/hive/src/test/resources/golden/windowing.q -- 39. testUDFOnOrderCols-0-7647562850dd367ef1e6c63117805423 new file mode 100644 index 0000000000000000000000000000000000000000..e5a541f56f6f33930707dc391344b867357f1cb7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 39. testUDFOnOrderCols-0-7647562850dd367ef1e6c63117805423 @@ -0,0 +1,26 @@ +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 PROMO PLATED TIN ROMO PLATED TIN 4 +Manufacturer#1 STANDARD ANODIZED STEEL TANDARD ANODIZED STEEL 6 +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/windowing.q -- 4. testCount-0-e6e97e884327df86f16b870527ec026c b/sql/hive/src/test/resources/golden/windowing.q -- 4. testCount-0-e6e97e884327df86f16b870527ec026c new file mode 100644 index 0000000000000000000000000000000000000000..bf8e620a304afd4400001a8a0ae7c0b426440224 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 4. testCount-0-e6e97e884327df86f16b870527ec026c @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique chartreuse lavender yellow 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 4 +Manufacturer#1 almond aquamarine burnished black steel 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 +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/windowing.q -- 40. testNoBetweenForRows-0-99007f45b6406869e048b0e4eb9213f1 b/sql/hive/src/test/resources/golden/windowing.q -- 40. testNoBetweenForRows-0-99007f45b6406869e048b0e4eb9213f1 new file mode 100644 index 0000000000000000000000000000000000000000..1e29df62901d0eedae90d06ee520317d100f7948 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 40. testNoBetweenForRows-0-99007f45b6406869e048b0e4eb9213f1 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 8749.730000000001 +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/windowing.q -- 41. testNoBetweenForRange-0-d81a591e90950de291d2f133793e9283 b/sql/hive/src/test/resources/golden/windowing.q -- 41. testNoBetweenForRange-0-d81a591e90950de291d2f133793e9283 new file mode 100644 index 0000000000000000000000000000000000000000..a620479fe406408a31b35c5b759c58b5d428c984 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 41. testNoBetweenForRange-0-d81a591e90950de291d2f133793e9283 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 3948.8900000000003 +Manufacturer#1 almond aquamarine burnished black steel 28 5363.31 +Manufacturer#1 almond antique chartreuse lavender yellow 34 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 8749.730000000001 +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/windowing.q -- 42. testUnboundedFollowingForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0 b/sql/hive/src/test/resources/golden/windowing.q -- 42. testUnboundedFollowingForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0 new file mode 100644 index 0000000000000000000000000000000000000000..74147d2571a1556eb4116a83dad7b9eee6156f7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 42. testUnboundedFollowingForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 8749.730000000001 +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/windowing.q -- 43. testUnboundedFollowingForRange-0-3cd04e5f2398853c4850f4f86142bb39 b/sql/hive/src/test/resources/golden/windowing.q -- 43. testUnboundedFollowingForRange-0-3cd04e5f2398853c4850f4f86142bb39 new file mode 100644 index 0000000000000000000000000000000000000000..49d003b5de1391cdcd8d385d48fb3f5a78289328 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 43. testUnboundedFollowingForRange-0-3cd04e5f2398853c4850f4f86142bb39 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 8749.730000000001 +Manufacturer#1 almond antique burnished rose metallic 2 8749.730000000001 +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/windowing.q -- 44. testOverNoPartitionSingleAggregate-0-cb3d2f8c1296044dc2658876bb6103ae b/sql/hive/src/test/resources/golden/windowing.q -- 44. testOverNoPartitionSingleAggregate-0-cb3d2f8c1296044dc2658876bb6103ae new file mode 100644 index 0000000000000000000000000000000000000000..5982c9ee2a4d417ac18bedbc8cfdd38186351479 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 44. testOverNoPartitionSingleAggregate-0-cb3d2f8c1296044dc2658876bb6103ae @@ -0,0 +1,26 @@ +almond antique blue firebrick mint 1789.69 1546.78 +almond antique burnished rose metallic 1173.15 1546.78 +almond antique burnished rose metallic 1173.15 1546.78 +almond antique chartreuse khaki white 1671.68 1546.78 +almond antique chartreuse lavender yellow 1753.76 1546.78 +almond antique forest lavender goldenrod 1190.27 1546.78 +almond antique gainsboro frosted violet 1620.67 1546.78 +almond antique medium spring khaki 1611.66 1546.78 +almond antique metallic orange dim 1410.39 1546.78 +almond antique misty red olive 1922.98 1546.78 +almond antique olive coral navajo 1337.29 1546.78 +almond antique salmon chartreuse burlywood 1602.59 1546.78 +almond antique sky peru orange 1788.73 1546.78 +almond antique violet chocolate turquoise 1690.68 1546.78 +almond antique violet mint lemon 1375.42 1546.78 +almond antique violet turquoise frosted 1800.7 1546.78 +almond aquamarine burnished black steel 1414.42 1546.78 +almond aquamarine dodger light gainsboro 1018.1 1546.78 +almond aquamarine floral ivory bisque 1206.26 1546.78 +almond aquamarine midnight light salmon 2031.98 1546.78 +almond aquamarine pink moccasin thistle 1632.66 1546.78 +almond aquamarine rose maroon antique 1698.66 1546.78 +almond aquamarine sandy cyan gainsboro 1701.6 1546.78 +almond aquamarine yellow dodger mint 1844.92 1546.78 +almond azure aquamarine papaya violet 1290.35 1546.78 +almond azure blanched chiffon midnight 1464.48 1546.78 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 5. testCountWithWindowingUDAF-0-3bde93728761b780a745c2ce0398aa0f b/sql/hive/src/test/resources/golden/windowing.q -- 5. testCountWithWindowingUDAF-0-3bde93728761b780a745c2ce0398aa0f new file mode 100644 index 0000000000000000000000000000000000000000..00d41fc0bcd9ccf0b53c129f1cd8ed33555b14a5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 5. testCountWithWindowingUDAF-0-3bde93728761b780a745c2ce0398aa0f @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 1 1 2 1173.15 1173.15 2 0 +Manufacturer#1 almond antique burnished rose metallic 1 1 2 1173.15 2346.3 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 3 2 3 1753.76 4100.06 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 4 3 4 1602.59 5702.650000000001 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 5 4 5 1414.42 7117.070000000001 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 5 6 1632.66 8749.730000000001 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 1 1690.68 1690.68 14 0 +Manufacturer#2 almond antique violet turquoise frosted 2 2 2 1800.7 3491.38 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 3 3 3 2031.98 5523.360000000001 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 4 4 4 1698.66 7222.02 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 5 5 1701.6 8923.62 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 1 1 1 1671.68 1671.68 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 2 2 2 1190.27 2861.95 14 -3 +Manufacturer#3 almond antique metallic orange dim 3 3 3 1410.39 4272.34 19 5 +Manufacturer#3 almond antique misty red olive 4 4 4 1922.98 6195.32 1 -18 +Manufacturer#3 almond antique olive coral navajo 5 5 5 1337.29 7532.61 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 1 1620.67 1620.67 10 0 +Manufacturer#4 almond antique violet mint lemon 2 2 2 1375.42 2996.09 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 3 3 3 1206.26 4202.35 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 4 4 4 1844.92 6047.27 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 5 5 5 1290.35 7337.620000000001 12 5 +Manufacturer#5 almond antique blue firebrick mint 1 1 1 1789.69 1789.69 31 0 +Manufacturer#5 almond antique medium spring khaki 2 2 2 1611.66 3401.3500000000004 6 -25 +Manufacturer#5 almond antique sky peru orange 3 3 3 1788.73 5190.08 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 4 4 1018.1 6208.18 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 5 5 5 1464.48 7672.66 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 6. testCountInSubQ-0-73d5274a21d4f4fd51d2a0f1d98516ce b/sql/hive/src/test/resources/golden/windowing.q -- 6. testCountInSubQ-0-73d5274a21d4f4fd51d2a0f1d98516ce new file mode 100644 index 0000000000000000000000000000000000000000..98c09e4fe15c38615859aae6989676a7e025ee92 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 6. testCountInSubQ-0-73d5274a21d4f4fd51d2a0f1d98516ce @@ -0,0 +1,26 @@ +1 1 2 1173.15 0 +1 1 2 2346.3 0 +3 2 3 4100.06 32 +4 3 4 5702.650000000001 -28 +5 4 5 7117.070000000001 22 +6 5 6 8749.730000000001 14 +1 1 1 1690.68 0 +2 2 2 3491.38 26 +3 3 3 5523.360000000001 -38 +4 4 4 7222.02 23 +5 5 5 8923.62 -7 +1 1 1 1671.68 0 +2 2 2 2861.95 -3 +3 3 3 4272.34 5 +4 4 4 6195.32 -18 +5 5 5 7532.61 44 +1 1 1 1620.67 0 +2 2 2 2996.09 29 +3 3 3 4202.35 -12 +4 4 4 6047.27 -20 +5 5 5 7337.620000000001 5 +1 1 1 1789.69 0 +2 2 2 3401.3500000000004 -25 +3 3 3 5190.08 -4 +4 4 4 6208.18 44 +5 5 5 7672.66 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 8. testMixedCaseAlias-0-4b1ad2515fb079012467e987f484a722 b/sql/hive/src/test/resources/golden/windowing.q -- 8. testMixedCaseAlias-0-4b1ad2515fb079012467e987f484a722 new file mode 100644 index 0000000000000000000000000000000000000000..e35257d98382ee79f5a22375f8ee3be3266144e9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 8. testMixedCaseAlias-0-4b1ad2515fb079012467e987f484a722 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 +Manufacturer#1 almond aquamarine burnished black steel 28 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 +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/windowing.q -- 9. testHavingWithWindowingNoGBY-0-70cdc0555a61ef08534a9ebebb95ebbf b/sql/hive/src/test/resources/golden/windowing.q -- 9. testHavingWithWindowingNoGBY-0-70cdc0555a61ef08534a9ebebb95ebbf new file mode 100644 index 0000000000000000000000000000000000000000..850c41c8115d6f4ee3c712324d68fc2a9b5ac3c7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 9. testHavingWithWindowingNoGBY-0-70cdc0555a61ef08534a9ebebb95ebbf @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-0-d3f50875bd5dff172cf813fdb7d738eb b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-0-d3f50875bd5dff172cf813fdb7d738eb new file mode 100644 index 0000000000000000000000000000000000000000..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-1-dda16565b98926fc3587de937b9401c7 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-1-dda16565b98926fc3587de937b9401c7 new file mode 100644 index 0000000000000000000000000000000000000000..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-2-374e39786feb745cd70f25be58bfa24 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-2-374e39786feb745cd70f25be58bfa24 new file mode 100644 index 0000000000000000000000000000000000000000..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-3-d2b5e23edec42a62e61750b110ecbaac b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-3-d2b5e23edec42a62e61750b110ecbaac new file mode 100644 index 0000000000000000000000000000000000000000..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-3-d2b5e23edec42a62e61750b110ecbaac @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-4-50d0c630159068b5b8ccdeb76493f1f7 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-4-50d0c630159068b5b8ccdeb76493f1f7 new file mode 100644 index 0000000000000000000000000000000000000000..850c41c8115d6f4ee3c712324d68fc2a9b5ac3c7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-4-50d0c630159068b5b8ccdeb76493f1f7 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-5-3f95cd6f4add7a2d0101fe3dd97e5082 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-5-3f95cd6f4add7a2d0101fe3dd97e5082 new file mode 100644 index 0000000000000000000000000000000000000000..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-5-3f95cd6f4add7a2d0101fe3dd97e5082 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-0-d3f50875bd5dff172cf813fdb7d738eb b/sql/hive/src/test/resources/golden/windowing_columnPruning-0-d3f50875bd5dff172cf813fdb7d738eb new file mode 100644 index 0000000000000000000000000000000000000000..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-1-dda16565b98926fc3587de937b9401c7 b/sql/hive/src/test/resources/golden/windowing_columnPruning-1-dda16565b98926fc3587de937b9401c7 new file mode 100644 index 0000000000000000000000000000000000000000..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-2-374e39786feb745cd70f25be58bfa24 b/sql/hive/src/test/resources/golden/windowing_columnPruning-2-374e39786feb745cd70f25be58bfa24 new file mode 100644 index 0000000000000000000000000000000000000000..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-3-9294b4a22bc396ff2accabd53c5da98b b/sql/hive/src/test/resources/golden/windowing_columnPruning-3-9294b4a22bc396ff2accabd53c5da98b new file mode 100644 index 0000000000000000000000000000000000000000..1b5ae55383a4a55fd7280a1e7a57cf21e0c27cf1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_columnPruning-3-9294b4a22bc396ff2accabd53c5da98b @@ -0,0 +1,26 @@ +2 1173.15 +2 2346.3 +34 4100.06 +6 5702.650000000001 +28 7117.070000000001 +42 8749.730000000001 +14 1690.68 +40 3491.38 +2 5523.360000000001 +25 7222.02 +18 8923.62 +17 1671.68 +14 2861.95 +19 4272.34 +1 6195.32 +45 7532.61 +10 1620.67 +39 2996.09 +27 4202.35 +7 6047.27 +12 7337.620000000001 +31 1789.69 +6 3401.3500000000004 +2 5190.08 +46 6208.18 +23 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-4-445cab062581c449ceffcb368cdf133 b/sql/hive/src/test/resources/golden/windowing_columnPruning-4-445cab062581c449ceffcb368cdf133 new file mode 100644 index 0000000000000000000000000000000000000000..1b5ae55383a4a55fd7280a1e7a57cf21e0c27cf1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_columnPruning-4-445cab062581c449ceffcb368cdf133 @@ -0,0 +1,26 @@ +2 1173.15 +2 2346.3 +34 4100.06 +6 5702.650000000001 +28 7117.070000000001 +42 8749.730000000001 +14 1690.68 +40 3491.38 +2 5523.360000000001 +25 7222.02 +18 8923.62 +17 1671.68 +14 2861.95 +19 4272.34 +1 6195.32 +45 7532.61 +10 1620.67 +39 2996.09 +27 4202.35 +7 6047.27 +12 7337.620000000001 +31 1789.69 +6 3401.3500000000004 +2 5190.08 +46 6208.18 +23 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-5-89110070c761eafb992eb9315128b53f b/sql/hive/src/test/resources/golden/windowing_columnPruning-5-89110070c761eafb992eb9315128b53f new file mode 100644 index 0000000000000000000000000000000000000000..e426c725b0e36aaca0bce055ba354e43c32b12a1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_columnPruning-5-89110070c761eafb992eb9315128b53f @@ -0,0 +1,26 @@ +Manufacturer#1 1173.15 +Manufacturer#1 2346.3 +Manufacturer#1 4100.06 +Manufacturer#1 5702.650000000001 +Manufacturer#1 7117.070000000001 +Manufacturer#1 8749.730000000001 +Manufacturer#2 1690.68 +Manufacturer#2 3491.38 +Manufacturer#2 5523.360000000001 +Manufacturer#2 7222.02 +Manufacturer#2 8923.62 +Manufacturer#3 1671.68 +Manufacturer#3 2861.95 +Manufacturer#3 4272.34 +Manufacturer#3 6195.32 +Manufacturer#3 7532.61 +Manufacturer#4 1620.67 +Manufacturer#4 2996.09 +Manufacturer#4 4202.35 +Manufacturer#4 6047.27 +Manufacturer#4 7337.620000000001 +Manufacturer#5 1789.69 +Manufacturer#5 3401.3500000000004 +Manufacturer#5 5190.08 +Manufacturer#5 6208.18 +Manufacturer#5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 1-0-12a92d8800e0da8b515ba3eaf6a7fd0f b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 1-0-12a92d8800e0da8b515ba3eaf6a7fd0f new file mode 100644 index 0000000000000000000000000000000000000000..acc4f3bc2a2d53efffb11184bcf81c566187ad18 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 1-0-12a92d8800e0da8b515ba3eaf6a7fd0f @@ -0,0 +1,1049 @@ + 1 4294967354 + 2 8589934811 + 3 12884902227 +alice allen 1 4294967487 +alice allen 2 8589935012 +alice allen 3 12884902543 +alice brown 1 4294967355 +alice carson 1 4294967370 +alice davidson 1 4294967517 +alice falkner 1 4294967316 +alice garcia 1 4294967369 +alice hernandez 1 4294967314 +alice hernandez 2 8589934613 +alice johnson 1 4294967424 +alice king 1 4294967387 +alice king 2 8589934903 +alice king 3 12884902449 +alice laertes 1 4294967519 +alice laertes 2 8589935038 +alice miller 1 4294967324 +alice nixon 1 4294967484 +alice nixon 2 8589934894 +alice nixon 3 12884902307 +alice ovid 1 4294967412 +alice polk 1 4294967366 +alice quirinius 1 4294967505 +alice quirinius 2 8589935054 +alice robinson 1 4294967502 +alice robinson 2 8589934947 +alice steinbeck 1 4294967549 +alice steinbeck 2 8589934913 +alice steinbeck 3 12884902387 +alice underhill 1 4294967441 +alice van buren 1 4294967428 +alice xylophone 1 4294967519 +alice xylophone 2 8589934877 +alice xylophone 3 12884902240 +alice zipper 1 4294967380 +alice zipper 2 8589934919 +alice zipper 3 12884902439 +bob brown 1 4294967431 +bob brown 2 8589934853 +bob brown 3 12884902280 +bob carson 1 4294967408 +bob davidson 1 4294967435 +bob davidson 2 8589934939 +bob davidson 3 12884902293 +bob ellison 1 4294967530 +bob ellison 2 8589934966 +bob ellison 3 12884902328 +bob ellison 4 17179869672 +bob falkner 1 4294967464 +bob garcia 1 4294967435 +bob garcia 2 8589934804 +bob garcia 3 12884902148 +bob garcia 4 17179869587 +bob garcia 5 21474836905 +bob hernandez 1 4294967500 +bob ichabod 1 4294967424 +bob king 1 4294967443 +bob king 2 8589934740 +bob king 3 12884902279 +bob laertes 1 4294967472 +bob laertes 2 8589934852 +bob miller 1 4294967349 +bob ovid 1 4294967401 +bob ovid 2 8589934801 +bob ovid 3 12884902313 +bob ovid 4 17179869708 +bob polk 1 4294967337 +bob quirinius 1 4294967346 +bob steinbeck 1 4294967342 +bob van buren 1 4294967422 +bob white 1 4294967493 +bob white 2 8589934855 +bob xylophone 1 4294967407 +bob xylophone 2 8589934872 +bob young 1 4294967413 +bob zipper 1 4294967416 +bob zipper 2 8589934769 +bob zipper 3 12884902068 +calvin allen 1 4294967373 +calvin brown 1 4294967530 +calvin brown 2 8589934967 +calvin brown 3 12884902378 +calvin carson 1 4294967534 +calvin davidson 1 4294967437 +calvin davidson 2 8589934905 +calvin ellison 1 4294967480 +calvin falkner 1 4294967305 +calvin falkner 2 8589934723 +calvin falkner 3 12884902155 +calvin falkner 4 17179869455 +calvin falkner 5 21474836800 +calvin falkner 6 25769804250 +calvin garcia 1 4294967492 +calvin hernandez 1 4294967341 +calvin johnson 1 4294967546 +calvin laertes 1 4294967499 +calvin laertes 2 8589934930 +calvin nixon 1 4294967488 +calvin nixon 2 8589934788 +calvin nixon 3 12884902200 +calvin ovid 1 4294967343 +calvin ovid 2 8589934881 +calvin ovid 3 12884902210 +calvin ovid 4 17179869559 +calvin polk 1 4294967416 +calvin quirinius 1 4294967532 +calvin quirinius 2 8589935053 +calvin robinson 1 4294967326 +calvin steinbeck 1 4294967417 +calvin steinbeck 2 8589934891 +calvin steinbeck 3 12884902396 +calvin thompson 1 4294967346 +calvin thompson 2 8589934859 +calvin underhill 1 4294967478 +calvin van buren 1 4294967300 +calvin van buren 2 8589934808 +calvin white 1 4294967304 +calvin white 2 8589934848 +calvin xylophone 1 4294967299 +calvin xylophone 2 8589934675 +calvin xylophone 3 12884902133 +calvin young 1 4294967410 +calvin young 2 8589934752 +calvin zipper 1 4294967441 +calvin zipper 2 8589934960 +david allen 1 4294967381 +david allen 2 8589934752 +david brown 1 4294967544 +david brown 2 8589934870 +david davidson 1 4294967487 +david davidson 2 8589934952 +david davidson 3 12884902474 +david davidson 4 17179869819 +david ellison 1 4294967477 +david ellison 2 8589934963 +david ellison 3 12884902426 +david hernandez 1 4294967324 +david ichabod 1 4294967385 +david ichabod 2 8589934872 +david laertes 1 4294967385 +david nixon 1 4294967381 +david ovid 1 4294967396 +david ovid 2 8589934839 +david quirinius 1 4294967375 +david quirinius 2 8589934905 +david quirinius 3 12884902362 +david robinson 1 4294967465 +david robinson 2 8589934933 +david thompson 1 4294967361 +david underhill 1 4294967331 +david underhill 2 8589934715 +david underhill 3 12884902185 +david van buren 1 4294967438 +david van buren 2 8589934747 +david white 1 4294967428 +david xylophone 1 8589934898 +david xylophone 1 8589934898 +david xylophone 3 12884902378 +david young 1 4294967296 +david young 2 8589934601 +ethan allen 1 4294967351 +ethan brown 1 4294967477 +ethan brown 2 8589934897 +ethan brown 3 12884902217 +ethan brown 4 17179869548 +ethan brown 5 21474836951 +ethan brown 6 25769804375 +ethan carson 1 4294967352 +ethan ellison 1 4294967514 +ethan ellison 2 8589934923 +ethan falkner 1 4294967318 +ethan falkner 2 8589934779 +ethan garcia 1 4294967310 +ethan hernandez 1 4294967349 +ethan johnson 1 4294967357 +ethan king 1 4294967413 +ethan laertes 1 4294967402 +ethan laertes 2 8589934859 +ethan laertes 3 12884902390 +ethan laertes 4 17179869880 +ethan laertes 5 21474837302 +ethan laertes 6 25769804603 +ethan laertes 7 30064771974 +ethan miller 1 4294967352 +ethan nixon 1 4294967499 +ethan ovid 1 4294967452 +ethan polk 1 4294967329 +ethan polk 2 8589934711 +ethan polk 3 12884902253 +ethan polk 4 17179869732 +ethan quirinius 1 4294967501 +ethan quirinius 2 8589934852 +ethan quirinius 3 12884902200 +ethan robinson 1 4294967353 +ethan robinson 2 8589934855 +ethan underhill 1 4294967466 +ethan van buren 1 4294967511 +ethan white 1 4294967445 +ethan white 2 8589934872 +ethan xylophone 1 4294967543 +ethan zipper 1 4294967462 +ethan zipper 2 8589934815 +fred davidson 1 4294967512 +fred davidson 2 8589934936 +fred davidson 3 12884902424 +fred ellison 1 4294967470 +fred ellison 2 8589934901 +fred ellison 3 12884902294 +fred falkner 1 4294967340 +fred falkner 2 8589934887 +fred falkner 3 12884902187 +fred hernandez 1 4294967365 +fred ichabod 1 4294967342 +fred ichabod 2 8589934831 +fred johnson 1 4294967373 +fred king 1 4294967346 +fred king 2 8589934766 +fred laertes 1 4294967351 +fred miller 1 4294967490 +fred nixon 1 4294967514 +fred nixon 2 8589934811 +fred nixon 3 12884902293 +fred nixon 4 17179869668 +fred polk 1 4294967332 +fred polk 2 8589934775 +fred polk 3 12884902233 +fred polk 4 17179869740 +fred quirinius 1 4294967426 +fred quirinius 2 8589934951 +fred robinson 1 4294967461 +fred steinbeck 1 4294967411 +fred steinbeck 2 8589934740 +fred steinbeck 3 12884902212 +fred underhill 1 4294967387 +fred van buren 1 4294967431 +fred van buren 2 8589934812 +fred van buren 3 12884902338 +fred van buren 4 17179869801 +fred white 1 4294967434 +fred young 1 4294967495 +fred young 2 8589934980 +fred zipper 1 4294967447 +gabriella allen 1 4294967405 +gabriella allen 2 8589934939 +gabriella brown 1 4294967543 +gabriella brown 2 8589934946 +gabriella carson 1 4294967540 +gabriella davidson 1 4294967507 +gabriella ellison 1 4294967393 +gabriella ellison 2 8589934733 +gabriella falkner 1 4294967378 +gabriella falkner 2 8589934901 +gabriella falkner 3 12884902335 +gabriella garcia 1 4294967419 +gabriella hernandez 1 4294967481 +gabriella hernandez 2 8589934943 +gabriella ichabod 1 4294967337 +gabriella ichabod 2 8589934725 +gabriella ichabod 3 12884902062 +gabriella ichabod 4 17179869382 +gabriella ichabod 5 21474836880 +gabriella king 1 4294967434 +gabriella king 2 8589934827 +gabriella laertes 1 4294967410 +gabriella miller 1 4294967363 +gabriella ovid 1 4294967482 +gabriella ovid 2 8589935004 +gabriella polk 1 4294967410 +gabriella polk 2 8589934712 +gabriella steinbeck 1 4294967500 +gabriella steinbeck 2 8589934935 +gabriella thompson 1 4294967299 +gabriella thompson 2 8589934711 +gabriella thompson 3 12884902196 +gabriella van buren 1 4294967457 +gabriella van buren 2 8589934927 +gabriella white 1 4294967335 +gabriella young 1 4294967493 +gabriella young 2 8589934924 +gabriella zipper 1 4294967357 +gabriella zipper 2 8589934867 +holly allen 1 4294967327 +holly brown 1 4294967321 +holly brown 2 8589934659 +holly falkner 1 4294967324 +holly hernandez 1 4294967378 +holly hernandez 2 8589934921 +holly hernandez 3 12884902465 +holly hernandez 4 17179869773 +holly ichabod 1 4294967342 +holly ichabod 2 8589934800 +holly ichabod 3 12884902129 +holly johnson 1 4294967517 +holly johnson 2 8589934897 +holly johnson 3 12884902432 +holly king 1 4294967392 +holly king 2 8589934753 +holly laertes 1 4294967406 +holly miller 1 4294967388 +holly nixon 1 4294967383 +holly nixon 2 8589934707 +holly polk 1 4294967398 +holly polk 2 8589934832 +holly robinson 1 4294967532 +holly thompson 1 4294967529 +holly thompson 2 8589934868 +holly thompson 3 12884902242 +holly underhill 1 4294967383 +holly underhill 2 8589934894 +holly underhill 3 12884902330 +holly underhill 4 17179869856 +holly van buren 1 4294967539 +holly white 1 4294967320 +holly white 2 8589934735 +holly xylophone 1 4294967435 +holly young 1 4294967487 +holly young 2 8589934987 +holly zipper 1 4294967337 +holly zipper 2 8589934846 +irene allen 1 4294967518 +irene brown 1 4294967434 +irene brown 2 8589934862 +irene brown 3 12884902272 +irene carson 1 4294967473 +irene ellison 1 4294967379 +irene ellison 2 8589934797 +irene falkner 1 4294967404 +irene falkner 2 8589934952 +irene garcia 1 4294967498 +irene garcia 2 8589934869 +irene garcia 3 12884902192 +irene ichabod 1 4294967529 +irene ichabod 2 8589935038 +irene johnson 1 4294967468 +irene laertes 1 4294967481 +irene laertes 2 8589934780 +irene laertes 3 12884902116 +irene miller 1 4294967387 +irene nixon 1 4294967323 +irene nixon 2 8589934824 +irene nixon 3 12884902362 +irene ovid 1 4294967499 +irene ovid 2 8589934870 +irene ovid 3 12884902230 +irene polk 1 4294967521 +irene polk 2 8589934930 +irene polk 3 12884902395 +irene polk 4 17179869941 +irene polk 5 21474837237 +irene quirinius 1 4294967365 +irene quirinius 2 8589934751 +irene quirinius 3 12884902141 +irene robinson 1 4294967347 +irene steinbeck 1 4294967549 +irene thompson 1 4294967479 +irene underhill 1 4294967371 +irene underhill 2 8589934753 +irene van buren 1 4294967439 +irene van buren 2 8589934906 +irene xylophone 1 4294967436 +jessica brown 1 4294967496 +jessica carson 1 4294967389 +jessica carson 2 8589934897 +jessica carson 3 12884902345 +jessica davidson 1 4294967325 +jessica davidson 2 8589934709 +jessica davidson 3 12884902098 +jessica davidson 4 17179869569 +jessica ellison 1 4294967316 +jessica ellison 2 8589934721 +jessica falkner 1 4294967549 +jessica garcia 1 4294967540 +jessica garcia 2 8589935041 +jessica ichabod 1 4294967413 +jessica johnson 1 4294967497 +jessica johnson 2 8589934870 +jessica miller 1 4294967495 +jessica nixon 1 4294967311 +jessica nixon 2 8589934754 +jessica ovid 1 4294967330 +jessica ovid 2 8589934855 +jessica polk 1 4294967403 +jessica quirinius 1 4294967523 +jessica quirinius 2 8589934942 +jessica quirinius 3 12884902388 +jessica quirinius 4 17179869696 +jessica robinson 1 4294967542 +jessica thompson 1 4294967449 +jessica thompson 2 8589934763 +jessica underhill 1 4294967541 +jessica underhill 2 8589934844 +jessica underhill 3 12884902153 +jessica van buren 1 4294967344 +jessica white 1 4294967482 +jessica white 2 8589934929 +jessica white 3 12884902378 +jessica white 4 17179869687 +jessica white 5 21474837086 +jessica xylophone 1 4294967421 +jessica young 1 4294967382 +jessica young 2 8589934903 +jessica zipper 1 4294967334 +jessica zipper 2 8589934785 +jessica zipper 3 12884902157 +katie allen 1 4294967443 +katie brown 1 4294967420 +katie davidson 1 4294967459 +katie ellison 1 4294967486 +katie ellison 2 8589934861 +katie falkner 1 4294967362 +katie garcia 1 4294967306 +katie garcia 2 8589934680 +katie hernandez 1 4294967451 +katie ichabod 1 4294967330 +katie ichabod 2 8589934742 +katie ichabod 3 12884902209 +katie king 1 4294967339 +katie king 2 8589934760 +katie king 3 12884902199 +katie miller 1 4294967425 +katie miller 2 8589934859 +katie nixon 1 4294967500 +katie ovid 1 4294967519 +katie polk 1 4294967384 +katie polk 2 8589934926 +katie robinson 1 4294967310 +katie van buren 1 4294967335 +katie van buren 2 8589934647 +katie white 1 4294967337 +katie white 2 8589934643 +katie xylophone 1 4294967486 +katie young 1 4294967349 +katie young 2 8589934681 +katie young 3 12884902225 +katie zipper 1 4294967354 +katie zipper 2 8589934766 +luke allen 1 4294967533 +luke allen 2 8589934836 +luke allen 3 12884902346 +luke allen 4 17179869863 +luke allen 5 21474837208 +luke brown 1 4294967473 +luke davidson 1 4294967550 +luke davidson 2 8589934904 +luke ellison 1 4294967322 +luke ellison 2 8589934675 +luke ellison 3 12884902103 +luke falkner 1 4294967359 +luke falkner 2 8589934782 +luke garcia 1 4294967304 +luke garcia 2 8589934683 +luke ichabod 1 4294967324 +luke ichabod 2 8589934766 +luke johnson 1 4294967527 +luke johnson 2 8589934987 +luke johnson 3 12884902342 +luke laertes 1 4294967505 +luke laertes 2 8589935011 +luke laertes 3 12884902497 +luke laertes 4 17179869806 +luke laertes 5 21474837193 +luke miller 1 4294967497 +luke ovid 1 4294967492 +luke ovid 2 8589934901 +luke polk 1 4294967545 +luke polk 2 8589934873 +luke quirinius 1 4294967320 +luke robinson 1 4294967299 +luke robinson 2 8589934606 +luke thompson 1 4294967521 +luke underhill 1 4294967393 +luke underhill 2 8589934803 +luke underhill 3 12884902122 +luke van buren 1 4294967424 +luke white 1 4294967505 +luke xylophone 1 4294967382 +luke zipper 1 4294967353 +mike allen 1 4294967466 +mike brown 1 4294967369 +mike carson 1 4294967477 +mike carson 2 8589934934 +mike carson 3 12884902482 +mike davidson 1 4294967501 +mike davidson 2 8589934965 +mike ellison 1 4294967353 +mike ellison 2 8589934747 +mike ellison 3 12884902282 +mike ellison 4 17179869806 +mike ellison 5 21474837309 +mike falkner 1 4294967301 +mike garcia 1 4294967428 +mike garcia 2 8589934826 +mike garcia 3 12884902289 +mike hernandez 1 4294967316 +mike hernandez 2 8589934800 +mike ichabod 1 4294967494 +mike king 1 4294967323 +mike king 2 8589934848 +mike king 3 12884902248 +mike king 4 17179869595 +mike king 5 21474837046 +mike king 6 25769804478 +mike miller 1 4294967449 +mike nixon 1 4294967527 +mike nixon 2 8589935004 +mike polk 1 4294967389 +mike polk 2 8589934848 +mike polk 3 12884902351 +mike quirinius 1 4294967422 +mike steinbeck 1 4294967519 +mike steinbeck 2 8589934827 +mike steinbeck 3 12884902316 +mike steinbeck 4 17179869850 +mike van buren 1 4294967544 +mike van buren 2 8589935061 +mike white 1 4294967336 +mike white 2 8589934882 +mike white 3 12884902374 +mike white 4 17179869843 +mike young 1 4294967453 +mike young 2 8589934804 +mike young 3 12884902198 +mike zipper 1 4294967402 +mike zipper 2 8589934727 +mike zipper 3 12884902228 +nick allen 1 4294967507 +nick allen 2 8589934807 +nick brown 1 4294967334 +nick davidson 1 4294967357 +nick ellison 1 4294967397 +nick ellison 2 8589934699 +nick falkner 1 4294967480 +nick falkner 2 8589934923 +nick garcia 1 4294967384 +nick garcia 2 8589934797 +nick garcia 3 12884902319 +nick ichabod 1 4294967388 +nick ichabod 2 8589934758 +nick ichabod 3 12884902225 +nick johnson 1 4294967398 +nick johnson 2 8589934809 +nick laertes 1 4294967389 +nick miller 1 4294967550 +nick nixon 1 4294967482 +nick ovid 1 4294967488 +nick polk 1 4294967551 +nick quirinius 1 4294967316 +nick quirinius 2 8589934612 +nick robinson 1 4294967409 +nick robinson 2 8589934731 +nick steinbeck 1 4294967355 +nick thompson 1 4294967401 +nick underhill 1 4294967527 +nick van buren 1 4294967303 +nick xylophone 1 4294967460 +nick young 1 4294967405 +nick young 2 8589934917 +nick zipper 1 4294967430 +nick zipper 2 8589934796 +oscar allen 1 4294967500 +oscar brown 1 4294967331 +oscar carson 1 4294967460 +oscar carson 2 8589934904 +oscar carson 3 12884902286 +oscar carson 4 17179869599 +oscar carson 5 21474836960 +oscar davidson 1 4294967482 +oscar ellison 1 8589934740 +oscar ellison 1 8589934740 +oscar falkner 1 4294967526 +oscar garcia 1 4294967301 +oscar hernandez 1 4294967343 +oscar hernandez 2 8589934843 +oscar ichabod 1 4294967513 +oscar ichabod 2 8589934837 +oscar ichabod 3 12884902165 +oscar ichabod 4 17179869569 +oscar johnson 1 4294967418 +oscar johnson 2 8589934763 +oscar king 1 4294967465 +oscar king 2 8589934936 +oscar king 3 12884902469 +oscar laertes 1 4294967425 +oscar laertes 2 8589934876 +oscar laertes 3 12884902426 +oscar laertes 4 17179869786 +oscar nixon 1 4294967532 +oscar ovid 1 4294967508 +oscar ovid 2 8589934910 +oscar ovid 3 12884902418 +oscar polk 1 4294967325 +oscar polk 2 8589934713 +oscar quirinius 1 4294967416 +oscar quirinius 2 8589934932 +oscar quirinius 3 12884902390 +oscar quirinius 4 17179869763 +oscar robinson 1 4294967355 +oscar robinson 2 8589934681 +oscar robinson 3 12884902031 +oscar robinson 4 17179869383 +oscar steinbeck 1 4294967548 +oscar thompson 1 4294967453 +oscar thompson 2 8589934824 +oscar thompson 3 12884902197 +oscar thompson 4 17179869496 +oscar underhill 1 4294967374 +oscar van buren 1 4294967520 +oscar van buren 2 8589934990 +oscar van buren 3 12884902490 +oscar white 1 4294967454 +oscar white 2 8589934761 +oscar white 3 12884902163 +oscar white 4 17179869512 +oscar xylophone 1 4294967400 +oscar xylophone 2 8589934806 +oscar xylophone 3 12884902124 +oscar zipper 1 4294967449 +oscar zipper 2 8589934969 +oscar zipper 3 12884902458 +priscilla brown 1 4294967369 +priscilla brown 2 8589934897 +priscilla brown 3 12884902360 +priscilla carson 1 4294967489 +priscilla carson 2 8589934838 +priscilla carson 3 12884902270 +priscilla ichabod 1 4294967379 +priscilla ichabod 2 8589934926 +priscilla johnson 1 4294967535 +priscilla johnson 2 8589935003 +priscilla johnson 3 12884902308 +priscilla johnson 4 17179869707 +priscilla johnson 5 21474837167 +priscilla king 1 4294967385 +priscilla nixon 1 4294967388 +priscilla nixon 2 8589934849 +priscilla ovid 1 4294967528 +priscilla ovid 2 8589935035 +priscilla polk 1 4294967434 +priscilla quirinius 1 4294967347 +priscilla thompson 1 4294967497 +priscilla underhill 1 4294967520 +priscilla underhill 2 8589934853 +priscilla van buren 1 4294967318 +priscilla van buren 2 8589934809 +priscilla van buren 3 12884902351 +priscilla white 1 4294967419 +priscilla xylophone 1 4294967503 +priscilla xylophone 2 8589934956 +priscilla xylophone 3 12884902406 +priscilla young 1 4294967401 +priscilla young 2 8589934931 +priscilla zipper 1 4294967516 +priscilla zipper 2 8589934950 +quinn allen 1 4294967339 +quinn allen 2 8589934881 +quinn brown 1 4294967335 +quinn brown 2 8589934651 +quinn brown 3 12884902099 +quinn davidson 1 4294967478 +quinn davidson 2 8589934849 +quinn davidson 3 12884902238 +quinn davidson 4 17179869565 +quinn ellison 1 4294967392 +quinn ellison 2 8589934907 +quinn garcia 1 4294967344 +quinn garcia 2 8589934882 +quinn garcia 3 12884902395 +quinn garcia 4 17179869868 +quinn ichabod 1 4294967405 +quinn king 1 4294967538 +quinn king 2 8589934996 +quinn laertes 1 4294967533 +quinn laertes 2 8589934977 +quinn laertes 3 12884902524 +quinn nixon 1 4294967432 +quinn ovid 1 4294967340 +quinn quirinius 1 4294967347 +quinn robinson 1 4294967365 +quinn steinbeck 1 4294967358 +quinn steinbeck 2 8589934810 +quinn thompson 1 4294967488 +quinn thompson 2 8589934888 +quinn underhill 1 4294967307 +quinn underhill 2 8589934744 +quinn underhill 3 12884902278 +quinn van buren 1 4294967362 +quinn young 1 4294967392 +quinn zipper 1 4294967521 +quinn zipper 2 8589934944 +rachel allen 1 4294967334 +rachel allen 2 8589934713 +rachel brown 1 4294967451 +rachel brown 2 8589934886 +rachel brown 3 12884902325 +rachel brown 4 17179869632 +rachel brown 5 21474836938 +rachel carson 1 4294967461 +rachel carson 2 8589934777 +rachel davidson 1 4294967387 +rachel ellison 1 4294967423 +rachel falkner 1 4294967544 +rachel falkner 2 8589934892 +rachel falkner 3 12884902350 +rachel falkner 4 17179869809 +rachel johnson 1 4294967541 +rachel king 1 4294967442 +rachel king 2 8589934771 +rachel laertes 1 4294967446 +rachel laertes 2 8589934804 +rachel ovid 1 4294967481 +rachel ovid 2 8589934832 +rachel polk 1 4294967335 +rachel quirinius 1 4294967297 +rachel robinson 1 4294967344 +rachel robinson 2 8589934807 +rachel robinson 3 12884902135 +rachel thompson 1 4294967518 +rachel thompson 2 8589934881 +rachel thompson 3 12884902306 +rachel underhill 1 4294967382 +rachel white 1 4294967457 +rachel white 2 8589934793 +rachel young 1 4294967391 +rachel zipper 1 4294967434 +rachel zipper 2 8589934813 +sarah carson 1 4294967503 +sarah carson 2 8589934822 +sarah carson 3 12884902167 +sarah ellison 1 4294967542 +sarah falkner 1 4294967525 +sarah falkner 2 8589934974 +sarah garcia 1 4294967391 +sarah garcia 2 8589934849 +sarah garcia 3 12884902247 +sarah ichabod 1 4294967370 +sarah ichabod 2 8589934909 +sarah johnson 1 4294967433 +sarah johnson 2 8589934926 +sarah johnson 3 12884902235 +sarah johnson 4 17179869559 +sarah king 1 4294967496 +sarah king 2 8589935039 +sarah miller 1 4294967458 +sarah ovid 1 4294967350 +sarah robinson 1 4294967419 +sarah robinson 2 8589934917 +sarah steinbeck 1 4294967456 +sarah white 1 4294967514 +sarah white 2 8589934882 +sarah xylophone 1 4294967355 +sarah young 1 4294967442 +sarah zipper 1 4294967432 +tom brown 1 4294967432 +tom brown 2 8589934950 +tom carson 1 4294967388 +tom carson 2 8589934693 +tom carson 3 12884902227 +tom davidson 1 4294967507 +tom ellison 1 4294967487 +tom ellison 2 8589934844 +tom ellison 3 12884902188 +tom falkner 1 4294967382 +tom falkner 2 8589934837 +tom hernandez 1 8589934733 +tom hernandez 1 8589934733 +tom ichabod 1 4294967445 +tom johnson 1 4294967492 +tom johnson 2 8589934923 +tom king 1 4294967331 +tom laertes 1 4294967431 +tom laertes 2 8589934744 +tom miller 1 4294967366 +tom miller 2 8589934723 +tom miller 3 12884902078 +tom nixon 1 4294967506 +tom ovid 1 4294967512 +tom polk 1 4294967329 +tom polk 2 8589934869 +tom quirinius 1 4294967507 +tom quirinius 2 8589934823 +tom robinson 1 4294967457 +tom robinson 2 8589935008 +tom robinson 3 12884902462 +tom robinson 4 17179869770 +tom steinbeck 1 4294967447 +tom van buren 1 4294967374 +tom van buren 2 8589934703 +tom van buren 3 12884902195 +tom white 1 4294967413 +tom young 1 4294967539 +tom young 2 8589935074 +tom zipper 1 4294967526 +ulysses brown 1 4294967537 +ulysses carson 1 4294967323 +ulysses carson 2 8589934815 +ulysses carson 3 12884902127 +ulysses carson 4 17179869485 +ulysses davidson 1 4294967467 +ulysses ellison 1 4294967442 +ulysses garcia 1 4294967470 +ulysses hernandez 1 4294967449 +ulysses hernandez 2 8589934995 +ulysses hernandez 3 12884902393 +ulysses ichabod 1 4294967353 +ulysses ichabod 2 8589934728 +ulysses johnson 1 4294967432 +ulysses king 1 4294967537 +ulysses laertes 1 4294967391 +ulysses laertes 2 8589934938 +ulysses laertes 3 12884902431 +ulysses miller 1 4294967373 +ulysses miller 2 8589934808 +ulysses nixon 1 4294967296 +ulysses ovid 1 4294967394 +ulysses polk 1 4294967509 +ulysses polk 2 8589934960 +ulysses polk 3 12884902440 +ulysses polk 4 17179869745 +ulysses quirinius 1 4294967449 +ulysses robinson 1 4294967531 +ulysses steinbeck 1 4294967303 +ulysses steinbeck 2 8589934788 +ulysses thompson 1 4294967389 +ulysses underhill 1 4294967544 +ulysses underhill 2 8589934949 +ulysses underhill 3 12884902275 +ulysses underhill 4 17179869726 +ulysses underhill 5 21474837190 +ulysses underhill 6 25769804570 +ulysses underhill 7 30064771927 +ulysses van buren 1 4294967439 +ulysses white 1 4294967429 +ulysses white 2 8589934878 +ulysses xylophone 1 4294967524 +ulysses xylophone 2 8589935025 +ulysses xylophone 3 12884902473 +ulysses young 1 4294967427 +ulysses young 2 8589934763 +ulysses young 3 12884902154 +victor allen 1 4294967450 +victor allen 2 8589934776 +victor brown 1 4294967521 +victor brown 2 8589934864 +victor brown 3 12884902170 +victor brown 4 17179869625 +victor davidson 1 4294967419 +victor davidson 2 8589934720 +victor davidson 3 12884902156 +victor ellison 1 4294967362 +victor ellison 2 8589934831 +victor hernandez 1 4294967428 +victor hernandez 2 8589934733 +victor hernandez 3 12884902062 +victor hernandez 4 17179869402 +victor hernandez 5 21474836874 +victor johnson 1 4294967496 +victor johnson 2 8589934824 +victor johnson 3 12884902246 +victor king 1 4294967401 +victor king 2 8589934884 +victor laertes 1 4294967407 +victor laertes 2 8589934862 +victor miller 1 4294967410 +victor nixon 1 4294967424 +victor nixon 2 8589934803 +victor ovid 1 4294967355 +victor polk 1 4294967333 +victor quirinius 1 4294967520 +victor quirinius 2 8589934846 +victor robinson 1 4294967440 +victor robinson 2 8589934930 +victor steinbeck 1 4294967390 +victor steinbeck 2 8589934707 +victor steinbeck 3 12884902037 +victor thompson 1 4294967319 +victor van buren 1 4294967365 +victor van buren 2 8589934906 +victor white 1 4294967403 +victor white 2 8589934862 +victor xylophone 1 4294967331 +victor xylophone 2 8589934864 +victor xylophone 3 12884902262 +victor xylophone 4 17179869633 +victor xylophone 5 21474837062 +victor young 1 4294967337 +victor zipper 1 4294967428 +wendy allen 1 4294967473 +wendy allen 2 8589934989 +wendy allen 3 12884902367 +wendy brown 1 4294967337 +wendy brown 2 8589934817 +wendy ellison 1 4294967475 +wendy ellison 2 8589934989 +wendy falkner 1 4294967313 +wendy falkner 2 8589934810 +wendy falkner 3 12884902236 +wendy garcia 1 4294967394 +wendy garcia 2 8589934775 +wendy garcia 3 12884902088 +wendy garcia 4 17179869400 +wendy hernandez 1 4294967299 +wendy ichabod 1 4294967516 +wendy king 1 4294967420 +wendy king 2 8589934811 +wendy king 3 12884902252 +wendy laertes 1 4294967519 +wendy laertes 2 8589934939 +wendy laertes 3 12884902315 +wendy miller 1 4294967478 +wendy miller 2 8589934957 +wendy nixon 1 4294967407 +wendy nixon 2 8589934901 +wendy ovid 1 4294967464 +wendy ovid 2 8589934894 +wendy polk 1 4294967434 +wendy polk 2 8589934824 +wendy quirinius 1 4294967334 +wendy quirinius 2 8589934782 +wendy robinson 1 4294967302 +wendy robinson 2 8589934613 +wendy robinson 3 12884901977 +wendy steinbeck 1 4294967444 +wendy thompson 1 4294967301 +wendy thompson 2 8589934621 +wendy underhill 1 4294967540 +wendy underhill 2 8589934993 +wendy underhill 3 12884902410 +wendy van buren 1 4294967488 +wendy van buren 2 8589934835 +wendy white 1 4294967490 +wendy xylophone 1 4294967488 +wendy xylophone 2 8589934939 +wendy young 1 4294967395 +wendy young 2 8589934708 +xavier allen 1 4294967304 +xavier allen 2 8589934743 +xavier allen 3 12884902129 +xavier brown 1 4294967546 +xavier brown 2 8589935074 +xavier brown 3 12884902532 +xavier carson 1 4294967547 +xavier carson 2 8589934862 +xavier davidson 1 4294967361 +xavier davidson 2 8589934760 +xavier davidson 3 12884902204 +xavier ellison 1 4294967441 +xavier ellison 2 8589934914 +xavier garcia 1 4294967465 +xavier hernandez 1 4294967383 +xavier hernandez 2 8589934743 +xavier hernandez 3 12884902274 +xavier ichabod 1 4294967511 +xavier ichabod 2 8589934950 +xavier johnson 1 4294967507 +xavier johnson 2 8589934898 +xavier king 1 4294967456 +xavier king 2 8589934758 +xavier laertes 1 4294967450 +xavier ovid 1 4294967403 +xavier polk 1 4294967506 +xavier polk 2 8589934925 +xavier polk 3 12884902406 +xavier polk 4 17179869906 +xavier quirinius 1 4294967383 +xavier quirinius 2 8589934748 +xavier quirinius 3 12884902060 +xavier quirinius 4 17179869562 +xavier thompson 1 4294967444 +xavier underhill 1 4294967332 +xavier white 1 4294967473 +xavier white 2 8589934952 +xavier xylophone 1 4294967499 +xavier zipper 1 4294967547 +yuri allen 1 4294967528 +yuri allen 2 8589935079 +yuri brown 1 4294967433 +yuri brown 2 8589934960 +yuri carson 1 4294967317 +yuri carson 2 8589934851 +yuri ellison 1 4294967299 +yuri ellison 2 8589934697 +yuri falkner 1 4294967368 +yuri falkner 2 8589934891 +yuri garcia 1 4294967362 +yuri hernandez 1 4294967367 +yuri johnson 1 4294967421 +yuri johnson 2 8589934877 +yuri johnson 3 12884902361 +yuri king 1 4294967376 +yuri laertes 1 4294967402 +yuri laertes 2 8589934924 +yuri nixon 1 4294967400 +yuri nixon 2 8589934706 +yuri polk 1 4294967391 +yuri polk 2 8589934861 +yuri polk 3 12884902167 +yuri quirinius 1 4294967398 +yuri quirinius 2 8589934768 +yuri quirinius 3 12884902081 +yuri steinbeck 1 4294967535 +yuri steinbeck 2 8589934873 +yuri thompson 1 4294967447 +yuri underhill 1 4294967499 +yuri underhill 2 8589934900 +yuri white 1 4294967341 +yuri xylophone 1 4294967420 +zach allen 1 4294967507 +zach brown 1 4294967316 +zach brown 2 8589934728 +zach brown 3 12884902099 +zach brown 4 17179869452 +zach brown 5 21474836769 +zach carson 1 4294967463 +zach ellison 1 4294967471 +zach falkner 1 4294967362 +zach falkner 2 8589934717 +zach garcia 1 4294967481 +zach garcia 2 8589934854 +zach garcia 3 12884902240 +zach garcia 4 17179869723 +zach ichabod 1 4294967539 +zach ichabod 2 8589934912 +zach king 1 4294967424 +zach king 2 8589934956 +zach king 3 12884902458 +zach miller 1 4294967442 +zach miller 2 8589934772 +zach miller 3 12884902163 +zach ovid 1 4294967412 +zach ovid 2 8589934775 +zach ovid 3 12884902244 +zach ovid 4 17179869574 +zach quirinius 1 4294967299 +zach robinson 1 4294967325 +zach steinbeck 1 4294967469 +zach steinbeck 2 8589934834 +zach thompson 1 4294967405 +zach thompson 2 8589934730 +zach underhill 1 4294967496 +zach white 1 4294967501 +zach xylophone 1 4294967452 +zach xylophone 2 8589934755 +zach young 1 4294967297 +zach zipper 1 4294967497 +zach zipper 2 8589934855 +zach zipper 3 12884902222 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 3-0-455e41d9949a2d22bab634fd8e42f2b1 b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 3-0-455e41d9949a2d22bab634fd8e42f2b1 new file mode 100644 index 0000000000000000000000000000000000000000..f47923618a1aa0b5f509f751f6cbcfccd2eddaa4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 3-0-455e41d9949a2d22bab634fd8e42f2b1 @@ -0,0 +1 @@ +bob steinbeck 65637 9.699999809265137 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 4-0-cfad06ae8eba6b047d32a6a61dd59392 b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 4-0-cfad06ae8eba6b047d32a6a61dd59392 new file mode 100644 index 0000000000000000000000000000000000000000..f41eaa259cec04de5242979983d01a49534059c5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 4-0-cfad06ae8eba6b047d32a6a61dd59392 @@ -0,0 +1 @@ +bob steinbeck 1 1 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 5-0-d7ca7a61377cef3a9f721a28afdae012 b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 5-0-d7ca7a61377cef3a9f721a28afdae012 new file mode 100644 index 0000000000000000000000000000000000000000..5308b2eb457e776ab558341f540df6f19064ec66 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 5-0-d7ca7a61377cef3a9f721a28afdae012 @@ -0,0 +1 @@ +bob steinbeck 9.699999809265137 1 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 6-0-287bcc7679822bc7b684532b267bf11f b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 6-0-287bcc7679822bc7b684532b267bf11f new file mode 100644 index 0000000000000000000000000000000000000000..f41eaa259cec04de5242979983d01a49534059c5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 6-0-287bcc7679822bc7b684532b267bf11f @@ -0,0 +1 @@ +bob steinbeck 1 1 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-0-36217f6074daaacddb9fcb50a3f4fb5b b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-0-36217f6074daaacddb9fcb50a3f4fb5b new file mode 100644 index 0000000000000000000000000000000000000000..8150409e62d33a5813571880269de7a59d38ebe1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-0-36217f6074daaacddb9fcb50a3f4fb5b @@ -0,0 +1,1049 @@ + 1 + 1 + 1 +alice allen 2 +alice allen 1 +alice allen 1 +alice brown 1 +alice carson 1 +alice davidson 1 +alice falkner 1 +alice garcia 1 +alice hernandez 2 +alice hernandez 1 +alice johnson 2 +alice king 1 +alice king 1 +alice king 1 +alice laertes 1 +alice laertes 1 +alice miller 1 +alice nixon 1 +alice nixon 1 +alice nixon 1 +alice ovid 1 +alice polk 3 +alice quirinius 1 +alice quirinius 1 +alice robinson 1 +alice robinson 1 +alice steinbeck 1 +alice steinbeck 1 +alice steinbeck 1 +alice underhill 2 +alice van buren 1 +alice xylophone 1 +alice xylophone 1 +alice xylophone 1 +alice zipper 1 +alice zipper 1 +alice zipper 1 +bob brown 1 +bob brown 1 +bob brown 1 +bob carson 1 +bob davidson 1 +bob davidson 1 +bob davidson 1 +bob ellison 2 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob falkner 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob hernandez 1 +bob ichabod 1 +bob king 2 +bob king 1 +bob king 1 +bob laertes 2 +bob laertes 1 +bob miller 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob polk 1 +bob quirinius 1 +bob steinbeck 1 +bob van buren 1 +bob white 1 +bob white 1 +bob xylophone 1 +bob xylophone 1 +bob young 1 +bob zipper 2 +bob zipper 1 +bob zipper 1 +calvin allen 1 +calvin brown 2 +calvin brown 1 +calvin brown 1 +calvin carson 2 +calvin davidson 2 +calvin davidson 1 +calvin ellison 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin garcia 1 +calvin hernandez 2 +calvin johnson 1 +calvin laertes 1 +calvin laertes 1 +calvin nixon 3 +calvin nixon 1 +calvin nixon 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin polk 1 +calvin quirinius 1 +calvin quirinius 1 +calvin robinson 2 +calvin steinbeck 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin thompson 1 +calvin thompson 1 +calvin underhill 1 +calvin van buren 1 +calvin van buren 1 +calvin white 1 +calvin white 1 +calvin xylophone 2 +calvin xylophone 1 +calvin xylophone 1 +calvin young 1 +calvin young 1 +calvin zipper 1 +calvin zipper 1 +david allen 1 +david allen 1 +david brown 1 +david brown 1 +david davidson 1 +david davidson 1 +david davidson 1 +david davidson 1 +david ellison 1 +david ellison 1 +david ellison 1 +david hernandez 1 +david ichabod 1 +david ichabod 1 +david laertes 1 +david nixon 1 +david ovid 1 +david ovid 1 +david quirinius 1 +david quirinius 1 +david quirinius 1 +david robinson 1 +david robinson 1 +david thompson 3 +david underhill 1 +david underhill 1 +david underhill 1 +david van buren 1 +david van buren 1 +david white 1 +david xylophone 1 +david xylophone 1 +david xylophone 1 +david young 1 +david young 1 +ethan allen 1 +ethan brown 2 +ethan brown 2 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan carson 1 +ethan ellison 1 +ethan ellison 1 +ethan falkner 3 +ethan falkner 1 +ethan garcia 1 +ethan hernandez 1 +ethan johnson 1 +ethan king 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan miller 2 +ethan nixon 1 +ethan ovid 1 +ethan polk 2 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan quirinius 1 +ethan quirinius 1 +ethan quirinius 1 +ethan robinson 1 +ethan robinson 1 +ethan underhill 1 +ethan van buren 2 +ethan white 1 +ethan white 1 +ethan xylophone 1 +ethan zipper 1 +ethan zipper 1 +fred davidson 1 +fred davidson 1 +fred davidson 1 +fred ellison 1 +fred ellison 1 +fred ellison 1 +fred falkner 2 +fred falkner 1 +fred falkner 1 +fred hernandez 1 +fred ichabod 1 +fred ichabod 1 +fred johnson 1 +fred king 2 +fred king 1 +fred laertes 1 +fred miller 1 +fred nixon 2 +fred nixon 1 +fred nixon 1 +fred nixon 1 +fred polk 1 +fred polk 1 +fred polk 1 +fred polk 1 +fred quirinius 1 +fred quirinius 1 +fred robinson 1 +fred steinbeck 1 +fred steinbeck 1 +fred steinbeck 1 +fred underhill 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred white 1 +fred young 3 +fred young 1 +fred zipper 1 +gabriella allen 1 +gabriella allen 1 +gabriella brown 2 +gabriella brown 1 +gabriella carson 1 +gabriella davidson 1 +gabriella ellison 1 +gabriella ellison 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella garcia 1 +gabriella hernandez 1 +gabriella hernandez 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella king 1 +gabriella king 1 +gabriella laertes 1 +gabriella miller 1 +gabriella ovid 1 +gabriella ovid 1 +gabriella polk 1 +gabriella polk 1 +gabriella steinbeck 1 +gabriella steinbeck 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella van buren 2 +gabriella van buren 1 +gabriella white 1 +gabriella young 1 +gabriella young 1 +gabriella zipper 1 +gabriella zipper 1 +holly allen 1 +holly brown 1 +holly brown 1 +holly falkner 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 1 +holly ichabod 1 +holly ichabod 1 +holly ichabod 1 +holly johnson 2 +holly johnson 1 +holly johnson 1 +holly king 1 +holly king 1 +holly laertes 1 +holly miller 1 +holly nixon 1 +holly nixon 1 +holly polk 2 +holly polk 1 +holly robinson 2 +holly thompson 1 +holly thompson 1 +holly thompson 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly van buren 1 +holly white 1 +holly white 1 +holly xylophone 1 +holly young 1 +holly young 1 +holly zipper 1 +holly zipper 1 +irene allen 1 +irene brown 1 +irene brown 1 +irene brown 1 +irene carson 1 +irene ellison 1 +irene ellison 1 +irene falkner 1 +irene falkner 1 +irene garcia 1 +irene garcia 1 +irene garcia 1 +irene ichabod 1 +irene ichabod 1 +irene johnson 2 +irene laertes 1 +irene laertes 1 +irene laertes 1 +irene miller 2 +irene nixon 1 +irene nixon 1 +irene nixon 1 +irene ovid 1 +irene ovid 1 +irene ovid 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene quirinius 1 +irene quirinius 1 +irene quirinius 1 +irene robinson 1 +irene steinbeck 1 +irene thompson 2 +irene underhill 2 +irene underhill 1 +irene van buren 1 +irene van buren 1 +irene xylophone 1 +jessica brown 2 +jessica carson 1 +jessica carson 1 +jessica carson 1 +jessica davidson 2 +jessica davidson 1 +jessica davidson 1 +jessica davidson 1 +jessica ellison 2 +jessica ellison 1 +jessica falkner 1 +jessica garcia 1 +jessica garcia 1 +jessica ichabod 1 +jessica johnson 1 +jessica johnson 1 +jessica miller 1 +jessica nixon 1 +jessica nixon 1 +jessica ovid 2 +jessica ovid 1 +jessica polk 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica robinson 1 +jessica thompson 1 +jessica thompson 1 +jessica underhill 1 +jessica underhill 1 +jessica underhill 1 +jessica van buren 1 +jessica white 2 +jessica white 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica xylophone 1 +jessica young 1 +jessica young 1 +jessica zipper 1 +jessica zipper 1 +jessica zipper 1 +katie allen 1 +katie brown 1 +katie davidson 1 +katie ellison 1 +katie ellison 1 +katie falkner 1 +katie garcia 1 +katie garcia 1 +katie hernandez 1 +katie ichabod 2 +katie ichabod 1 +katie ichabod 1 +katie king 1 +katie king 1 +katie king 1 +katie miller 1 +katie miller 1 +katie nixon 1 +katie ovid 1 +katie polk 1 +katie polk 1 +katie robinson 1 +katie van buren 2 +katie van buren 1 +katie white 1 +katie white 1 +katie xylophone 1 +katie young 1 +katie young 1 +katie young 1 +katie zipper 1 +katie zipper 1 +luke allen 2 +luke allen 1 +luke allen 1 +luke allen 1 +luke allen 1 +luke brown 2 +luke davidson 1 +luke davidson 1 +luke ellison 1 +luke ellison 1 +luke ellison 1 +luke falkner 2 +luke falkner 1 +luke garcia 1 +luke garcia 1 +luke ichabod 1 +luke ichabod 1 +luke johnson 1 +luke johnson 1 +luke johnson 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke miller 2 +luke ovid 2 +luke ovid 1 +luke polk 1 +luke polk 1 +luke quirinius 1 +luke robinson 1 +luke robinson 1 +luke thompson 1 +luke underhill 1 +luke underhill 1 +luke underhill 1 +luke van buren 2 +luke white 1 +luke xylophone 1 +luke zipper 1 +mike allen 2 +mike brown 1 +mike carson 1 +mike carson 1 +mike carson 1 +mike davidson 1 +mike davidson 1 +mike ellison 2 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike falkner 1 +mike garcia 1 +mike garcia 1 +mike garcia 1 +mike hernandez 1 +mike hernandez 1 +mike ichabod 1 +mike king 2 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike miller 1 +mike nixon 2 +mike nixon 1 +mike polk 2 +mike polk 1 +mike polk 1 +mike quirinius 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike van buren 2 +mike van buren 1 +mike white 1 +mike white 1 +mike white 1 +mike white 1 +mike young 1 +mike young 1 +mike young 1 +mike zipper 1 +mike zipper 1 +mike zipper 1 +nick allen 1 +nick allen 1 +nick brown 1 +nick davidson 1 +nick ellison 2 +nick ellison 1 +nick falkner 1 +nick falkner 1 +nick garcia 1 +nick garcia 1 +nick garcia 1 +nick ichabod 1 +nick ichabod 1 +nick ichabod 1 +nick johnson 1 +nick johnson 1 +nick laertes 1 +nick miller 1 +nick nixon 1 +nick ovid 1 +nick polk 2 +nick quirinius 2 +nick quirinius 1 +nick robinson 1 +nick robinson 1 +nick steinbeck 1 +nick thompson 2 +nick underhill 1 +nick van buren 1 +nick xylophone 1 +nick young 1 +nick young 1 +nick zipper 2 +nick zipper 1 +oscar allen 1 +oscar brown 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar davidson 1 +oscar ellison 2 +oscar ellison 2 +oscar falkner 1 +oscar garcia 1 +oscar hernandez 1 +oscar hernandez 1 +oscar ichabod 2 +oscar ichabod 1 +oscar ichabod 1 +oscar ichabod 1 +oscar johnson 2 +oscar johnson 1 +oscar king 1 +oscar king 1 +oscar king 1 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar nixon 1 +oscar ovid 1 +oscar ovid 1 +oscar ovid 1 +oscar polk 1 +oscar polk 1 +oscar quirinius 2 +oscar quirinius 2 +oscar quirinius 1 +oscar quirinius 1 +oscar robinson 2 +oscar robinson 1 +oscar robinson 1 +oscar robinson 1 +oscar steinbeck 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 1 +oscar underhill 1 +oscar van buren 1 +oscar van buren 1 +oscar van buren 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar xylophone 2 +oscar xylophone 1 +oscar xylophone 1 +oscar zipper 2 +oscar zipper 1 +oscar zipper 1 +priscilla brown 2 +priscilla brown 1 +priscilla brown 1 +priscilla carson 2 +priscilla carson 1 +priscilla carson 1 +priscilla ichabod 2 +priscilla ichabod 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla king 1 +priscilla nixon 2 +priscilla nixon 2 +priscilla ovid 2 +priscilla ovid 1 +priscilla polk 1 +priscilla quirinius 1 +priscilla thompson 1 +priscilla underhill 1 +priscilla underhill 1 +priscilla van buren 2 +priscilla van buren 1 +priscilla van buren 1 +priscilla white 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla young 1 +priscilla young 1 +priscilla zipper 1 +priscilla zipper 1 +quinn allen 1 +quinn allen 1 +quinn brown 1 +quinn brown 1 +quinn brown 1 +quinn davidson 2 +quinn davidson 1 +quinn davidson 1 +quinn davidson 1 +quinn ellison 1 +quinn ellison 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn ichabod 1 +quinn king 1 +quinn king 1 +quinn laertes 1 +quinn laertes 1 +quinn laertes 1 +quinn nixon 2 +quinn ovid 1 +quinn quirinius 1 +quinn robinson 2 +quinn steinbeck 1 +quinn steinbeck 1 +quinn thompson 1 +quinn thompson 1 +quinn underhill 2 +quinn underhill 1 +quinn underhill 1 +quinn van buren 1 +quinn young 2 +quinn zipper 1 +quinn zipper 1 +rachel allen 1 +rachel allen 1 +rachel brown 3 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel carson 2 +rachel carson 1 +rachel davidson 1 +rachel ellison 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel johnson 1 +rachel king 2 +rachel king 1 +rachel laertes 1 +rachel laertes 1 +rachel ovid 1 +rachel ovid 1 +rachel polk 1 +rachel quirinius 1 +rachel robinson 1 +rachel robinson 1 +rachel robinson 1 +rachel thompson 1 +rachel thompson 1 +rachel thompson 1 +rachel underhill 1 +rachel white 1 +rachel white 1 +rachel young 1 +rachel zipper 1 +rachel zipper 1 +sarah carson 1 +sarah carson 1 +sarah carson 1 +sarah ellison 1 +sarah falkner 1 +sarah falkner 1 +sarah garcia 1 +sarah garcia 1 +sarah garcia 1 +sarah ichabod 1 +sarah ichabod 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah king 1 +sarah king 1 +sarah miller 1 +sarah ovid 1 +sarah robinson 1 +sarah robinson 1 +sarah steinbeck 1 +sarah white 1 +sarah white 1 +sarah xylophone 1 +sarah young 1 +sarah zipper 1 +tom brown 1 +tom brown 1 +tom carson 1 +tom carson 1 +tom carson 1 +tom davidson 1 +tom ellison 1 +tom ellison 1 +tom ellison 1 +tom falkner 1 +tom falkner 1 +tom hernandez 1 +tom hernandez 1 +tom ichabod 1 +tom johnson 1 +tom johnson 1 +tom king 1 +tom laertes 2 +tom laertes 1 +tom miller 2 +tom miller 1 +tom miller 1 +tom nixon 2 +tom ovid 1 +tom polk 1 +tom polk 1 +tom quirinius 1 +tom quirinius 1 +tom robinson 1 +tom robinson 1 +tom robinson 1 +tom robinson 1 +tom steinbeck 3 +tom van buren 1 +tom van buren 1 +tom van buren 1 +tom white 2 +tom young 1 +tom young 1 +tom zipper 3 +ulysses brown 1 +ulysses carson 4 +ulysses carson 1 +ulysses carson 1 +ulysses carson 1 +ulysses davidson 2 +ulysses ellison 1 +ulysses garcia 1 +ulysses hernandez 1 +ulysses hernandez 1 +ulysses hernandez 1 +ulysses ichabod 1 +ulysses ichabod 1 +ulysses johnson 2 +ulysses king 1 +ulysses laertes 2 +ulysses laertes 1 +ulysses laertes 1 +ulysses miller 1 +ulysses miller 1 +ulysses nixon 1 +ulysses ovid 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 1 +ulysses quirinius 1 +ulysses robinson 1 +ulysses steinbeck 1 +ulysses steinbeck 1 +ulysses thompson 1 +ulysses underhill 2 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses van buren 1 +ulysses white 1 +ulysses white 1 +ulysses xylophone 2 +ulysses xylophone 1 +ulysses xylophone 1 +ulysses young 2 +ulysses young 1 +ulysses young 1 +victor allen 1 +victor allen 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor davidson 1 +victor davidson 1 +victor davidson 1 +victor ellison 2 +victor ellison 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor johnson 2 +victor johnson 1 +victor johnson 1 +victor king 1 +victor king 1 +victor laertes 1 +victor laertes 1 +victor miller 1 +victor nixon 1 +victor nixon 1 +victor ovid 2 +victor polk 2 +victor quirinius 1 +victor quirinius 1 +victor robinson 2 +victor robinson 1 +victor steinbeck 2 +victor steinbeck 1 +victor steinbeck 1 +victor thompson 1 +victor van buren 1 +victor van buren 1 +victor white 2 +victor white 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor young 1 +victor zipper 1 +wendy allen 1 +wendy allen 1 +wendy allen 1 +wendy brown 1 +wendy brown 1 +wendy ellison 1 +wendy ellison 1 +wendy falkner 2 +wendy falkner 1 +wendy falkner 1 +wendy garcia 2 +wendy garcia 1 +wendy garcia 1 +wendy garcia 1 +wendy hernandez 1 +wendy ichabod 1 +wendy king 1 +wendy king 1 +wendy king 1 +wendy laertes 1 +wendy laertes 1 +wendy laertes 1 +wendy miller 1 +wendy miller 1 +wendy nixon 1 +wendy nixon 1 +wendy ovid 1 +wendy ovid 1 +wendy polk 2 +wendy polk 1 +wendy quirinius 1 +wendy quirinius 1 +wendy robinson 2 +wendy robinson 1 +wendy robinson 1 +wendy steinbeck 1 +wendy thompson 2 +wendy thompson 1 +wendy underhill 2 +wendy underhill 1 +wendy underhill 1 +wendy van buren 1 +wendy van buren 1 +wendy white 1 +wendy xylophone 1 +wendy xylophone 1 +wendy young 1 +wendy young 1 +xavier allen 1 +xavier allen 1 +xavier allen 1 +xavier brown 1 +xavier brown 1 +xavier brown 1 +xavier carson 1 +xavier carson 1 +xavier davidson 1 +xavier davidson 1 +xavier davidson 1 +xavier ellison 1 +xavier ellison 1 +xavier garcia 1 +xavier hernandez 1 +xavier hernandez 1 +xavier hernandez 1 +xavier ichabod 1 +xavier ichabod 1 +xavier johnson 1 +xavier johnson 1 +xavier king 1 +xavier king 1 +xavier laertes 1 +xavier ovid 2 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier quirinius 2 +xavier quirinius 2 +xavier quirinius 1 +xavier quirinius 1 +xavier thompson 1 +xavier underhill 1 +xavier white 2 +xavier white 1 +xavier xylophone 1 +xavier zipper 1 +yuri allen 1 +yuri allen 1 +yuri brown 1 +yuri brown 1 +yuri carson 1 +yuri carson 1 +yuri ellison 1 +yuri ellison 1 +yuri falkner 1 +yuri falkner 1 +yuri garcia 1 +yuri hernandez 1 +yuri johnson 1 +yuri johnson 1 +yuri johnson 1 +yuri king 2 +yuri laertes 1 +yuri laertes 1 +yuri nixon 1 +yuri nixon 1 +yuri polk 1 +yuri polk 1 +yuri polk 1 +yuri quirinius 1 +yuri quirinius 1 +yuri quirinius 1 +yuri steinbeck 1 +yuri steinbeck 1 +yuri thompson 1 +yuri underhill 1 +yuri underhill 1 +yuri white 1 +yuri xylophone 1 +zach allen 1 +zach brown 2 +zach brown 1 +zach brown 1 +zach brown 1 +zach brown 1 +zach carson 2 +zach ellison 1 +zach falkner 1 +zach falkner 1 +zach garcia 2 +zach garcia 1 +zach garcia 1 +zach garcia 1 +zach ichabod 1 +zach ichabod 1 +zach king 2 +zach king 1 +zach king 1 +zach miller 1 +zach miller 1 +zach miller 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach quirinius 1 +zach robinson 1 +zach steinbeck 1 +zach steinbeck 1 +zach thompson 2 +zach thompson 1 +zach underhill 1 +zach white 1 +zach xylophone 2 +zach xylophone 1 +zach young 1 +zach zipper 1 +zach zipper 1 +zach zipper 1 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-1-9ee79e711248dd6e0a6ce27e439e55f4 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-1-9ee79e711248dd6e0a6ce27e439e55f4 new file mode 100644 index 0000000000000000000000000000000000000000..275772e1f643adcc0e4b71ab3941881ea91bb25f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-1-9ee79e711248dd6e0a6ce27e439e55f4 @@ -0,0 +1,1049 @@ +65791 calvin nixon +65791 katie garcia +65790 fred nixon +65790 victor polk +65790 yuri ellison +65789 NULL +65789 alice laertes +65789 gabriella king +65789 katie zipper +65789 oscar zipper +65789 quinn davidson +65789 wendy brown +65788 irene brown +65788 oscar zipper +65788 rachel king +65788 xavier thompson +65787 alice laertes +65787 david laertes +65787 katie ichabod +65787 ulysses king +65786 bob carson +65786 quinn king +65786 victor xylophone +65786 xavier allen +65786 xavier davidson +65785 sarah carson +65785 sarah johnson +65784 holly young +65784 jessica ellison +65784 jessica ovid +65784 jessica zipper +65784 quinn van buren +65783 david allen +65783 fred ellison +65783 irene nixon +65783 mike ichabod +65783 wendy miller +65783 zach garcia +65782 tom miller +65782 ulysses underhill +65782 victor nixon +65782 yuri white +65782 zach ovid +65781 ethan carson +65781 luke laertes +65781 quinn brown +65780 holly hernandez +65780 holly zipper +65780 wendy falkner +65779 gabriella ichabod +65779 irene brown +65779 irene underhill +65779 ulysses xylophone +65779 wendy hernandez +65779 yuri nixon +65779 zach ellison +65778 jessica davidson +65778 katie allen +65778 oscar van buren +65778 oscar white +65778 rachel johnson +65778 victor quirinius +65778 yuri polk +65778 yuri quirinius +65777 ethan garcia +65777 irene garcia +65777 katie polk +65777 rachel thompson +65776 NULL +65776 luke garcia +65776 luke quirinius +65776 priscilla ovid +65776 sarah king +65775 bob ellison +65775 calvin steinbeck +65775 ethan laertes +65775 luke robinson +65775 oscar polk +65774 calvin young +65774 irene quirinius +65774 katie brown +65774 oscar hernandez +65774 sarah ichabod +65774 zach king +65773 NULL +65773 calvin garcia +65773 irene polk +65773 jessica davidson +65773 nick ichabod +65773 nick zipper +65773 quinn ovid +65773 zach falkner +65772 oscar garcia +65771 ethan underhill +65771 yuri hernandez +65771 zach brown +65770 alice nixon +65770 gabriella ellison +65769 mike king +65769 nick steinbeck +65769 oscar nixon +65769 rachel zipper +65769 xavier quirinius +65769 zach ichabod +65768 jessica davidson +65767 tom robinson +65767 victor davidson +65767 xavier king +65766 fred davidson +65766 luke underhill +65766 ulysses carson +65766 wendy nixon +65766 xavier brown +65765 bob polk +65765 gabriella brown +65765 wendy allen +65764 alice hernandez +65764 alice robinson +65764 luke ellison +65763 calvin van buren +65763 ethan brown +65763 priscilla johnson +65763 tom ichabod +65763 tom miller +65762 david xylophone +65762 priscilla xylophone +65762 quinn allen +65762 sarah johnson +65762 tom carson +65761 david ovid +65760 NULL +65760 ethan king +65760 priscilla johnson +65760 wendy van buren +65759 alice xylophone +65759 ethan quirinius +65759 fred steinbeck +65759 sarah robinson +65759 xavier johnson +65758 alice van buren +65758 ethan white +65758 irene johnson +65758 jessica underhill +65758 rachel laertes +65757 bob quirinius +65757 nick quirinius +65757 tom van buren +65757 xavier carson +65757 zach brown +65756 gabriella falkner +65756 mike polk +65756 nick allen +65756 yuri xylophone +65755 alice johnson +65755 david ellison +65755 fred polk +65755 gabriella hernandez +65755 irene van buren +65755 jessica nixon +65755 mike ellison +65755 nick van buren +65755 zach miller +65754 luke johnson +65754 xavier quirinius +65753 rachel robinson +65753 yuri garcia +65752 bob miller +65752 oscar zipper +65751 calvin quirinius +65751 katie king +65751 mike allen +65751 mike quirinius +65751 mike white +65751 zach zipper +65750 mike white +65750 nick thompson +65750 oscar quirinius +65750 priscilla polk +65750 rachel brown +65749 david allen +65749 gabriella polk +65749 mike ellison +65749 sarah carson +65749 wendy thompson +65748 david davidson +65748 oscar laertes +65747 calvin falkner +65747 fred steinbeck +65747 priscilla zipper +65747 quinn underhill +65747 rachel falkner +65747 wendy falkner +65747 yuri falkner +65746 +65746 katie robinson +65746 luke garcia +65746 tom brown +65746 zach steinbeck +65745 oscar steinbeck +65745 oscar xylophone +65745 priscilla nixon +65745 victor laertes +65744 alice allen +65744 priscilla ichabod +65743 jessica carson +65743 oscar ichabod +65743 sarah falkner +65743 tom brown +65742 jessica brown +65742 jessica ellison +65742 wendy polk +65741 calvin thompson +65740 irene van buren +65740 mike zipper +65740 rachel quirinius +65739 gabriella van buren +65739 victor robinson +65739 wendy nixon +65738 sarah ichabod +65738 wendy robinson +65738 wendy young +65738 xavier king +65737 holly brown +65737 irene falkner +65737 jessica white +65737 quinn garcia +65737 wendy laertes +65736 fred young +65736 quinn laertes +65736 rachel allen +65736 victor ellison +65735 oscar laertes +65735 tom falkner +65735 ulysses thompson +65734 calvin falkner +65734 mike garcia +65733 gabriella van buren +65733 holly nixon +65733 luke underhill +65733 oscar allen +65733 xavier polk +65733 yuri ellison +65732 mike steinbeck +65732 tom carson +65732 tom ellison +65731 alice robinson +65731 priscilla carson +65731 tom johnson +65731 ulysses ovid +65730 bob king +65730 calvin polk +65730 gabriella ichabod +65730 rachel brown +65729 fred quirinius +65728 priscilla king +65728 victor brown +65727 ethan brown +65727 fred polk +65727 jessica white +65727 priscilla underhill +65727 sarah white +65726 mike garcia +65726 ulysses hernandez +65726 wendy quirinius +65725 zach thompson +65724 NULL +65724 alice king +65724 alice steinbeck +65724 mike ellison +65724 nick ovid +65724 priscilla zipper +65723 nick allen +65723 rachel white +65722 fred falkner +65722 jessica underhill +65722 luke ichabod +65721 ethan falkner +65721 jessica zipper +65721 luke laertes +65721 luke underhill +65721 mike carson +65721 oscar laertes +65721 ulysses ichabod +65720 calvin nixon +65720 calvin thompson +65720 gabriella young +65720 holly polk +65720 mike king +65719 bob brown +65719 holly ichabod +65719 ulysses ellison +65719 ulysses young +65718 jessica ichabod +65718 oscar johnson +65718 victor hernandez +65718 zach ovid +65717 holly hernandez +65717 mike nixon +65717 quinn garcia +65717 ulysses davidson +65717 ulysses polk +65716 ethan zipper +65716 holly xylophone +65716 jessica garcia +65716 nick falkner +65716 sarah king +65716 victor xylophone +65715 alice hernandez +65715 david young +65715 ethan polk +65715 oscar falkner +65715 priscilla brown +65714 NULL +65714 ethan laertes +65714 luke allen +65713 bob ellison +65713 nick nixon +65713 ulysses carson +65713 ulysses ichabod +65713 zach garcia +65712 NULL +65712 katie king +65712 luke davidson +65712 quinn garcia +65712 zach carson +65711 bob zipper +65711 fred miller +65711 holly nixon +65711 katie ellison +65711 wendy ovid +65711 zach zipper +65710 luke polk +65710 mike young +65709 bob laertes +65709 calvin laertes +65709 oscar laertes +65708 gabriella laertes +65708 priscilla van buren +65707 bob garcia +65707 mike steinbeck +65706 bob ellison +65706 bob xylophone +65706 luke allen +65706 ulysses xylophone +65706 wendy ichabod +65705 david ellison +65705 mike white +65705 priscilla johnson +65705 tom ellison +65704 ethan brown +65704 holly king +65704 jessica young +65704 tom steinbeck +65704 victor steinbeck +65704 zach falkner +65703 holly underhill +65703 mike ellison +65703 victor davidson +65703 xavier underhill +65702 NULL +65702 bob brown +65702 bob zipper +65702 ulysses quirinius +65701 alice allen +65701 mike steinbeck +65701 oscar thompson +65700 luke allen +65700 quinn laertes +65700 tom quirinius +65699 ethan brown +65699 ethan van buren +65699 irene laertes +65699 mike young +65699 nick garcia +65699 sarah white +65698 gabriella thompson +65698 nick polk +65697 NULL +65697 NULL +65697 holly miller +65697 oscar quirinius +65697 oscar thompson +65697 xavier davidson +65696 alice xylophone +65696 jessica davidson +65696 luke allen +65696 oscar xylophone +65695 oscar king +65695 rachel young +65695 wendy ellison +65695 yuri quirinius +65694 david brown +65694 holly underhill +65694 victor quirinius +65694 zach brown +65693 bob hernandez +65693 bob young +65693 david brown +65693 holly hernandez +65693 tom polk +65693 ulysses polk +65693 victor brown +65692 holly johnson +65692 tom robinson +65691 calvin ovid +65691 ethan nixon +65691 ethan robinson +65691 fred underhill +65691 holly white +65691 irene polk +65691 oscar white +65691 rachel brown +65690 fred van buren +65690 jessica quirinius +65689 oscar ovid +65689 wendy thompson +65688 bob steinbeck +65688 victor steinbeck +65687 gabriella ichabod +65687 jessica underhill +65687 mike zipper +65687 quinn thompson +65686 bob king +65686 bob zipper +65686 david quirinius +65686 luke ichabod +65685 ethan robinson +65685 gabriella hernandez +65685 katie garcia +65685 sarah ellison +65685 victor hernandez +65685 victor nixon +65684 priscilla brown +65684 victor laertes +65684 wendy van buren +65683 NULL +65683 mike king +65683 tom laertes +65682 calvin quirinius +65682 ethan brown +65682 katie ellison +65681 gabriella allen +65681 luke laertes +65681 oscar quirinius +65681 ulysses laertes +65681 wendy ellison +65681 xavier polk +65680 NULL +65680 alice nixon +65680 gabriella ovid +65680 jessica carson +65680 ulysses nixon +65680 zach zipper +65679 bob garcia +65679 wendy underhill +65678 bob falkner +65678 victor xylophone +65678 wendy king +65677 alice allen +65677 fred van buren +65677 mike brown +65677 nick xylophone +65677 ulysses underhill +65677 zach robinson +65676 bob davidson +65676 bob laertes +65676 tom ovid +65676 xavier johnson +65675 david hernandez +65675 david nixon +65675 holly falkner +65675 quinn steinbeck +65675 rachel robinson +65675 sarah zipper +65675 tom polk +65675 victor allen +65674 gabriella falkner +65673 nick johnson +65673 quinn brown +65673 quinn underhill +65673 rachel ovid +65673 wendy brown +65672 nick laertes +65672 nick underhill +65672 rachel zipper +65672 tom white +65672 victor king +65671 fred ellison +65671 fred falkner +65671 zach white +65670 david robinson +65670 jessica zipper +65670 luke van buren +65670 oscar ovid +65670 quinn steinbeck +65669 NULL +65669 alice king +65669 calvin hernandez +65669 katie polk +65669 nick miller +65669 oscar van buren +65668 luke ellison +65667 bob brown +65667 irene nixon +65667 oscar brown +65667 tom falkner +65666 +65666 david underhill +65666 fred van buren +65665 rachel brown +65664 NULL +65664 bob davidson +65664 david ichabod +65664 ethan laertes +65664 irene robinson +65664 mike carson +65664 priscilla young +65664 victor king +65663 calvin underhill +65663 jessica johnson +65663 priscilla carson +65663 zach ichabod +65662 ethan allen +65662 katie ovid +65662 oscar johnson +65662 ulysses carson +65662 ulysses polk +65662 victor ovid +65661 david van buren +65661 luke xylophone +65661 mike falkner +65661 priscilla van buren +65661 victor johnson +65660 holly ichabod +65660 priscilla johnson +65660 victor thompson +65659 david robinson +65659 gabriella king +65659 luke davidson +65659 mike king +65659 mike zipper +65659 nick brown +65659 nick zipper +65659 yuri underhill +65658 NULL +65658 alice zipper +65658 calvin allen +65658 calvin johnson +65658 jessica garcia +65658 quinn davidson +65658 sarah ovid +65658 ulysses brown +65658 ulysses miller +65658 yuri king +65657 ethan falkner +65657 holly zipper +65657 irene ovid +65657 luke ovid +65657 priscilla white +65656 david davidson +65656 irene ovid +65656 jessica xylophone +65656 luke laertes +65656 oscar ichabod +65656 xavier ellison +65655 calvin falkner +65655 yuri laertes +65654 alice carson +65654 alice quirinius +65654 gabriella falkner +65654 nick young +65654 oscar robinson +65654 quinn robinson +65654 rachel falkner +65654 tom laertes +65654 yuri johnson +65653 calvin ellison +65653 holly underhill +65653 ulysses polk +65653 wendy xylophone +65652 NULL +65652 tom ellison +65652 victor johnson +65651 NULL +65651 ethan laertes +65651 ethan laertes +65651 irene garcia +65651 mike young +65650 irene ellison +65650 oscar white +65650 sarah steinbeck +65650 ulysses underhill +65650 ulysses xylophone +65650 victor xylophone +65649 irene underhill +65649 priscilla quirinius +65649 quinn ellison +65649 tom quirinius +65648 alice nixon +65648 calvin brown +65648 sarah carson +65648 xavier ellison +65647 irene allen +65647 mike ellison +65646 bob ovid +65646 xavier brown +65646 xavier ovid +65645 jessica quirinius +65645 katie miller +65645 ulysses hernandez +65644 alice king +65644 calvin ovid +65644 jessica white +65644 katie van buren +65644 sarah young +65644 ulysses hernandez +65644 yuri carson +65643 david davidson +65643 ethan polk +65643 ethan zipper +65643 gabriella ichabod +65643 mike davidson +65643 mike hernandez +65643 oscar robinson +65643 priscilla underhill +65643 zach king +65642 gabriella thompson +65641 fred laertes +65641 sarah garcia +65641 tom miller +65641 xavier hernandez +65640 david underhill +65639 wendy garcia +65638 fred nixon +65638 luke polk +65638 rachel carson +65637 alice underhill +65637 david davidson +65637 fred davidson +65637 gabriella davidson +65637 oscar carson +65637 rachel laertes +65637 sarah garcia +65637 wendy garcia +65636 +65636 irene polk +65636 wendy allen +65635 alice steinbeck +65635 alice zipper +65635 ulysses white +65634 NULL +65634 calvin white +65634 holly underhill +65634 sarah falkner +65633 NULL +65633 holly polk +65633 jessica nixon +65633 oscar quirinius +65632 alice falkner +65632 zach quirinius +65631 fred ellison +65630 rachel brown +65630 xavier hernandez +65629 jessica quirinius +65629 priscilla carson +65629 victor young +65629 xavier carson +65628 bob ovid +65628 ethan ovid +65628 irene ichabod +65628 oscar hernandez +65628 oscar robinson +65628 xavier quirinius +65627 alice miller +65627 holly johnson +65627 luke falkner +65627 yuri polk +65626 ethan polk +65626 holly robinson +65626 tom young +65626 yuri johnson +65625 david xylophone +65625 fred ichabod +65625 katie white +65625 ulysses garcia +65624 calvin steinbeck +65624 calvin xylophone +65624 rachel carson +65624 tom van buren +65624 yuri brown +65623 alice quirinius +65623 jessica miller +65623 oscar ichabod +65623 quinn zipper +65623 tom van buren +65623 victor brown +65623 wendy young +65622 nick davidson +65622 rachel king +65622 wendy robinson +65622 xavier ichabod +65622 zach xylophone +65622 zach young +65621 quinn underhill +65621 ulysses young +65620 nick garcia +65620 oscar thompson +65620 quinn quirinius +65620 victor white +65620 victor xylophone +65620 wendy quirinius +65619 calvin brown +65619 gabriella polk +65619 oscar king +65619 ulysses miller +65619 ulysses robinson +65619 ulysses steinbeck +65618 gabriella ovid +65618 irene laertes +65618 katie king +65618 oscar ovid +65618 quinn thompson +65617 fred van buren +65617 gabriella carson +65617 sarah johnson +65617 ulysses underhill +65616 calvin steinbeck +65616 xavier ichabod +65615 alice ovid +65615 david quirinius +65615 irene quirinius +65615 katie nixon +65614 wendy king +65614 xavier quirinius +65614 xavier white +65613 xavier zipper +65612 irene miller +65612 victor hernandez +65612 wendy white +65612 yuri polk +65611 ethan johnson +65611 fred zipper +65611 irene carson +65611 nick quirinius +65610 tom king +65610 victor steinbeck +65610 wendy garcia +65610 yuri carson +65610 zach ovid +65609 sarah robinson +65608 katie van buren +65608 mike van buren +65608 quinn ichabod +65608 zach underhill +65607 katie miller +65607 luke falkner +65607 mike polk +65607 priscilla xylophone +65607 yuri allen +65607 yuri allen +65606 bob white +65606 gabriella white +65606 oscar carson +65606 victor white +65606 xavier allen +65606 zach allen +65605 holly king +65604 katie zipper +65604 oscar davidson +65604 wendy laertes +65604 zach brown +65603 alice davidson +65603 ethan miller +65603 katie davidson +65603 katie young +65603 mike garcia +65602 NULL +65602 calvin laertes +65602 ethan laertes +65602 fred steinbeck +65602 jessica young +65602 xavier brown +65601 priscilla ovid +65601 sarah xylophone +65601 tom robinson +65600 gabriella thompson +65600 jessica polk +65600 nick robinson +65600 rachel allen +65599 fred quirinius +65599 luke johnson +65599 nick garcia +65599 oscar xylophone +65599 ulysses underhill +65598 ulysses van buren +65598 victor zipper +65597 ethan ellison +65597 nick ellison +65597 quinn davidson +65596 NULL +65596 calvin zipper +65596 david ellison +65596 irene ichabod +65596 wendy laertes +65595 bob white +65595 holly hernandez +65595 luke brown +65595 oscar ellison +65595 oscar ichabod +65595 quinn ellison +65594 gabriella ellison +65594 oscar robinson +65594 ulysses underhill +65594 victor robinson +65593 oscar white +65593 zach xylophone +65592 calvin xylophone +65591 alice zipper +65591 nick ichabod +65591 priscilla ichabod +65591 rachel underhill +65590 NULL +65590 katie falkner +65590 oscar van buren +65590 xavier garcia +65590 yuri underhill +65589 ethan white +65589 gabriella zipper +65589 irene ovid +65589 oscar king +65589 wendy xylophone +65588 bob van buren +65588 david ichabod +65588 mike miller +65588 tom hernandez +65588 victor van buren +65587 bob garcia +65587 luke johnson +65587 mike king +65587 victor allen +65587 xavier white +65586 david young +65586 irene brown +65586 priscilla brown +65586 wendy allen +65586 xavier laertes +65585 alice garcia +65585 bob garcia +65585 ethan ellison +65585 nick ellison +65585 priscilla thompson +65584 jessica carson +65584 jessica van buren +65584 jessica white +65583 bob xylophone +65583 nick ichabod +65583 yuri brown +65583 yuri steinbeck +65582 holly johnson +65582 mike carson +65582 victor van buren +65582 zach miller +65581 gabriella steinbeck +65581 irene quirinius +65581 luke allen +65581 nick robinson +65581 nick young +65581 wendy robinson +65580 alice steinbeck +65580 alice xylophone +65580 irene xylophone +65579 irene polk +65579 luke ovid +65579 quinn nixon +65579 sarah garcia +65579 wendy ovid +65578 calvin robinson +65578 fred king +65578 holly thompson +65578 katie ichabod +65578 quinn king +65578 rachel davidson +65578 victor hernandez +65577 holly white +65576 calvin falkner +65576 calvin ovid +65576 fred polk +65576 luke robinson +65575 calvin falkner +65575 irene steinbeck +65575 luke zipper +65575 zach king +65574 gabriella steinbeck +65574 priscilla nixon +65574 rachel thompson +65573 victor ellison +65573 victor hernandez +65573 yuri nixon +65572 calvin davidson +65572 calvin young +65572 katie young +65572 oscar ellison +65572 quinn garcia +65571 bob king +65571 irene polk +65571 katie ichabod +65571 mike steinbeck +65570 NULL +65570 bob ovid +65570 fred polk +65570 luke ellison +65570 mike hernandez +65570 yuri quirinius +65569 nick falkner +65568 bob ichabod +65568 holly thompson +65568 jessica thompson +65567 katie xylophone +65566 gabriella garcia +65566 rachel white +65565 katie young +65565 quinn young +65564 alice polk +65564 calvin carson +65564 calvin white +65564 ethan hernandez +65564 ethan quirinius +65564 jessica thompson +65564 katie hernandez +65563 calvin zipper +65563 priscilla young +65563 xavier davidson +65563 yuri steinbeck +65562 calvin falkner +65562 ethan xylophone +65562 luke white +65562 quinn allen +65562 rachel polk +65562 wendy polk +65561 bob davidson +65561 ethan polk +65561 jessica robinson +65560 fred white +65560 jessica johnson +65560 oscar thompson +65560 ulysses steinbeck +65560 zach brown +65559 NULL +65559 ethan laertes +65559 gabriella ichabod +65559 gabriella zipper +65559 irene garcia +65558 fred robinson +65557 fred hernandez +65557 nick johnson +65556 oscar underhill +65556 xavier hernandez +65556 yuri falkner +65556 zach garcia +65556 zach steinbeck +65555 fred nixon +65554 gabriella miller +65554 rachel falkner +65553 calvin van buren +65553 david van buren +65553 irene nixon +65553 luke laertes +65553 oscar carson +65552 NULL +65552 irene ellison +65552 oscar polk +65552 wendy falkner +65552 zach miller +65551 fred young +65551 ulysses underhill +65551 wendy underhill +65550 ethan quirinius +65550 fred davidson +65550 holly young +65550 jessica ovid +65550 quinn brown +65550 quinn laertes +65550 tom johnson +65549 bob garcia +65549 bob ovid +65549 fred ichabod +65549 fred king +65549 jessica white +65549 ulysses laertes +65549 victor davidson +65549 victor miller +65548 calvin ovid +65548 gabriella allen +65548 holly ichabod +65548 priscilla johnson +65548 quinn zipper +65548 tom hernandez +65548 wendy king +65547 bob ellison +65547 jessica quirinius +65547 mike davidson +65547 xavier allen +65546 katie white +65545 mike king +65545 tom carson +65545 victor brown +65544 calvin davidson +65544 calvin nixon +65544 david ovid +65544 irene thompson +65544 ulysses young +65544 xavier polk +65544 xavier xylophone +65544 zach ovid +65543 fred johnson +65543 sarah johnson +65542 fred falkner +65542 holly thompson +65542 luke miller +65542 mike white +65542 tom davidson +65541 calvin brown +65541 ethan brown +65541 holly brown +65541 jessica falkner +65541 rachel thompson +65541 tom zipper +65541 wendy underhill +65541 xavier polk +65541 yuri johnson +65540 rachel falkner +65539 gabriella young +65539 holly laertes +65539 oscar carson +65538 irene laertes +65538 mike polk +65538 tom robinson +65537 NULL +65537 david quirinius +65537 rachel ovid +65537 ulysses laertes +65537 zach garcia +65536 calvin xylophone +65536 david thompson +65536 irene falkner +65536 ulysses johnson +65536 victor johnson +65536 wendy miller +65536 yuri thompson diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 new file mode 100644 index 0000000000000000000000000000000000000000..62d71abc6fc7ddbcf84c3813842e22540cddd5e2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 @@ -0,0 +1,1049 @@ +65536 NULL +65536 16.85 +65536 32.68 +65536 33.45 +65536 58.86 +65536 75.7 +65536 83.48 +65537 NULL +65537 4.49 +65537 11.87 +65537 51.91 +65537 99.34 +65538 NULL +65538 9.53 +65538 30.27 +65539 NULL +65539 58.85 +65539 96.64 +65540 NULL +65541 NULL +65541 9.04 +65541 14.94 +65541 15.85 +65541 27.89 +65541 35.38 +65541 72.33 +65541 89.14 +65541 98.87 +65542 NULL +65542 42.85 +65542 73.72 +65542 73.93 +65542 84.63 +65543 NULL +65543 21.59 +65544 NULL +65544 26.94 +65544 32.03 +65544 48.84 +65544 65.77 +65544 65.85 +65544 73.08 +65544 79.21 +65545 NULL +65545 34.65 +65545 65.81 +65546 NULL +65547 NULL +65547 17.71 +65547 62.31 +65547 83.21 +65548 NULL +65548 52.94 +65548 53.52 +65548 57.61 +65548 58.51 +65548 75.39 +65548 77.24 +65549 NULL +65549 13.3 +65549 28.93 +65549 50.6 +65549 55.04 +65549 64.91 +65549 76.06 +65549 80.09 +65550 NULL +65550 1.75 +65550 8.46 +65550 33.01 +65550 57.63 +65550 91.38 +65550 96.9 +65551 NULL +65551 39.43 +65551 73.93 +65552 NULL +65552 3.43 +65552 18.11 +65552 48.96 +65552 82.21 +65553 NULL +65553 25.31 +65553 29.62 +65553 71.07 +65553 72.16 +65554 NULL +65554 0.24 +65555 NULL +65556 NULL +65556 9.96 +65556 45.81 +65556 50.42 +65556 95.12 +65557 NULL +65557 21.14 +65558 NULL +65559 NULL +65559 29.55 +65559 56.06 +65559 73.94 +65559 83.5 +65560 NULL +65560 16.86 +65560 21.81 +65560 64.38 +65560 71.59 +65561 NULL +65561 32.86 +65561 47.71 +65562 NULL +65562 26.86 +65562 45.94 +65562 54.94 +65562 69.41 +65562 72.04 +65563 NULL +65563 14.36 +65563 33.29 +65563 39.96 +65564 NULL +65564 9.47 +65564 15.82 +65564 26.97 +65564 30.78 +65564 90.34 +65564 98.36 +65565 NULL +65565 81.72 +65566 NULL +65566 7.8 +65567 NULL +65568 NULL +65568 21.79 +65568 58.66 +65569 NULL +65570 NULL +65570 17.09 +65570 18.2 +65570 25.57 +65570 45.23 +65570 76.8 +65571 NULL +65571 26.64 +65571 40.68 +65571 82.5 +65572 NULL +65572 22.64 +65572 43.49 +65572 70.91 +65572 93.48 +65573 NULL +65573 53.56 +65573 96.32 +65574 NULL +65574 31.28 +65574 38.54 +65575 NULL +65575 17 +65575 32.85 +65575 83.4 +65576 NULL +65576 2.04 +65576 4.88 +65576 66.55 +65577 NULL +65578 NULL +65578 16.01 +65578 41.1 +65578 51.36 +65578 54.35 +65578 58.78 +65578 75.62 +65579 NULL +65579 21.36 +65579 33.37 +65579 73.48 +65579 91.42 +65580 NULL +65580 56.33 +65580 81.42 +65581 NULL +65581 29.74 +65581 45.48 +65581 56.59 +65581 60.88 +65581 88.09 +65582 NULL +65582 1.23 +65582 9.35 +65582 96.6 +65583 NULL +65583 28.07 +65583 50.57 +65583 57.67 +65584 NULL +65584 6.88 +65584 65.78 +65585 NULL +65585 31.23 +65585 37.34 +65585 39.32 +65585 50.38 +65586 NULL +65586 1.15 +65586 18.65 +65586 87.57 +65586 94.25 +65587 NULL +65587 5.83 +65587 11.86 +65587 53.84 +65587 94.47 +65588 NULL +65588 64.63 +65588 69.48 +65588 76.44 +65588 98.33 +65589 NULL +65589 49.49 +65589 72.3 +65589 74.83 +65589 94.73 +65590 NULL +65590 13.51 +65590 22.13 +65590 42.53 +65590 92.71 +65591 NULL +65591 9.85 +65591 11.43 +65591 60.78 +65592 NULL +65593 NULL +65593 35.15 +65594 NULL +65594 35.28 +65594 60.57 +65594 65.61 +65595 NULL +65595 8.76 +65595 67.56 +65595 72.7 +65595 89.6 +65595 90.24 +65596 NULL +65596 12.72 +65596 54.75 +65596 85.74 +65596 94.35 +65597 NULL +65597 37.41 +65597 69.05 +65598 NULL +65598 63.3 +65599 NULL +65599 0.56 +65599 4.93 +65599 41.61 +65599 76.29 +65600 NULL +65600 36.57 +65600 43.03 +65600 92.92 +65601 NULL +65601 26.54 +65601 37.93 +65602 NULL +65602 8.13 +65602 47.16 +65602 83.67 +65602 86.23 +65602 95.58 +65603 NULL +65603 41.44 +65603 45.63 +65603 69.26 +65603 80.24 +65604 NULL +65604 63.36 +65604 63.65 +65604 85.91 +65605 NULL +65606 NULL +65606 7.51 +65606 24.8 +65606 57.69 +65606 67.94 +65606 87.16 +65607 NULL +65607 9.67 +65607 36.58 +65607 71.75 +65607 75.86 +65607 91.52 +65608 NULL +65608 48.9 +65608 69.42 +65608 87.9 +65609 NULL +65610 NULL +65610 7.59 +65610 11.99 +65610 36.77 +65610 39.74 +65611 NULL +65611 21.21 +65611 25.92 +65611 64.89 +65612 NULL +65612 16.05 +65612 25.1 +65612 52.64 +65613 NULL +65614 NULL +65614 1.42 +65614 94.47 +65615 NULL +65615 10.79 +65615 39.4 +65615 99.88 +65616 NULL +65616 75.2 +65617 NULL +65617 18.51 +65617 47.45 +65617 64.9 +65618 NULL +65618 10.06 +65618 16.6 +65618 81.99 +65618 88.38 +65619 NULL +65619 27.32 +65619 32.64 +65619 34.72 +65619 36.48 +65619 36.59 +65620 NULL +65620 6.85 +65620 8.16 +65620 29.14 +65620 64.65 +65620 81.28 +65621 NULL +65621 95.14 +65622 NULL +65622 28.37 +65622 50.08 +65622 74.31 +65622 88.6 +65622 93.7 +65623 NULL +65623 30.83 +65623 31.22 +65623 39.74 +65623 48.51 +65623 95.58 +65623 97.2 +65624 NULL +65624 58.02 +65624 65.31 +65624 70.08 +65624 93.3 +65625 NULL +65625 20.61 +65625 42.86 +65625 55.06 +65626 NULL +65626 63.54 +65626 64.61 +65626 75.15 +65627 NULL +65627 19.65 +65627 61.89 +65627 93.29 +65628 NULL +65628 14.83 +65628 30.43 +65628 37.8 +65628 74.31 +65628 83.26 +65629 NULL +65629 19.33 +65629 58.81 +65629 72.9 +65630 NULL +65630 72.13 +65631 NULL +65632 NULL +65632 88.51 +65633 NULL +65633 59.56 +65633 72.54 +65633 81.02 +65634 NULL +65634 57.09 +65634 64.36 +65634 99.34 +65635 NULL +65635 64.99 +65635 82.29 +65636 NULL +65636 21.15 +65636 86.29 +65637 NULL +65637 16.89 +65637 26.78 +65637 29.34 +65637 35.51 +65637 44.32 +65637 48.88 +65637 93.41 +65638 NULL +65638 11.2 +65638 19.13 +65639 NULL +65640 NULL +65641 NULL +65641 26.02 +65641 84.27 +65641 91.46 +65642 NULL +65643 NULL +65643 22.05 +65643 50.79 +65643 52.56 +65643 61.29 +65643 71.29 +65643 80.96 +65643 92.24 +65643 93.11 +65644 NULL +65644 1.97 +65644 30.25 +65644 58.05 +65644 87.31 +65644 89.95 +65644 96.45 +65645 NULL +65645 3.95 +65645 63.22 +65646 NULL +65646 17.92 +65646 27.34 +65647 NULL +65647 58.03 +65648 NULL +65648 0.08 +65648 17.66 +65648 64.06 +65649 NULL +65649 8.69 +65649 43.92 +65649 91.03 +65650 NULL +65650 23.55 +65650 59.55 +65650 85.89 +65650 89.12 +65650 90.77 +65651 NULL +65651 24.25 +65651 58.25 +65651 74.13 +65651 84.42 +65652 NULL +65652 55.04 +65652 73.61 +65653 NULL +65653 3.81 +65653 52.23 +65653 85.09 +65654 NULL +65654 8.91 +65654 11.64 +65654 26.73 +65654 29.85 +65654 37.74 +65654 37.8 +65654 53.55 +65654 88.23 +65655 NULL +65655 77.41 +65656 NULL +65656 14 +65656 14.96 +65656 53.27 +65656 64.44 +65656 82.67 +65657 NULL +65657 11.93 +65657 26.4 +65657 64.39 +65657 65.01 +65658 NULL +65658 2.63 +65658 20.69 +65658 42.93 +65658 46.61 +65658 60.94 +65658 66.53 +65658 68.85 +65658 77.66 +65658 92.67 +65659 NULL +65659 8.95 +65659 46.57 +65659 53.8 +65659 94.3 +65659 94.69 +65659 95.71 +65659 99.87 +65660 NULL +65660 28.05 +65660 62.82 +65661 NULL +65661 5.24 +65661 8.06 +65661 26.8 +65661 68.98 +65662 NULL +65662 59.92 +65662 76.11 +65662 76.51 +65662 88.64 +65662 99.18 +65663 NULL +65663 5.42 +65663 78.56 +65663 94.16 +65664 NULL +65664 11.46 +65664 27.6 +65664 34.71 +65664 38.42 +65664 45.4 +65664 55.82 +65664 97.64 +65665 NULL +65666 NULL +65666 32.73 +65666 83.95 +65667 NULL +65667 13.96 +65667 63.9 +65667 97.87 +65668 NULL +65669 NULL +65669 1.76 +65669 16.95 +65669 38.6 +65669 54.25 +65669 93.79 +65670 NULL +65670 5.37 +65670 61.06 +65670 61.54 +65670 92.97 +65671 NULL +65671 8.65 +65671 52.05 +65672 NULL +65672 52.6 +65672 58.1 +65672 64.09 +65672 75.27 +65673 NULL +65673 0.9 +65673 33.27 +65673 43.81 +65673 87.78 +65674 NULL +65675 NULL +65675 4.19 +65675 24.19 +65675 35.33 +65675 35.78 +65675 79.9 +65675 83.09 +65675 87.36 +65676 NULL +65676 8.77 +65676 58.12 +65676 80.13 +65677 NULL +65677 5.06 +65677 25.37 +65677 44.47 +65677 48.79 +65677 87.67 +65678 NULL +65678 8.72 +65678 33.9 +65679 NULL +65679 64.15 +65680 NULL +65680 1.01 +65680 34.08 +65680 54.11 +65680 55.3 +65680 65.88 +65681 NULL +65681 35.45 +65681 41.57 +65681 61.3 +65681 71.17 +65681 75.85 +65682 NULL +65682 67.17 +65682 92.95 +65683 NULL +65683 17.62 +65683 99.56 +65684 NULL +65684 3.51 +65684 67.34 +65685 NULL +65685 38.71 +65685 43.48 +65685 63.27 +65685 87.84 +65685 90.69 +65686 NULL +65686 31.75 +65686 58.87 +65686 98.68 +65687 NULL +65687 3.37 +65687 21.79 +65687 48.73 +65688 NULL +65688 76.21 +65689 NULL +65689 9.12 +65690 NULL +65690 3.43 +65691 NULL +65691 5.01 +65691 6.93 +65691 28.47 +65691 56.02 +65691 58.01 +65691 69.8 +65691 76.98 +65692 NULL +65692 54.76 +65693 NULL +65693 8.38 +65693 32.33 +65693 45.69 +65693 69.32 +65693 71.72 +65693 84.88 +65694 NULL +65694 58.23 +65694 82.24 +65694 88.5 +65695 NULL +65695 57.33 +65695 59.96 +65695 77.09 +65696 NULL +65696 17.35 +65696 40.3 +65696 54.02 +65697 NULL +65697 3.18 +65697 50.01 +65697 67.9 +65697 86.79 +65697 90.16 +65698 NULL +65698 42.98 +65699 NULL +65699 13.29 +65699 38.71 +65699 68.94 +65699 84.79 +65699 88.09 +65700 NULL +65700 2.83 +65700 37.61 +65701 NULL +65701 1.81 +65701 6.35 +65702 NULL +65702 37.6 +65702 55.68 +65702 79.5 +65703 NULL +65703 37.18 +65703 40.81 +65703 90.89 +65704 NULL +65704 16.22 +65704 37.12 +65704 48.48 +65704 54.76 +65704 93.21 +65705 NULL +65705 20.57 +65705 25.89 +65705 65.13 +65706 NULL +65706 3.91 +65706 9.74 +65706 55.94 +65706 72.87 +65707 NULL +65707 76.2 +65708 NULL +65708 1.29 +65709 NULL +65709 5.64 +65709 49.79 +65710 NULL +65710 86.7 +65711 NULL +65711 8.66 +65711 50.26 +65711 71.89 +65711 78.69 +65711 96.1 +65712 NULL +65712 30.27 +65712 34.7 +65712 49.69 +65712 53.65 +65713 NULL +65713 10.94 +65713 39.47 +65713 72.37 +65713 90.91 +65714 NULL +65714 14.85 +65714 47.42 +65715 NULL +65715 39.62 +65715 54.79 +65715 81.28 +65715 89.4 +65716 NULL +65716 9 +65716 10.07 +65716 33.4 +65716 71.53 +65716 85.93 +65717 NULL +65717 1.23 +65717 5.81 +65717 57.61 +65717 80.05 +65718 NULL +65718 63.06 +65718 84.35 +65718 89.67 +65719 NULL +65719 51.13 +65719 66.85 +65719 82.1 +65720 NULL +65720 2.72 +65720 18.8 +65720 22.34 +65720 62.04 +65721 NULL +65721 23.78 +65721 39.19 +65721 55.75 +65721 72.82 +65721 95.12 +65721 95.38 +65722 NULL +65722 1.76 +65722 38.82 +65723 NULL +65723 39.9 +65724 NULL +65724 10.52 +65724 36.05 +65724 50.96 +65724 71.66 +65724 85.52 +65725 NULL +65726 NULL +65726 6 +65726 60.46 +65727 NULL +65727 19.81 +65727 49.19 +65727 87.37 +65727 88.11 +65728 NULL +65728 55.37 +65729 NULL +65730 NULL +65730 1.35 +65730 30.6 +65730 81.44 +65731 NULL +65731 24.48 +65731 61.52 +65731 97.18 +65732 NULL +65732 30.06 +65732 91.15 +65733 NULL +65733 11.44 +65733 20.72 +65733 88.46 +65733 93.45 +65733 99.8 +65734 NULL +65734 31.71 +65735 NULL +65735 12.67 +65735 61.16 +65736 NULL +65736 28.9 +65736 48.54 +65736 86.51 +65737 NULL +65737 3.98 +65737 20.85 +65737 29.92 +65737 80.97 +65738 NULL +65738 30.94 +65738 82.32 +65738 95.1 +65739 NULL +65739 74.77 +65739 92.4 +65740 NULL +65740 7.49 +65740 58.65 +65741 NULL +65742 NULL +65742 6.61 +65742 43.84 +65743 NULL +65743 26.6 +65743 52.65 +65743 62 +65744 NULL +65744 46.98 +65745 NULL +65745 25.19 +65745 66.36 +65745 80.12 +65746 NULL +65746 36.74 +65746 93.21 +65746 97.52 +65746 98.1 +65747 NULL +65747 11.16 +65747 15.07 +65747 21.8 +65747 39.77 +65747 52.77 +65747 71.87 +65748 NULL +65748 29.49 +65749 NULL +65749 15.14 +65749 45 +65749 65.49 +65749 73.24 +65750 NULL +65750 20.91 +65750 83.44 +65750 85.44 +65750 96.85 +65751 NULL +65751 2.96 +65751 9.02 +65751 30.68 +65751 47.81 +65751 78.75 +65752 NULL +65752 47.82 +65753 NULL +65753 86.97 +65754 NULL +65754 54.35 +65755 NULL +65755 11.23 +65755 22.44 +65755 64 +65755 67.54 +65755 76.75 +65755 81.44 +65755 90.08 +65755 96.8 +65756 NULL +65756 1.45 +65756 11.81 +65756 63.51 +65757 NULL +65757 1.86 +65757 9.24 +65757 34.84 +65757 90.09 +65758 NULL +65758 25.62 +65758 56.56 +65758 60.88 +65758 94.9 +65759 NULL +65759 10.63 +65759 14.1 +65759 47.54 +65759 92.81 +65760 NULL +65760 21.14 +65760 27.52 +65760 95.45 +65761 NULL +65762 NULL +65762 5.49 +65762 45.7 +65762 77.96 +65762 87.5 +65763 NULL +65763 0.72 +65763 43.8 +65763 86.43 +65763 87.99 +65764 NULL +65764 31.41 +65764 57.1 +65765 NULL +65765 88.52 +65765 88.56 +65766 NULL +65766 37.06 +65766 66.34 +65766 86.53 +65766 98.9 +65767 NULL +65767 90.88 +65767 95.57 +65768 NULL +65769 NULL +65769 11.45 +65769 38.98 +65769 58.05 +65769 70.52 +65769 91.49 +65770 NULL +65770 51.9 +65771 NULL +65771 6.15 +65771 7.5 +65772 NULL +65773 NULL +65773 3.81 +65773 18.2 +65773 30.49 +65773 47.09 +65773 53.09 +65773 63.26 +65773 76.46 +65774 NULL +65774 45.74 +65774 45.97 +65774 48.8 +65774 56.84 +65774 94.77 +65775 NULL +65775 7.88 +65775 66.56 +65775 66.68 +65775 98.43 +65776 NULL +65776 18.7 +65776 28.47 +65776 49.73 +65776 98.87 +65777 NULL +65777 54.39 +65777 73.79 +65777 82.62 +65778 NULL +65778 7.37 +65778 51.64 +65778 59.03 +65778 62.17 +65778 64.69 +65778 89.51 +65778 95.69 +65779 NULL +65779 11.87 +65779 28.2 +65779 39.48 +65779 45.61 +65779 64.41 +65779 65.24 +65780 NULL +65780 10.95 +65780 38.58 +65781 NULL +65781 70.59 +65781 95.52 +65782 NULL +65782 30.24 +65782 34.31 +65782 76.14 +65782 81.9 +65783 NULL +65783 46.34 +65783 51.08 +65783 52.43 +65783 62.58 +65783 77.4 +65784 NULL +65784 15.7 +65784 31.35 +65784 68.18 +65784 93.95 +65785 NULL +65785 29.61 +65786 NULL +65786 8.99 +65786 29.32 +65786 66.89 +65786 80.94 +65787 NULL +65787 18.78 +65787 31.19 +65787 64.88 +65788 NULL +65788 16.1 +65788 21.81 +65788 25.77 +65789 NULL +65789 20.44 +65789 43.53 +65789 52.49 +65789 83.18 +65789 92.74 +65789 96.9 +65790 NULL +65790 46.91 +65790 84.87 +65791 NULL +65791 4.24 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-3-34d9ee4120f21d0d0ae914fba0acc60c b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-3-34d9ee4120f21d0d0ae914fba0acc60c new file mode 100644 index 0000000000000000000000000000000000000000..569c1d4e5f7b6df2a9605e5b4b2560a995991768 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-3-34d9ee4120f21d0d0ae914fba0acc60c @@ -0,0 +1,1049 @@ + 7 + 11 + 38 +alice allen 48 +alice allen 78 +alice allen 79 +alice brown 91 +alice carson 28 +alice davidson 88 +alice falkner 117 +alice garcia 106 +alice hernandez 37 +alice hernandez 85 +alice johnson 65 +alice king 109 +alice king 113 +alice king 118 +alice laertes 6 +alice laertes 65 +alice miller 97 +alice nixon 15 +alice nixon 31 +alice nixon 56 +alice ovid 15 +alice polk 90 +alice quirinius 27 +alice quirinius 89 +alice robinson 5 +alice robinson 68 +alice steinbeck 45 +alice steinbeck 50 +alice steinbeck 77 +alice underhill 34 +alice van buren 31 +alice xylophone 0 +alice xylophone 5 +alice xylophone 113 +alice zipper 19 +alice zipper 51 +alice zipper 74 +bob brown 35 +bob brown 61 +bob brown 71 +bob carson 40 +bob davidson 44 +bob davidson 103 +bob davidson 124 +bob ellison 7 +bob ellison 46 +bob ellison 85 +bob ellison 103 +bob falkner 2 +bob garcia 16 +bob garcia 42 +bob garcia 71 +bob garcia 77 +bob garcia 83 +bob hernandez 93 +bob ichabod 72 +bob king 9 +bob king 22 +bob king 81 +bob laertes -1 +bob laertes 105 +bob miller 31 +bob ovid 26 +bob ovid 27 +bob ovid 81 +bob ovid 86 +bob polk 55 +bob quirinius 26 +bob steinbeck 95 +bob van buren 88 +bob white 1 +bob white 16 +bob xylophone -2 +bob xylophone 49 +bob young -1 +bob zipper 36 +bob zipper 78 +bob zipper 92 +calvin allen 98 +calvin brown 81 +calvin brown 87 +calvin brown 121 +calvin carson 105 +calvin davidson 34 +calvin davidson 89 +calvin ellison 34 +calvin falkner -1 +calvin falkner 12 +calvin falkner 46 +calvin falkner 50 +calvin falkner 53 +calvin falkner 101 +calvin garcia 79 +calvin hernandez 22 +calvin johnson 34 +calvin laertes 37 +calvin laertes 100 +calvin nixon 50 +calvin nixon 71 +calvin nixon 72 +calvin ovid -1 +calvin ovid 50 +calvin ovid 65 +calvin ovid 71 +calvin polk 23 +calvin quirinius 5 +calvin quirinius 96 +calvin robinson 0 +calvin steinbeck 13 +calvin steinbeck 14 +calvin steinbeck 33 +calvin thompson 12 +calvin thompson 40 +calvin underhill 19 +calvin van buren 57 +calvin van buren 78 +calvin white 26 +calvin white 80 +calvin xylophone 17 +calvin xylophone 48 +calvin xylophone 78 +calvin young 8 +calvin young 99 +calvin zipper 31 +calvin zipper 46 +david allen 27 +david allen 80 +david brown 75 +david brown 117 +david davidson 11 +david davidson 38 +david davidson 54 +david davidson 74 +david ellison 50 +david ellison 54 +david ellison 120 +david hernandez 72 +david ichabod 6 +david ichabod 55 +david laertes 88 +david nixon 118 +david ovid 61 +david ovid 95 +david quirinius 43 +david quirinius 85 +david quirinius 121 +david robinson 47 +david robinson 59 +david thompson 89 +david underhill 69 +david underhill 87 +david underhill 98 +david van buren 1 +david van buren 38 +david white 93 +david xylophone 0 +david xylophone 22 +david xylophone 82 +david young 2 +david young 79 +ethan allen 24 +ethan brown 3 +ethan brown 29 +ethan brown 55 +ethan brown 64 +ethan brown 84 +ethan brown 108 +ethan carson 83 +ethan ellison 28 +ethan ellison 65 +ethan falkner 45 +ethan falkner 49 +ethan garcia 16 +ethan hernandez 71 +ethan johnson 108 +ethan king 44 +ethan laertes -1 +ethan laertes 27 +ethan laertes 46 +ethan laertes 68 +ethan laertes 81 +ethan laertes 103 +ethan laertes 114 +ethan miller 40 +ethan nixon 18 +ethan ovid 66 +ethan polk 46 +ethan polk 71 +ethan polk 114 +ethan polk 117 +ethan quirinius 16 +ethan quirinius 41 +ethan quirinius 85 +ethan robinson 32 +ethan robinson 34 +ethan underhill 89 +ethan van buren 43 +ethan white 38 +ethan white 51 +ethan xylophone 107 +ethan zipper 37 +ethan zipper 89 +fred davidson 18 +fred davidson 18 +fred davidson 77 +fred ellison -3 +fred ellison 44 +fred ellison 116 +fred falkner 18 +fred falkner 33 +fred falkner 100 +fred hernandez 36 +fred ichabod 1 +fred ichabod 77 +fred johnson 62 +fred king 33 +fred king 92 +fred laertes 17 +fred miller NULL +fred nixon 33 +fred nixon 36 +fred nixon 102 +fred nixon 111 +fred polk -2 +fred polk 39 +fred polk 60 +fred polk 85 +fred quirinius 25 +fred quirinius 124 +fred robinson 89 +fred steinbeck 79 +fred steinbeck 118 +fred steinbeck 119 +fred underhill 122 +fred van buren 4 +fred van buren 24 +fred van buren 63 +fred van buren 106 +fred white 97 +fred young 33 +fred young 103 +fred zipper 66 +gabriella allen 114 +gabriella allen 119 +gabriella brown 25 +gabriella brown 92 +gabriella carson 112 +gabriella davidson 45 +gabriella ellison 21 +gabriella ellison 101 +gabriella falkner 14 +gabriella falkner 66 +gabriella falkner 77 +gabriella garcia 110 +gabriella hernandez 20 +gabriella hernandez 36 +gabriella ichabod 17 +gabriella ichabod 66 +gabriella ichabod 71 +gabriella ichabod 90 +gabriella ichabod 91 +gabriella king 100 +gabriella king 115 +gabriella laertes 50 +gabriella miller 35 +gabriella ovid 38 +gabriella ovid 89 +gabriella polk 42 +gabriella polk 105 +gabriella steinbeck 18 +gabriella steinbeck 115 +gabriella thompson 45 +gabriella thompson 70 +gabriella thompson 88 +gabriella van buren 5 +gabriella van buren 117 +gabriella white 37 +gabriella young 48 +gabriella young 107 +gabriella zipper 57 +gabriella zipper 71 +holly allen 63 +holly brown 50 +holly brown 117 +holly falkner NULL +holly hernandez 31 +holly hernandez 43 +holly hernandez 48 +holly hernandez 100 +holly ichabod 28 +holly ichabod 53 +holly ichabod 83 +holly johnson 60 +holly johnson 112 +holly johnson 121 +holly king 90 +holly king 114 +holly laertes 13 +holly miller 28 +holly nixon -2 +holly nixon 120 +holly polk 54 +holly polk 124 +holly robinson 1 +holly thompson 28 +holly thompson 33 +holly thompson 83 +holly underhill 5 +holly underhill 31 +holly underhill 101 +holly underhill 113 +holly van buren 19 +holly white 18 +holly white 90 +holly xylophone 18 +holly young 32 +holly young 72 +holly zipper 78 +holly zipper 88 +irene allen 56 +irene brown 78 +irene brown 93 +irene brown 108 +irene carson 0 +irene ellison -3 +irene ellison 40 +irene falkner 13 +irene falkner 104 +irene garcia 34 +irene garcia 62 +irene garcia 124 +irene ichabod 83 +irene ichabod 112 +irene johnson 88 +irene laertes 9 +irene laertes 54 +irene laertes 60 +irene miller 108 +irene nixon -1 +irene nixon 12 +irene nixon 101 +irene ovid 26 +irene ovid 32 +irene ovid 53 +irene polk 18 +irene polk 92 +irene polk 99 +irene polk 109 +irene polk 116 +irene quirinius 7 +irene quirinius 76 +irene quirinius 97 +irene robinson 51 +irene steinbeck 46 +irene thompson 10 +irene underhill 27 +irene underhill 63 +irene van buren 17 +irene van buren 104 +irene xylophone 18 +jessica brown 117 +jessica carson 3 +jessica carson 13 +jessica carson 88 +jessica davidson 11 +jessica davidson 28 +jessica davidson 89 +jessica davidson 124 +jessica ellison 38 +jessica ellison 50 +jessica falkner 71 +jessica garcia 25 +jessica garcia 43 +jessica ichabod 104 +jessica johnson 31 +jessica johnson 69 +jessica miller 74 +jessica nixon 22 +jessica nixon 120 +jessica ovid 47 +jessica ovid 73 +jessica polk 118 +jessica quirinius 0 +jessica quirinius 87 +jessica quirinius 105 +jessica quirinius 114 +jessica robinson 15 +jessica thompson 1 +jessica thompson 77 +jessica underhill 32 +jessica underhill 46 +jessica underhill 83 +jessica van buren 54 +jessica white 5 +jessica white 30 +jessica white 45 +jessica white 65 +jessica white 98 +jessica xylophone 67 +jessica young 61 +jessica young 123 +jessica zipper 27 +jessica zipper 33 +jessica zipper 54 +katie allen 114 +katie brown 39 +katie davidson 35 +katie ellison 5 +katie ellison 58 +katie falkner 15 +katie garcia 49 +katie garcia 65 +katie hernandez 83 +katie ichabod 9 +katie ichabod 75 +katie ichabod 104 +katie king 44 +katie king 59 +katie king 93 +katie miller 23 +katie miller 117 +katie nixon 43 +katie ovid 81 +katie polk 17 +katie polk 85 +katie robinson 92 +katie van buren 25 +katie van buren 88 +katie white 34 +katie white 86 +katie xylophone 84 +katie young 2 +katie young 24 +katie young 70 +katie zipper 25 +katie zipper 87 +luke allen 7 +luke allen 44 +luke allen 62 +luke allen 100 +luke allen 114 +luke brown 112 +luke davidson 51 +luke davidson 84 +luke ellison 35 +luke ellison 40 +luke ellison 86 +luke falkner 59 +luke falkner 97 +luke garcia 51 +luke garcia 100 +luke ichabod 42 +luke ichabod 123 +luke johnson 9 +luke johnson 17 +luke johnson 53 +luke laertes 66 +luke laertes 73 +luke laertes 76 +luke laertes 101 +luke laertes 118 +luke miller 93 +luke ovid 43 +luke ovid 70 +luke polk 53 +luke polk 88 +luke quirinius 82 +luke robinson 0 +luke robinson 114 +luke thompson 51 +luke underhill 2 +luke underhill 109 +luke underhill 119 +luke van buren 43 +luke white 110 +luke xylophone 15 +luke zipper 10 +mike allen 0 +mike brown 88 +mike carson 12 +mike carson 17 +mike carson 122 +mike davidson 9 +mike davidson 110 +mike ellison 5 +mike ellison 50 +mike ellison 70 +mike ellison 94 +mike ellison 95 +mike falkner 61 +mike garcia 2 +mike garcia 68 +mike garcia 110 +mike hernandez 91 +mike hernandez 106 +mike ichabod 18 +mike king 4 +mike king 58 +mike king 83 +mike king 96 +mike king 103 +mike king 118 +mike miller 51 +mike nixon 97 +mike nixon 106 +mike polk 6 +mike polk 65 +mike polk 119 +mike quirinius 22 +mike steinbeck 75 +mike steinbeck 85 +mike steinbeck 101 +mike steinbeck 116 +mike van buren 16 +mike van buren 111 +mike white -1 +mike white 22 +mike white 45 +mike white 61 +mike young 37 +mike young 53 +mike young 72 +mike zipper 27 +mike zipper 76 +mike zipper 106 +nick allen 8 +nick allen 57 +nick brown 114 +nick davidson 84 +nick ellison 10 +nick ellison 107 +nick falkner 83 +nick falkner 86 +nick garcia 53 +nick garcia 69 +nick garcia 108 +nick ichabod 59 +nick ichabod 71 +nick ichabod 84 +nick johnson 47 +nick johnson 88 +nick laertes 17 +nick miller 101 +nick nixon 43 +nick ovid 42 +nick polk 1 +nick quirinius 22 +nick quirinius 36 +nick robinson 48 +nick robinson 54 +nick steinbeck 33 +nick thompson 73 +nick underhill 122 +nick van buren 53 +nick xylophone 80 +nick young 6 +nick young 60 +nick zipper 3 +nick zipper 21 +oscar allen 58 +oscar brown 80 +oscar carson 10 +oscar carson 27 +oscar carson 36 +oscar carson 72 +oscar carson 88 +oscar davidson 14 +oscar ellison 50 +oscar ellison 74 +oscar falkner 96 +oscar garcia 44 +oscar hernandez 1 +oscar hernandez 93 +oscar ichabod 20 +oscar ichabod 28 +oscar ichabod 69 +oscar ichabod 120 +oscar johnson 44 +oscar johnson 53 +oscar king 67 +oscar king 71 +oscar king 81 +oscar laertes 4 +oscar laertes 28 +oscar laertes 53 +oscar laertes 63 +oscar nixon 58 +oscar ovid 3 +oscar ovid 27 +oscar ovid 47 +oscar polk 8 +oscar polk 112 +oscar quirinius NULL +oscar quirinius 0 +oscar quirinius 17 +oscar quirinius 114 +oscar robinson 16 +oscar robinson 42 +oscar robinson 59 +oscar robinson 93 +oscar steinbeck 51 +oscar thompson 44 +oscar thompson 44 +oscar thompson 60 +oscar thompson 66 +oscar underhill 86 +oscar van buren 40 +oscar van buren 51 +oscar van buren 114 +oscar white 2 +oscar white 20 +oscar white 49 +oscar white 58 +oscar xylophone 18 +oscar xylophone 73 +oscar xylophone 74 +oscar zipper 0 +oscar zipper 23 +oscar zipper 95 +priscilla brown 51 +priscilla brown 75 +priscilla brown 97 +priscilla carson 16 +priscilla carson 52 +priscilla carson 124 +priscilla ichabod 117 +priscilla ichabod 122 +priscilla johnson 5 +priscilla johnson 17 +priscilla johnson 62 +priscilla johnson 77 +priscilla johnson 117 +priscilla king 43 +priscilla nixon 61 +priscilla nixon 66 +priscilla ovid 46 +priscilla ovid 118 +priscilla polk 45 +priscilla quirinius 83 +priscilla thompson 82 +priscilla underhill 117 +priscilla underhill 122 +priscilla van buren 0 +priscilla van buren 22 +priscilla van buren 102 +priscilla white 88 +priscilla xylophone 8 +priscilla xylophone 90 +priscilla xylophone 109 +priscilla young 17 +priscilla young 113 +priscilla zipper 27 +priscilla zipper 35 +quinn allen 27 +quinn allen 114 +quinn brown 70 +quinn brown 88 +quinn brown 117 +quinn davidson 93 +quinn davidson 93 +quinn davidson 109 +quinn davidson 121 +quinn ellison 83 +quinn ellison 116 +quinn garcia 78 +quinn garcia 104 +quinn garcia 110 +quinn garcia 120 +quinn ichabod 60 +quinn king 14 +quinn king 46 +quinn laertes -2 +quinn laertes 65 +quinn laertes 95 +quinn nixon 11 +quinn ovid 123 +quinn quirinius 94 +quinn robinson 60 +quinn steinbeck 82 +quinn steinbeck 122 +quinn thompson 41 +quinn thompson 60 +quinn underhill 19 +quinn underhill 28 +quinn underhill 34 +quinn van buren 18 +quinn young 15 +quinn zipper 44 +quinn zipper 103 +rachel allen 76 +rachel allen 122 +rachel brown 23 +rachel brown 56 +rachel brown 71 +rachel brown 101 +rachel brown 108 +rachel carson 27 +rachel carson 74 +rachel davidson 84 +rachel ellison 51 +rachel falkner -2 +rachel falkner 43 +rachel falkner 72 +rachel falkner 104 +rachel johnson 32 +rachel king 84 +rachel king 95 +rachel laertes 37 +rachel laertes 106 +rachel ovid 5 +rachel ovid 31 +rachel polk 79 +rachel quirinius 108 +rachel robinson 24 +rachel robinson 41 +rachel robinson 91 +rachel thompson -3 +rachel thompson -2 +rachel thompson 74 +rachel underhill 11 +rachel white 108 +rachel white 119 +rachel young 77 +rachel zipper 16 +rachel zipper 116 +sarah carson 41 +sarah carson 58 +sarah carson 119 +sarah ellison 14 +sarah falkner 112 +sarah falkner 123 +sarah garcia 72 +sarah garcia 91 +sarah garcia 98 +sarah ichabod 38 +sarah ichabod 80 +sarah johnson 5 +sarah johnson 51 +sarah johnson 69 +sarah johnson 116 +sarah king 13 +sarah king 120 +sarah miller 31 +sarah ovid 122 +sarah robinson 26 +sarah robinson 35 +sarah steinbeck 30 +sarah white 11 +sarah white 32 +sarah xylophone 28 +sarah young 120 +sarah zipper 107 +tom brown 27 +tom brown 89 +tom carson 11 +tom carson 70 +tom carson 123 +tom davidson 72 +tom ellison 28 +tom ellison 118 +tom ellison 120 +tom falkner 11 +tom falkner 35 +tom hernandez -3 +tom hernandez 118 +tom ichabod 19 +tom johnson 42 +tom johnson 82 +tom king 59 +tom laertes 33 +tom laertes 54 +tom miller 9 +tom miller 48 +tom miller 94 +tom nixon 45 +tom ovid 68 +tom polk 70 +tom polk 107 +tom quirinius 10 +tom quirinius 38 +tom robinson 52 +tom robinson 104 +tom robinson 109 +tom robinson 115 +tom steinbeck 113 +tom van buren 5 +tom van buren 48 +tom van buren 63 +tom white 81 +tom young 13 +tom young 99 +tom zipper 31 +ulysses brown 46 +ulysses carson 5 +ulysses carson 26 +ulysses carson 55 +ulysses carson 109 +ulysses davidson 18 +ulysses ellison 61 +ulysses garcia 12 +ulysses hernandez 9 +ulysses hernandez 22 +ulysses hernandez 53 +ulysses ichabod 32 +ulysses ichabod 99 +ulysses johnson 41 +ulysses king 2 +ulysses laertes 40 +ulysses laertes 51 +ulysses laertes 95 +ulysses miller 23 +ulysses miller 85 +ulysses nixon 92 +ulysses ovid 31 +ulysses polk 28 +ulysses polk 74 +ulysses polk 86 +ulysses polk 89 +ulysses quirinius 7 +ulysses robinson 79 +ulysses steinbeck 6 +ulysses steinbeck 45 +ulysses thompson 24 +ulysses underhill 6 +ulysses underhill 27 +ulysses underhill 42 +ulysses underhill 51 +ulysses underhill 93 +ulysses underhill 98 +ulysses underhill 111 +ulysses van buren 58 +ulysses white 67 +ulysses white 109 +ulysses xylophone 47 +ulysses xylophone 105 +ulysses xylophone 123 +ulysses young 61 +ulysses young 86 +ulysses young 89 +victor allen 2 +victor allen 17 +victor brown 0 +victor brown 23 +victor brown 60 +victor brown 64 +victor davidson 42 +victor davidson 89 +victor davidson 123 +victor ellison 35 +victor ellison 84 +victor hernandez 1 +victor hernandez 17 +victor hernandez 91 +victor hernandez 94 +victor hernandez 116 +victor johnson 34 +victor johnson 53 +victor johnson 57 +victor king 59 +victor king 112 +victor laertes 18 +victor laertes 118 +victor miller 79 +victor nixon 50 +victor nixon 104 +victor ovid 120 +victor polk 106 +victor quirinius 77 +victor quirinius 85 +victor robinson 29 +victor robinson 105 +victor steinbeck 20 +victor steinbeck 92 +victor steinbeck 100 +victor thompson 124 +victor van buren 41 +victor van buren 71 +victor white 15 +victor white 49 +victor xylophone -3 +victor xylophone 41 +victor xylophone 43 +victor xylophone 54 +victor xylophone 91 +victor young 24 +victor zipper 3 +wendy allen 25 +wendy allen 38 +wendy allen 95 +wendy brown 92 +wendy brown 119 +wendy ellison 53 +wendy ellison 103 +wendy falkner 23 +wendy falkner 28 +wendy falkner 58 +wendy garcia 3 +wendy garcia 48 +wendy garcia 60 +wendy garcia 99 +wendy hernandez 53 +wendy ichabod 87 +wendy king -2 +wendy king 45 +wendy king 124 +wendy laertes 26 +wendy laertes 51 +wendy laertes 72 +wendy miller 51 +wendy miller 105 +wendy nixon 5 +wendy nixon 25 +wendy ovid 17 +wendy ovid 85 +wendy polk 44 +wendy polk 99 +wendy quirinius 77 +wendy quirinius 88 +wendy robinson -3 +wendy robinson 71 +wendy robinson 97 +wendy steinbeck 37 +wendy thompson 28 +wendy thompson 31 +wendy underhill 58 +wendy underhill 82 +wendy underhill 120 +wendy van buren 27 +wendy van buren 82 +wendy white 63 +wendy xylophone 53 +wendy xylophone 119 +wendy young 66 +wendy young 112 +xavier allen 18 +xavier allen 41 +xavier allen 106 +xavier brown 10 +xavier brown 63 +xavier brown 108 +xavier carson 20 +xavier carson 57 +xavier davidson 21 +xavier davidson 24 +xavier davidson 106 +xavier ellison 0 +xavier ellison 53 +xavier garcia 42 +xavier hernandez 9 +xavier hernandez 80 +xavier hernandez 114 +xavier ichabod 20 +xavier ichabod 58 +xavier johnson 44 +xavier johnson 85 +xavier king 26 +xavier king 107 +xavier laertes 60 +xavier ovid 3 +xavier polk 29 +xavier polk 83 +xavier polk 91 +xavier polk 122 +xavier quirinius 27 +xavier quirinius 35 +xavier quirinius 39 +xavier quirinius 111 +xavier thompson 2 +xavier underhill 102 +xavier white 8 +xavier white 56 +xavier xylophone 24 +xavier zipper 48 +yuri allen 31 +yuri allen 121 +yuri brown 101 +yuri brown 106 +yuri carson 1 +yuri carson 36 +yuri ellison -1 +yuri ellison 43 +yuri falkner 31 +yuri falkner 96 +yuri garcia 49 +yuri hernandez 92 +yuri johnson 1 +yuri johnson 2 +yuri johnson 111 +yuri king 44 +yuri laertes 84 +yuri laertes 115 +yuri nixon 5 +yuri nixon 111 +yuri polk 13 +yuri polk 49 +yuri polk 115 +yuri quirinius 24 +yuri quirinius 28 +yuri quirinius 90 +yuri steinbeck 8 +yuri steinbeck 65 +yuri thompson 42 +yuri underhill 10 +yuri underhill 66 +yuri white 73 +yuri xylophone 63 +zach allen 35 +zach brown 7 +zach brown 15 +zach brown 37 +zach brown 61 +zach brown 94 +zach carson 114 +zach ellison 16 +zach falkner 70 +zach falkner 115 +zach garcia -2 +zach garcia 59 +zach garcia 68 +zach garcia 97 +zach ichabod 14 +zach ichabod 73 +zach king 66 +zach king 70 +zach king 81 +zach miller 4 +zach miller 9 +zach miller 73 +zach ovid 61 +zach ovid 68 +zach ovid 77 +zach ovid 114 +zach quirinius 79 +zach robinson 69 +zach steinbeck 6 +zach steinbeck 122 +zach thompson 75 +zach thompson 95 +zach underhill 123 +zach white 58 +zach xylophone 19 +zach xylophone 85 +zach young 11 +zach zipper 68 +zach zipper 100 +zach zipper 101 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-4-dfd39236756a3951bc1ec354799d69e4 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-4-dfd39236756a3951bc1ec354799d69e4 new file mode 100644 index 0000000000000000000000000000000000000000..86ca4e49d21bd44ab1fcf545390f06b32111ceb3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-4-dfd39236756a3951bc1ec354799d69e4 @@ -0,0 +1,1049 @@ + + + +alice allen +alice allen +alice allen +alice brown +alice carson +alice davidson +alice falkner +alice garcia +alice hernandez +alice hernandez +alice johnson +alice king +alice king +alice king +alice laertes +alice laertes +alice miller +alice nixon +alice nixon +alice nixon +alice ovid +alice polk +alice quirinius +alice quirinius +alice robinson +alice robinson +alice steinbeck +alice steinbeck +alice steinbeck +alice underhill +alice van buren +alice xylophone +alice xylophone +alice xylophone +alice zipper +alice zipper +alice zipper +bob brown +bob brown +bob brown +bob carson +bob davidson +bob davidson +bob davidson +bob ellison +bob ellison +bob ellison +bob ellison +bob falkner +bob garcia +bob garcia +bob garcia +bob garcia +bob garcia +bob hernandez +bob ichabod +bob king +bob king +bob king +bob laertes +bob laertes +bob miller +bob ovid +bob ovid +bob ovid +bob ovid +bob polk +bob quirinius +bob steinbeck +bob van buren +bob white +bob white +bob xylophone +bob xylophone +bob young +bob zipper +bob zipper +bob zipper +calvin allen +calvin brown +calvin brown +calvin brown +calvin carson +calvin davidson +calvin davidson +calvin ellison +calvin falkner +calvin falkner +calvin falkner +calvin falkner +calvin falkner +calvin falkner +calvin garcia +calvin hernandez +calvin johnson +calvin laertes +calvin laertes +calvin nixon +calvin nixon +calvin nixon +calvin ovid +calvin ovid +calvin ovid +calvin ovid +calvin polk +calvin quirinius +calvin quirinius +calvin robinson +calvin steinbeck +calvin steinbeck +calvin steinbeck +calvin thompson +calvin thompson +calvin underhill +calvin van buren +calvin van buren +calvin white +calvin white +calvin xylophone +calvin xylophone +calvin xylophone +calvin young +calvin young +calvin zipper +calvin zipper +david allen +david allen +david brown +david brown +david davidson +david davidson +david davidson +david davidson +david ellison +david ellison +david ellison +david hernandez +david ichabod +david ichabod +david laertes +david nixon +david ovid +david ovid +david quirinius +david quirinius +david quirinius +david robinson +david robinson +david thompson +david underhill +david underhill +david underhill +david van buren +david van buren +david white +david xylophone +david xylophone +david xylophone +david young +david young +ethan allen +ethan brown +ethan brown +ethan brown +ethan brown +ethan brown +ethan brown +ethan carson +ethan ellison +ethan ellison +ethan falkner +ethan falkner +ethan garcia +ethan hernandez +ethan johnson +ethan king +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan miller +ethan nixon +ethan ovid +ethan polk +ethan polk +ethan polk +ethan polk +ethan quirinius +ethan quirinius +ethan quirinius +ethan robinson +ethan robinson +ethan underhill +ethan van buren +ethan white +ethan white +ethan xylophone +ethan zipper +ethan zipper +fred davidson +fred davidson +fred davidson +fred ellison +fred ellison +fred ellison +fred falkner +fred falkner +fred falkner +fred hernandez +fred ichabod +fred ichabod +fred johnson +fred king +fred king +fred laertes +fred miller +fred nixon +fred nixon +fred nixon +fred nixon +fred polk +fred polk +fred polk +fred polk +fred quirinius +fred quirinius +fred robinson +fred steinbeck +fred steinbeck +fred steinbeck +fred underhill +fred van buren +fred van buren +fred van buren +fred van buren +fred white +fred young +fred young +fred zipper +gabriella allen +gabriella allen +gabriella brown +gabriella brown +gabriella carson +gabriella davidson +gabriella ellison +gabriella ellison +gabriella falkner +gabriella falkner +gabriella falkner +gabriella garcia +gabriella hernandez +gabriella hernandez +gabriella ichabod +gabriella ichabod +gabriella ichabod +gabriella ichabod +gabriella ichabod +gabriella king +gabriella king +gabriella laertes +gabriella miller +gabriella ovid +gabriella ovid +gabriella polk +gabriella polk +gabriella steinbeck +gabriella steinbeck +gabriella thompson +gabriella thompson +gabriella thompson +gabriella van buren +gabriella van buren +gabriella white +gabriella young +gabriella young +gabriella zipper +gabriella zipper +holly allen +holly brown +holly brown +holly falkner +holly hernandez +holly hernandez +holly hernandez +holly hernandez +holly ichabod +holly ichabod +holly ichabod +holly johnson +holly johnson +holly johnson +holly king +holly king +holly laertes +holly miller +holly nixon +holly nixon +holly polk +holly polk +holly robinson +holly thompson +holly thompson +holly thompson +holly underhill +holly underhill +holly underhill +holly underhill +holly van buren +holly white +holly white +holly xylophone +holly young +holly young +holly zipper +holly zipper +irene allen +irene brown +irene brown +irene brown +irene carson +irene ellison +irene ellison +irene falkner +irene falkner +irene garcia +irene garcia +irene garcia +irene ichabod +irene ichabod +irene johnson +irene laertes +irene laertes +irene laertes +irene miller +irene nixon +irene nixon +irene nixon +irene ovid +irene ovid +irene ovid +irene polk +irene polk +irene polk +irene polk +irene polk +irene quirinius +irene quirinius +irene quirinius +irene robinson +irene steinbeck +irene thompson +irene underhill +irene underhill +irene van buren +irene van buren +irene xylophone +jessica brown +jessica carson +jessica carson +jessica carson +jessica davidson +jessica davidson +jessica davidson +jessica davidson +jessica ellison +jessica ellison +jessica falkner +jessica garcia +jessica garcia +jessica ichabod +jessica johnson +jessica johnson +jessica miller +jessica nixon +jessica nixon +jessica ovid +jessica ovid +jessica polk +jessica quirinius +jessica quirinius +jessica quirinius +jessica quirinius +jessica robinson +jessica thompson +jessica thompson +jessica underhill +jessica underhill +jessica underhill +jessica van buren +jessica white +jessica white +jessica white +jessica white +jessica white +jessica xylophone +jessica young +jessica young +jessica zipper +jessica zipper +jessica zipper +katie allen +katie brown +katie davidson +katie ellison +katie ellison +katie falkner +katie garcia +katie garcia +katie hernandez +katie ichabod +katie ichabod +katie ichabod +katie king +katie king +katie king +katie miller +katie miller +katie nixon +katie ovid +katie polk +katie polk +katie robinson +katie van buren +katie van buren +katie white +katie white +katie xylophone +katie young +katie young +katie young +katie zipper +katie zipper +luke allen +luke allen +luke allen +luke allen +luke allen +luke brown +luke davidson +luke davidson +luke ellison +luke ellison +luke ellison +luke falkner +luke falkner +luke garcia +luke garcia +luke ichabod +luke ichabod +luke johnson +luke johnson +luke johnson +luke laertes +luke laertes +luke laertes +luke laertes +luke laertes +luke miller +luke ovid +luke ovid +luke polk +luke polk +luke quirinius +luke robinson +luke robinson +luke thompson +luke underhill +luke underhill +luke underhill +luke van buren +luke white +luke xylophone +luke zipper +mike allen +mike brown +mike carson +mike carson +mike carson +mike davidson +mike davidson +mike ellison +mike ellison +mike ellison +mike ellison +mike ellison +mike falkner +mike garcia +mike garcia +mike garcia +mike hernandez +mike hernandez +mike ichabod +mike king +mike king +mike king +mike king +mike king +mike king +mike miller +mike nixon +mike nixon +mike polk +mike polk +mike polk +mike quirinius +mike steinbeck +mike steinbeck +mike steinbeck +mike steinbeck +mike van buren +mike van buren +mike white +mike white +mike white +mike white +mike young +mike young +mike young +mike zipper +mike zipper +mike zipper +nick allen +nick allen +nick brown +nick davidson +nick ellison +nick ellison +nick falkner +nick falkner +nick garcia +nick garcia +nick garcia +nick ichabod +nick ichabod +nick ichabod +nick johnson +nick johnson +nick laertes +nick miller +nick nixon +nick ovid +nick polk +nick quirinius +nick quirinius +nick robinson +nick robinson +nick steinbeck +nick thompson +nick underhill +nick van buren +nick xylophone +nick young +nick young +nick zipper +nick zipper +oscar allen +oscar brown +oscar carson +oscar carson +oscar carson +oscar carson +oscar carson +oscar davidson +oscar ellison +oscar ellison +oscar falkner +oscar garcia +oscar hernandez +oscar hernandez +oscar ichabod +oscar ichabod +oscar ichabod +oscar ichabod +oscar johnson +oscar johnson +oscar king +oscar king +oscar king +oscar laertes +oscar laertes +oscar laertes +oscar laertes +oscar nixon +oscar ovid +oscar ovid +oscar ovid +oscar polk +oscar polk +oscar quirinius +oscar quirinius +oscar quirinius +oscar quirinius +oscar robinson +oscar robinson +oscar robinson +oscar robinson +oscar steinbeck +oscar thompson +oscar thompson +oscar thompson +oscar thompson +oscar underhill +oscar van buren +oscar van buren +oscar van buren +oscar white +oscar white +oscar white +oscar white +oscar xylophone +oscar xylophone +oscar xylophone +oscar zipper +oscar zipper +oscar zipper +priscilla brown +priscilla brown +priscilla brown +priscilla carson +priscilla carson +priscilla carson +priscilla ichabod +priscilla ichabod +priscilla johnson +priscilla johnson +priscilla johnson +priscilla johnson +priscilla johnson +priscilla king +priscilla nixon +priscilla nixon +priscilla ovid +priscilla ovid +priscilla polk +priscilla quirinius +priscilla thompson +priscilla underhill +priscilla underhill +priscilla van buren +priscilla van buren +priscilla van buren +priscilla white +priscilla xylophone +priscilla xylophone +priscilla xylophone +priscilla young +priscilla young +priscilla zipper +priscilla zipper +quinn allen +quinn allen +quinn brown +quinn brown +quinn brown +quinn davidson +quinn davidson +quinn davidson +quinn davidson +quinn ellison +quinn ellison +quinn garcia +quinn garcia +quinn garcia +quinn garcia +quinn ichabod +quinn king +quinn king +quinn laertes +quinn laertes +quinn laertes +quinn nixon +quinn ovid +quinn quirinius +quinn robinson +quinn steinbeck +quinn steinbeck +quinn thompson +quinn thompson +quinn underhill +quinn underhill +quinn underhill +quinn van buren +quinn young +quinn zipper +quinn zipper +rachel allen +rachel allen +rachel brown +rachel brown +rachel brown +rachel brown +rachel brown +rachel carson +rachel carson +rachel davidson +rachel ellison +rachel falkner +rachel falkner +rachel falkner +rachel falkner +rachel johnson +rachel king +rachel king +rachel laertes +rachel laertes +rachel ovid +rachel ovid +rachel polk +rachel quirinius +rachel robinson +rachel robinson +rachel robinson +rachel thompson +rachel thompson +rachel thompson +rachel underhill +rachel white +rachel white +rachel young +rachel zipper +rachel zipper +sarah carson +sarah carson +sarah carson +sarah ellison +sarah falkner +sarah falkner +sarah garcia +sarah garcia +sarah garcia +sarah ichabod +sarah ichabod +sarah johnson +sarah johnson +sarah johnson +sarah johnson +sarah king +sarah king +sarah miller +sarah ovid +sarah robinson +sarah robinson +sarah steinbeck +sarah white +sarah white +sarah xylophone +sarah young +sarah zipper +tom brown +tom brown +tom carson +tom carson +tom carson +tom davidson +tom ellison +tom ellison +tom ellison +tom falkner +tom falkner +tom hernandez +tom hernandez +tom ichabod +tom johnson +tom johnson +tom king +tom laertes +tom laertes +tom miller +tom miller +tom miller +tom nixon +tom ovid +tom polk +tom polk +tom quirinius +tom quirinius +tom robinson +tom robinson +tom robinson +tom robinson +tom steinbeck +tom van buren +tom van buren +tom van buren +tom white +tom young +tom young +tom zipper +ulysses brown +ulysses carson +ulysses carson +ulysses carson +ulysses carson +ulysses davidson +ulysses ellison +ulysses garcia +ulysses hernandez +ulysses hernandez +ulysses hernandez +ulysses ichabod +ulysses ichabod +ulysses johnson +ulysses king +ulysses laertes +ulysses laertes +ulysses laertes +ulysses miller +ulysses miller +ulysses nixon +ulysses ovid +ulysses polk +ulysses polk +ulysses polk +ulysses polk +ulysses quirinius +ulysses robinson +ulysses steinbeck +ulysses steinbeck +ulysses thompson +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses van buren +ulysses white +ulysses white +ulysses xylophone +ulysses xylophone +ulysses xylophone +ulysses young +ulysses young +ulysses young +victor allen +victor allen +victor brown +victor brown +victor brown +victor brown +victor davidson +victor davidson +victor davidson +victor ellison +victor ellison +victor hernandez +victor hernandez +victor hernandez +victor hernandez +victor hernandez +victor johnson +victor johnson +victor johnson +victor king +victor king +victor laertes +victor laertes +victor miller +victor nixon +victor nixon +victor ovid +victor polk +victor quirinius +victor quirinius +victor robinson +victor robinson +victor steinbeck +victor steinbeck +victor steinbeck +victor thompson +victor van buren +victor van buren +victor white +victor white +victor xylophone +victor xylophone +victor xylophone +victor xylophone +victor xylophone +victor young +victor zipper +wendy allen +wendy allen +wendy allen +wendy brown +wendy brown +wendy ellison +wendy ellison +wendy falkner +wendy falkner +wendy falkner +wendy garcia +wendy garcia +wendy garcia +wendy garcia +wendy hernandez +wendy ichabod +wendy king +wendy king +wendy king +wendy laertes +wendy laertes +wendy laertes +wendy miller +wendy miller +wendy nixon +wendy nixon +wendy ovid +wendy ovid +wendy polk +wendy polk +wendy quirinius +wendy quirinius +wendy robinson +wendy robinson +wendy robinson +wendy steinbeck +wendy thompson +wendy thompson +wendy underhill +wendy underhill +wendy underhill +wendy van buren +wendy van buren +wendy white +wendy xylophone +wendy xylophone +wendy young +wendy young +xavier allen +xavier allen +xavier allen +xavier brown +xavier brown +xavier brown +xavier carson +xavier carson +xavier davidson +xavier davidson +xavier davidson +xavier ellison +xavier ellison +xavier garcia +xavier hernandez +xavier hernandez +xavier hernandez +xavier ichabod +xavier ichabod +xavier johnson +xavier johnson +xavier king +xavier king +xavier laertes +xavier ovid +xavier polk +xavier polk +xavier polk +xavier polk +xavier quirinius +xavier quirinius +xavier quirinius +xavier quirinius +xavier thompson +xavier underhill +xavier white +xavier white +xavier xylophone +xavier zipper +yuri allen +yuri allen +yuri brown +yuri brown +yuri carson +yuri carson +yuri ellison +yuri ellison +yuri falkner +yuri falkner +yuri garcia +yuri hernandez +yuri johnson +yuri johnson +yuri johnson +yuri king +yuri laertes +yuri laertes +yuri nixon +yuri nixon +yuri polk +yuri polk +yuri polk +yuri quirinius +yuri quirinius +yuri quirinius +yuri steinbeck +yuri steinbeck +yuri thompson +yuri underhill +yuri underhill +yuri white +yuri xylophone +zach allen +zach brown +zach brown +zach brown +zach brown +zach brown +zach carson +zach ellison +zach falkner +zach falkner +zach garcia +zach garcia +zach garcia +zach garcia +zach ichabod +zach ichabod +zach king +zach king +zach king +zach miller +zach miller +zach miller +zach ovid +zach ovid +zach ovid +zach ovid +zach quirinius +zach robinson +zach steinbeck +zach steinbeck +zach thompson +zach thompson +zach underhill +zach white +zach xylophone +zach xylophone +zach young +zach zipper +zach zipper +zach zipper diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-5-8d0ee3e1605f38214bfad28a5ce897cc b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-5-8d0ee3e1605f38214bfad28a5ce897cc new file mode 100644 index 0000000000000000000000000000000000000000..ddb15e338263f45f956ef02504c3f6f58799bbe9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-5-8d0ee3e1605f38214bfad28a5ce897cc @@ -0,0 +1 @@ +10 oscar carson 65549 65549 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-0-b7cb25303831392a51cd996e758ac79a b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-0-b7cb25303831392a51cd996e758ac79a new file mode 100644 index 0000000000000000000000000000000000000000..42e5151fe211bd57cab35b137fcd3337b4ee68f6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-0-b7cb25303831392a51cd996e758ac79a @@ -0,0 +1,1049 @@ +65536 1 +65536 1 +65536 1 +65536 1 +65536 1 +65536 1 +65536 1 +65537 1 +65537 1 +65537 1 +65537 1 +65537 1 +65538 1 +65538 1 +65538 1 +65539 1 +65539 1 +65539 1 +65540 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65542 1 +65542 1 +65542 1 +65542 1 +65542 1 +65543 1 +65543 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 2 +65545 1 +65545 1 +65545 1 +65546 2 +65547 1 +65547 1 +65547 1 +65547 1 +65548 1 +65548 1 +65548 1 +65548 1 +65548 1 +65548 1 +65548 2 +65549 1 +65549 1 +65549 1 +65549 1 +65549 1 +65549 1 +65549 1 +65549 3 +65550 1 +65550 1 +65550 1 +65550 1 +65550 1 +65550 1 +65550 2 +65551 1 +65551 1 +65551 1 +65552 1 +65552 1 +65552 1 +65552 1 +65552 2 +65553 1 +65553 1 +65553 1 +65553 1 +65553 1 +65554 1 +65554 1 +65555 2 +65556 1 +65556 1 +65556 1 +65556 1 +65556 1 +65557 1 +65557 1 +65558 1 +65559 1 +65559 1 +65559 1 +65559 1 +65559 1 +65560 1 +65560 1 +65560 1 +65560 2 +65560 2 +65561 1 +65561 2 +65561 2 +65562 1 +65562 1 +65562 1 +65562 1 +65562 1 +65562 2 +65563 1 +65563 1 +65563 1 +65563 1 +65564 1 +65564 1 +65564 1 +65564 1 +65564 2 +65564 2 +65564 2 +65565 1 +65565 1 +65566 1 +65566 2 +65567 1 +65568 1 +65568 1 +65568 1 +65569 1 +65570 1 +65570 1 +65570 1 +65570 1 +65570 1 +65570 2 +65571 1 +65571 1 +65571 1 +65571 2 +65572 1 +65572 1 +65572 1 +65572 1 +65572 1 +65573 1 +65573 1 +65573 3 +65574 1 +65574 1 +65574 1 +65575 1 +65575 1 +65575 1 +65575 2 +65576 1 +65576 1 +65576 1 +65576 3 +65577 1 +65578 1 +65578 1 +65578 1 +65578 1 +65578 1 +65578 1 +65578 2 +65579 1 +65579 1 +65579 1 +65579 1 +65579 3 +65580 1 +65580 2 +65580 2 +65581 1 +65581 1 +65581 1 +65581 1 +65581 2 +65581 2 +65582 1 +65582 1 +65582 1 +65582 1 +65583 1 +65583 1 +65583 2 +65583 2 +65584 1 +65584 1 +65584 1 +65585 1 +65585 1 +65585 1 +65585 1 +65585 2 +65586 1 +65586 1 +65586 1 +65586 1 +65586 2 +65587 1 +65587 1 +65587 1 +65587 2 +65587 3 +65588 1 +65588 1 +65588 1 +65588 1 +65588 2 +65589 1 +65589 1 +65589 1 +65589 1 +65589 2 +65590 1 +65590 1 +65590 1 +65590 1 +65590 1 +65591 1 +65591 1 +65591 2 +65591 3 +65592 1 +65593 1 +65593 1 +65594 1 +65594 1 +65594 1 +65594 3 +65595 1 +65595 1 +65595 1 +65595 2 +65595 2 +65595 4 +65596 1 +65596 1 +65596 1 +65596 2 +65596 2 +65597 1 +65597 1 +65597 2 +65598 1 +65598 2 +65599 1 +65599 1 +65599 1 +65599 1 +65599 3 +65600 1 +65600 1 +65600 2 +65600 2 +65601 1 +65601 2 +65601 2 +65602 1 +65602 1 +65602 1 +65602 1 +65602 1 +65602 2 +65603 1 +65603 1 +65603 1 +65603 2 +65603 2 +65604 1 +65604 2 +65604 2 +65604 3 +65605 2 +65606 1 +65606 1 +65606 1 +65606 1 +65606 2 +65606 2 +65607 1 +65607 1 +65607 1 +65607 1 +65607 2 +65607 3 +65608 1 +65608 1 +65608 1 +65608 2 +65609 1 +65610 1 +65610 1 +65610 3 +65610 4 +65610 5 +65611 1 +65611 1 +65611 1 +65611 1 +65612 1 +65612 2 +65612 2 +65612 3 +65613 1 +65614 1 +65614 1 +65614 2 +65615 1 +65615 1 +65615 2 +65615 2 +65616 1 +65616 2 +65617 1 +65617 2 +65617 2 +65617 2 +65618 1 +65618 1 +65618 2 +65618 2 +65618 2 +65619 1 +65619 1 +65619 1 +65619 1 +65619 2 +65619 2 +65620 1 +65620 1 +65620 1 +65620 1 +65620 1 +65620 3 +65621 1 +65621 1 +65622 1 +65622 1 +65622 1 +65622 3 +65622 3 +65622 4 +65623 1 +65623 1 +65623 1 +65623 1 +65623 1 +65623 2 +65623 4 +65624 1 +65624 2 +65624 2 +65624 2 +65624 4 +65625 1 +65625 1 +65625 1 +65625 1 +65626 1 +65626 1 +65626 1 +65626 2 +65627 1 +65627 1 +65627 2 +65627 3 +65628 1 +65628 1 +65628 1 +65628 2 +65628 2 +65628 3 +65629 1 +65629 1 +65629 1 +65629 3 +65630 1 +65630 2 +65631 1 +65632 2 +65632 2 +65633 1 +65633 2 +65633 2 +65633 2 +65634 1 +65634 1 +65634 1 +65634 2 +65635 1 +65635 2 +65635 3 +65636 1 +65636 1 +65636 2 +65637 1 +65637 1 +65637 1 +65637 1 +65637 1 +65637 1 +65637 2 +65637 3 +65638 1 +65638 1 +65638 2 +65639 1 +65640 2 +65641 1 +65641 1 +65641 1 +65641 5 +65642 4 +65643 1 +65643 1 +65643 2 +65643 2 +65643 2 +65643 2 +65643 2 +65643 2 +65643 4 +65644 1 +65644 1 +65644 1 +65644 1 +65644 2 +65644 2 +65644 3 +65645 1 +65645 1 +65645 1 +65646 1 +65646 1 +65646 1 +65647 1 +65647 2 +65648 1 +65648 1 +65648 2 +65648 2 +65649 1 +65649 1 +65649 2 +65649 2 +65650 1 +65650 1 +65650 1 +65650 1 +65650 2 +65650 2 +65651 1 +65651 1 +65651 2 +65651 2 +65651 3 +65652 1 +65652 1 +65652 2 +65653 1 +65653 2 +65653 2 +65653 3 +65654 1 +65654 1 +65654 1 +65654 1 +65654 1 +65654 1 +65654 2 +65654 2 +65654 2 +65655 1 +65655 1 +65656 1 +65656 1 +65656 1 +65656 1 +65656 2 +65656 3 +65657 1 +65657 2 +65657 2 +65657 2 +65657 2 +65658 1 +65658 1 +65658 1 +65658 1 +65658 1 +65658 2 +65658 2 +65658 2 +65658 2 +65658 2 +65659 1 +65659 1 +65659 1 +65659 1 +65659 1 +65659 2 +65659 2 +65659 3 +65660 1 +65660 1 +65660 2 +65661 1 +65661 2 +65661 2 +65661 2 +65661 3 +65662 1 +65662 1 +65662 2 +65662 2 +65662 2 +65662 2 +65663 2 +65663 2 +65663 2 +65663 3 +65664 1 +65664 1 +65664 1 +65664 1 +65664 1 +65664 2 +65664 2 +65664 2 +65665 2 +65666 1 +65666 1 +65666 2 +65667 1 +65667 1 +65667 1 +65667 1 +65668 3 +65669 1 +65669 1 +65669 1 +65669 1 +65669 1 +65669 3 +65670 1 +65670 1 +65670 2 +65670 2 +65670 3 +65671 2 +65671 2 +65671 3 +65672 1 +65672 1 +65672 1 +65672 2 +65672 2 +65673 2 +65673 2 +65673 3 +65673 4 +65673 4 +65674 1 +65675 1 +65675 1 +65675 2 +65675 2 +65675 2 +65675 2 +65675 3 +65675 3 +65676 1 +65676 1 +65676 2 +65676 3 +65677 1 +65677 1 +65677 1 +65677 2 +65677 3 +65677 4 +65678 1 +65678 1 +65678 1 +65679 1 +65679 2 +65680 1 +65680 1 +65680 2 +65680 2 +65680 2 +65680 5 +65681 1 +65681 2 +65681 2 +65681 2 +65681 4 +65681 4 +65682 1 +65682 2 +65682 2 +65683 1 +65683 1 +65683 2 +65684 1 +65684 1 +65684 2 +65685 2 +65685 2 +65685 2 +65685 3 +65685 3 +65685 4 +65686 1 +65686 2 +65686 3 +65686 3 +65687 1 +65687 1 +65687 2 +65687 3 +65688 1 +65688 2 +65689 1 +65689 3 +65690 2 +65690 2 +65691 1 +65691 1 +65691 1 +65691 1 +65691 1 +65691 2 +65691 3 +65691 3 +65692 1 +65692 2 +65693 1 +65693 2 +65693 2 +65693 2 +65693 2 +65693 3 +65693 5 +65694 1 +65694 2 +65694 2 +65694 3 +65695 1 +65695 2 +65695 2 +65695 3 +65696 1 +65696 2 +65696 2 +65696 4 +65697 1 +65697 1 +65697 1 +65697 1 +65697 2 +65697 5 +65698 1 +65698 2 +65699 1 +65699 1 +65699 1 +65699 2 +65699 2 +65699 2 +65700 1 +65700 2 +65700 2 +65701 1 +65701 2 +65701 3 +65702 2 +65702 2 +65702 2 +65702 3 +65703 1 +65703 2 +65703 3 +65703 3 +65704 1 +65704 1 +65704 2 +65704 2 +65704 3 +65704 3 +65705 1 +65705 1 +65705 3 +65705 4 +65706 1 +65706 1 +65706 1 +65706 3 +65706 4 +65707 2 +65707 2 +65708 2 +65708 3 +65709 1 +65709 1 +65709 2 +65710 1 +65710 2 +65711 1 +65711 2 +65711 2 +65711 2 +65711 2 +65711 2 +65712 1 +65712 2 +65712 3 +65712 3 +65712 4 +65713 1 +65713 2 +65713 3 +65713 3 +65713 6 +65714 1 +65714 1 +65714 2 +65715 1 +65715 1 +65715 2 +65715 2 +65715 4 +65716 1 +65716 2 +65716 2 +65716 2 +65716 4 +65716 4 +65717 1 +65717 2 +65717 2 +65717 2 +65717 5 +65718 1 +65718 2 +65718 3 +65718 3 +65719 1 +65719 1 +65719 2 +65719 3 +65720 1 +65720 2 +65720 2 +65720 3 +65720 4 +65721 1 +65721 1 +65721 1 +65721 1 +65721 3 +65721 3 +65721 3 +65722 2 +65722 3 +65722 5 +65723 2 +65723 3 +65724 1 +65724 1 +65724 2 +65724 2 +65724 3 +65724 3 +65725 1 +65726 2 +65726 2 +65726 4 +65727 1 +65727 1 +65727 3 +65727 3 +65727 4 +65728 1 +65728 2 +65729 2 +65730 1 +65730 1 +65730 2 +65730 4 +65731 1 +65731 1 +65731 1 +65731 3 +65732 1 +65732 2 +65732 2 +65733 1 +65733 1 +65733 2 +65733 3 +65733 3 +65733 6 +65734 2 +65734 2 +65735 1 +65735 4 +65735 4 +65736 1 +65736 2 +65736 2 +65736 3 +65737 1 +65737 1 +65737 2 +65737 3 +65737 4 +65738 3 +65738 3 +65738 4 +65738 4 +65739 1 +65739 2 +65739 3 +65740 2 +65740 2 +65740 3 +65741 1 +65742 1 +65742 2 +65742 3 +65743 1 +65743 1 +65743 1 +65743 2 +65744 1 +65744 2 +65745 2 +65745 2 +65745 3 +65745 6 +65746 1 +65746 2 +65746 2 +65746 2 +65746 2 +65747 1 +65747 1 +65747 1 +65747 2 +65747 2 +65747 3 +65747 3 +65748 1 +65748 3 +65749 2 +65749 3 +65749 3 +65749 3 +65749 4 +65750 1 +65750 1 +65750 2 +65750 3 +65750 3 +65751 1 +65751 2 +65751 2 +65751 3 +65751 3 +65751 4 +65752 1 +65752 3 +65753 2 +65753 3 +65754 2 +65754 4 +65755 1 +65755 2 +65755 2 +65755 3 +65755 3 +65755 3 +65755 3 +65755 5 +65755 5 +65756 2 +65756 3 +65756 3 +65756 5 +65757 1 +65757 1 +65757 1 +65757 2 +65757 3 +65758 1 +65758 2 +65758 2 +65758 3 +65758 4 +65759 2 +65759 2 +65759 2 +65759 4 +65759 4 +65760 2 +65760 4 +65760 5 +65760 7 +65761 1 +65762 1 +65762 1 +65762 2 +65762 4 +65762 5 +65763 1 +65763 2 +65763 2 +65763 2 +65763 4 +65764 3 +65764 3 +65764 4 +65765 2 +65765 2 +65765 3 +65766 1 +65766 1 +65766 3 +65766 3 +65766 3 +65767 2 +65767 3 +65767 3 +65768 3 +65769 1 +65769 2 +65769 2 +65769 2 +65769 3 +65769 5 +65770 2 +65770 3 +65771 2 +65771 3 +65771 3 +65772 2 +65773 1 +65773 2 +65773 2 +65773 2 +65773 2 +65773 3 +65773 3 +65773 4 +65774 2 +65774 2 +65774 2 +65774 2 +65774 3 +65774 3 +65775 1 +65775 2 +65775 3 +65775 3 +65775 5 +65776 1 +65776 3 +65776 3 +65776 5 +65776 6 +65777 2 +65777 3 +65777 4 +65777 4 +65778 1 +65778 2 +65778 2 +65778 2 +65778 3 +65778 3 +65778 4 +65778 5 +65779 3 +65779 3 +65779 3 +65779 3 +65779 4 +65779 4 +65779 5 +65780 1 +65780 2 +65780 3 +65781 1 +65781 3 +65781 3 +65782 2 +65782 2 +65782 3 +65782 4 +65782 5 +65783 2 +65783 3 +65783 3 +65783 3 +65783 3 +65783 3 +65784 2 +65784 2 +65784 2 +65784 4 +65784 6 +65785 2 +65785 7 +65786 1 +65786 2 +65786 3 +65786 3 +65786 4 +65787 1 +65787 2 +65787 3 +65787 3 +65788 1 +65788 1 +65788 2 +65788 4 +65789 1 +65789 1 +65789 2 +65789 2 +65789 2 +65789 2 +65789 4 +65790 2 +65790 2 +65790 4 +65791 2 +65791 2 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-1-a3d352560ac835993001665db6954965 b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-1-a3d352560ac835993001665db6954965 new file mode 100644 index 0000000000000000000000000000000000000000..dc72606a83db6f37694cdebbf744e2f0f1b91c8a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-1-a3d352560ac835993001665db6954965 @@ -0,0 +1,1049 @@ + 1 + 1 + 1 +alice allen 1 +alice allen 1 +alice allen 1 +alice brown 1 +alice carson 1 +alice davidson 1 +alice falkner 1 +alice garcia 1 +alice hernandez 1 +alice hernandez 1 +alice johnson 1 +alice king 1 +alice king 1 +alice king 1 +alice laertes 1 +alice laertes 2 +alice miller 1 +alice nixon 1 +alice nixon 1 +alice nixon 1 +alice ovid 2 +alice polk 1 +alice quirinius 1 +alice quirinius 1 +alice robinson 1 +alice robinson 1 +alice steinbeck 1 +alice steinbeck 1 +alice steinbeck 1 +alice underhill 1 +alice van buren 1 +alice xylophone 1 +alice xylophone 1 +alice xylophone 2 +alice zipper 1 +alice zipper 1 +alice zipper 2 +bob brown 1 +bob brown 1 +bob brown 1 +bob carson 1 +bob davidson 1 +bob davidson 1 +bob davidson 1 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob falkner 2 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 2 +bob garcia 2 +bob hernandez 1 +bob ichabod 1 +bob king 1 +bob king 1 +bob king 2 +bob laertes 1 +bob laertes 1 +bob miller 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob polk 1 +bob quirinius 1 +bob steinbeck 1 +bob van buren 1 +bob white 1 +bob white 1 +bob xylophone 1 +bob xylophone 1 +bob young 1 +bob zipper 1 +bob zipper 2 +bob zipper 2 +calvin allen 3 +calvin brown 1 +calvin brown 1 +calvin brown 1 +calvin carson 2 +calvin davidson 1 +calvin davidson 2 +calvin ellison 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 2 +calvin falkner 2 +calvin falkner 2 +calvin garcia 2 +calvin hernandez 3 +calvin johnson 1 +calvin laertes 1 +calvin laertes 1 +calvin nixon 1 +calvin nixon 1 +calvin nixon 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 2 +calvin ovid 2 +calvin polk 2 +calvin quirinius 1 +calvin quirinius 1 +calvin robinson 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin steinbeck 2 +calvin thompson 1 +calvin thompson 2 +calvin underhill 1 +calvin van buren 1 +calvin van buren 1 +calvin white 1 +calvin white 2 +calvin xylophone 1 +calvin xylophone 2 +calvin xylophone 2 +calvin young 1 +calvin young 2 +calvin zipper 3 +calvin zipper 4 +david allen 1 +david allen 1 +david brown 2 +david brown 3 +david davidson 1 +david davidson 2 +david davidson 3 +david davidson 3 +david ellison 1 +david ellison 2 +david ellison 3 +david hernandez 1 +david ichabod 1 +david ichabod 3 +david laertes 3 +david nixon 1 +david ovid 1 +david ovid 1 +david quirinius 1 +david quirinius 1 +david quirinius 3 +david robinson 1 +david robinson 4 +david thompson 1 +david underhill 1 +david underhill 2 +david underhill 3 +david van buren 1 +david van buren 2 +david white 1 +david xylophone 1 +david xylophone 1 +david xylophone 2 +david young 1 +david young 1 +ethan allen 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 2 +ethan brown 2 +ethan carson 1 +ethan ellison 1 +ethan ellison 2 +ethan falkner 1 +ethan falkner 1 +ethan garcia 1 +ethan hernandez 1 +ethan johnson 2 +ethan king 2 +ethan laertes 1 +ethan laertes 1 +ethan laertes 2 +ethan laertes 2 +ethan laertes 2 +ethan laertes 2 +ethan laertes 3 +ethan miller 1 +ethan nixon 2 +ethan ovid 2 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan polk 2 +ethan quirinius 1 +ethan quirinius 1 +ethan quirinius 2 +ethan robinson 1 +ethan robinson 2 +ethan underhill 2 +ethan van buren 1 +ethan white 1 +ethan white 2 +ethan xylophone 2 +ethan zipper 1 +ethan zipper 3 +fred davidson 2 +fred davidson 2 +fred davidson 3 +fred ellison 1 +fred ellison 2 +fred ellison 2 +fred falkner 1 +fred falkner 3 +fred falkner 4 +fred hernandez 2 +fred ichabod 2 +fred ichabod 3 +fred johnson 2 +fred king 2 +fred king 2 +fred laertes 2 +fred miller 3 +fred nixon 1 +fred nixon 1 +fred nixon 1 +fred nixon 3 +fred polk 1 +fred polk 1 +fred polk 1 +fred polk 2 +fred quirinius 2 +fred quirinius 3 +fred robinson 2 +fred steinbeck 1 +fred steinbeck 1 +fred steinbeck 1 +fred underhill 1 +fred van buren 2 +fred van buren 3 +fred van buren 3 +fred van buren 4 +fred white 2 +fred young 1 +fred young 2 +fred zipper 3 +gabriella allen 1 +gabriella allen 3 +gabriella brown 1 +gabriella brown 1 +gabriella carson 2 +gabriella davidson 1 +gabriella ellison 1 +gabriella ellison 3 +gabriella falkner 1 +gabriella falkner 1 +gabriella falkner 3 +gabriella garcia 1 +gabriella hernandez 1 +gabriella hernandez 1 +gabriella ichabod 1 +gabriella ichabod 2 +gabriella ichabod 2 +gabriella ichabod 2 +gabriella ichabod 4 +gabriella king 1 +gabriella king 2 +gabriella laertes 1 +gabriella miller 2 +gabriella ovid 1 +gabriella ovid 2 +gabriella polk 1 +gabriella polk 2 +gabriella steinbeck 1 +gabriella steinbeck 1 +gabriella thompson 1 +gabriella thompson 2 +gabriella thompson 3 +gabriella van buren 1 +gabriella van buren 2 +gabriella white 1 +gabriella young 1 +gabriella young 2 +gabriella zipper 1 +gabriella zipper 2 +holly allen 3 +holly brown 2 +holly brown 2 +holly falkner 2 +holly hernandez 2 +holly hernandez 2 +holly hernandez 2 +holly hernandez 3 +holly ichabod 1 +holly ichabod 2 +holly ichabod 2 +holly johnson 1 +holly johnson 3 +holly johnson 4 +holly king 2 +holly king 2 +holly laertes 3 +holly miller 2 +holly nixon 1 +holly nixon 2 +holly polk 1 +holly polk 2 +holly robinson 3 +holly thompson 1 +holly thompson 3 +holly thompson 4 +holly underhill 2 +holly underhill 2 +holly underhill 3 +holly underhill 3 +holly van buren 1 +holly white 4 +holly white 4 +holly xylophone 2 +holly young 1 +holly young 2 +holly zipper 1 +holly zipper 4 +irene allen 3 +irene brown 1 +irene brown 2 +irene brown 3 +irene carson 2 +irene ellison 2 +irene ellison 2 +irene falkner 1 +irene falkner 1 +irene garcia 1 +irene garcia 2 +irene garcia 3 +irene ichabod 1 +irene ichabod 2 +irene johnson 2 +irene laertes 1 +irene laertes 3 +irene laertes 4 +irene miller 1 +irene nixon 1 +irene nixon 3 +irene nixon 3 +irene ovid 2 +irene ovid 2 +irene ovid 2 +irene polk 1 +irene polk 1 +irene polk 2 +irene polk 2 +irene polk 4 +irene quirinius 2 +irene quirinius 3 +irene quirinius 4 +irene robinson 2 +irene steinbeck 1 +irene thompson 1 +irene underhill 2 +irene underhill 3 +irene van buren 2 +irene van buren 3 +irene xylophone 2 +jessica brown 2 +jessica carson 1 +jessica carson 2 +jessica carson 4 +jessica davidson 1 +jessica davidson 2 +jessica davidson 3 +jessica davidson 3 +jessica ellison 1 +jessica ellison 3 +jessica falkner 2 +jessica garcia 1 +jessica garcia 5 +jessica ichabod 2 +jessica johnson 2 +jessica johnson 3 +jessica miller 2 +jessica nixon 2 +jessica nixon 3 +jessica ovid 2 +jessica ovid 3 +jessica polk 5 +jessica quirinius 2 +jessica quirinius 2 +jessica quirinius 3 +jessica quirinius 3 +jessica robinson 1 +jessica thompson 2 +jessica thompson 3 +jessica underhill 2 +jessica underhill 2 +jessica underhill 4 +jessica van buren 2 +jessica white 3 +jessica white 3 +jessica white 3 +jessica white 3 +jessica white 4 +jessica xylophone 4 +jessica young 4 +jessica young 4 +jessica zipper 1 +jessica zipper 2 +jessica zipper 4 +katie allen 2 +katie brown 4 +katie davidson 3 +katie ellison 3 +katie ellison 3 +katie falkner 2 +katie garcia 2 +katie garcia 3 +katie hernandez 2 +katie ichabod 2 +katie ichabod 2 +katie ichabod 2 +katie king 1 +katie king 1 +katie king 2 +katie miller 2 +katie miller 3 +katie nixon 5 +katie ovid 1 +katie polk 2 +katie polk 3 +katie robinson 4 +katie van buren 2 +katie van buren 4 +katie white 1 +katie white 2 +katie xylophone 3 +katie young 2 +katie young 2 +katie young 3 +katie zipper 1 +katie zipper 3 +luke allen 2 +luke allen 2 +luke allen 2 +luke allen 3 +luke allen 3 +luke brown 2 +luke davidson 1 +luke davidson 3 +luke ellison 3 +luke ellison 5 +luke ellison 5 +luke falkner 2 +luke falkner 4 +luke garcia 1 +luke garcia 5 +luke ichabod 3 +luke ichabod 3 +luke johnson 1 +luke johnson 2 +luke johnson 3 +luke laertes 2 +luke laertes 3 +luke laertes 3 +luke laertes 3 +luke laertes 3 +luke miller 2 +luke ovid 1 +luke ovid 2 +luke polk 2 +luke polk 3 +luke quirinius 2 +luke robinson 1 +luke robinson 4 +luke thompson 1 +luke underhill 2 +luke underhill 3 +luke underhill 5 +luke van buren 2 +luke white 3 +luke xylophone 2 +luke zipper 1 +mike allen 4 +mike brown 4 +mike carson 1 +mike carson 2 +mike carson 4 +mike davidson 3 +mike davidson 4 +mike ellison 2 +mike ellison 3 +mike ellison 3 +mike ellison 4 +mike ellison 4 +mike falkner 1 +mike garcia 1 +mike garcia 2 +mike garcia 3 +mike hernandez 2 +mike hernandez 3 +mike ichabod 1 +mike king 1 +mike king 1 +mike king 3 +mike king 3 +mike king 4 +mike king 4 +mike miller 4 +mike nixon 3 +mike nixon 4 +mike polk 4 +mike polk 5 +mike polk 5 +mike quirinius 3 +mike steinbeck 2 +mike steinbeck 3 +mike steinbeck 3 +mike steinbeck 4 +mike van buren 2 +mike van buren 3 +mike white 3 +mike white 4 +mike white 5 +mike white 6 +mike young 2 +mike young 2 +mike young 4 +mike zipper 1 +mike zipper 4 +mike zipper 6 +nick allen 2 +nick allen 2 +nick brown 5 +nick davidson 1 +nick ellison 3 +nick ellison 4 +nick falkner 2 +nick falkner 3 +nick garcia 2 +nick garcia 4 +nick garcia 4 +nick ichabod 3 +nick ichabod 3 +nick ichabod 3 +nick johnson 4 +nick johnson 4 +nick laertes 2 +nick miller 3 +nick nixon 2 +nick ovid 3 +nick polk 4 +nick quirinius 1 +nick quirinius 3 +nick robinson 4 +nick robinson 4 +nick steinbeck 1 +nick thompson 2 +nick underhill 6 +nick van buren 2 +nick xylophone 3 +nick young 3 +nick young 5 +nick zipper 3 +nick zipper 4 +oscar allen 4 +oscar brown 3 +oscar carson 3 +oscar carson 4 +oscar carson 4 +oscar carson 5 +oscar carson 5 +oscar davidson 4 +oscar ellison 1 +oscar ellison 2 +oscar falkner 1 +oscar garcia 5 +oscar hernandez 1 +oscar hernandez 2 +oscar ichabod 2 +oscar ichabod 5 +oscar ichabod 5 +oscar ichabod 6 +oscar johnson 3 +oscar johnson 7 +oscar king 3 +oscar king 4 +oscar king 4 +oscar laertes 1 +oscar laertes 2 +oscar laertes 3 +oscar laertes 3 +oscar nixon 4 +oscar ovid 3 +oscar ovid 3 +oscar ovid 6 +oscar polk 5 +oscar polk 5 +oscar quirinius 2 +oscar quirinius 4 +oscar quirinius 5 +oscar quirinius 6 +oscar robinson 2 +oscar robinson 3 +oscar robinson 5 +oscar robinson 6 +oscar steinbeck 1 +oscar thompson 2 +oscar thompson 3 +oscar thompson 3 +oscar thompson 4 +oscar underhill 2 +oscar van buren 1 +oscar van buren 2 +oscar van buren 5 +oscar white 1 +oscar white 2 +oscar white 5 +oscar white 5 +oscar xylophone 3 +oscar xylophone 3 +oscar xylophone 4 +oscar zipper 2 +oscar zipper 2 +oscar zipper 2 +priscilla brown 2 +priscilla brown 2 +priscilla brown 4 +priscilla carson 3 +priscilla carson 5 +priscilla carson 7 +priscilla ichabod 1 +priscilla ichabod 4 +priscilla johnson 1 +priscilla johnson 2 +priscilla johnson 4 +priscilla johnson 4 +priscilla johnson 6 +priscilla king 3 +priscilla nixon 3 +priscilla nixon 6 +priscilla ovid 3 +priscilla ovid 7 +priscilla polk 4 +priscilla quirinius 3 +priscilla thompson 6 +priscilla underhill 1 +priscilla underhill 4 +priscilla van buren 3 +priscilla van buren 5 +priscilla van buren 5 +priscilla white 1 +priscilla xylophone 2 +priscilla xylophone 3 +priscilla xylophone 3 +priscilla young 5 +priscilla young 8 +priscilla zipper 3 +priscilla zipper 3 +quinn allen 1 +quinn allen 4 +quinn brown 3 +quinn brown 4 +quinn brown 4 +quinn davidson 2 +quinn davidson 4 +quinn davidson 6 +quinn davidson 7 +quinn ellison 3 +quinn ellison 8 +quinn garcia 2 +quinn garcia 3 +quinn garcia 3 +quinn garcia 5 +quinn ichabod 7 +quinn king 1 +quinn king 1 +quinn laertes 2 +quinn laertes 4 +quinn laertes 5 +quinn nixon 4 +quinn ovid 4 +quinn quirinius 5 +quinn robinson 3 +quinn steinbeck 4 +quinn steinbeck 5 +quinn thompson 4 +quinn thompson 6 +quinn underhill 2 +quinn underhill 3 +quinn underhill 7 +quinn van buren 1 +quinn young 2 +quinn zipper 3 +quinn zipper 4 +rachel allen 2 +rachel allen 3 +rachel brown 2 +rachel brown 3 +rachel brown 4 +rachel brown 4 +rachel brown 5 +rachel carson 2 +rachel carson 4 +rachel davidson 7 +rachel ellison 1 +rachel falkner 1 +rachel falkner 3 +rachel falkner 5 +rachel falkner 6 +rachel johnson 9 +rachel king 3 +rachel king 7 +rachel laertes 4 +rachel laertes 6 +rachel ovid 3 +rachel ovid 4 +rachel polk 3 +rachel quirinius 4 +rachel robinson 4 +rachel robinson 4 +rachel robinson 6 +rachel thompson 4 +rachel thompson 5 +rachel thompson 5 +rachel underhill 2 +rachel white 4 +rachel white 5 +rachel young 4 +rachel zipper 1 +rachel zipper 5 +sarah carson 1 +sarah carson 4 +sarah carson 7 +sarah ellison 1 +sarah falkner 4 +sarah falkner 5 +sarah garcia 2 +sarah garcia 2 +sarah garcia 4 +sarah ichabod 3 +sarah ichabod 3 +sarah johnson 3 +sarah johnson 5 +sarah johnson 5 +sarah johnson 6 +sarah king 3 +sarah king 5 +sarah miller 2 +sarah ovid 5 +sarah robinson 5 +sarah robinson 5 +sarah steinbeck 5 +sarah white 4 +sarah white 6 +sarah xylophone 3 +sarah young 5 +sarah zipper 6 +tom brown 2 +tom brown 5 +tom carson 1 +tom carson 3 +tom carson 5 +tom davidson 2 +tom ellison 3 +tom ellison 4 +tom ellison 6 +tom falkner 3 +tom falkner 4 +tom hernandez 1 +tom hernandez 3 +tom ichabod 4 +tom johnson 6 +tom johnson 7 +tom king 3 +tom laertes 3 +tom laertes 3 +tom miller 3 +tom miller 4 +tom miller 5 +tom nixon 4 +tom ovid 3 +tom polk 2 +tom polk 2 +tom quirinius 3 +tom quirinius 5 +tom robinson 2 +tom robinson 3 +tom robinson 3 +tom robinson 5 +tom steinbeck 2 +tom van buren 2 +tom van buren 3 +tom van buren 6 +tom white 5 +tom young 1 +tom young 5 +tom zipper 7 +ulysses brown 2 +ulysses carson 2 +ulysses carson 5 +ulysses carson 6 +ulysses carson 8 +ulysses davidson 3 +ulysses ellison 4 +ulysses garcia 3 +ulysses hernandez 3 +ulysses hernandez 3 +ulysses hernandez 4 +ulysses ichabod 1 +ulysses ichabod 3 +ulysses johnson 5 +ulysses king 2 +ulysses laertes 2 +ulysses laertes 5 +ulysses laertes 6 +ulysses miller 5 +ulysses miller 7 +ulysses nixon 4 +ulysses ovid 4 +ulysses polk 2 +ulysses polk 2 +ulysses polk 3 +ulysses polk 4 +ulysses quirinius 4 +ulysses robinson 1 +ulysses steinbeck 2 +ulysses steinbeck 5 +ulysses thompson 3 +ulysses underhill 2 +ulysses underhill 2 +ulysses underhill 3 +ulysses underhill 4 +ulysses underhill 4 +ulysses underhill 4 +ulysses underhill 5 +ulysses van buren 2 +ulysses white 6 +ulysses white 7 +ulysses xylophone 2 +ulysses xylophone 3 +ulysses xylophone 6 +ulysses young 1 +ulysses young 4 +ulysses young 7 +victor allen 2 +victor allen 3 +victor brown 1 +victor brown 4 +victor brown 5 +victor brown 7 +victor davidson 4 +victor davidson 4 +victor davidson 6 +victor ellison 4 +victor ellison 4 +victor hernandez 2 +victor hernandez 3 +victor hernandez 4 +victor hernandez 4 +victor hernandez 6 +victor johnson 4 +victor johnson 5 +victor johnson 6 +victor king 2 +victor king 6 +victor laertes 3 +victor laertes 5 +victor miller 5 +victor nixon 2 +victor nixon 3 +victor ovid 3 +victor polk 4 +victor quirinius 5 +victor quirinius 5 +victor robinson 5 +victor robinson 5 +victor steinbeck 3 +victor steinbeck 4 +victor steinbeck 5 +victor thompson 6 +victor van buren 5 +victor van buren 6 +victor white 2 +victor white 7 +victor xylophone 4 +victor xylophone 6 +victor xylophone 6 +victor xylophone 8 +victor xylophone 8 +victor young 5 +victor zipper 3 +wendy allen 5 +wendy allen 6 +wendy allen 6 +wendy brown 3 +wendy brown 5 +wendy ellison 3 +wendy ellison 5 +wendy falkner 2 +wendy falkner 4 +wendy falkner 6 +wendy garcia 4 +wendy garcia 4 +wendy garcia 7 +wendy garcia 7 +wendy hernandez 4 +wendy ichabod 4 +wendy king 4 +wendy king 5 +wendy king 7 +wendy laertes 2 +wendy laertes 3 +wendy laertes 5 +wendy miller 4 +wendy miller 4 +wendy nixon 3 +wendy nixon 5 +wendy ovid 5 +wendy ovid 9 +wendy polk 2 +wendy polk 5 +wendy quirinius 3 +wendy quirinius 4 +wendy robinson 5 +wendy robinson 6 +wendy robinson 6 +wendy steinbeck 3 +wendy thompson 4 +wendy thompson 5 +wendy underhill 4 +wendy underhill 5 +wendy underhill 6 +wendy van buren 6 +wendy van buren 6 +wendy white 4 +wendy xylophone 4 +wendy xylophone 6 +wendy young 1 +wendy young 6 +xavier allen 3 +xavier allen 4 +xavier allen 5 +xavier brown 2 +xavier brown 4 +xavier brown 6 +xavier carson 4 +xavier carson 5 +xavier davidson 7 +xavier davidson 8 +xavier davidson 9 +xavier ellison 8 +xavier ellison 8 +xavier garcia 5 +xavier hernandez 5 +xavier hernandez 6 +xavier hernandez 9 +xavier ichabod 3 +xavier ichabod 4 +xavier johnson 2 +xavier johnson 9 +xavier king 3 +xavier king 5 +xavier laertes 4 +xavier ovid 4 +xavier polk 3 +xavier polk 4 +xavier polk 4 +xavier polk 8 +xavier quirinius 3 +xavier quirinius 5 +xavier quirinius 6 +xavier quirinius 6 +xavier thompson 4 +xavier underhill 2 +xavier white 3 +xavier white 3 +xavier xylophone 4 +xavier zipper 3 +yuri allen 2 +yuri allen 3 +yuri brown 2 +yuri brown 3 +yuri carson 5 +yuri carson 6 +yuri ellison 6 +yuri ellison 6 +yuri falkner 6 +yuri falkner 10 +yuri garcia 1 +yuri hernandez 5 +yuri johnson 5 +yuri johnson 5 +yuri johnson 6 +yuri king 7 +yuri laertes 7 +yuri laertes 8 +yuri nixon 3 +yuri nixon 3 +yuri polk 3 +yuri polk 5 +yuri polk 6 +yuri quirinius 3 +yuri quirinius 4 +yuri quirinius 7 +yuri steinbeck 1 +yuri steinbeck 2 +yuri thompson 3 +yuri underhill 4 +yuri underhill 4 +yuri white 8 +yuri xylophone 3 +zach allen 4 +zach brown 5 +zach brown 5 +zach brown 5 +zach brown 5 +zach brown 7 +zach carson 5 +zach ellison 2 +zach falkner 4 +zach falkner 6 +zach garcia 4 +zach garcia 5 +zach garcia 7 +zach garcia 8 +zach ichabod 4 +zach ichabod 4 +zach king 4 +zach king 5 +zach king 8 +zach miller 1 +zach miller 3 +zach miller 4 +zach ovid 4 +zach ovid 5 +zach ovid 5 +zach ovid 7 +zach quirinius 8 +zach robinson 5 +zach steinbeck 4 +zach steinbeck 6 +zach thompson 3 +zach thompson 4 +zach underhill 3 +zach white 6 +zach xylophone 3 +zach xylophone 5 +zach young 4 +zach zipper 4 +zach zipper 4 +zach zipper 5 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-2-fafa16c0f7697ca28aeb6f2698799562 b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-2-fafa16c0f7697ca28aeb6f2698799562 new file mode 100644 index 0000000000000000000000000000000000000000..76cbeb254c0e96772e1091ee07c43b0727e3de40 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-2-fafa16c0f7697ca28aeb6f2698799562 @@ -0,0 +1,1049 @@ +0.08 1 +0.1 1 +0.13 1 +0.15 1 +0.27 1 +0.28 1 +0.43 1 +0.52 1 +0.56 1 +0.6 1 +0.61 1 +0.79 1 +0.84 1 +0.98 1 +1.02 1 +1.08 1 +1.08 1 +1.12 1 +1.21 1 +1.25 1 +1.27 1 +1.29 1 +1.31 1 +1.58 1 +1.87 1 +1.91 1 +1.92 1 +2.07 1 +2.18 1 +2.2 1 +2.35 1 +2.6 1 +2.79 1 +2.92 1 +2.96 1 +2.96 1 +2.97 1 +3.0 1 +3.21 1 +3.28 1 +3.33 1 +3.61 1 +3.62 1 +3.82 1 +3.86 1 +3.96 1 +3.97 1 +4.17 1 +4.32 1 +4.35 1 +4.41 1 +4.46 1 +4.47 1 +4.57 1 +4.59 1 +4.71 1 +4.72 1 +4.79 1 +4.8 1 +4.92 1 +5.08 1 +5.24 1 +5.28 1 +5.4 1 +5.44 1 +5.45 1 +5.51 1 +5.54 1 +5.62 1 +5.67 1 +5.85 1 +5.88 1 +6.29 1 +6.55 1 +6.57 1 +6.63 1 +6.67 1 +6.72 1 +6.74 1 +6.84 1 +6.87 1 +7.05 1 +7.06 1 +7.11 1 +7.54 1 +7.56 1 +7.79 1 +7.82 1 +7.96 1 +7.96 1 +7.98 1 +8.07 1 +8.07 1 +8.32 1 +8.37 1 +8.42 1 +8.45 1 +8.45 1 +8.45 1 +8.45 1 +8.57 1 +8.61 1 +8.67 2 +8.71 1 +8.79 1 +8.91 1 +9.04 1 +9.13 1 +9.19 1 +9.22 1 +9.25 1 +9.26 1 +9.35 1 +9.48 1 +9.56 1 +9.57 1 +9.57 1 +9.68 1 +9.7 1 +9.71 1 +9.74 1 +9.8 1 +9.81 1 +9.93 1 +10.09 1 +10.09 1 +10.13 1 +10.16 1 +10.17 1 +10.19 1 +10.2 1 +10.22 1 +10.25 1 +10.26 1 +10.29 1 +10.6 1 +10.66 1 +10.67 1 +10.73 1 +11.15 1 +11.18 1 +11.19 1 +11.22 1 +11.34 1 +11.55 1 +11.57 1 +11.68 1 +11.82 1 +11.89 1 +11.91 1 +12.02 1 +12.16 1 +12.19 1 +12.32 1 +12.42 1 +12.44 1 +12.45 1 +12.46 1 +12.5 1 +12.54 1 +12.85 1 +12.9 1 +13.01 1 +13.1 1 +13.15 1 +13.35 1 +13.87 1 +13.89 1 +13.94 1 +13.99 1 +14.13 1 +14.21 2 +14.3 1 +14.44 1 +14.84 2 +14.92 1 +14.92 1 +14.93 1 +15.1 1 +15.15 1 +15.18 1 +15.22 1 +15.26 2 +15.3 1 +15.37 1 +15.45 1 +15.63 2 +15.75 1 +15.81 1 +15.86 1 +15.9 1 +15.92 1 +16.08 1 +16.09 1 +16.24 1 +16.25 1 +16.48 1 +16.69 1 +16.99 1 +16.99 1 +17.16 1 +17.37 1 +17.74 1 +17.79 1 +17.87 1 +18.2 1 +18.5 1 +18.56 1 +18.63 1 +18.63 1 +18.86 1 +18.89 1 +18.93 1 +19.0 1 +19.03 1 +19.06 1 +19.06 1 +19.13 1 +19.14 1 +19.28 1 +19.69 1 +20.07 1 +20.38 1 +20.64 1 +20.67 1 +20.79 1 +20.81 1 +20.82 1 +20.82 1 +21.18 1 +21.19 1 +21.23 1 +21.28 1 +21.32 1 +21.45 1 +21.49 1 +21.61 1 +21.7 1 +21.8 1 +21.94 1 +22.01 1 +22.08 1 +22.12 1 +22.12 1 +22.25 1 +22.27 1 +22.36 1 +22.68 1 +22.78 1 +22.85 1 +22.85 1 +22.94 1 +23.07 1 +23.13 1 +23.17 1 +23.19 1 +23.44 1 +23.45 1 +23.6 1 +23.77 1 +23.96 1 +24.02 1 +24.28 1 +24.49 1 +24.52 1 +24.73 1 +24.79 1 +24.8 1 +24.83 1 +24.86 1 +25.11 1 +25.28 1 +25.37 1 +25.42 1 +25.55 1 +25.67 1 +25.88 1 +26.08 1 +26.39 1 +26.43 1 +26.47 1 +26.49 1 +26.49 1 +26.64 1 +26.71 2 +26.73 1 +26.76 1 +27.07 1 +27.12 1 +27.3 1 +27.31 1 +27.63 1 +27.66 1 +27.72 2 +27.87 1 +28.11 1 +28.31 1 +28.45 1 +28.5 1 +28.56 1 +28.69 1 +28.71 1 +28.79 1 +28.89 1 +28.95 1 +29.02 1 +29.24 1 +29.36 1 +29.4 1 +29.41 2 +29.54 1 +29.59 1 +29.78 1 +30.25 2 +30.36 1 +30.37 1 +30.61 1 +30.62 1 +30.63 1 +30.65 1 +30.71 1 +30.81 1 +31.01 1 +31.15 1 +31.4 1 +31.61 1 +31.67 1 +31.77 1 +31.86 1 +31.91 1 +32.01 1 +32.18 1 +32.2 1 +32.23 1 +32.25 1 +32.37 1 +32.41 1 +32.47 1 +32.52 1 +32.75 1 +32.89 2 +32.92 1 +33.36 1 +33.52 1 +33.55 1 +33.58 1 +33.67 1 +33.76 1 +33.83 1 +33.85 2 +33.87 1 +34.03 1 +34.21 1 +34.35 1 +34.41 2 +34.58 1 +34.73 1 +34.97 1 +35.0 2 +35.08 1 +35.13 1 +35.17 1 +35.17 1 +35.56 1 +35.62 1 +35.65 1 +35.68 1 +35.72 1 +35.8 1 +35.89 1 +36.22 1 +36.26 1 +36.58 1 +36.7 1 +36.79 1 +36.89 1 +36.95 1 +37.07 2 +37.1 1 +37.14 1 +37.14 1 +37.24 1 +37.59 1 +37.6 1 +37.72 2 +37.78 1 +37.8 1 +37.85 1 +37.9 1 +38.05 1 +38.05 1 +38.3 2 +38.33 1 +38.57 1 +38.62 1 +38.79 1 +38.85 1 +38.88 1 +38.94 1 +39.01 1 +39.03 1 +39.18 1 +39.21 1 +39.34 1 +39.69 1 +39.81 1 +39.82 1 +39.83 1 +39.87 1 +39.9 1 +39.98 1 +40.0 1 +40.04 1 +40.17 1 +40.24 1 +40.42 1 +40.44 1 +40.78 1 +40.8 1 +40.98 1 +41.2 2 +41.29 1 +41.29 1 +41.31 2 +41.34 1 +41.34 1 +41.36 1 +41.44 1 +41.45 2 +41.62 1 +41.68 1 +41.71 1 +41.81 1 +41.85 1 +41.87 1 +41.89 2 +42.24 1 +42.31 1 +42.42 2 +42.48 1 +42.51 1 +42.55 1 +42.56 1 +42.67 2 +42.76 1 +42.85 1 +43.01 1 +43.02 1 +43.13 1 +43.16 1 +43.17 1 +43.19 1 +43.31 1 +43.37 1 +43.57 1 +43.71 1 +43.73 1 +43.92 1 +44.1 1 +44.22 1 +44.27 1 +44.43 1 +44.57 1 +45.06 2 +45.1 1 +45.19 1 +45.19 1 +45.24 1 +45.34 1 +45.35 1 +45.42 1 +45.45 1 +45.56 1 +45.59 1 +45.68 1 +45.92 1 +45.99 2 +46.02 1 +46.09 1 +46.1 1 +46.15 1 +46.18 1 +46.21 1 +46.27 1 +46.43 1 +46.45 1 +46.62 1 +46.8 1 +46.86 1 +46.87 1 +46.88 1 +46.97 2 +47.08 2 +47.27 1 +47.32 2 +47.57 1 +47.59 1 +47.69 1 +47.88 1 +48.01 1 +48.08 1 +48.11 1 +48.15 1 +48.22 1 +48.23 1 +48.25 1 +48.28 1 +48.37 1 +48.45 1 +48.45 1 +48.52 1 +48.59 1 +49.12 1 +49.28 2 +49.44 1 +49.68 1 +49.77 2 +49.78 1 +50.02 3 +50.08 1 +50.09 1 +50.26 1 +50.28 1 +50.31 1 +50.32 1 +50.4 1 +50.41 1 +50.66 1 +50.7 1 +50.83 2 +50.92 2 +50.96 1 +51.25 1 +51.29 1 +51.29 2 +51.72 1 +51.79 2 +51.84 1 +51.85 1 +52.17 1 +52.23 1 +52.44 1 +52.5 1 +52.53 2 +52.72 1 +52.73 1 +52.85 2 +52.87 1 +53.02 1 +53.06 1 +53.18 2 +53.27 1 +53.59 2 +53.78 1 +53.93 1 +53.94 1 +54.1 2 +54.31 1 +54.34 1 +54.43 1 +54.44 1 +54.47 1 +54.73 1 +54.75 1 +54.83 1 +54.99 1 +55.1 1 +55.18 1 +55.2 1 +55.39 1 +55.51 1 +55.63 1 +55.99 1 +56.04 1 +56.07 1 +56.1 1 +56.15 1 +56.33 1 +56.62 1 +56.68 2 +56.81 1 +57.08 1 +57.11 1 +57.12 2 +57.23 1 +57.25 1 +57.29 1 +57.35 1 +57.37 1 +57.46 1 +57.64 1 +57.67 1 +57.89 2 +57.93 1 +58.0 1 +58.08 2 +58.09 2 +58.13 1 +58.43 1 +58.52 1 +58.66 1 +58.67 1 +58.75 1 +58.86 2 +59.07 1 +59.16 1 +59.21 1 +59.34 1 +59.43 1 +59.45 1 +59.45 2 +59.5 1 +59.55 1 +59.61 1 +59.62 1 +59.68 1 +59.68 1 +59.7 1 +59.71 1 +59.83 1 +59.87 1 +59.99 1 +60.02 1 +60.06 1 +60.12 1 +60.13 1 +60.22 2 +60.26 1 +60.26 1 +60.53 1 +60.6 2 +60.71 1 +60.85 1 +61.21 1 +61.7 1 +61.86 1 +61.88 1 +61.92 2 +61.94 1 +62.14 1 +62.2 1 +62.23 1 +62.3 1 +62.39 1 +62.52 1 +62.72 1 +62.74 1 +62.85 2 +62.9 3 +62.92 1 +63.12 1 +63.33 1 +63.35 2 +63.42 1 +63.51 1 +63.9 1 +64.0 1 +64.22 1 +64.25 1 +64.3 1 +64.36 2 +64.46 1 +64.65 2 +64.67 1 +64.77 1 +64.87 1 +64.95 1 +65.02 1 +65.02 1 +65.38 1 +65.43 1 +65.43 1 +65.44 2 +65.55 1 +65.62 1 +65.7 1 +65.72 1 +66.17 1 +66.17 2 +66.36 1 +66.51 1 +66.61 1 +66.61 1 +66.67 1 +66.89 1 +67.12 1 +67.18 1 +67.26 1 +67.38 1 +67.45 1 +67.48 1 +67.59 1 +67.94 1 +67.98 1 +68.01 2 +68.04 1 +68.22 1 +68.25 1 +68.25 1 +68.32 1 +68.41 1 +68.5 1 +68.81 1 +68.85 2 +68.89 1 +68.95 1 +68.96 1 +69.32 2 +69.53 1 +69.74 3 +69.8 2 +69.88 1 +69.96 1 +69.97 1 +70.0 2 +70.04 1 +70.06 1 +70.24 1 +70.35 1 +70.38 1 +70.39 1 +70.52 1 +70.53 1 +70.56 1 +70.85 1 +70.89 1 +70.93 1 +71.01 1 +71.07 2 +71.13 1 +71.19 1 +71.26 1 +71.31 1 +71.32 1 +71.35 1 +71.5 1 +71.54 1 +71.55 3 +71.68 1 +71.68 2 +71.78 2 +71.8 1 +71.89 2 +72.04 1 +72.18 1 +72.51 1 +72.53 2 +72.56 1 +72.62 1 +72.79 1 +72.98 1 +73.18 1 +73.32 1 +73.48 2 +73.63 2 +73.65 1 +73.68 1 +73.88 1 +73.93 1 +74.0 1 +74.02 1 +74.15 1 +74.19 1 +74.19 1 +74.3 1 +74.42 1 +74.45 1 +74.52 1 +74.53 1 +74.59 1 +74.62 1 +74.72 1 +74.78 1 +75.03 2 +75.1 3 +75.19 1 +75.29 1 +75.35 2 +75.42 2 +75.66 2 +75.73 2 +75.83 1 +75.88 1 +76.05 3 +76.1 1 +76.28 1 +76.28 1 +76.33 1 +76.52 1 +76.69 1 +76.7 1 +76.71 1 +76.72 2 +76.72 2 +76.74 1 +76.92 1 +76.93 1 +77.02 3 +77.1 1 +77.36 2 +77.42 1 +77.57 2 +77.66 1 +77.81 1 +77.84 2 +77.89 1 +77.97 1 +78.21 2 +78.26 1 +78.28 1 +78.3 2 +78.31 1 +78.62 1 +78.64 1 +78.73 1 +78.89 1 +78.98 2 +79.12 1 +79.19 1 +79.21 1 +79.38 1 +79.42 1 +79.48 1 +79.48 2 +79.49 2 +79.54 1 +79.55 2 +79.75 1 +79.83 1 +79.96 1 +79.97 2 +79.99 1 +80.23 2 +80.3 1 +80.3 2 +80.46 1 +80.52 1 +80.58 1 +80.6 1 +80.71 1 +80.74 1 +80.84 2 +80.92 1 +80.96 1 +80.97 1 +80.99 1 +81.17 1 +81.32 1 +81.32 1 +81.47 1 +81.58 1 +81.64 1 +81.66 1 +82.24 1 +82.3 1 +82.34 2 +82.41 1 +82.52 1 +82.55 1 +82.56 1 +82.72 1 +82.97 1 +83.08 1 +83.27 1 +83.33 1 +83.4 1 +83.54 2 +83.57 1 +83.58 1 +83.87 1 +83.92 1 +83.93 1 +84.03 1 +84.23 2 +84.31 1 +84.38 1 +84.4 1 +84.69 1 +84.72 1 +84.83 1 +85.0 1 +85.03 2 +85.1 1 +85.14 1 +85.23 1 +85.49 1 +85.49 2 +85.51 2 +85.74 1 +85.76 1 +85.87 1 +85.9 1 +86.0 1 +86.22 1 +86.23 1 +86.63 1 +86.69 1 +86.92 2 +86.93 1 +86.93 1 +87.14 2 +87.22 1 +87.4 1 +87.48 1 +87.57 1 +87.61 1 +87.67 1 +87.83 2 +87.94 1 +87.99 1 +88.02 1 +88.05 1 +88.07 2 +88.17 1 +88.22 1 +88.36 1 +88.47 1 +88.48 1 +88.55 1 +88.77 1 +88.78 1 +88.8 1 +88.91 2 +89.01 2 +89.03 1 +89.1 3 +89.15 2 +89.28 1 +89.38 1 +89.53 1 +89.55 1 +89.55 1 +89.55 1 +89.8 1 +89.81 1 +89.93 1 +90.05 1 +90.05 1 +90.07 1 +90.12 1 +90.2 1 +90.25 1 +90.28 2 +90.35 1 +90.38 1 +90.51 1 +90.56 2 +90.69 1 +90.69 1 +90.73 1 +90.77 1 +91.05 1 +91.16 1 +91.42 1 +91.48 1 +91.53 1 +91.61 1 +91.63 1 +91.78 1 +91.88 1 +91.97 1 +92.05 2 +92.11 2 +92.33 2 +92.37 1 +92.4 2 +92.55 1 +92.61 1 +92.82 1 +92.96 2 +92.98 1 +93.03 1 +93.09 1 +93.11 1 +93.61 1 +93.64 1 +93.73 1 +94.08 1 +94.15 1 +94.25 1 +94.27 1 +94.31 1 +94.33 1 +94.34 1 +94.38 1 +94.43 1 +94.54 1 +94.66 1 +94.68 1 +94.68 1 +94.72 1 +95.07 1 +95.11 1 +95.28 1 +95.33 1 +95.34 1 +95.38 2 +95.48 1 +95.53 1 +95.53 1 +95.81 1 +95.81 2 +95.84 1 +96.09 1 +96.23 1 +96.25 1 +96.29 1 +96.38 1 +96.62 1 +96.68 1 +96.73 1 +96.78 1 +96.91 2 +96.94 1 +97.09 1 +97.24 1 +97.26 1 +97.39 1 +97.46 1 +97.51 2 +97.56 1 +97.57 1 +97.65 2 +97.68 1 +97.71 4 +97.81 1 +97.83 1 +97.87 1 +98.18 2 +98.22 1 +98.23 1 +98.31 1 +98.48 1 +98.51 1 +98.57 1 +98.72 1 +98.96 1 +99.13 1 +99.15 1 +99.21 1 +99.24 1 +99.29 1 +99.36 1 +99.62 1 +99.65 1 +99.67 1 +99.68 1 +99.91 1 +99.92 1 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-3-bda0e7c77d6f4712a03389cb5032bc6d b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-3-bda0e7c77d6f4712a03389cb5032bc6d new file mode 100644 index 0000000000000000000000000000000000000000..a9ec53c0cb21c9c3698706db0cac271c64fa0dfc --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-3-bda0e7c77d6f4712a03389cb5032bc6d @@ -0,0 +1,1049 @@ +0.02 1 +0.21 1 +0.27 1 +0.37 1 +0.37 1 +0.47 1 +0.48 1 +0.52 1 +0.6 1 +0.63 1 +0.63 1 +0.66 1 +0.73 1 +0.74 1 +0.74 1 +0.78 1 +0.8 1 +0.86 1 +0.86 1 +0.98 1 +1.17 1 +1.29 1 +1.3 1 +1.31 1 +1.37 1 +1.42 1 +1.45 1 +1.5 1 +1.53 1 +1.61 1 +1.62 1 +1.69 1 +1.71 1 +1.79 1 +1.98 1 +2.07 1 +2.09 1 +2.13 1 +2.16 1 +2.26 1 +2.29 1 +2.34 1 +2.34 1 +2.36 1 +2.43 1 +2.43 1 +2.52 1 +2.53 1 +2.53 1 +2.61 1 +2.63 1 +2.74 1 +2.75 1 +2.75 1 +2.79 1 +2.8 1 +2.82 1 +2.89 1 +2.89 1 +2.89 1 +2.9 1 +2.92 1 +3.03 1 +3.07 1 +3.07 1 +3.12 1 +3.15 1 +3.17 1 +3.27 1 +3.27 1 +3.29 1 +3.4 1 +3.4 1 +3.4 1 +3.4 1 +3.42 1 +3.66 1 +3.67 1 +3.69 1 +3.71 1 +3.78 1 +3.83 1 +3.86 1 +3.92 1 +3.98 1 +3.99 1 +4.04 1 +4.16 1 +4.25 1 +4.27 1 +4.44 1 +4.48 1 +4.53 1 +4.57 1 +4.58 1 +4.62 1 +4.81 1 +4.82 1 +4.83 1 +4.92 1 +4.95 1 +4.96 1 +4.97 1 +4.98 1 +5.09 1 +5.11 1 +5.19 1 +5.23 1 +5.3 1 +5.31 1 +5.31 1 +5.35 1 +5.42 1 +5.51 1 +5.55 1 +5.58 1 +5.74 1 +5.82 1 +5.84 1 +5.93 1 +5.93 1 +5.96 1 +6.06 1 +6.06 1 +6.21 1 +6.28 1 +6.46 1 +6.52 2 +6.54 1 +6.56 1 +6.57 1 +6.58 1 +6.61 1 +6.62 1 +6.76 1 +6.81 1 +6.81 1 +6.96 1 +6.98 1 +7.02 1 +7.03 1 +7.14 1 +7.18 1 +7.24 1 +7.24 1 +7.31 1 +7.36 1 +7.37 1 +7.45 1 +7.53 1 +7.62 1 +7.66 1 +7.71 1 +7.71 1 +7.8 1 +7.92 1 +8.05 1 +8.09 1 +8.21 1 +8.33 1 +8.33 1 +8.49 1 +8.49 1 +8.52 1 +8.56 1 +8.61 1 +8.62 2 +8.72 1 +8.76 1 +8.79 1 +8.82 1 +8.84 1 +8.95 1 +8.98 1 +9.14 1 +9.19 1 +9.21 1 +9.22 1 +9.26 1 +9.27 1 +9.39 2 +9.4 1 +9.42 1 +9.5 1 +9.51 1 +9.56 1 +9.6 1 +9.61 1 +9.62 1 +9.64 1 +9.81 1 +9.87 1 +9.88 1 +9.93 1 +9.94 1 +9.96 1 +9.99 1 +10.15 1 +10.21 1 +10.22 1 +10.23 1 +10.24 1 +10.36 1 +10.38 1 +10.38 1 +10.41 1 +10.47 1 +10.49 1 +10.49 1 +10.51 1 +10.52 1 +10.7 1 +10.71 1 +10.85 1 +10.99 1 +11.02 1 +11.12 1 +11.12 1 +11.16 1 +11.2 1 +11.26 1 +11.27 1 +11.35 1 +11.35 1 +11.4 1 +11.43 1 +11.44 1 +11.44 1 +11.46 1 +11.48 1 +11.5 1 +11.54 1 +11.63 1 +11.66 1 +11.69 1 +11.83 1 +11.9 1 +11.91 1 +11.96 1 +12.02 1 +12.13 1 +12.14 1 +12.15 1 +12.3 1 +12.3 1 +12.3 2 +12.34 1 +12.35 1 +12.43 1 +12.43 1 +12.64 1 +12.66 1 +12.7 1 +12.72 1 +12.73 1 +12.74 2 +12.82 1 +12.85 1 +13.02 1 +13.04 1 +13.08 1 +13.14 1 +13.2 1 +13.2 1 +13.22 1 +13.23 1 +13.3 1 +13.3 1 +13.44 1 +13.44 1 +13.44 1 +13.49 1 +13.6 1 +13.66 1 +13.71 1 +13.72 1 +13.8 1 +13.83 1 +13.84 1 +13.88 1 +13.95 1 +14.07 1 +14.16 1 +14.17 1 +14.22 1 +14.24 1 +14.26 1 +14.29 1 +14.33 1 +14.39 1 +14.44 1 +14.51 1 +14.51 1 +14.52 1 +14.62 1 +14.69 1 +14.72 1 +14.75 1 +14.83 1 +14.83 1 +14.84 1 +14.9 1 +14.91 1 +14.92 1 +14.99 1 +15.0 1 +15.01 1 +15.09 1 +15.09 1 +15.09 1 +15.1 1 +15.12 1 +15.13 1 +15.16 1 +15.18 1 +15.22 1 +15.27 1 +15.28 1 +15.32 1 +15.38 1 +15.46 1 +15.46 1 +15.51 1 +15.54 1 +15.87 1 +15.94 1 +15.97 1 +15.98 1 +16.04 2 +16.1 1 +16.12 1 +16.13 1 +16.15 1 +16.29 1 +16.35 1 +16.36 1 +16.38 1 +16.4 1 +16.42 1 +16.47 1 +16.49 1 +16.54 1 +16.61 1 +16.66 1 +16.79 1 +16.79 1 +16.82 1 +16.87 1 +16.87 1 +16.9 1 +16.9 1 +16.91 1 +16.92 1 +17.03 1 +17.03 2 +17.08 1 +17.15 1 +17.19 1 +17.29 1 +17.33 1 +17.44 1 +17.46 1 +17.47 1 +17.51 1 +17.52 2 +17.55 1 +17.59 1 +17.63 1 +17.69 1 +17.76 1 +17.86 1 +17.89 1 +17.99 1 +18.09 1 +18.19 1 +18.2 1 +18.28 1 +18.29 1 +18.31 1 +18.34 1 +18.35 1 +18.36 1 +18.38 1 +18.38 1 +18.41 1 +18.47 1 +18.48 1 +18.79 1 +18.82 1 +18.83 1 +18.86 1 +18.86 1 +19.03 1 +19.12 1 +19.15 1 +19.2 1 +19.31 1 +19.32 1 +19.41 1 +19.47 1 +19.47 2 +19.56 1 +19.59 1 +19.63 1 +19.65 1 +19.72 1 +19.72 1 +19.79 1 +19.79 1 +19.85 1 +19.87 1 +19.9 1 +19.92 1 +19.93 1 +19.98 1 +20.02 1 +20.02 1 +20.17 1 +20.19 1 +20.22 1 +20.3 1 +20.3 1 +20.34 1 +20.39 1 +20.42 1 +20.42 1 +20.44 1 +20.55 1 +20.55 1 +20.56 1 +20.56 1 +20.58 1 +20.58 1 +20.64 1 +20.65 1 +20.75 1 +20.76 1 +20.76 1 +20.8 1 +20.82 1 +20.91 1 +20.93 1 +20.94 1 +20.94 1 +20.97 1 +21.0 1 +21.01 1 +21.01 1 +21.02 1 +21.02 1 +21.11 1 +21.11 1 +21.14 1 +21.16 1 +21.26 1 +21.27 1 +21.3 1 +21.3 1 +21.33 1 +21.33 1 +21.38 1 +21.42 1 +21.52 1 +21.53 1 +21.57 1 +21.66 1 +21.67 1 +21.69 1 +21.77 1 +21.81 1 +21.86 1 +21.91 1 +21.95 1 +22.15 1 +22.19 1 +22.19 1 +22.22 1 +22.27 1 +22.27 1 +22.33 1 +22.48 1 +22.6 1 +22.61 1 +22.64 1 +22.68 1 +22.73 1 +22.75 1 +22.94 1 +22.95 1 +23.03 1 +23.07 1 +23.15 1 +23.15 1 +23.18 1 +23.18 1 +23.25 1 +23.27 1 +23.3 1 +23.31 1 +23.45 1 +23.48 1 +23.53 1 +23.55 1 +23.59 1 +23.61 1 +23.63 1 +23.73 1 +23.77 1 +23.78 2 +23.88 1 +23.91 1 +24.03 1 +24.03 1 +24.13 1 +24.17 1 +24.18 1 +24.25 1 +24.35 1 +24.35 2 +24.42 1 +24.46 1 +24.53 2 +24.59 1 +24.61 1 +24.61 1 +24.84 1 +24.85 1 +24.86 1 +24.95 1 +25.01 1 +25.02 1 +25.03 1 +25.08 1 +25.11 1 +25.17 1 +25.32 2 +25.36 1 +25.36 1 +25.43 1 +25.49 1 +25.51 1 +25.51 1 +25.58 1 +25.59 1 +25.63 1 +25.71 1 +25.75 1 +25.8 1 +25.92 1 +25.92 1 +25.95 1 +25.97 2 +26.0 1 +26.17 2 +26.21 1 +26.22 1 +26.24 1 +26.28 1 +26.44 1 +26.55 1 +26.55 1 +26.65 1 +26.67 1 +26.71 1 +26.71 1 +26.73 1 +26.74 1 +26.79 1 +26.84 1 +26.87 1 +27.02 1 +27.14 1 +27.2 1 +27.27 1 +27.29 1 +27.36 1 +27.39 1 +27.4 1 +27.42 1 +27.46 1 +27.54 1 +27.54 1 +27.61 1 +27.62 1 +27.89 1 +28.02 1 +28.1 1 +28.13 1 +28.14 1 +28.15 1 +28.17 1 +28.19 1 +28.29 1 +28.36 1 +28.4 1 +28.42 2 +28.44 1 +28.52 1 +28.52 1 +28.61 1 +28.64 1 +28.68 1 +28.69 1 +28.69 1 +28.71 1 +28.71 2 +28.77 1 +28.77 1 +28.85 1 +28.86 1 +28.91 1 +28.96 1 +28.96 1 +28.98 1 +29.0 1 +29.11 1 +29.19 1 +29.22 1 +29.24 1 +29.25 1 +29.36 1 +29.41 1 +29.46 1 +29.49 1 +29.52 2 +29.62 1 +29.63 1 +29.66 1 +29.73 1 +29.76 1 +29.78 1 +29.88 1 +29.96 1 +30.04 1 +30.04 1 +30.09 1 +30.12 1 +30.13 1 +30.16 1 +30.17 1 +30.21 1 +30.22 1 +30.28 1 +30.35 1 +30.37 2 +30.41 1 +30.41 1 +30.49 1 +30.55 1 +30.55 1 +30.58 1 +30.61 1 +30.62 1 +30.66 1 +30.67 1 +30.78 1 +30.78 1 +30.87 1 +30.89 1 +30.9 1 +30.92 1 +30.98 2 +30.99 1 +31.01 1 +31.16 1 +31.23 1 +31.3 1 +31.33 1 +31.36 1 +31.45 1 +31.46 1 +31.5 1 +31.61 1 +31.63 1 +31.64 1 +31.66 1 +31.68 1 +31.74 1 +31.75 1 +31.76 1 +31.84 1 +32.04 1 +32.06 1 +32.13 1 +32.17 1 +32.2 1 +32.25 1 +32.25 1 +32.31 1 +32.33 1 +32.39 1 +32.56 1 +32.56 1 +32.61 1 +32.74 1 +32.85 1 +32.89 1 +32.98 1 +33.0 1 +33.01 1 +33.02 1 +33.02 1 +33.02 1 +33.11 1 +33.12 1 +33.18 1 +33.18 1 +33.19 1 +33.24 1 +33.3 1 +33.36 1 +33.36 1 +33.38 1 +33.49 1 +33.52 2 +33.6 2 +33.64 1 +33.64 1 +33.66 1 +33.67 1 +33.72 1 +33.76 1 +33.9 1 +34.05 1 +34.06 1 +34.11 1 +34.14 1 +34.15 1 +34.17 1 +34.2 1 +34.41 2 +34.48 2 +34.49 1 +34.52 1 +34.53 1 +34.54 1 +34.62 1 +34.68 1 +34.68 1 +34.72 1 +34.81 1 +34.83 1 +34.84 1 +34.9 1 +34.95 1 +34.95 1 +34.97 1 +34.97 1 +34.98 1 +35.01 1 +35.02 1 +35.1 1 +35.15 1 +35.16 1 +35.23 1 +35.24 1 +35.36 1 +35.36 1 +35.49 1 +35.62 1 +35.68 1 +35.72 1 +35.84 1 +35.85 1 +35.9 1 +36.05 2 +36.09 1 +36.11 1 +36.12 1 +36.13 1 +36.22 1 +36.56 1 +36.57 1 +36.57 1 +36.62 1 +36.7 1 +36.72 1 +36.73 1 +36.84 1 +36.86 1 +36.93 1 +36.93 1 +37.02 1 +37.08 1 +37.12 1 +37.23 2 +37.32 1 +37.34 1 +37.37 1 +37.76 1 +37.77 1 +37.8 1 +37.93 1 +37.94 1 +37.96 1 +38.0 1 +38.04 1 +38.04 1 +38.04 1 +38.05 1 +38.07 1 +38.14 1 +38.22 1 +38.28 1 +38.37 2 +38.39 1 +38.43 1 +38.53 1 +38.6 1 +38.62 1 +38.66 1 +38.67 1 +38.73 1 +38.74 1 +38.92 1 +38.94 1 +39.01 1 +39.04 1 +39.05 1 +39.11 1 +39.27 1 +39.29 1 +39.41 1 +39.46 1 +39.49 1 +39.55 1 +39.57 1 +39.6 1 +39.63 1 +39.73 1 +39.74 1 +39.78 2 +39.8 1 +39.84 1 +39.84 1 +39.85 1 +39.92 1 +40.01 1 +40.04 1 +40.15 1 +40.21 2 +40.22 1 +40.24 1 +40.26 1 +40.27 1 +40.39 2 +40.43 1 +40.46 1 +40.5 1 +40.59 1 +40.6 1 +40.63 1 +40.76 1 +40.79 1 +40.84 1 +40.89 1 +40.91 1 +40.94 1 +40.96 1 +41.02 1 +41.08 1 +41.2 1 +41.24 1 +41.33 1 +41.34 1 +41.34 2 +41.36 1 +41.36 1 +41.37 1 +41.54 1 +41.56 1 +41.69 1 +41.73 1 +41.75 1 +41.75 1 +41.83 1 +41.86 1 +41.9 1 +42.0 2 +42.02 1 +42.03 1 +42.04 1 +42.22 1 +42.3 1 +42.37 1 +42.41 2 +42.47 1 +42.55 1 +42.56 1 +42.57 1 +42.76 1 +42.89 1 +42.96 1 +42.96 1 +43.0 1 +43.04 1 +43.04 3 +43.18 1 +43.23 1 +43.3 1 +43.34 1 +43.34 2 +43.4 1 +43.42 1 +43.58 1 +43.64 1 +43.67 1 +43.76 2 +43.84 1 +43.85 1 +43.92 1 +43.95 1 +43.95 1 +43.96 1 +43.96 1 +44.04 1 +44.11 1 +44.12 1 +44.12 1 +44.22 3 +44.24 1 +44.27 1 +44.32 1 +44.36 1 +44.4 1 +44.57 1 +44.6 1 +44.63 1 +44.66 1 +44.73 1 +44.75 1 +44.8 1 +44.83 1 +44.9 1 +44.92 1 +44.93 1 +45.0 1 +45.02 1 +45.06 1 +45.06 1 +45.09 1 +45.1 1 +45.1 1 +45.11 1 +45.12 2 +45.14 1 +45.14 2 +45.28 2 +45.29 1 +45.46 1 +45.49 1 +45.53 1 +45.53 2 +45.54 1 +45.69 1 +45.71 1 +45.78 1 +45.81 2 +45.86 1 +45.9 1 +45.94 1 +46.03 1 +46.03 1 +46.09 1 +46.18 1 +46.19 1 +46.28 1 +46.3 1 +46.31 1 +46.33 1 +46.36 1 +46.39 1 +46.52 1 +46.53 1 +46.54 1 +46.57 1 +46.59 1 +46.67 1 +46.69 1 +46.73 1 +46.73 1 +46.73 2 +46.74 1 +46.81 1 +46.87 1 +46.88 1 +46.9 1 +46.93 1 +46.98 1 +47.0 1 +47.03 1 +47.03 1 +47.06 1 +47.15 1 +47.22 1 +47.3 1 +47.31 2 +47.37 2 +47.4 1 +47.46 1 +47.49 1 +47.55 1 +47.6 1 +47.66 1 +47.68 1 +47.71 1 +47.72 1 +47.82 1 +47.86 2 +47.91 1 +47.91 2 +47.95 1 +47.98 1 +48.0 1 +48.08 1 +48.5 1 +48.52 1 +48.71 1 +48.78 1 +48.8 1 +48.85 1 +48.89 1 +48.96 1 +48.98 1 +49.04 1 +49.05 1 +49.16 1 +49.21 1 +49.32 1 +49.34 1 +49.34 1 +49.38 1 +49.44 2 +49.45 1 +49.45 1 +49.46 1 +49.46 1 +49.52 1 +49.56 1 +49.59 1 +49.63 1 +49.67 1 +49.69 1 +49.71 2 +49.72 1 +49.73 1 +49.79 1 +49.84 1 +49.85 2 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-0-2e0cbc2d7c5f16657edacd9e7209e6e7 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-0-2e0cbc2d7c5f16657edacd9e7209e6e7 new file mode 100644 index 0000000000000000000000000000000000000000..612bdf44c0cdf4a72201b6fbdf231764a1643aa1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-0-2e0cbc2d7c5f16657edacd9e7209e6e7 @@ -0,0 +1,1049 @@ + 1 + 1 + 1 +alice allen 1 +alice allen 1 +alice allen 1 +alice brown 1 +alice carson 1 +alice davidson 1 +alice falkner 1 +alice garcia 2 +alice hernandez 1 +alice hernandez 1 +alice johnson 1 +alice king 1 +alice king 1 +alice king 1 +alice laertes 1 +alice laertes 1 +alice miller 1 +alice nixon 1 +alice nixon 1 +alice nixon 1 +alice ovid 1 +alice polk 1 +alice quirinius 1 +alice quirinius 1 +alice robinson 1 +alice robinson 1 +alice steinbeck 1 +alice steinbeck 1 +alice steinbeck 1 +alice underhill 1 +alice van buren 1 +alice xylophone 1 +alice xylophone 1 +alice xylophone 1 +alice zipper 1 +alice zipper 1 +alice zipper 1 +bob brown 1 +bob brown 1 +bob brown 2 +bob carson 1 +bob davidson 1 +bob davidson 1 +bob davidson 1 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob ellison 2 +bob falkner 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 2 +bob hernandez 1 +bob ichabod 1 +bob king 1 +bob king 1 +bob king 1 +bob laertes 1 +bob laertes 1 +bob miller 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob polk 1 +bob quirinius 1 +bob steinbeck 1 +bob van buren 1 +bob white 1 +bob white 1 +bob xylophone 1 +bob xylophone 1 +bob young 1 +bob zipper 1 +bob zipper 1 +bob zipper 1 +calvin allen 1 +calvin brown 1 +calvin brown 1 +calvin brown 1 +calvin carson 1 +calvin davidson 1 +calvin davidson 1 +calvin ellison 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin garcia 1 +calvin hernandez 1 +calvin johnson 2 +calvin laertes 1 +calvin laertes 1 +calvin nixon 1 +calvin nixon 1 +calvin nixon 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin polk 1 +calvin quirinius 1 +calvin quirinius 1 +calvin robinson 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin thompson 1 +calvin thompson 1 +calvin underhill 1 +calvin van buren 1 +calvin van buren 1 +calvin white 1 +calvin white 1 +calvin xylophone 1 +calvin xylophone 1 +calvin xylophone 1 +calvin young 1 +calvin young 1 +calvin zipper 1 +calvin zipper 1 +david allen 1 +david allen 1 +david brown 1 +david brown 1 +david davidson 1 +david davidson 1 +david davidson 1 +david davidson 2 +david ellison 1 +david ellison 1 +david ellison 1 +david hernandez 1 +david ichabod 1 +david ichabod 1 +david laertes 1 +david nixon 1 +david ovid 1 +david ovid 1 +david quirinius 1 +david quirinius 1 +david quirinius 1 +david robinson 1 +david robinson 1 +david thompson 1 +david underhill 1 +david underhill 1 +david underhill 1 +david van buren 1 +david van buren 1 +david white 2 +david xylophone 1 +david xylophone 1 +david xylophone 1 +david young 1 +david young 1 +ethan allen 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan carson 1 +ethan ellison 1 +ethan ellison 1 +ethan falkner 1 +ethan falkner 1 +ethan garcia 1 +ethan hernandez 1 +ethan johnson 1 +ethan king 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan miller 1 +ethan nixon 1 +ethan ovid 1 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan quirinius 1 +ethan quirinius 1 +ethan quirinius 1 +ethan robinson 1 +ethan robinson 1 +ethan underhill 1 +ethan van buren 1 +ethan white 1 +ethan white 1 +ethan xylophone 1 +ethan zipper 1 +ethan zipper 1 +fred davidson 1 +fred davidson 1 +fred davidson 1 +fred ellison 1 +fred ellison 1 +fred ellison 1 +fred falkner 1 +fred falkner 1 +fred falkner 1 +fred hernandez 1 +fred ichabod 1 +fred ichabod 2 +fred johnson 1 +fred king 1 +fred king 1 +fred laertes 1 +fred miller 1 +fred nixon 1 +fred nixon 1 +fred nixon 1 +fred nixon 2 +fred polk 1 +fred polk 1 +fred polk 1 +fred polk 1 +fred quirinius 1 +fred quirinius 1 +fred robinson 1 +fred steinbeck 1 +fred steinbeck 1 +fred steinbeck 1 +fred underhill 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred white 1 +fred young 1 +fred young 1 +fred zipper 1 +gabriella allen 1 +gabriella allen 1 +gabriella brown 1 +gabriella brown 1 +gabriella carson 1 +gabriella davidson 1 +gabriella ellison 1 +gabriella ellison 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella garcia 1 +gabriella hernandez 1 +gabriella hernandez 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella king 1 +gabriella king 1 +gabriella laertes 1 +gabriella miller 1 +gabriella ovid 1 +gabriella ovid 1 +gabriella polk 1 +gabriella polk 1 +gabriella steinbeck 1 +gabriella steinbeck 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella van buren 1 +gabriella van buren 1 +gabriella white 1 +gabriella young 1 +gabriella young 1 +gabriella zipper 1 +gabriella zipper 1 +holly allen 1 +holly brown 1 +holly brown 1 +holly falkner 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 2 +holly ichabod 1 +holly ichabod 1 +holly ichabod 1 +holly johnson 1 +holly johnson 1 +holly johnson 1 +holly king 1 +holly king 1 +holly laertes 1 +holly miller 1 +holly nixon 1 +holly nixon 1 +holly polk 1 +holly polk 1 +holly robinson 1 +holly thompson 1 +holly thompson 1 +holly thompson 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly van buren 1 +holly white 1 +holly white 2 +holly xylophone 1 +holly young 1 +holly young 1 +holly zipper 1 +holly zipper 1 +irene allen 1 +irene brown 1 +irene brown 1 +irene brown 1 +irene carson 1 +irene ellison 1 +irene ellison 1 +irene falkner 1 +irene falkner 1 +irene garcia 1 +irene garcia 1 +irene garcia 1 +irene ichabod 1 +irene ichabod 1 +irene johnson 1 +irene laertes 1 +irene laertes 1 +irene laertes 1 +irene miller 1 +irene nixon 1 +irene nixon 1 +irene nixon 1 +irene ovid 1 +irene ovid 1 +irene ovid 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene quirinius 1 +irene quirinius 1 +irene quirinius 1 +irene robinson 1 +irene steinbeck 1 +irene thompson 1 +irene underhill 1 +irene underhill 1 +irene van buren 1 +irene van buren 1 +irene xylophone 2 +jessica brown 2 +jessica carson 1 +jessica carson 1 +jessica carson 1 +jessica davidson 1 +jessica davidson 1 +jessica davidson 1 +jessica davidson 1 +jessica ellison 1 +jessica ellison 1 +jessica falkner 1 +jessica garcia 1 +jessica garcia 1 +jessica ichabod 1 +jessica johnson 1 +jessica johnson 1 +jessica miller 1 +jessica nixon 1 +jessica nixon 1 +jessica ovid 1 +jessica ovid 2 +jessica polk 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica robinson 1 +jessica thompson 1 +jessica thompson 3 +jessica underhill 1 +jessica underhill 1 +jessica underhill 1 +jessica van buren 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica xylophone 1 +jessica young 1 +jessica young 1 +jessica zipper 1 +jessica zipper 1 +jessica zipper 1 +katie allen 1 +katie brown 1 +katie davidson 1 +katie ellison 1 +katie ellison 1 +katie falkner 1 +katie garcia 1 +katie garcia 1 +katie hernandez 1 +katie ichabod 1 +katie ichabod 1 +katie ichabod 1 +katie king 1 +katie king 1 +katie king 1 +katie miller 1 +katie miller 1 +katie nixon 1 +katie ovid 1 +katie polk 1 +katie polk 1 +katie robinson 1 +katie van buren 1 +katie van buren 1 +katie white 1 +katie white 1 +katie xylophone 1 +katie young 1 +katie young 1 +katie young 1 +katie zipper 1 +katie zipper 1 +luke allen 1 +luke allen 1 +luke allen 1 +luke allen 1 +luke allen 2 +luke brown 1 +luke davidson 1 +luke davidson 1 +luke ellison 1 +luke ellison 1 +luke ellison 1 +luke falkner 1 +luke falkner 1 +luke garcia 1 +luke garcia 1 +luke ichabod 1 +luke ichabod 1 +luke johnson 1 +luke johnson 1 +luke johnson 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke miller 1 +luke ovid 1 +luke ovid 1 +luke polk 1 +luke polk 1 +luke quirinius 1 +luke robinson 1 +luke robinson 1 +luke thompson 1 +luke underhill 1 +luke underhill 1 +luke underhill 2 +luke van buren 1 +luke white 1 +luke xylophone 1 +luke zipper 1 +mike allen 1 +mike brown 1 +mike carson 1 +mike carson 1 +mike carson 1 +mike davidson 1 +mike davidson 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike falkner 1 +mike garcia 1 +mike garcia 1 +mike garcia 1 +mike hernandez 1 +mike hernandez 2 +mike ichabod 1 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike king 2 +mike miller 1 +mike nixon 1 +mike nixon 1 +mike polk 1 +mike polk 1 +mike polk 1 +mike quirinius 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike van buren 1 +mike van buren 1 +mike white 1 +mike white 1 +mike white 1 +mike white 2 +mike young 1 +mike young 1 +mike young 1 +mike zipper 1 +mike zipper 1 +mike zipper 1 +nick allen 1 +nick allen 1 +nick brown 1 +nick davidson 1 +nick ellison 1 +nick ellison 1 +nick falkner 1 +nick falkner 1 +nick garcia 1 +nick garcia 1 +nick garcia 1 +nick ichabod 1 +nick ichabod 1 +nick ichabod 1 +nick johnson 1 +nick johnson 1 +nick laertes 1 +nick miller 1 +nick nixon 1 +nick ovid 1 +nick polk 1 +nick quirinius 1 +nick quirinius 1 +nick robinson 1 +nick robinson 1 +nick steinbeck 1 +nick thompson 1 +nick underhill 1 +nick van buren 1 +nick xylophone 1 +nick young 1 +nick young 1 +nick zipper 1 +nick zipper 1 +oscar allen 2 +oscar brown 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar davidson 1 +oscar ellison 1 +oscar ellison 1 +oscar falkner 1 +oscar garcia 1 +oscar hernandez 1 +oscar hernandez 1 +oscar ichabod 1 +oscar ichabod 1 +oscar ichabod 1 +oscar ichabod 1 +oscar johnson 1 +oscar johnson 1 +oscar king 1 +oscar king 1 +oscar king 2 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar nixon 1 +oscar ovid 1 +oscar ovid 1 +oscar ovid 2 +oscar polk 1 +oscar polk 1 +oscar quirinius 1 +oscar quirinius 1 +oscar quirinius 1 +oscar quirinius 1 +oscar robinson 1 +oscar robinson 1 +oscar robinson 1 +oscar robinson 1 +oscar steinbeck 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 2 +oscar underhill 1 +oscar van buren 1 +oscar van buren 1 +oscar van buren 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar xylophone 1 +oscar xylophone 1 +oscar xylophone 1 +oscar zipper 1 +oscar zipper 1 +oscar zipper 1 +priscilla brown 1 +priscilla brown 1 +priscilla brown 1 +priscilla carson 1 +priscilla carson 1 +priscilla carson 1 +priscilla ichabod 1 +priscilla ichabod 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla king 1 +priscilla nixon 1 +priscilla nixon 2 +priscilla ovid 1 +priscilla ovid 1 +priscilla polk 1 +priscilla quirinius 1 +priscilla thompson 1 +priscilla underhill 1 +priscilla underhill 1 +priscilla van buren 1 +priscilla van buren 1 +priscilla van buren 1 +priscilla white 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla young 1 +priscilla young 1 +priscilla zipper 1 +priscilla zipper 1 +quinn allen 1 +quinn allen 1 +quinn brown 1 +quinn brown 1 +quinn brown 1 +quinn davidson 1 +quinn davidson 1 +quinn davidson 1 +quinn davidson 1 +quinn ellison 1 +quinn ellison 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn ichabod 1 +quinn king 1 +quinn king 1 +quinn laertes 1 +quinn laertes 1 +quinn laertes 1 +quinn nixon 1 +quinn ovid 1 +quinn quirinius 1 +quinn robinson 1 +quinn steinbeck 1 +quinn steinbeck 4 +quinn thompson 1 +quinn thompson 2 +quinn underhill 1 +quinn underhill 1 +quinn underhill 2 +quinn van buren 1 +quinn young 1 +quinn zipper 1 +quinn zipper 1 +rachel allen 1 +rachel allen 1 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel brown 2 +rachel carson 1 +rachel carson 1 +rachel davidson 1 +rachel ellison 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel johnson 1 +rachel king 1 +rachel king 1 +rachel laertes 1 +rachel laertes 1 +rachel ovid 1 +rachel ovid 1 +rachel polk 1 +rachel quirinius 1 +rachel robinson 1 +rachel robinson 1 +rachel robinson 1 +rachel thompson 1 +rachel thompson 1 +rachel thompson 1 +rachel underhill 1 +rachel white 1 +rachel white 1 +rachel young 1 +rachel zipper 1 +rachel zipper 1 +sarah carson 1 +sarah carson 1 +sarah carson 1 +sarah ellison 2 +sarah falkner 1 +sarah falkner 1 +sarah garcia 1 +sarah garcia 1 +sarah garcia 2 +sarah ichabod 1 +sarah ichabod 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah king 1 +sarah king 1 +sarah miller 1 +sarah ovid 1 +sarah robinson 1 +sarah robinson 1 +sarah steinbeck 1 +sarah white 1 +sarah white 1 +sarah xylophone 1 +sarah young 1 +sarah zipper 1 +tom brown 1 +tom brown 1 +tom carson 1 +tom carson 1 +tom carson 1 +tom davidson 1 +tom ellison 1 +tom ellison 1 +tom ellison 1 +tom falkner 1 +tom falkner 1 +tom hernandez 1 +tom hernandez 1 +tom ichabod 1 +tom johnson 1 +tom johnson 2 +tom king 1 +tom laertes 1 +tom laertes 1 +tom miller 1 +tom miller 1 +tom miller 2 +tom nixon 1 +tom ovid 1 +tom polk 1 +tom polk 1 +tom quirinius 1 +tom quirinius 1 +tom robinson 1 +tom robinson 1 +tom robinson 2 +tom robinson 2 +tom steinbeck 2 +tom van buren 1 +tom van buren 1 +tom van buren 1 +tom white 1 +tom young 1 +tom young 2 +tom zipper 1 +ulysses brown 1 +ulysses carson 1 +ulysses carson 1 +ulysses carson 1 +ulysses carson 1 +ulysses davidson 1 +ulysses ellison 1 +ulysses garcia 1 +ulysses hernandez 1 +ulysses hernandez 1 +ulysses hernandez 2 +ulysses ichabod 1 +ulysses ichabod 1 +ulysses johnson 1 +ulysses king 1 +ulysses laertes 1 +ulysses laertes 1 +ulysses laertes 1 +ulysses miller 1 +ulysses miller 1 +ulysses nixon 1 +ulysses ovid 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 2 +ulysses quirinius 1 +ulysses robinson 1 +ulysses steinbeck 1 +ulysses steinbeck 1 +ulysses thompson 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses van buren 1 +ulysses white 1 +ulysses white 1 +ulysses xylophone 1 +ulysses xylophone 1 +ulysses xylophone 1 +ulysses young 1 +ulysses young 1 +ulysses young 1 +victor allen 1 +victor allen 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor davidson 1 +victor davidson 1 +victor davidson 2 +victor ellison 1 +victor ellison 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor johnson 1 +victor johnson 1 +victor johnson 1 +victor king 1 +victor king 1 +victor laertes 1 +victor laertes 1 +victor miller 2 +victor nixon 1 +victor nixon 1 +victor ovid 1 +victor polk 1 +victor quirinius 1 +victor quirinius 1 +victor robinson 1 +victor robinson 1 +victor steinbeck 1 +victor steinbeck 1 +victor steinbeck 1 +victor thompson 1 +victor van buren 1 +victor van buren 1 +victor white 1 +victor white 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 2 +victor young 1 +victor zipper 1 +wendy allen 1 +wendy allen 1 +wendy allen 1 +wendy brown 1 +wendy brown 1 +wendy ellison 1 +wendy ellison 1 +wendy falkner 1 +wendy falkner 1 +wendy falkner 1 +wendy garcia 1 +wendy garcia 1 +wendy garcia 1 +wendy garcia 1 +wendy hernandez 1 +wendy ichabod 1 +wendy king 1 +wendy king 1 +wendy king 1 +wendy laertes 1 +wendy laertes 1 +wendy laertes 1 +wendy miller 1 +wendy miller 1 +wendy nixon 1 +wendy nixon 1 +wendy ovid 1 +wendy ovid 1 +wendy polk 1 +wendy polk 1 +wendy quirinius 1 +wendy quirinius 1 +wendy robinson 1 +wendy robinson 1 +wendy robinson 1 +wendy steinbeck 1 +wendy thompson 1 +wendy thompson 1 +wendy underhill 1 +wendy underhill 1 +wendy underhill 1 +wendy van buren 1 +wendy van buren 1 +wendy white 1 +wendy xylophone 1 +wendy xylophone 1 +wendy young 1 +wendy young 3 +xavier allen 1 +xavier allen 1 +xavier allen 1 +xavier brown 1 +xavier brown 1 +xavier brown 1 +xavier carson 1 +xavier carson 1 +xavier davidson 1 +xavier davidson 1 +xavier davidson 1 +xavier ellison 1 +xavier ellison 1 +xavier garcia 1 +xavier hernandez 1 +xavier hernandez 1 +xavier hernandez 1 +xavier ichabod 1 +xavier ichabod 1 +xavier johnson 1 +xavier johnson 1 +xavier king 1 +xavier king 1 +xavier laertes 1 +xavier ovid 1 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier quirinius 1 +xavier quirinius 1 +xavier quirinius 1 +xavier quirinius 1 +xavier thompson 1 +xavier underhill 1 +xavier white 1 +xavier white 1 +xavier xylophone 1 +xavier zipper 2 +yuri allen 1 +yuri allen 1 +yuri brown 1 +yuri brown 1 +yuri carson 1 +yuri carson 1 +yuri ellison 1 +yuri ellison 1 +yuri falkner 1 +yuri falkner 1 +yuri garcia 1 +yuri hernandez 1 +yuri johnson 1 +yuri johnson 1 +yuri johnson 1 +yuri king 1 +yuri laertes 1 +yuri laertes 1 +yuri nixon 1 +yuri nixon 1 +yuri polk 1 +yuri polk 1 +yuri polk 1 +yuri quirinius 1 +yuri quirinius 1 +yuri quirinius 1 +yuri steinbeck 1 +yuri steinbeck 1 +yuri thompson 1 +yuri underhill 1 +yuri underhill 1 +yuri white 1 +yuri xylophone 1 +zach allen 2 +zach brown 1 +zach brown 1 +zach brown 1 +zach brown 1 +zach brown 1 +zach carson 1 +zach ellison 1 +zach falkner 1 +zach falkner 1 +zach garcia 1 +zach garcia 1 +zach garcia 1 +zach garcia 1 +zach ichabod 1 +zach ichabod 1 +zach king 1 +zach king 1 +zach king 2 +zach miller 1 +zach miller 1 +zach miller 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach quirinius 1 +zach robinson 2 +zach steinbeck 2 +zach steinbeck 2 +zach thompson 1 +zach thompson 1 +zach underhill 1 +zach white 1 +zach xylophone 1 +zach xylophone 1 +zach young 1 +zach zipper 1 +zach zipper 1 +zach zipper 1 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-1-5c5f373e325115d710a7a23fcb1626f1 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-1-5c5f373e325115d710a7a23fcb1626f1 new file mode 100644 index 0000000000000000000000000000000000000000..22a6f27253dcf3302d53ed87a723bc5fb67854b3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-1-5c5f373e325115d710a7a23fcb1626f1 @@ -0,0 +1,1049 @@ +zach zipper 4 +zach zipper 3 +zach zipper 1 +zach young 4 +zach xylophone 4 +zach xylophone 1 +zach white 1 +zach underhill 1 +zach thompson 2 +zach thompson 2 +zach steinbeck 5 +zach steinbeck 1 +zach robinson 1 +zach quirinius 3 +zach ovid 5 +zach ovid 4 +zach ovid 3 +zach ovid 1 +zach miller 5 +zach miller 3 +zach miller 1 +zach king 6 +zach king 4 +zach king 1 +zach ichabod 3 +zach ichabod 2 +zach garcia 6 +zach garcia 3 +zach garcia 1 +zach garcia 1 +zach falkner 2 +zach falkner 1 +zach ellison 4 +zach carson 3 +zach brown 5 +zach brown 4 +zach brown 3 +zach brown 2 +zach brown 1 +zach allen 4 +yuri xylophone 3 +yuri white 2 +yuri underhill 6 +yuri underhill 4 +yuri thompson 4 +yuri steinbeck 6 +yuri steinbeck 2 +yuri quirinius 4 +yuri quirinius 3 +yuri quirinius 1 +yuri polk 4 +yuri polk 3 +yuri polk 2 +yuri nixon 3 +yuri nixon 2 +yuri laertes 3 +yuri laertes 1 +yuri king 5 +yuri johnson 4 +yuri johnson 3 +yuri johnson 1 +yuri hernandez 4 +yuri garcia 3 +yuri falkner 7 +yuri falkner 3 +yuri ellison 1 +yuri ellison 1 +yuri carson 7 +yuri carson 4 +yuri brown 3 +yuri brown 1 +yuri allen 3 +yuri allen 2 +xavier zipper 1 +xavier xylophone 1 +xavier white 3 +xavier white 3 +xavier underhill 2 +xavier thompson 3 +xavier quirinius 6 +xavier quirinius 5 +xavier quirinius 2 +xavier quirinius 1 +xavier polk 5 +xavier polk 3 +xavier polk 3 +xavier polk 3 +xavier ovid 5 +xavier laertes 4 +xavier king 3 +xavier king 1 +xavier johnson 3 +xavier johnson 1 +xavier ichabod 2 +xavier ichabod 2 +xavier hernandez 3 +xavier hernandez 1 +xavier hernandez 1 +xavier garcia 4 +xavier ellison 1 +xavier ellison 1 +xavier davidson 5 +xavier davidson 4 +xavier davidson 1 +xavier carson 5 +xavier carson 3 +xavier brown 4 +xavier brown 2 +xavier brown 2 +xavier allen 6 +xavier allen 3 +xavier allen 1 +wendy young 8 +wendy young 2 +wendy xylophone 6 +wendy xylophone 4 +wendy white 5 +wendy van buren 2 +wendy van buren 2 +wendy underhill 6 +wendy underhill 5 +wendy underhill 4 +wendy thompson 5 +wendy thompson 2 +wendy steinbeck 1 +wendy robinson 5 +wendy robinson 3 +wendy robinson 2 +wendy quirinius 6 +wendy quirinius 4 +wendy polk 2 +wendy polk 2 +wendy ovid 4 +wendy ovid 1 +wendy nixon 3 +wendy nixon 1 +wendy miller 2 +wendy miller 1 +wendy laertes 3 +wendy laertes 3 +wendy laertes 1 +wendy king 5 +wendy king 4 +wendy king 1 +wendy ichabod 3 +wendy hernandez 1 +wendy garcia 7 +wendy garcia 5 +wendy garcia 4 +wendy garcia 1 +wendy falkner 3 +wendy falkner 1 +wendy falkner 1 +wendy ellison 2 +wendy ellison 1 +wendy brown 5 +wendy brown 2 +wendy allen 6 +wendy allen 2 +wendy allen 2 +victor zipper 3 +victor young 1 +victor xylophone 6 +victor xylophone 6 +victor xylophone 2 +victor xylophone 1 +victor xylophone 1 +victor white 2 +victor white 1 +victor van buren 4 +victor van buren 4 +victor thompson 2 +victor steinbeck 5 +victor steinbeck 2 +victor steinbeck 1 +victor robinson 4 +victor robinson 2 +victor quirinius 3 +victor quirinius 1 +victor polk 3 +victor ovid 2 +victor nixon 6 +victor nixon 4 +victor miller 1 +victor laertes 4 +victor laertes 3 +victor king 6 +victor king 1 +victor johnson 2 +victor johnson 2 +victor johnson 1 +victor hernandez 6 +victor hernandez 4 +victor hernandez 3 +victor hernandez 1 +victor hernandez 1 +victor ellison 7 +victor ellison 4 +victor davidson 6 +victor davidson 2 +victor davidson 2 +victor brown 4 +victor brown 3 +victor brown 2 +victor brown 1 +victor allen 4 +victor allen 2 +ulysses young 7 +ulysses young 6 +ulysses young 3 +ulysses xylophone 6 +ulysses xylophone 3 +ulysses xylophone 2 +ulysses white 6 +ulysses white 2 +ulysses van buren 3 +ulysses underhill 8 +ulysses underhill 4 +ulysses underhill 3 +ulysses underhill 2 +ulysses underhill 2 +ulysses underhill 1 +ulysses underhill 1 +ulysses thompson 5 +ulysses steinbeck 3 +ulysses steinbeck 1 +ulysses robinson 5 +ulysses quirinius 8 +ulysses polk 6 +ulysses polk 4 +ulysses polk 1 +ulysses polk 1 +ulysses ovid 3 +ulysses nixon 1 +ulysses miller 3 +ulysses miller 2 +ulysses laertes 5 +ulysses laertes 4 +ulysses laertes 2 +ulysses king 2 +ulysses johnson 5 +ulysses ichabod 1 +ulysses ichabod 1 +ulysses hernandez 6 +ulysses hernandez 3 +ulysses hernandez 2 +ulysses garcia 2 +ulysses ellison 2 +ulysses davidson 8 +ulysses carson 4 +ulysses carson 3 +ulysses carson 2 +ulysses carson 1 +ulysses brown 3 +tom zipper 5 +tom young 2 +tom young 1 +tom white 1 +tom van buren 5 +tom van buren 2 +tom van buren 1 +tom steinbeck 4 +tom robinson 8 +tom robinson 4 +tom robinson 3 +tom robinson 2 +tom quirinius 5 +tom quirinius 1 +tom polk 3 +tom polk 2 +tom ovid 2 +tom nixon 5 +tom miller 1 +tom miller 1 +tom miller 1 +tom laertes 4 +tom laertes 2 +tom king 1 +tom johnson 8 +tom johnson 1 +tom ichabod 1 +tom hernandez 3 +tom hernandez 2 +tom falkner 3 +tom falkner 2 +tom ellison 5 +tom ellison 3 +tom ellison 1 +tom davidson 7 +tom carson 3 +tom carson 3 +tom carson 1 +tom brown 4 +tom brown 2 +sarah zipper 1 +sarah young 1 +sarah xylophone 2 +sarah white 4 +sarah white 3 +sarah steinbeck 6 +sarah robinson 3 +sarah robinson 2 +sarah ovid 1 +sarah miller 1 +sarah king 3 +sarah king 2 +sarah johnson 7 +sarah johnson 6 +sarah johnson 4 +sarah johnson 2 +sarah ichabod 4 +sarah ichabod 3 +sarah garcia 2 +sarah garcia 2 +sarah garcia 2 +sarah falkner 7 +sarah falkner 1 +sarah ellison 1 +sarah carson 6 +sarah carson 4 +sarah carson 4 +rachel zipper 8 +rachel zipper 5 +rachel young 3 +rachel white 2 +rachel white 2 +rachel underhill 2 +rachel thompson 5 +rachel thompson 4 +rachel thompson 3 +rachel robinson 10 +rachel robinson 3 +rachel robinson 1 +rachel quirinius 5 +rachel polk 4 +rachel ovid 5 +rachel ovid 4 +rachel laertes 1 +rachel laertes 1 +rachel king 3 +rachel king 1 +rachel johnson 1 +rachel falkner 8 +rachel falkner 5 +rachel falkner 5 +rachel falkner 2 +rachel ellison 6 +rachel davidson 6 +rachel carson 7 +rachel carson 2 +rachel brown 5 +rachel brown 4 +rachel brown 3 +rachel brown 3 +rachel brown 1 +rachel allen 5 +rachel allen 1 +quinn zipper 2 +quinn zipper 2 +quinn young 2 +quinn van buren 2 +quinn underhill 7 +quinn underhill 6 +quinn underhill 2 +quinn thompson 5 +quinn thompson 2 +quinn steinbeck 3 +quinn steinbeck 2 +quinn robinson 2 +quinn quirinius 5 +quinn ovid 6 +quinn nixon 3 +quinn laertes 2 +quinn laertes 2 +quinn laertes 1 +quinn king 2 +quinn king 1 +quinn ichabod 1 +quinn garcia 6 +quinn garcia 3 +quinn garcia 2 +quinn garcia 1 +quinn ellison 7 +quinn ellison 5 +quinn davidson 7 +quinn davidson 4 +quinn davidson 3 +quinn davidson 2 +quinn brown 5 +quinn brown 3 +quinn brown 2 +quinn allen 5 +quinn allen 2 +priscilla zipper 5 +priscilla zipper 2 +priscilla young 4 +priscilla young 1 +priscilla xylophone 7 +priscilla xylophone 2 +priscilla xylophone 1 +priscilla white 4 +priscilla van buren 3 +priscilla van buren 3 +priscilla van buren 2 +priscilla underhill 5 +priscilla underhill 4 +priscilla thompson 2 +priscilla quirinius 4 +priscilla polk 5 +priscilla ovid 4 +priscilla ovid 1 +priscilla nixon 2 +priscilla nixon 1 +priscilla king 4 +priscilla johnson 4 +priscilla johnson 2 +priscilla johnson 2 +priscilla johnson 2 +priscilla johnson 1 +priscilla ichabod 3 +priscilla ichabod 2 +priscilla carson 6 +priscilla carson 5 +priscilla carson 4 +priscilla brown 5 +priscilla brown 5 +priscilla brown 3 +oscar zipper 4 +oscar zipper 4 +oscar zipper 2 +oscar xylophone 7 +oscar xylophone 5 +oscar xylophone 3 +oscar white 5 +oscar white 5 +oscar white 3 +oscar white 2 +oscar van buren 5 +oscar van buren 3 +oscar van buren 2 +oscar underhill 1 +oscar thompson 6 +oscar thompson 3 +oscar thompson 3 +oscar thompson 2 +oscar steinbeck 7 +oscar robinson 7 +oscar robinson 3 +oscar robinson 3 +oscar robinson 1 +oscar quirinius 3 +oscar quirinius 3 +oscar quirinius 2 +oscar quirinius 1 +oscar polk 2 +oscar polk 2 +oscar ovid 4 +oscar ovid 2 +oscar ovid 1 +oscar nixon 1 +oscar laertes 6 +oscar laertes 4 +oscar laertes 3 +oscar laertes 2 +oscar king 4 +oscar king 2 +oscar king 1 +oscar johnson 6 +oscar johnson 3 +oscar ichabod 3 +oscar ichabod 3 +oscar ichabod 1 +oscar ichabod 1 +oscar hernandez 6 +oscar hernandez 6 +oscar garcia 4 +oscar falkner 2 +oscar ellison 2 +oscar ellison 1 +oscar davidson 1 +oscar carson 4 +oscar carson 2 +oscar carson 2 +oscar carson 1 +oscar carson 1 +oscar brown 4 +oscar allen 2 +nick zipper 7 +nick zipper 5 +nick young 4 +nick young 2 +nick xylophone 2 +nick van buren 2 +nick underhill 2 +nick thompson 2 +nick steinbeck 4 +nick robinson 3 +nick robinson 1 +nick quirinius 5 +nick quirinius 1 +nick polk 5 +nick ovid 6 +nick nixon 4 +nick miller 2 +nick laertes 3 +nick johnson 4 +nick johnson 4 +nick ichabod 3 +nick ichabod 3 +nick ichabod 1 +nick garcia 5 +nick garcia 4 +nick garcia 4 +nick falkner 3 +nick falkner 1 +nick ellison 3 +nick ellison 2 +nick davidson 4 +nick brown 3 +nick allen 5 +nick allen 4 +mike zipper 4 +mike zipper 4 +mike zipper 1 +mike young 3 +mike young 1 +mike young 1 +mike white 9 +mike white 7 +mike white 5 +mike white 2 +mike van buren 2 +mike van buren 1 +mike steinbeck 4 +mike steinbeck 2 +mike steinbeck 2 +mike steinbeck 1 +mike quirinius 7 +mike polk 4 +mike polk 2 +mike polk 2 +mike nixon 3 +mike nixon 2 +mike miller 1 +mike king 6 +mike king 5 +mike king 4 +mike king 3 +mike king 1 +mike king 1 +mike ichabod 3 +mike hernandez 2 +mike hernandez 1 +mike garcia 3 +mike garcia 2 +mike garcia 1 +mike falkner 2 +mike ellison 6 +mike ellison 5 +mike ellison 3 +mike ellison 1 +mike ellison 1 +mike davidson 5 +mike davidson 5 +mike carson 9 +mike carson 4 +mike carson 3 +mike brown 2 +mike allen 3 +luke zipper 2 +luke xylophone 1 +luke white 1 +luke van buren 2 +luke underhill 2 +luke underhill 2 +luke underhill 1 +luke thompson 3 +luke robinson 6 +luke robinson 1 +luke quirinius 3 +luke polk 3 +luke polk 1 +luke ovid 3 +luke ovid 1 +luke miller 4 +luke laertes 4 +luke laertes 2 +luke laertes 2 +luke laertes 2 +luke laertes 1 +luke johnson 4 +luke johnson 2 +luke johnson 1 +luke ichabod 4 +luke ichabod 1 +luke garcia 5 +luke garcia 2 +luke falkner 4 +luke falkner 2 +luke ellison 3 +luke ellison 2 +luke ellison 1 +luke davidson 2 +luke davidson 2 +luke brown 5 +luke allen 5 +luke allen 2 +luke allen 1 +luke allen 1 +luke allen 1 +katie zipper 1 +katie zipper 1 +katie young 11 +katie young 6 +katie young 1 +katie xylophone 1 +katie white 5 +katie white 3 +katie van buren 6 +katie van buren 4 +katie robinson 2 +katie polk 5 +katie polk 2 +katie ovid 3 +katie nixon 1 +katie miller 1 +katie miller 1 +katie king 7 +katie king 5 +katie king 4 +katie ichabod 6 +katie ichabod 2 +katie ichabod 1 +katie hernandez 1 +katie garcia 4 +katie garcia 3 +katie falkner 4 +katie ellison 5 +katie ellison 4 +katie davidson 1 +katie brown 6 +katie allen 1 +jessica zipper 7 +jessica zipper 6 +jessica zipper 1 +jessica young 4 +jessica young 3 +jessica xylophone 3 +jessica white 8 +jessica white 6 +jessica white 3 +jessica white 1 +jessica white 1 +jessica van buren 1 +jessica underhill 5 +jessica underhill 3 +jessica underhill 2 +jessica thompson 3 +jessica thompson 2 +jessica robinson 2 +jessica quirinius 4 +jessica quirinius 4 +jessica quirinius 3 +jessica quirinius 1 +jessica polk 4 +jessica ovid 2 +jessica ovid 1 +jessica nixon 3 +jessica nixon 2 +jessica miller 5 +jessica johnson 4 +jessica johnson 3 +jessica ichabod 5 +jessica garcia 4 +jessica garcia 3 +jessica falkner 2 +jessica ellison 5 +jessica ellison 2 +jessica davidson 5 +jessica davidson 2 +jessica davidson 2 +jessica davidson 1 +jessica carson 4 +jessica carson 2 +jessica carson 1 +jessica brown 3 +irene xylophone 3 +irene van buren 2 +irene van buren 1 +irene underhill 5 +irene underhill 1 +irene thompson 6 +irene steinbeck 1 +irene robinson 1 +irene quirinius 6 +irene quirinius 5 +irene quirinius 5 +irene polk 3 +irene polk 2 +irene polk 2 +irene polk 1 +irene polk 1 +irene ovid 6 +irene ovid 6 +irene ovid 5 +irene nixon 4 +irene nixon 4 +irene nixon 1 +irene miller 6 +irene laertes 5 +irene laertes 3 +irene laertes 3 +irene johnson 2 +irene ichabod 7 +irene ichabod 1 +irene garcia 4 +irene garcia 2 +irene garcia 2 +irene falkner 5 +irene falkner 2 +irene ellison 4 +irene ellison 3 +irene carson 1 +irene brown 4 +irene brown 4 +irene brown 1 +irene allen 2 +holly zipper 3 +holly zipper 3 +holly young 2 +holly young 2 +holly xylophone 1 +holly white 3 +holly white 1 +holly van buren 4 +holly underhill 6 +holly underhill 3 +holly underhill 3 +holly underhill 2 +holly thompson 2 +holly thompson 1 +holly thompson 1 +holly robinson 2 +holly polk 7 +holly polk 4 +holly nixon 5 +holly nixon 1 +holly miller 4 +holly laertes 5 +holly king 4 +holly king 1 +holly johnson 2 +holly johnson 2 +holly johnson 2 +holly ichabod 4 +holly ichabod 4 +holly ichabod 2 +holly hernandez 9 +holly hernandez 3 +holly hernandez 3 +holly hernandez 2 +holly falkner 6 +holly brown 3 +holly brown 2 +holly allen 1 +gabriella zipper 5 +gabriella zipper 1 +gabriella young 3 +gabriella young 1 +gabriella white 3 +gabriella van buren 3 +gabriella van buren 1 +gabriella thompson 5 +gabriella thompson 5 +gabriella thompson 5 +gabriella steinbeck 4 +gabriella steinbeck 1 +gabriella polk 4 +gabriella polk 4 +gabriella ovid 2 +gabriella ovid 1 +gabriella miller 1 +gabriella laertes 4 +gabriella king 3 +gabriella king 3 +gabriella ichabod 3 +gabriella ichabod 3 +gabriella ichabod 3 +gabriella ichabod 2 +gabriella ichabod 1 +gabriella hernandez 9 +gabriella hernandez 5 +gabriella garcia 2 +gabriella falkner 4 +gabriella falkner 3 +gabriella falkner 2 +gabriella ellison 3 +gabriella ellison 1 +gabriella davidson 2 +gabriella carson 1 +gabriella brown 6 +gabriella brown 3 +gabriella allen 5 +gabriella allen 3 +fred zipper 1 +fred young 2 +fred young 1 +fred white 3 +fred van buren 7 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred underhill 4 +fred steinbeck 4 +fred steinbeck 2 +fred steinbeck 1 +fred robinson 3 +fred quirinius 7 +fred quirinius 4 +fred polk 7 +fred polk 6 +fred polk 4 +fred polk 2 +fred nixon 7 +fred nixon 5 +fred nixon 1 +fred nixon 1 +fred miller 1 +fred laertes 4 +fred king 6 +fred king 3 +fred johnson 4 +fred ichabod 3 +fred ichabod 2 +fred hernandez 1 +fred falkner 4 +fred falkner 3 +fred falkner 3 +fred ellison 5 +fred ellison 2 +fred ellison 1 +fred davidson 2 +fred davidson 2 +fred davidson 1 +ethan zipper 2 +ethan zipper 1 +ethan xylophone 3 +ethan white 5 +ethan white 2 +ethan van buren 1 +ethan underhill 1 +ethan robinson 3 +ethan robinson 1 +ethan quirinius 6 +ethan quirinius 2 +ethan quirinius 1 +ethan polk 3 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan ovid 2 +ethan nixon 7 +ethan miller 5 +ethan laertes 4 +ethan laertes 4 +ethan laertes 3 +ethan laertes 2 +ethan laertes 2 +ethan laertes 2 +ethan laertes 1 +ethan king 1 +ethan johnson 1 +ethan hernandez 3 +ethan garcia 8 +ethan falkner 2 +ethan falkner 1 +ethan ellison 6 +ethan ellison 4 +ethan carson 6 +ethan brown 4 +ethan brown 3 +ethan brown 3 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan allen 4 +david young 4 +david young 1 +david xylophone 6 +david xylophone 4 +david xylophone 1 +david white 2 +david van buren 3 +david van buren 2 +david underhill 7 +david underhill 4 +david underhill 1 +david thompson 1 +david robinson 3 +david robinson 2 +david quirinius 4 +david quirinius 4 +david quirinius 2 +david ovid 4 +david ovid 3 +david nixon 1 +david laertes 4 +david ichabod 6 +david ichabod 3 +david hernandez 7 +david ellison 5 +david ellison 3 +david ellison 3 +david davidson 4 +david davidson 3 +david davidson 1 +david davidson 1 +david brown 6 +david brown 2 +david allen 5 +david allen 2 +calvin zipper 9 +calvin zipper 3 +calvin young 3 +calvin young 1 +calvin xylophone 6 +calvin xylophone 3 +calvin xylophone 1 +calvin white 1 +calvin white 1 +calvin van buren 9 +calvin van buren 1 +calvin underhill 4 +calvin thompson 3 +calvin thompson 2 +calvin steinbeck 6 +calvin steinbeck 3 +calvin steinbeck 3 +calvin robinson 2 +calvin quirinius 4 +calvin quirinius 3 +calvin polk 2 +calvin ovid 5 +calvin ovid 4 +calvin ovid 3 +calvin ovid 1 +calvin nixon 7 +calvin nixon 3 +calvin nixon 2 +calvin laertes 3 +calvin laertes 1 +calvin johnson 2 +calvin hernandez 1 +calvin garcia 3 +calvin falkner 8 +calvin falkner 4 +calvin falkner 4 +calvin falkner 3 +calvin falkner 2 +calvin falkner 1 +calvin ellison 3 +calvin davidson 1 +calvin davidson 1 +calvin carson 1 +calvin brown 5 +calvin brown 3 +calvin brown 1 +calvin allen 1 +bob zipper 4 +bob zipper 1 +bob zipper 1 +bob young 1 +bob xylophone 3 +bob xylophone 2 +bob white 3 +bob white 1 +bob van buren 3 +bob steinbeck 2 +bob quirinius 4 +bob polk 2 +bob ovid 7 +bob ovid 2 +bob ovid 2 +bob ovid 1 +bob miller 1 +bob laertes 5 +bob laertes 1 +bob king 3 +bob king 3 +bob king 2 +bob ichabod 1 +bob hernandez 1 +bob garcia 4 +bob garcia 3 +bob garcia 2 +bob garcia 1 +bob garcia 1 +bob falkner 6 +bob ellison 3 +bob ellison 2 +bob ellison 1 +bob ellison 1 +bob davidson 5 +bob davidson 2 +bob davidson 2 +bob carson 3 +bob brown 8 +bob brown 6 +bob brown 2 +alice zipper 2 +alice zipper 1 +alice zipper 1 +alice xylophone 2 +alice xylophone 2 +alice xylophone 1 +alice van buren 2 +alice underhill 2 +alice steinbeck 7 +alice steinbeck 3 +alice steinbeck 1 +alice robinson 4 +alice robinson 1 +alice quirinius 6 +alice quirinius 4 +alice polk 1 +alice ovid 2 +alice nixon 2 +alice nixon 2 +alice nixon 1 +alice miller 2 +alice laertes 3 +alice laertes 2 +alice king 8 +alice king 4 +alice king 2 +alice johnson 5 +alice hernandez 8 +alice hernandez 8 +alice garcia 1 +alice falkner 5 +alice davidson 2 +alice carson 1 +alice brown 5 +alice allen 5 +alice allen 5 +alice allen 4 + 5 + 4 + 3 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-2-ac487cc1b94130bf9ce00e07c7075f65 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-2-ac487cc1b94130bf9ce00e07c7075f65 new file mode 100644 index 0000000000000000000000000000000000000000..c38e7bbabc21e78cc50f16b465ea552b44b98f50 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-2-ac487cc1b94130bf9ce00e07c7075f65 @@ -0,0 +1,1049 @@ + 0.25047801147227533 + 0.47992351816443596 + 0.6197718631178707 +alice allen 0.7300380228136882 +alice allen 0.8954372623574145 +alice allen 0.9216061185468452 +alice brown 0.22053231939163498 +alice carson 0.2889733840304182 +alice davidson 0.8593155893536122 +alice falkner 0.08604206500956023 +alice garcia 0.2870722433460076 +alice hernandez 0.011472275334608031 +alice hernandez 0.07604562737642585 +alice johnson 0.5181644359464627 +alice king 0.3652007648183556 +alice king 0.8536121673003803 +alice king 0.9771863117870723 +alice laertes 0.870722433460076 +alice laertes 0.870722433460076 +alice miller 0.12045889101338432 +alice nixon 0.4372623574144487 +alice nixon 0.47036328871892924 +alice nixon 0.768642447418738 +alice ovid 0.4665391969407266 +alice polk 0.279467680608365 +alice quirinius 0.8432122370936902 +alice quirinius 0.9923518164435946 +alice robinson 0.5722433460076045 +alice robinson 0.7984790874524715 +alice steinbeck 0.27151051625239003 +alice steinbeck 0.739961759082218 +alice steinbeck 0.9923954372623575 +alice underhill 0.5513307984790875 +alice van buren 0.4923954372623574 +alice xylophone 0.2376425855513308 +alice xylophone 0.26806083650190116 +alice xylophone 0.8776290630975143 +alice zipper 0.33460076045627374 +alice zipper 0.8814531548757171 +alice zipper 0.9445506692160612 +bob brown 0.5038022813688213 +bob brown 0.5066921606118547 +bob brown 0.5372848948374761 +bob carson 0.43346007604562736 +bob davidson 0.21673003802281368 +bob davidson 0.5285171102661597 +bob davidson 0.8413001912045889 +bob ellison 0.2045889101338432 +bob ellison 0.26577437858508607 +bob ellison 0.5793499043977055 +bob ellison 0.9144486692015209 +bob falkner 0.6940726577437859 +bob garcia 0.08555133079847908 +bob garcia 0.17680608365019013 +bob garcia 0.2887189292543021 +bob garcia 0.5418250950570342 +bob garcia 0.5736137667304015 +bob hernandez 0.7813688212927756 +bob ichabod 0.5200764818355641 +bob king 0.0076481835564053535 +bob king 0.5627376425855514 +bob king 0.9524714828897338 +bob laertes 0.32887189292543023 +bob laertes 0.6825095057034221 +bob miller 0.19771863117870722 +bob ovid 0.40304182509505704 +bob ovid 0.40344168260038243 +bob ovid 0.42065009560229444 +bob ovid 0.8403041825095057 +bob polk 0.15019011406844107 +bob quirinius 0.1844106463878327 +bob steinbeck 0.16920152091254753 +bob van buren 0.5086042065009561 +bob white 0.26045627376425856 +bob white 0.7623574144486692 +bob xylophone 0.4474187380497132 +bob xylophone 0.6539923954372624 +bob young 0.4722753346080306 +bob zipper 0.009505703422053232 +bob zipper 0.24091778202676864 +bob zipper 0.4600760456273764 +calvin allen 0.30975143403441685 +calvin brown 0.4448669201520912 +calvin brown 0.5361216730038023 +calvin brown 0.9196940726577438 +calvin carson 0.9315589353612167 +calvin davidson 0.5869980879541109 +calvin davidson 0.6653992395437263 +calvin ellison 0.6977186311787072 +calvin falkner 0.02091254752851711 +calvin falkner 0.03824091778202677 +calvin falkner 0.21223709369024857 +calvin falkner 0.46577946768060835 +calvin falkner 0.5114068441064639 +calvin falkner 0.5950570342205324 +calvin garcia 0.7896749521988528 +calvin hernandez 0.16730038022813687 +calvin johnson 0.9790874524714829 +calvin laertes 0.5487571701720841 +calvin laertes 0.8145315487571702 +calvin nixon 0.019120458891013385 +calvin nixon 0.4467680608365019 +calvin nixon 0.7395437262357415 +calvin ovid 0.14531548757170173 +calvin ovid 0.17490494296577946 +calvin ovid 0.19961977186311788 +calvin ovid 0.9407265774378585 +calvin polk 0.4619771863117871 +calvin quirinius 0.8802281368821293 +calvin quirinius 0.9254302103250478 +calvin robinson 0.13193116634799235 +calvin steinbeck 0.4818355640535373 +calvin steinbeck 0.7418738049713193 +calvin steinbeck 0.8060836501901141 +calvin thompson 0.2179732313575526 +calvin thompson 0.8422053231939164 +calvin underhill 0.7495219885277247 +calvin van buren 0.022813688212927757 +calvin van buren 0.8508604206500956 +calvin white 0.04182509505703422 +calvin white 0.9674952198852772 +calvin xylophone 0.011406844106463879 +calvin xylophone 0.3193116634799235 +calvin xylophone 0.6634799235181644 +calvin young 0.1988527724665392 +calvin young 0.4391634980988593 +calvin zipper 0.5532319391634981 +calvin zipper 0.8726235741444867 +david allen 0.30019120458891013 +david allen 0.3326959847036329 +david brown 0.1338432122370937 +david brown 0.9694072657743786 +david davidson 0.21414913957934992 +david davidson 0.655893536121673 +david davidson 0.7319391634980988 +david davidson 0.8878326996197718 +david ellison 0.6863117870722434 +david ellison 0.6883365200764818 +david ellison 0.7243346007604563 +david hernandez 0.12237093690248566 +david ichabod 0.35564053537284895 +david ichabod 0.7338403041825095 +david laertes 0.3575525812619503 +david nixon 0.33460803059273425 +david ovid 0.3916349809885932 +david ovid 0.6022944550669216 +david quirinius 0.3155893536121673 +david quirinius 0.6577437858508605 +david quirinius 0.9163498098859315 +david robinson 0.6673003802281369 +david robinson 0.6998087954110899 +david thompson 0.25285171102661597 +david underhill 0.1586998087954111 +david underhill 0.35181644359464626 +david underhill 0.7189292543021033 +david van buren 0.05927342256214149 +david van buren 0.5889101338432122 +david white 0.49429657794676807 +david xylophone 0.4875717017208413 +david xylophone 0.6901140684410646 +david xylophone 0.7571701720841301 +david young 0.0019011406844106464 +david young 0.040152963671128104 +ethan allen 0.20532319391634982 +ethan brown 0.10707456978967496 +ethan brown 0.13307984790874525 +ethan brown 0.4340344168260038 +ethan brown 0.4752851711026616 +ethan brown 0.5219885277246654 +ethan brown 0.745697896749522 +ethan carson 0.20912547528517111 +ethan ellison 0.45124282982791586 +ethan ellison 0.8680688336520076 +ethan falkner 0.0994263862332696 +ethan falkner 0.6845124282982792 +ethan garcia 0.06653992395437262 +ethan hernandez 0.2237093690248566 +ethan johnson 0.2300380228136882 +ethan king 0.47418738049713194 +ethan laertes 0.022944550669216062 +ethan laertes 0.2908745247148289 +ethan laertes 0.42638623326959846 +ethan laertes 0.48098859315589354 +ethan laertes 0.6596558317399618 +ethan laertes 0.7839388145315488 +ethan laertes 0.9201520912547528 +ethan miller 0.23709369024856597 +ethan nixon 0.8164435946462715 +ethan ovid 0.6121673003802282 +ethan polk 0.12167300380228137 +ethan polk 0.3384321223709369 +ethan polk 0.6920152091254753 +ethan polk 0.9619771863117871 +ethan quirinius 0.19391634980988592 +ethan quirinius 0.23135755258126195 +ethan quirinius 0.7908745247148289 +ethan robinson 0.24282982791587 +ethan robinson 0.8003802281368821 +ethan underhill 0.6615969581749049 +ethan van buren 0.8365019011406845 +ethan white 0.48859315589353614 +ethan white 0.5741444866920152 +ethan xylophone 0.9695817490494296 +ethan zipper 0.21102661596958175 +ethan zipper 0.6425855513307985 +fred davidson 0.5239005736137667 +fred davidson 0.7414448669201521 +fred davidson 0.8604206500956023 +fred ellison 0.3977055449330784 +fred ellison 0.5506692160611855 +fred ellison 0.7208413001912046 +fred falkner 0.024714828897338403 +fred falkner 0.19120458891013384 +fred falkner 0.9809885931558935 +fred hernandez 0.2734225621414914 +fred ichabod 0.17110266159695817 +fred ichabod 0.780114722753346 +fred johnson 0.30038022813688214 +fred king 0.2198852772466539 +fred king 0.47718631178707227 +fred laertes 0.2332695984703633 +fred miller 0.7858508604206501 +fred nixon 0.005703422053231939 +fred nixon 0.31749049429657794 +fred nixon 0.7648183556405354 +fred nixon 0.8460076045627376 +fred polk 0.16252390057361377 +fred polk 0.564638783269962 +fred polk 0.6273764258555133 +fred polk 0.8155893536121673 +fred quirinius 0.4866920152091255 +fred quirinius 0.8973384030418251 +fred robinson 0.6387832699619772 +fred steinbeck 0.14722753346080306 +fred steinbeck 0.4627151051625239 +fred steinbeck 0.7265774378585086 +fred underhill 0.35361216730038025 +fred van buren 0.3365200764818356 +fred van buren 0.5057034220532319 +fred van buren 0.6463878326996197 +fred van buren 0.904397705544933 +fred white 0.5171102661596958 +fred young 0.7705544933078394 +fred young 0.7992351816443595 +fred zipper 0.615678776290631 +gabriella allen 0.4435946462715105 +gabriella allen 0.9334600760456274 +gabriella brown 0.4359464627151052 +gabriella brown 0.9636711281070746 +gabriella carson 0.9562737642585551 +gabriella davidson 0.8174904942965779 +gabriella ellison 0.1931166347992352 +gabriella ellison 0.38022813688212925 +gabriella falkner 0.3231939163498099 +gabriella falkner 0.5659655831739961 +gabriella falkner 0.8948374760994264 +gabriella garcia 0.4695817490494297 +gabriella hernandez 0.6444866920152091 +gabriella hernandez 0.7015209125475285 +gabriella ichabod 0.09125475285171103 +gabriella ichabod 0.1520912547528517 +gabriella ichabod 0.1835564053537285 +gabriella ichabod 0.372848948374761 +gabriella ichabod 0.8107074569789675 +gabriella king 0.39961759082217974 +gabriella king 0.5190114068441065 +gabriella laertes 0.4569789674952199 +gabriella miller 0.26996197718631176 +gabriella ovid 0.7091254752851711 +gabriella ovid 0.8897338403041825 +gabriella polk 0.030418250950570342 +gabriella polk 0.44106463878326996 +gabriella steinbeck 0.5755258126195029 +gabriella steinbeck 0.8221797323135756 +gabriella thompson 0.013307984790874524 +gabriella thompson 0.44866920152091255 +gabriella thompson 0.7224334600760456 +gabriella van buren 0.6216730038022814 +gabriella van buren 0.6730038022813688 +gabriella white 0.17208413001912046 +gabriella young 0.5076045627376425 +gabriella young 0.7934990439770554 +gabriella zipper 0.23193916349809887 +gabriella zipper 0.8565965583173997 +holly allen 0.11596958174904944 +holly brown 0.11281070745697896 +holly brown 0.155893536121673 +holly falkner 0.124282982791587 +holly hernandez 0.055449330783938815 +holly hernandez 0.32509505703422054 +holly hernandez 0.97131931166348 +holly hernandez 0.9714828897338403 +holly ichabod 0.12357414448669202 +holly ichabod 0.17300380228136883 +holly ichabod 0.629277566539924 +holly johnson 0.33078393881453155 +holly johnson 0.8612167300380228 +holly johnson 0.9391634980988594 +holly king 0.25475285171102663 +holly king 0.3745247148288973 +holly laertes 0.42775665399239543 +holly miller 0.37476099426386233 +holly nixon 0.10076045627376426 +holly nixon 0.34608030592734224 +holly polk 0.40535372848948376 +holly polk 0.5209125475285171 +holly robinson 0.9273422562141491 +holly thompson 0.1596958174904943 +holly thompson 0.311787072243346 +holly thompson 0.9125475285171103 +holly underhill 0.3479923518164436 +holly underhill 0.5812619502868069 +holly underhill 0.8384030418250951 +holly underhill 0.903041825095057 +holly van buren 0.9464627151051626 +holly white 0.1089866156787763 +holly white 0.4780114722753346 +holly xylophone 0.5304182509505704 +holly young 0.7357414448669202 +holly young 0.8240917782026769 +holly zipper 0.15399239543726237 +holly zipper 0.8546845124282982 +irene allen 0.8738049713193117 +irene brown 0.4588910133843212 +irene brown 0.49619771863117873 +irene brown 0.5678776290630975 +irene carson 0.6844106463878327 +irene ellison 0.32504780114722753 +irene ellison 0.48565965583174 +irene falkner 0.41825095057034223 +irene falkner 0.9866920152091255 +irene garcia 0.11663479923518165 +irene garcia 0.29277566539923955 +irene garcia 0.8126195028680688 +irene ichabod 0.8307984790874525 +irene ichabod 0.9177820267686424 +irene johnson 0.7112810707456979 +irene laertes 0.01338432122370937 +irene laertes 0.1482889733840304 +irene laertes 0.7034220532319392 +irene miller 0.367112810707457 +irene nixon 0.11854684512428298 +irene nixon 0.7927756653992395 +irene nixon 0.9426386233269598 +irene ovid 0.24714828897338403 +irene ovid 0.30210325047801145 +irene ovid 0.779467680608365 +irene polk 0.0038022813688212928 +irene polk 0.45315487571701724 +irene polk 0.6577946768060836 +irene polk 0.8891013384321224 +irene polk 0.9789674952198852 +irene quirinius 0.27533460803059273 +irene quirinius 0.35946462715105165 +irene quirinius 0.384321223709369 +irene robinson 0.18631178707224336 +irene steinbeck 0.9942965779467681 +irene thompson 0.6939163498098859 +irene underhill 0.30401529636711283 +irene underhill 0.3403041825095057 +irene van buren 0.5908221797323135 +irene van buren 0.6634980988593155 +irene xylophone 0.5342205323193916 +jessica brown 0.7680608365019012 +jessica carson 0.3574144486692015 +jessica carson 0.6195028680688337 +jessica carson 0.8269961977186312 +jessica davidson 0.10646387832699619 +jessica davidson 0.34790874524714827 +jessica davidson 0.3593155893536122 +jessica davidson 0.6768060836501901 +jessica ellison 0.0779467680608365 +jessica ellison 0.42015209125475284 +jessica falkner 0.994263862332696 +jessica garcia 0.8279158699808795 +jessica garcia 0.9581749049429658 +jessica ichabod 0.45627376425855515 +jessica johnson 0.30228136882129275 +jessica johnson 0.8049713193116634 +jessica miller 0.8011472275334608 +jessica nixon 0.06500956022944551 +jessica nixon 0.6042065009560229 +jessica ovid 0.15105162523900573 +jessica ovid 0.8992395437262357 +jessica polk 0.4378585086042065 +jessica quirinius 0.058935361216730035 +jessica quirinius 0.4714828897338403 +jessica quirinius 0.5760456273764258 +jessica quirinius 0.8935361216730038 +jessica robinson 0.9638783269961977 +jessica thompson 0.08221797323135756 +jessica thompson 0.5893536121673004 +jessica underhill 0.034220532319391636 +jessica underhill 0.06118546845124283 +jessica underhill 0.9541108986615678 +jessica van buren 0.20650095602294455 +jessica white 0.06273764258555133 +jessica white 0.4149139579349904 +jessica white 0.5798479087452472 +jessica white 0.591254752851711 +jessica white 0.7667304015296367 +jessica xylophone 0.5009560229445507 +jessica young 0.3403441682600382 +jessica young 0.8821292775665399 +jessica zipper 0.14068441064638784 +jessica zipper 0.2984790874524715 +jessica zipper 0.6007604562737643 +katie allen 0.5665399239543726 +katie brown 0.49521988527724664 +katie davidson 0.6730401529636711 +katie ellison 0.3173996175908222 +katie ellison 0.7262357414448669 +katie falkner 0.2676864244741874 +katie garcia 0.049429657794676805 +katie garcia 0.3135755258126195 +katie hernandez 0.6026615969581749 +katie ichabod 0.15296367112810708 +katie ichabod 0.4684512428298279 +katie ichabod 0.7055449330783938 +katie king 0.16159695817490494 +katie king 0.502868068833652 +katie king 0.5927342256214149 +katie miller 0.5228136882129277 +katie miller 0.5296367112810707 +katie nixon 0.7832699619771863 +katie ovid 0.8795411089866156 +katie polk 0.35372848948374763 +katie polk 0.9657794676806084 +katie robinson 0.06844106463878327 +katie van buren 0.06883365200764818 +katie van buren 0.1739961759082218 +katie white 0.045889101338432124 +katie white 0.18546845124282982 +katie xylophone 0.7281368821292775 +katie young 0.16443594646271512 +katie young 0.20152091254752852 +katie young 0.9732313575525813 +katie zipper 0.21863117870722434 +katie zipper 0.4505703422053232 +luke allen 0.03612167300380228 +luke allen 0.21606118546845124 +luke allen 0.8346007604562737 +luke allen 0.8631178707224335 +luke allen 0.9311663479923518 +luke brown 0.7304015296367112 +luke davidson 0.25239005736137665 +luke davidson 0.9961977186311787 +luke ellison 0.1147227533460803 +luke ellison 0.2447418738049713 +luke ellison 0.49809885931558934 +luke falkner 0.24524714828897337 +luke falkner 0.5124282982791587 +luke garcia 0.03441682600382409 +luke garcia 0.32695984703632885 +luke ichabod 0.10266159695817491 +luke ichabod 0.5551330798479087 +luke johnson 0.25430210325047803 +luke johnson 0.6787762906309751 +luke johnson 0.9082217973231358 +luke laertes 0.06309751434034416 +luke laertes 0.3690248565965583 +luke laertes 0.7743785850860421 +luke laertes 0.8079847908745247 +luke laertes 0.811787072243346 +luke miller 0.8068833652007649 +luke ovid 0.435361216730038 +luke ovid 0.7547528517110266 +luke polk 0.13957934990439771 +luke polk 0.9770554493307839 +luke quirinius 0.09315589353612168 +luke robinson 0.015209125475285171 +luke robinson 0.053231939163498096 +luke thompson 0.8840304182509505 +luke underhill 0.08745247148288973 +luke underhill 0.40152963671128106 +luke underhill 0.4608030592734226 +luke van buren 0.4847908745247148 +luke white 0.8098859315589354 +luke xylophone 0.34220532319391633 +luke zipper 0.21292775665399238 +mike allen 0.7036328871892925 +mike brown 0.29063097514340347 +mike carson 0.623574144486692 +mike carson 0.7476099426386233 +mike carson 0.9885931558935361 +mike davidson 0.6520912547528517 +mike davidson 0.8298279158699808 +mike ellison 0.24665391969407266 +mike ellison 0.3821292775665399 +mike ellison 0.8355640535372849 +mike ellison 0.8986615678776291 +mike ellison 0.94106463878327 +mike falkner 0.0248565965583174 +mike garcia 0.39543726235741444 +mike garcia 0.5391969407265774 +mike garcia 0.6482889733840305 +mike hernandez 0.07984790874524715 +mike hernandez 0.7186311787072244 +mike ichabod 0.7642585551330798 +mike king 0.09695817490494296 +mike king 0.188212927756654 +mike king 0.4049429657794677 +mike king 0.5544933078393881 +mike king 0.6045627376425855 +mike king 0.9011406844106464 +mike miller 0.621414913957935 +mike nixon 0.688212927756654 +mike nixon 0.9068441064638784 +mike polk 0.3612167300380228 +mike polk 0.6749521988527725 +mike polk 0.8374760994263862 +mike quirinius 0.5105162523900574 +mike steinbeck 0.05736137667304015 +mike steinbeck 0.747148288973384 +mike steinbeck 0.8745247148288974 +mike steinbeck 0.9330783938814532 +mike van buren 0.8650190114068441 +mike van buren 0.973384030418251 +mike white 0.17782026768642448 +mike white 0.7151051625239006 +mike white 0.7566539923954373 +mike white 0.9808795411089866 +mike young 0.20722433460076045 +mike young 0.3840304182509506 +mike young 0.6405353728489483 +mike zipper 0.12810707456978968 +mike zipper 0.42829827915869984 +mike zipper 0.7946768060836502 +nick allen 0.021032504780114723 +nick allen 0.847036328871893 +nick brown 0.14258555133079848 +nick davidson 0.26003824091778205 +nick ellison 0.028680688336520075 +nick ellison 0.3935361216730038 +nick falkner 0.5684410646387833 +nick falkner 0.7590822179732314 +nick garcia 0.34980988593155893 +nick garcia 0.45817490494296575 +nick garcia 0.892925430210325 +nick ichabod 0.2944550669216061 +nick ichabod 0.37667304015296366 +nick ichabod 0.7074569789674953 +nick johnson 0.3973384030418251 +nick johnson 0.4646271510516252 +nick laertes 0.36311787072243346 +nick miller 0.9961759082217974 +nick nixon 0.7110266159695817 +nick ovid 0.7762906309751434 +nick polk 1.0 +nick quirinius 0.0019120458891013384 +nick quirinius 0.08795411089866156 +nick robinson 0.09505703422053231 +nick robinson 0.45506692160611856 +nick steinbeck 0.2224334600760456 +nick thompson 0.4225621414913958 +nick underhill 0.9101338432122371 +nick van buren 0.03802281368821293 +nick xylophone 0.6806883365200764 +nick young 0.4220532319391635 +nick young 0.8623326959847036 +nick zipper 0.2829827915869981 +nick zipper 0.5468451242829828 +oscar allen 0.785171102661597 +oscar brown 0.13498098859315588 +oscar carson 0.07224334600760456 +oscar carson 0.25665399239543724 +oscar carson 0.3422562141491396 +oscar carson 0.6061185468451242 +oscar carson 0.6826003824091779 +oscar davidson 0.7129277566539924 +oscar ellison 0.036328871892925434 +oscar ellison 0.5831739961759083 +oscar falkner 0.9049429657794676 +oscar garcia 0.02676864244741874 +oscar hernandez 0.20076481835564053 +oscar hernandez 0.7870722433460076 +oscar ichabod 0.12619502868068833 +oscar ichabod 0.14149139579349904 +oscar ichabod 0.4416826003824092 +oscar ichabod 0.8661567877629063 +oscar johnson 0.1806083650190114 +oscar johnson 0.467680608365019 +oscar king 0.6596958174904943 +oscar king 0.6787072243346007 +oscar king 0.9258555133079848 +oscar laertes 0.24904942965779467 +oscar laertes 0.5315487571701721 +oscar laertes 0.6328871892925431 +oscar laertes 0.9980988593155894 +oscar nixon 0.9292543021032504 +oscar ovid 0.43021032504780116 +oscar ovid 0.8288973384030418 +oscar ovid 0.8527724665391969 +oscar polk 0.10836501901140684 +oscar polk 0.37858508604206503 +oscar quirinius 0.3041825095057034 +oscar quirinius 0.46387832699619774 +oscar quirinius 0.6311787072243346 +oscar quirinius 0.8555133079847909 +oscar robinson 0.11216730038022814 +oscar robinson 0.22433460076045628 +oscar robinson 0.2294455066921606 +oscar robinson 0.2390057361376673 +oscar steinbeck 0.9904942965779467 +oscar thompson 0.015296367112810707 +oscar thompson 0.2946768060836502 +oscar thompson 0.3060836501901141 +oscar thompson 0.6140684410646388 +oscar underhill 0.31368821292775667 +oscar van buren 0.722753346080306 +oscar van buren 0.7889733840304183 +oscar van buren 0.8833652007648184 +oscar white 0.055133079847908745 +oscar white 0.22562141491395793 +oscar white 0.4321223709369025 +oscar white 0.6443594646271511 +oscar xylophone 0.10133843212237094 +oscar xylophone 0.4187380497131931 +oscar xylophone 0.4296577946768061 +oscar zipper 0.6233269598470363 +oscar zipper 0.7490494296577946 +oscar zipper 0.8783269961977186 +priscilla brown 0.2925430210325048 +priscilla brown 0.6501901140684411 +priscilla brown 0.9120458891013384 +priscilla carson 0.22753346080305928 +priscilla carson 0.5564053537284895 +priscilla carson 0.7820267686424475 +priscilla ichabod 0.3269961977186312 +priscilla ichabod 0.9828897338403042 +priscilla johnson 0.04206500956022945 +priscilla johnson 0.4011406844106464 +priscilla johnson 0.6368821292775665 +priscilla johnson 0.7131931166347992 +priscilla johnson 0.9429657794676806 +priscilla king 0.3517110266159696 +priscilla nixon 0.38049713193116635 +priscilla nixon 0.6864244741873805 +priscilla ovid 0.8193916349809885 +priscilla ovid 0.9139579349904398 +priscilla polk 0.5697896749521989 +priscilla quirinius 0.22179732313575526 +priscilla thompson 0.7737642585551331 +priscilla underhill 0.1682600382409178 +priscilla underhill 0.8852772466539197 +priscilla van buren 0.10325047801147227 +priscilla van buren 0.7877629063097514 +priscilla van buren 0.9598470363288719 +priscilla white 0.4894837476099426 +priscilla xylophone 0.596958174904943 +priscilla xylophone 0.6159695817490495 +priscilla xylophone 0.8393881453154876 +priscilla young 0.41064638783269963 +priscilla young 0.9182509505703422 +priscilla zipper 0.5247148288973384 +priscilla zipper 0.8574144486692015 +quinn allen 0.1634980988593156 +quinn allen 0.9617590822179732 +quinn brown 0.08986615678776291 +quinn brown 0.17590822179732313 +quinn brown 0.5836501901140685 +quinn davidson 0.11787072243346007 +quinn davidson 0.30592734225621415 +quinn davidson 0.3650190114068441 +quinn davidson 0.751434034416826 +quinn ellison 0.376425855513308 +quinn ellison 0.8517110266159695 +quinn garcia 0.17870722433460076 +quinn garcia 0.7323135755258127 +quinn garcia 0.844106463878327 +quinn garcia 0.9486692015209125 +quinn ichabod 0.42395437262357416 +quinn king 0.6653919694072657 +quinn king 0.9505703422053232 +quinn laertes 0.6080305927342257 +quinn laertes 0.9277566539923955 +quinn laertes 0.9847908745247148 +quinn nixon 0.5133079847908745 +quinn ovid 0.16539923954372623 +quinn quirinius 0.19011406844106463 +quinn robinson 0.27756653992395436 +quinn steinbeck 0.23954372623574144 +quinn steinbeck 0.6367112810707457 +quinn thompson 0.4068441064638783 +quinn thompson 0.7782026768642447 +quinn underhill 0.05353728489483748 +quinn underhill 0.5380228136882129 +quinn underhill 0.9349904397705545 +quinn van buren 0.2623574144486692 +quinn young 0.37832699619771865 +quinn zipper 0.51434034416826 +quinn zipper 0.8859315589353612 +rachel allen 0.1701720841300191 +rachel allen 0.3288973384030418 +rachel brown 0.04780114722753346 +rachel brown 0.057034220532319393 +rachel brown 0.532319391634981 +rachel brown 0.5946462715105163 +rachel brown 0.6064638783269962 +rachel carson 0.09177820267686425 +rachel carson 0.6406844106463878 +rachel davidson 0.37093690248565964 +rachel ellison 0.5162523900573613 +rachel falkner 0.1958174904942966 +rachel falkner 0.6330798479087453 +rachel falkner 0.6768642447418738 +rachel falkner 0.9751434034416826 +rachel johnson 0.9560229445506692 +rachel king 0.12547528517110265 +rachel king 0.6003824091778203 +rachel laertes 0.2638623326959847 +rachel laertes 0.5779467680608364 +rachel ovid 0.23518164435946462 +rachel ovid 0.7053231939163498 +rachel polk 0.14638783269961977 +rachel quirinius 0.0076045627376425855 +rachel robinson 0.14340344168260039 +rachel robinson 0.2084130019120459 +rachel robinson 0.6902485659655831 +rachel thompson 0.2718631178707224 +rachel thompson 0.5334608030592735 +rachel thompson 0.875717017208413 +rachel underhill 0.344106463878327 +rachel white 0.17973231357552583 +rachel white 0.6615678776290631 +rachel young 0.3862332695984704 +rachel zipper 0.33079847908745247 +rachel zipper 0.5717017208413002 +sarah carson 0.08935361216730038 +sarah carson 0.18250950570342206 +sarah carson 0.8041825095057035 +sarah ellison 0.967680608365019 +sarah falkner 0.6252390057361377 +sarah falkner 0.9024856596558317 +sarah garcia 0.3881453154875717 +sarah garcia 0.4072657743785851 +sarah garcia 0.6673040152963671 +sarah ichabod 0.29636711281070743 +sarah ichabod 0.9483747609942639 +sarah johnson 0.06463878326996197 +sarah johnson 0.10456273764258556 +sarah johnson 0.5640535372848948 +sarah johnson 0.7954110898661568 +sarah king 0.8030592734225621 +sarah king 0.9655831739961759 +sarah miller 0.6692160611854685 +sarah ovid 0.20342205323193915 +sarah robinson 0.47338403041825095 +sarah robinson 0.7775665399239544 +sarah steinbeck 0.6520076481835564 +sarah white 0.28517110266159695 +sarah white 0.8479087452471483 +sarah xylophone 0.25621414913957935 +sarah young 0.5570342205323194 +sarah zipper 0.5583173996175909 +tom brown 0.5602294455066922 +tom brown 0.8669201520912547 +tom carson 0.045627376425855515 +tom carson 0.35551330798479086 +tom carson 0.935361216730038 +tom davidson 0.8212927756653993 +tom ellison 0.21032504780114722 +tom ellison 0.26195028680688337 +tom ellison 0.7376425855513308 +tom falkner 0.3441682600382409 +tom falkner 0.6481835564053537 +tom hernandez 0.0038240917782026767 +tom hernandez 0.5399239543726235 +tom ichabod 0.6137667304015296 +tom johnson 0.5525812619502868 +tom johnson 0.7915869980879541 +tom king 0.16061185468451242 +tom laertes 0.0745697896749522 +tom laertes 0.5095057034220533 +tom miller 0.2262357414448669 +tom miller 0.2338403041825095 +tom miller 0.2813688212927757 +tom nixon 0.8451242829827916 +tom ovid 0.864244741873805 +tom polk 0.1491395793499044 +tom polk 0.9521988527724665 +tom quirinius 0.09369024856596558 +tom quirinius 0.8489483747609943 +tom robinson 0.060836501901140684 +tom robinson 0.6254752851711026 +tom robinson 0.6462715105162524 +tom robinson 0.9980879541108987 +tom steinbeck 0.5817490494296578 +tom van buren 0.12737642585551331 +tom van buren 0.3154875717017208 +tom van buren 0.7585551330798479 +tom white 0.47609942638623326 +tom young 0.9369024856596558 +tom young 0.9543726235741445 +tom zipper 0.9063097514340345 +ulysses brown 0.9448669201520913 +ulysses carson 0.07034220532319392 +ulysses carson 0.09885931558935361 +ulysses carson 0.2414448669201521 +ulysses carson 0.7604562737642585 +ulysses davidson 0.7093690248565966 +ulysses ellison 0.55893536121673 +ulysses garcia 0.7246653919694073 +ulysses hernandez 0.4091778202676864 +ulysses hernandez 0.627151051625239 +ulysses hernandez 0.982791586998088 +ulysses ichabod 0.21482889733840305 +ulysses ichabod 0.3193916349809886 +ulysses johnson 0.5621414913957935 +ulysses king 0.9467680608365019 +ulysses laertes 0.390057361376673 +ulysses laertes 0.7973231357552581 +ulysses laertes 0.9866156787762906 +ulysses miller 0.31166347992351817 +ulysses miller 0.5774378585086042 +ulysses nixon 0.0057361376673040155 +ulysses ovid 0.38593155893536124 +ulysses polk 0.04752851711026616 +ulysses polk 0.6083650190114068 +ulysses polk 0.7609942638623327 +ulysses polk 0.8326996197718631 +ulysses quirinius 0.6290630975143403 +ulysses robinson 0.9235181644359465 +ulysses steinbeck 0.039923954372623575 +ulysses steinbeck 0.7724665391969407 +ulysses thompson 0.3824091778202677 +ulysses underhill 0.11406844106463879 +ulysses underhill 0.23574144486692014 +ulysses underhill 0.3365019011406844 +ulysses underhill 0.42585551330798477 +ulysses underhill 0.6102661596958175 +ulysses underhill 0.6959847036328872 +ulysses underhill 0.9752851711026616 +ulysses van buren 0.5437262357414449 +ulysses white 0.5 +ulysses white 0.5931558935361216 +ulysses xylophone 0.5855513307984791 +ulysses xylophone 0.8317399617590823 +ulysses xylophone 0.9005736137667304 +ulysses young 0.18164435946462715 +ulysses young 0.3919694072657744 +ulysses young 0.49049429657794674 +victor allen 0.13575525812619502 +victor allen 0.6309751434034416 +victor brown 0.0497131931166348 +victor brown 0.20267686424474188 +victor brown 0.6178707224334601 +victor brown 0.8910133843212237 +victor davidson 0.026615969581749048 +victor davidson 0.491395793499044 +victor davidson 0.5850860420650096 +victor ellison 0.26425855513307983 +victor ellison 0.6692015209125475 +victor hernandez 0.04397705544933078 +victor hernandez 0.12927756653992395 +victor hernandez 0.1950286806883365 +victor hernandez 0.5411089866156787 +victor hernandez 0.7284894837476099 +victor johnson 0.11977186311787072 +victor johnson 0.4828897338403042 +victor johnson 0.7699619771863118 +victor king 0.41254752851711024 +victor king 0.714828897338403 +victor laertes 0.43155893536121676 +victor laertes 0.6500956022944551 +victor miller 0.4429657794676806 +victor nixon 0.33269961977186313 +victor nixon 0.5258126195028681 +victor ovid 0.22813688212927757 +victor polk 0.13878326996197718 +victor quirinius 0.13766730401529637 +victor quirinius 0.887189292543021 +victor robinson 0.5494296577946768 +victor robinson 0.7509505703422054 +victor steinbeck 0.08365019011406843 +victor steinbeck 0.15487571701720843 +victor steinbeck 0.3669201520912547 +victor thompson 0.10516252390057361 +victor van buren 0.27724665391969405 +victor van buren 0.9579349904397706 +victor white 0.41634980988593157 +victor white 0.6349809885931559 +victor xylophone 0.13688212927756654 +victor xylophone 0.3078393881453155 +victor xylophone 0.4110898661567878 +victor xylophone 0.5449330783938815 +victor xylophone 0.9296577946768061 +victor young 0.18738049713193117 +victor zipper 0.5430210325047801 +wendy allen 0.3231357552581262 +wendy allen 0.734225621414914 +wendy allen 0.869980879541109 +wendy brown 0.18929254302103252 +wendy brown 0.6996197718631179 +wendy ellison 0.7437858508604207 +wendy ellison 0.8498098859315589 +wendy falkner 0.07648183556405354 +wendy falkner 0.5353728489483748 +wendy falkner 0.7756653992395437 +wendy garcia 0.07074569789674952 +wendy garcia 0.0741444866920152 +wendy garcia 0.33840304182509506 +wendy garcia 0.38783269961977185 +wendy hernandez 0.017110266159695818 +wendy ichabod 0.8718929254302104 +wendy king 0.37072243346007605 +wendy king 0.497131931166348 +wendy king 0.5965583173996176 +wendy laertes 0.32122370936902483 +wendy laertes 0.49904397705544934 +wendy laertes 0.876425855513308 +wendy miller 0.7533460803059273 +wendy miller 0.7552581261950286 +wendy nixon 0.44933078393881454 +wendy nixon 0.7661596958174905 +wendy ovid 0.5019011406844106 +wendy ovid 0.6978967495219885 +wendy polk 0.3688212927756654 +wendy polk 0.526615969581749 +wendy quirinius 0.1444866920152091 +wendy quirinius 0.5874524714828897 +wendy robinson 0.030592734225621414 +wendy robinson 0.06692160611854685 +wendy robinson 0.27566539923954375 +wendy steinbeck 0.5703422053231939 +wendy thompson 0.028517110266159697 +wendy thompson 0.11089866156787763 +wendy underhill 0.4837476099426386 +wendy underhill 0.6424474187380497 +wendy underhill 0.9600760456273765 +wendy van buren 0.1920152091254753 +wendy van buren 0.7433460076045627 +wendy white 0.752851711026616 +wendy xylophone 0.6347992351816444 +wendy xylophone 0.7452471482889734 +wendy young 0.07839388145315487 +wendy young 0.3897338403041825 +xavier allen 0.043726235741444866 +xavier allen 0.361376673040153 +xavier allen 0.5456273764258555 +xavier brown 0.6711281070745698 +xavier brown 0.9158699808795411 +xavier brown 0.9847036328871893 +xavier carson 0.0841300191204589 +xavier carson 0.988527724665392 +xavier davidson 0.2585551330798479 +xavier davidson 0.4168260038240918 +xavier davidson 0.609942638623327 +xavier ellison 0.5984703632887189 +xavier ellison 0.7361376673040153 +xavier garcia 0.7017208413001912 +xavier hernandez 0.2509505703422053 +xavier hernandez 0.34990439770554493 +xavier hernandez 0.9220532319391636 +xavier ichabod 0.5475285171102662 +xavier ichabod 0.858508604206501 +xavier johnson 0.3938814531548757 +xavier johnson 0.8231939163498099 +xavier king 0.03231939163498099 +xavier king 0.6539196940726577 +xavier laertes 0.5988593155893536 +xavier ovid 0.4397705544933078 +xavier polk 0.4933078393881453 +xavier polk 0.762906309751434 +xavier polk 0.8136882129277566 +xavier polk 0.8260038240917782 +xavier quirinius 0.07265774378585087 +xavier quirinius 0.27915869980879543 +xavier quirinius 0.34600760456273766 +xavier quirinius 0.8022813688212928 +xavier thompson 0.6118546845124283 +xavier underhill 0.16634799235181644 +xavier white 0.6958174904942965 +xavier white 0.7380497131931166 +xavier xylophone 0.8183556405353728 +xavier zipper 0.9904397705544933 +yuri allen 0.9106463878326996 +yuri allen 1.0 +yuri brown 0.5152091254752852 +yuri brown 0.908745247148289 +yuri carson 0.09560229445506692 +yuri carson 0.9372623574144486 +yuri ellison 0.017208413001912046 +yuri ellison 0.39923954372623577 +yuri falkner 0.28680688336520077 +yuri falkner 0.8967495219885278 +yuri garcia 0.2661596958174905 +yuri hernandez 0.28489483747609945 +yuri johnson 0.5047801147227533 +yuri johnson 0.655831739961759 +yuri johnson 0.720532319391635 +yuri king 0.32129277566539927 +yuri laertes 0.4144486692015209 +yuri laertes 0.8916349809885932 +yuri nixon 0.05162523900573614 +yuri nixon 0.40874524714828897 +yuri polk 0.051330798479087454 +yuri polk 0.39579349904397704 +yuri polk 0.6749049429657795 +yuri quirinius 0.08030592734225621 +yuri quirinius 0.2982791586998088 +yuri quirinius 0.4130019120458891 +yuri steinbeck 0.15779467680608364 +yuri steinbeck 0.9388145315487572 +yuri thompson 0.6175908221797323 +yuri underhill 0.42447418738049714 +yuri underhill 0.8202676864244742 +yuri white 0.19694072657743786 +yuri xylophone 0.4790874524714829 +zach allen 0.8250950570342205 +zach brown 0.0817490494296578 +zach brown 0.09751434034416825 +zach brown 0.248565965583174 +zach brown 0.2965779467680608 +zach brown 0.4524714828897338 +zach carson 0.6921606118546845 +zach ellison 0.6806083650190115 +zach falkner 0.25812619502868067 +zach falkner 0.2695984703632887 +zach garcia 0.30798479087452474 +zach garcia 0.3632887189292543 +zach garcia 0.7072243346007605 +zach garcia 0.7167300380228137 +zach ichabod 0.30988593155893535 +zach ichabod 0.9502868068833652 +zach king 0.5277246653919694 +zach king 0.8336520076481836 +zach king 0.9239543726235742 +zach miller 0.15678776290630975 +zach miller 0.3726235741444867 +zach miller 0.5608365019011406 +zach ovid 0.1311787072243346 +zach ovid 0.2737642585551331 +zach ovid 0.4543726235741445 +zach ovid 0.6711026615969582 +zach quirinius 0.019011406844106463 +zach robinson 0.11026615969581749 +zach steinbeck 0.28107074569789675 +zach steinbeck 0.7170172084130019 +zach thompson 0.13001912045889102 +zach thompson 0.44550669216061184 +zach underhill 0.7718631178707225 +zach white 0.7965779467680608 +zach xylophone 0.032504780114722756 +zach xylophone 0.638623326959847 +zach young 0.009560229445506692 +zach zipper 0.24334600760456274 +zach zipper 0.2832699619771863 +zach zipper 0.8087954110898662 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-3-b82dfa24123047be4b4e3c27c3997d34 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-3-b82dfa24123047be4b4e3c27c3997d34 new file mode 100644 index 0000000000000000000000000000000000000000..1e0cf03db63a020c37426ffefa60ffe420a5f838 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-3-b82dfa24123047be4b4e3c27c3997d34 @@ -0,0 +1,1049 @@ +zach zipper 0.0 +zach zipper 0.0 +zach zipper 0.0 +zach young 0.0 +zach xylophone 0.0 +zach xylophone 0.0 +zach white 0.0 +zach underhill 0.0 +zach thompson 0.0 +zach thompson 0.0 +zach steinbeck 0.0 +zach steinbeck 0.0 +zach robinson 0.0 +zach quirinius 0.0 +zach ovid 0.0 +zach ovid 0.0 +zach ovid 0.0 +zach ovid 0.0 +zach miller 0.0 +zach miller 0.0 +zach miller 0.0 +zach king 0.0 +zach king 0.0 +zach king 0.0 +zach ichabod 0.0 +zach ichabod 0.0 +zach garcia 0.0 +zach garcia 0.0 +zach garcia 0.0 +zach garcia 0.0 +zach falkner 0.0 +zach falkner 0.0 +zach ellison 0.0 +zach carson 0.0 +zach brown 0.0 +zach brown 0.0 +zach brown 0.0 +zach brown 0.0 +zach brown 0.0 +zach allen 0.0 +yuri xylophone 0.0 +yuri white 0.0 +yuri underhill 0.0 +yuri underhill 0.0 +yuri thompson 0.0 +yuri steinbeck 0.0 +yuri steinbeck 0.0 +yuri quirinius 0.0 +yuri quirinius 0.0 +yuri quirinius 0.0 +yuri polk 0.0 +yuri polk 0.0 +yuri polk 0.0 +yuri nixon 0.0 +yuri nixon 0.0 +yuri laertes 0.0 +yuri laertes 0.0 +yuri king 0.0 +yuri johnson 0.0 +yuri johnson 0.0 +yuri johnson 0.0 +yuri hernandez 0.0 +yuri garcia 0.0 +yuri falkner 0.0 +yuri falkner 0.0 +yuri ellison 0.0 +yuri ellison 0.0 +yuri carson 0.0 +yuri carson 0.0 +yuri brown 0.0 +yuri brown 0.0 +yuri allen 0.0 +yuri allen 0.0 +xavier zipper 1.0 +xavier xylophone 0.0 +xavier white 0.0 +xavier white 0.0 +xavier underhill 0.0 +xavier thompson 0.0 +xavier quirinius 0.0 +xavier quirinius 0.0 +xavier quirinius 0.0 +xavier quirinius 0.0 +xavier polk 0.0 +xavier polk 0.0 +xavier polk 0.0 +xavier polk 0.0 +xavier ovid 0.0 +xavier laertes 0.0 +xavier king 0.0 +xavier king 0.0 +xavier johnson 0.0 +xavier johnson 0.0 +xavier ichabod 0.0 +xavier ichabod 0.0 +xavier hernandez 0.0 +xavier hernandez 0.0 +xavier hernandez 0.0 +xavier garcia 0.0 +xavier ellison 0.0 +xavier ellison 0.0 +xavier davidson 0.0 +xavier davidson 0.0 +xavier davidson 0.0 +xavier carson 1.0 +xavier carson 0.0 +xavier brown 0.0 +xavier brown 0.0 +xavier brown 0.0 +xavier allen 0.0 +xavier allen 0.0 +xavier allen 0.0 +wendy young 1.0 +wendy young 0.0 +wendy xylophone 0.0 +wendy xylophone 0.0 +wendy white 0.0 +wendy van buren 0.0 +wendy van buren 0.0 +wendy underhill 0.0 +wendy underhill 0.0 +wendy underhill 0.0 +wendy thompson 0.0 +wendy thompson 0.0 +wendy steinbeck 0.0 +wendy robinson 0.0 +wendy robinson 0.0 +wendy robinson 0.0 +wendy quirinius 0.0 +wendy quirinius 0.0 +wendy polk 0.0 +wendy polk 0.0 +wendy ovid 0.0 +wendy ovid 0.0 +wendy nixon 0.0 +wendy nixon 0.0 +wendy miller 0.0 +wendy miller 0.0 +wendy laertes 0.0 +wendy laertes 0.0 +wendy laertes 0.0 +wendy king 0.0 +wendy king 0.0 +wendy king 0.0 +wendy ichabod 0.0 +wendy hernandez 0.0 +wendy garcia 0.0 +wendy garcia 0.0 +wendy garcia 0.0 +wendy garcia 0.0 +wendy falkner 0.0 +wendy falkner 0.0 +wendy falkner 0.0 +wendy ellison 0.0 +wendy ellison 0.0 +wendy brown 0.0 +wendy brown 0.0 +wendy allen 0.0 +wendy allen 0.0 +wendy allen 0.0 +victor zipper 0.0 +victor young 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor white 1.0 +victor white 0.0 +victor van buren 0.0 +victor van buren 0.0 +victor thompson 0.0 +victor steinbeck 0.0 +victor steinbeck 0.0 +victor steinbeck 0.0 +victor robinson 0.0 +victor robinson 0.0 +victor quirinius 0.0 +victor quirinius 0.0 +victor polk 0.0 +victor ovid 0.0 +victor nixon 0.0 +victor nixon 0.0 +victor miller 0.0 +victor laertes 0.0 +victor laertes 0.0 +victor king 0.0 +victor king 0.0 +victor johnson 0.0 +victor johnson 0.0 +victor johnson 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor ellison 0.0 +victor ellison 0.0 +victor davidson 0.0 +victor davidson 0.0 +victor davidson 0.0 +victor brown 0.0 +victor brown 0.0 +victor brown 0.0 +victor brown 0.0 +victor allen 0.0 +victor allen 0.0 +ulysses young 0.0 +ulysses young 0.0 +ulysses young 0.0 +ulysses xylophone 0.0 +ulysses xylophone 0.0 +ulysses xylophone 0.0 +ulysses white 0.0 +ulysses white 0.0 +ulysses van buren 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses thompson 1.0 +ulysses steinbeck 0.0 +ulysses steinbeck 0.0 +ulysses robinson 0.0 +ulysses quirinius 0.0 +ulysses polk 1.0 +ulysses polk 0.0 +ulysses polk 0.0 +ulysses polk 0.0 +ulysses ovid 0.0 +ulysses nixon 0.0 +ulysses miller 0.0 +ulysses miller 0.0 +ulysses laertes 0.0 +ulysses laertes 0.0 +ulysses laertes 0.0 +ulysses king 0.0 +ulysses johnson 0.0 +ulysses ichabod 0.0 +ulysses ichabod 0.0 +ulysses hernandez 1.0 +ulysses hernandez 0.0 +ulysses hernandez 0.0 +ulysses garcia 0.0 +ulysses ellison 1.0 +ulysses davidson 0.0 +ulysses carson 0.0 +ulysses carson 0.0 +ulysses carson 0.0 +ulysses carson 0.0 +ulysses brown 0.0 +tom zipper 0.0 +tom young 0.0 +tom young 0.0 +tom white 0.0 +tom van buren 0.0 +tom van buren 0.0 +tom van buren 0.0 +tom steinbeck 0.0 +tom robinson 0.0 +tom robinson 0.0 +tom robinson 0.0 +tom robinson 0.0 +tom quirinius 0.0 +tom quirinius 0.0 +tom polk 0.0 +tom polk 0.0 +tom ovid 0.0 +tom nixon 0.0 +tom miller 0.0 +tom miller 0.0 +tom miller 0.0 +tom laertes 0.0 +tom laertes 0.0 +tom king 0.0 +tom johnson 0.0 +tom johnson 0.0 +tom ichabod 0.0 +tom hernandez 0.0 +tom hernandez 0.0 +tom falkner 0.0 +tom falkner 0.0 +tom ellison 0.0 +tom ellison 0.0 +tom ellison 0.0 +tom davidson 0.0 +tom carson 0.0 +tom carson 0.0 +tom carson 0.0 +tom brown 0.0 +tom brown 0.0 +sarah zipper 0.0 +sarah young 0.0 +sarah xylophone 0.0 +sarah white 0.0 +sarah white 0.0 +sarah steinbeck 0.0 +sarah robinson 0.0 +sarah robinson 0.0 +sarah ovid 0.0 +sarah miller 0.0 +sarah king 0.0 +sarah king 0.0 +sarah johnson 0.0 +sarah johnson 0.0 +sarah johnson 0.0 +sarah johnson 0.0 +sarah ichabod 0.0 +sarah ichabod 0.0 +sarah garcia 0.0 +sarah garcia 0.0 +sarah garcia 0.0 +sarah falkner 0.0 +sarah falkner 0.0 +sarah ellison 0.0 +sarah carson 0.0 +sarah carson 0.0 +sarah carson 0.0 +rachel zipper 0.0 +rachel zipper 0.0 +rachel young 0.0 +rachel white 0.0 +rachel white 0.0 +rachel underhill 0.0 +rachel thompson 0.0 +rachel thompson 0.0 +rachel thompson 0.0 +rachel robinson 1.0 +rachel robinson 0.0 +rachel robinson 0.0 +rachel quirinius 0.0 +rachel polk 0.0 +rachel ovid 0.0 +rachel ovid 0.0 +rachel laertes 0.0 +rachel laertes 0.0 +rachel king 0.0 +rachel king 0.0 +rachel johnson 0.0 +rachel falkner 0.0 +rachel falkner 0.0 +rachel falkner 0.0 +rachel falkner 0.0 +rachel ellison 0.0 +rachel davidson 0.0 +rachel carson 0.0 +rachel carson 0.0 +rachel brown 1.0 +rachel brown 0.0 +rachel brown 0.0 +rachel brown 0.0 +rachel brown 0.0 +rachel allen 0.0 +rachel allen 0.0 +quinn zipper 0.0 +quinn zipper 0.0 +quinn young 0.0 +quinn van buren 0.0 +quinn underhill 0.0 +quinn underhill 0.0 +quinn underhill 0.0 +quinn thompson 0.0 +quinn thompson 0.0 +quinn steinbeck 0.0 +quinn steinbeck 0.0 +quinn robinson 0.0 +quinn quirinius 0.0 +quinn ovid 0.0 +quinn nixon 0.0 +quinn laertes 1.0 +quinn laertes 0.0 +quinn laertes 0.0 +quinn king 1.0 +quinn king 0.0 +quinn ichabod 0.0 +quinn garcia 1.0 +quinn garcia 0.0 +quinn garcia 0.0 +quinn garcia 0.0 +quinn ellison 0.0 +quinn ellison 0.0 +quinn davidson 1.0 +quinn davidson 0.0 +quinn davidson 0.0 +quinn davidson 0.0 +quinn brown 0.0 +quinn brown 0.0 +quinn brown 0.0 +quinn allen 1.0 +quinn allen 0.0 +priscilla zipper 0.0 +priscilla zipper 0.0 +priscilla young 0.0 +priscilla young 0.0 +priscilla xylophone 0.0 +priscilla xylophone 0.0 +priscilla xylophone 0.0 +priscilla white 1.0 +priscilla van buren 0.0 +priscilla van buren 0.0 +priscilla van buren 0.0 +priscilla underhill 0.0 +priscilla underhill 0.0 +priscilla thompson 0.0 +priscilla quirinius 0.0 +priscilla polk 0.0 +priscilla ovid 0.0 +priscilla ovid 0.0 +priscilla nixon 0.0 +priscilla nixon 0.0 +priscilla king 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla ichabod 0.0 +priscilla ichabod 0.0 +priscilla carson 0.0 +priscilla carson 0.0 +priscilla carson 0.0 +priscilla brown 0.0 +priscilla brown 0.0 +priscilla brown 0.0 +oscar zipper 0.0 +oscar zipper 0.0 +oscar zipper 0.0 +oscar xylophone 0.0 +oscar xylophone 0.0 +oscar xylophone 0.0 +oscar white 0.0 +oscar white 0.0 +oscar white 0.0 +oscar white 0.0 +oscar van buren 1.0 +oscar van buren 0.0 +oscar van buren 0.0 +oscar underhill 0.0 +oscar thompson 0.0 +oscar thompson 0.0 +oscar thompson 0.0 +oscar thompson 0.0 +oscar steinbeck 0.0 +oscar robinson 0.0 +oscar robinson 0.0 +oscar robinson 0.0 +oscar robinson 0.0 +oscar quirinius 0.0 +oscar quirinius 0.0 +oscar quirinius 0.0 +oscar quirinius 0.0 +oscar polk 1.0 +oscar polk 0.0 +oscar ovid 0.0 +oscar ovid 0.0 +oscar ovid 0.0 +oscar nixon 0.0 +oscar laertes 0.0 +oscar laertes 0.0 +oscar laertes 0.0 +oscar laertes 0.0 +oscar king 0.0 +oscar king 0.0 +oscar king 0.0 +oscar johnson 0.0 +oscar johnson 0.0 +oscar ichabod 0.0 +oscar ichabod 0.0 +oscar ichabod 0.0 +oscar ichabod 0.0 +oscar hernandez 0.0 +oscar hernandez 0.0 +oscar garcia 0.0 +oscar falkner 1.0 +oscar ellison 0.0 +oscar ellison 0.0 +oscar davidson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar brown 0.0 +oscar allen 0.0 +nick zipper 0.0 +nick zipper 0.0 +nick young 1.0 +nick young 0.0 +nick xylophone 0.0 +nick van buren 0.0 +nick underhill 0.0 +nick thompson 0.0 +nick steinbeck 0.0 +nick robinson 0.0 +nick robinson 0.0 +nick quirinius 0.0 +nick quirinius 0.0 +nick polk 0.0 +nick ovid 0.0 +nick nixon 0.0 +nick miller 0.0 +nick laertes 0.0 +nick johnson 0.0 +nick johnson 0.0 +nick ichabod 0.0 +nick ichabod 0.0 +nick ichabod 0.0 +nick garcia 0.0 +nick garcia 0.0 +nick garcia 0.0 +nick falkner 0.0 +nick falkner 0.0 +nick ellison 0.0 +nick ellison 0.0 +nick davidson 0.0 +nick brown 0.0 +nick allen 0.0 +nick allen 0.0 +mike zipper 0.0 +mike zipper 0.0 +mike zipper 0.0 +mike young 0.0 +mike young 0.0 +mike young 0.0 +mike white 0.0 +mike white 0.0 +mike white 0.0 +mike white 0.0 +mike van buren 0.0 +mike van buren 0.0 +mike steinbeck 0.0 +mike steinbeck 0.0 +mike steinbeck 0.0 +mike steinbeck 0.0 +mike quirinius 0.0 +mike polk 0.0 +mike polk 0.0 +mike polk 0.0 +mike nixon 0.0 +mike nixon 0.0 +mike miller 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike ichabod 0.0 +mike hernandez 0.0 +mike hernandez 0.0 +mike garcia 0.0 +mike garcia 0.0 +mike garcia 0.0 +mike falkner 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike davidson 0.0 +mike davidson 0.0 +mike carson 0.0 +mike carson 0.0 +mike carson 0.0 +mike brown 0.0 +mike allen 0.0 +luke zipper 0.0 +luke xylophone 0.0 +luke white 0.0 +luke van buren 0.0 +luke underhill 1.0 +luke underhill 0.0 +luke underhill 0.0 +luke thompson 0.0 +luke robinson 0.0 +luke robinson 0.0 +luke quirinius 0.0 +luke polk 0.0 +luke polk 0.0 +luke ovid 0.0 +luke ovid 0.0 +luke miller 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke johnson 0.0 +luke johnson 0.0 +luke johnson 0.0 +luke ichabod 0.0 +luke ichabod 0.0 +luke garcia 0.0 +luke garcia 0.0 +luke falkner 0.0 +luke falkner 0.0 +luke ellison 0.0 +luke ellison 0.0 +luke ellison 0.0 +luke davidson 0.0 +luke davidson 0.0 +luke brown 0.0 +luke allen 1.0 +luke allen 0.0 +luke allen 0.0 +luke allen 0.0 +luke allen 0.0 +katie zipper 1.0 +katie zipper 0.0 +katie young 1.0 +katie young 0.0 +katie young 0.0 +katie xylophone 0.0 +katie white 0.0 +katie white 0.0 +katie van buren 0.0 +katie van buren 0.0 +katie robinson 0.0 +katie polk 0.0 +katie polk 0.0 +katie ovid 0.0 +katie nixon 1.0 +katie miller 0.0 +katie miller 0.0 +katie king 0.0 +katie king 0.0 +katie king 0.0 +katie ichabod 0.0 +katie ichabod 0.0 +katie ichabod 0.0 +katie hernandez 0.0 +katie garcia 0.0 +katie garcia 0.0 +katie falkner 0.0 +katie ellison 0.0 +katie ellison 0.0 +katie davidson 0.0 +katie brown 0.0 +katie allen 0.0 +jessica zipper 0.0 +jessica zipper 0.0 +jessica zipper 0.0 +jessica young 0.0 +jessica young 0.0 +jessica xylophone 0.0 +jessica white 0.0 +jessica white 0.0 +jessica white 0.0 +jessica white 0.0 +jessica white 0.0 +jessica van buren 0.0 +jessica underhill 1.0 +jessica underhill 0.0 +jessica underhill 0.0 +jessica thompson 0.0 +jessica thompson 0.0 +jessica robinson 0.0 +jessica quirinius 1.0 +jessica quirinius 0.0 +jessica quirinius 0.0 +jessica quirinius 0.0 +jessica polk 0.0 +jessica ovid 0.0 +jessica ovid 0.0 +jessica nixon 0.0 +jessica nixon 0.0 +jessica miller 0.0 +jessica johnson 0.0 +jessica johnson 0.0 +jessica ichabod 1.0 +jessica garcia 0.0 +jessica garcia 0.0 +jessica falkner 0.0 +jessica ellison 0.0 +jessica ellison 0.0 +jessica davidson 0.0 +jessica davidson 0.0 +jessica davidson 0.0 +jessica davidson 0.0 +jessica carson 0.0 +jessica carson 0.0 +jessica carson 0.0 +jessica brown 0.0 +irene xylophone 0.0 +irene van buren 0.0 +irene van buren 0.0 +irene underhill 0.0 +irene underhill 0.0 +irene thompson 0.0 +irene steinbeck 0.0 +irene robinson 0.0 +irene quirinius 0.5 +irene quirinius 0.0 +irene quirinius 0.0 +irene polk 0.0 +irene polk 0.0 +irene polk 0.0 +irene polk 0.0 +irene polk 0.0 +irene ovid 0.0 +irene ovid 0.0 +irene ovid 0.0 +irene nixon 0.0 +irene nixon 0.0 +irene nixon 0.0 +irene miller 0.0 +irene laertes 0.0 +irene laertes 0.0 +irene laertes 0.0 +irene johnson 0.0 +irene ichabod 0.0 +irene ichabod 0.0 +irene garcia 0.0 +irene garcia 0.0 +irene garcia 0.0 +irene falkner 0.0 +irene falkner 0.0 +irene ellison 0.0 +irene ellison 0.0 +irene carson 0.0 +irene brown 0.0 +irene brown 0.0 +irene brown 0.0 +irene allen 0.0 +holly zipper 1.0 +holly zipper 0.0 +holly young 0.0 +holly young 0.0 +holly xylophone 0.0 +holly white 0.0 +holly white 0.0 +holly van buren 0.0 +holly underhill 1.0 +holly underhill 0.0 +holly underhill 0.0 +holly underhill 0.0 +holly thompson 1.0 +holly thompson 0.0 +holly thompson 0.0 +holly robinson 0.0 +holly polk 0.0 +holly polk 0.0 +holly nixon 0.0 +holly nixon 0.0 +holly miller 1.0 +holly laertes 0.0 +holly king 0.0 +holly king 0.0 +holly johnson 0.0 +holly johnson 0.0 +holly johnson 0.0 +holly ichabod 0.0 +holly ichabod 0.0 +holly ichabod 0.0 +holly hernandez 0.0 +holly hernandez 0.0 +holly hernandez 0.0 +holly hernandez 0.0 +holly falkner 0.0 +holly brown 0.0 +holly brown 0.0 +holly allen 0.0 +gabriella zipper 0.0 +gabriella zipper 0.0 +gabriella young 0.0 +gabriella young 0.0 +gabriella white 0.0 +gabriella van buren 0.0 +gabriella van buren 0.0 +gabriella thompson 0.0 +gabriella thompson 0.0 +gabriella thompson 0.0 +gabriella steinbeck 0.0 +gabriella steinbeck 0.0 +gabriella polk 0.0 +gabriella polk 0.0 +gabriella ovid 0.0 +gabriella ovid 0.0 +gabriella miller 0.0 +gabriella laertes 0.0 +gabriella king 0.0 +gabriella king 0.0 +gabriella ichabod 1.0 +gabriella ichabod 0.0 +gabriella ichabod 0.0 +gabriella ichabod 0.0 +gabriella ichabod 0.0 +gabriella hernandez 1.0 +gabriella hernandez 0.0 +gabriella garcia 0.0 +gabriella falkner 0.0 +gabriella falkner 0.0 +gabriella falkner 0.0 +gabriella ellison 0.0 +gabriella ellison 0.0 +gabriella davidson 0.0 +gabriella carson 0.0 +gabriella brown 0.0 +gabriella brown 0.0 +gabriella allen 0.0 +gabriella allen 0.0 +fred zipper 0.0 +fred young 0.0 +fred young 0.0 +fred white 0.0 +fred van buren 0.0 +fred van buren 0.0 +fred van buren 0.0 +fred van buren 0.0 +fred underhill 0.0 +fred steinbeck 0.0 +fred steinbeck 0.0 +fred steinbeck 0.0 +fred robinson 1.0 +fred quirinius 0.0 +fred quirinius 0.0 +fred polk 0.0 +fred polk 0.0 +fred polk 0.0 +fred polk 0.0 +fred nixon 0.0 +fred nixon 0.0 +fred nixon 0.0 +fred nixon 0.0 +fred miller 0.0 +fred laertes 0.0 +fred king 0.0 +fred king 0.0 +fred johnson 1.0 +fred ichabod 0.0 +fred ichabod 0.0 +fred hernandez 0.0 +fred falkner 1.0 +fred falkner 0.0 +fred falkner 0.0 +fred ellison 0.0 +fred ellison 0.0 +fred ellison 0.0 +fred davidson 0.0 +fred davidson 0.0 +fred davidson 0.0 +ethan zipper 0.0 +ethan zipper 0.0 +ethan xylophone 0.0 +ethan white 0.0 +ethan white 0.0 +ethan van buren 0.0 +ethan underhill 0.5 +ethan robinson 0.0 +ethan robinson 0.0 +ethan quirinius 0.0 +ethan quirinius 0.0 +ethan quirinius 0.0 +ethan polk 1.0 +ethan polk 0.0 +ethan polk 0.0 +ethan polk 0.0 +ethan ovid 0.0 +ethan nixon 0.0 +ethan miller 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan king 0.0 +ethan johnson 0.0 +ethan hernandez 0.0 +ethan garcia 0.0 +ethan falkner 0.0 +ethan falkner 0.0 +ethan ellison 0.0 +ethan ellison 0.0 +ethan carson 0.0 +ethan brown 1.0 +ethan brown 0.0 +ethan brown 0.0 +ethan brown 0.0 +ethan brown 0.0 +ethan brown 0.0 +ethan allen 0.0 +david young 0.0 +david young 0.0 +david xylophone 1.0 +david xylophone 0.0 +david xylophone 0.0 +david white 0.0 +david van buren 0.0 +david van buren 0.0 +david underhill 0.0 +david underhill 0.0 +david underhill 0.0 +david thompson 1.0 +david robinson 0.0 +david robinson 0.0 +david quirinius 0.0 +david quirinius 0.0 +david quirinius 0.0 +david ovid 0.0 +david ovid 0.0 +david nixon 0.0 +david laertes 0.0 +david ichabod 1.0 +david ichabod 0.0 +david hernandez 1.0 +david ellison 0.0 +david ellison 0.0 +david ellison 0.0 +david davidson 0.0 +david davidson 0.0 +david davidson 0.0 +david davidson 0.0 +david brown 0.0 +david brown 0.0 +david allen 0.0 +david allen 0.0 +calvin zipper 0.0 +calvin zipper 0.0 +calvin young 0.0 +calvin young 0.0 +calvin xylophone 0.0 +calvin xylophone 0.0 +calvin xylophone 0.0 +calvin white 0.0 +calvin white 0.0 +calvin van buren 1.0 +calvin van buren 0.0 +calvin underhill 0.0 +calvin thompson 0.0 +calvin thompson 0.0 +calvin steinbeck 0.0 +calvin steinbeck 0.0 +calvin steinbeck 0.0 +calvin robinson 0.0 +calvin quirinius 0.0 +calvin quirinius 0.0 +calvin polk 0.0 +calvin ovid 0.0 +calvin ovid 0.0 +calvin ovid 0.0 +calvin ovid 0.0 +calvin nixon 0.0 +calvin nixon 0.0 +calvin nixon 0.0 +calvin laertes 0.0 +calvin laertes 0.0 +calvin johnson 0.0 +calvin hernandez 0.0 +calvin garcia 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin ellison 0.0 +calvin davidson 0.0 +calvin davidson 0.0 +calvin carson 0.0 +calvin brown 0.0 +calvin brown 0.0 +calvin brown 0.0 +calvin allen 0.0 +bob zipper 0.0 +bob zipper 0.0 +bob zipper 0.0 +bob young 0.0 +bob xylophone 0.0 +bob xylophone 0.0 +bob white 0.0 +bob white 0.0 +bob van buren 0.0 +bob steinbeck 0.0 +bob quirinius 0.0 +bob polk 0.0 +bob ovid 0.0 +bob ovid 0.0 +bob ovid 0.0 +bob ovid 0.0 +bob miller 0.0 +bob laertes 0.0 +bob laertes 0.0 +bob king 1.0 +bob king 0.0 +bob king 0.0 +bob ichabod 0.0 +bob hernandez 1.0 +bob garcia 0.0 +bob garcia 0.0 +bob garcia 0.0 +bob garcia 0.0 +bob garcia 0.0 +bob falkner 0.0 +bob ellison 1.0 +bob ellison 0.0 +bob ellison 0.0 +bob ellison 0.0 +bob davidson 0.0 +bob davidson 0.0 +bob davidson 0.0 +bob carson 0.0 +bob brown 0.0 +bob brown 0.0 +bob brown 0.0 +alice zipper 0.0 +alice zipper 0.0 +alice zipper 0.0 +alice xylophone 0.0 +alice xylophone 0.0 +alice xylophone 0.0 +alice van buren 0.0 +alice underhill 0.0 +alice steinbeck 0.0 +alice steinbeck 0.0 +alice steinbeck 0.0 +alice robinson 0.0 +alice robinson 0.0 +alice quirinius 0.0 +alice quirinius 0.0 +alice polk 1.0 +alice ovid 0.0 +alice nixon 0.0 +alice nixon 0.0 +alice nixon 0.0 +alice miller 0.0 +alice laertes 0.0 +alice laertes 0.0 +alice king 0.0 +alice king 0.0 +alice king 0.0 +alice johnson 0.0 +alice hernandez 0.0 +alice hernandez 0.0 +alice garcia 0.0 +alice falkner 0.0 +alice davidson 0.0 +alice carson 0.0 +alice brown 0.0 +alice allen 0.0 +alice allen 0.0 +alice allen 0.0 + 0.0 + 0.0 + 0.0 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 new file mode 100644 index 0000000000000000000000000000000000000000..9091a9156134c5e406ad805d82e6f3809542b2eb --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 @@ -0,0 +1,1294 @@ +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703093 14.85 1 +2013-03-01 09:11:58.703093 14.85 1 +2013-03-01 09:11:58.703093 14.85 1 +2013-03-01 09:11:58.703094 57.11 1 +2013-03-01 09:11:58.703094 57.11 1 +2013-03-01 09:11:58.703094 57.11 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703099 11.69 1 +2013-03-01 09:11:58.703099 11.69 1 +2013-03-01 09:11:58.703099 11.69 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703105 28.47 1 +2013-03-01 09:11:58.703105 28.47 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.703119 58.02 1 +2013-03-01 09:11:58.703119 58.02 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703124 8.93 1 +2013-03-01 09:11:58.703124 8.93 1 +2013-03-01 09:11:58.703124 8.93 1 +2013-03-01 09:11:58.703125 14.94 1 +2013-03-01 09:11:58.703125 14.94 1 +2013-03-01 09:11:58.703125 14.94 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703135 29.14 1 +2013-03-01 09:11:58.703135 29.14 1 +2013-03-01 09:11:58.703135 29.14 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703139 12.67 1 +2013-03-01 09:11:58.703139 12.67 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703142 24.25 1 +2013-03-01 09:11:58.703142 24.25 1 +2013-03-01 09:11:58.703142 24.25 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703144 3.43 1 +2013-03-01 09:11:58.703144 3.43 1 +2013-03-01 09:11:58.703144 3.43 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703158 71.89 1 +2013-03-01 09:11:58.703158 71.89 1 +2013-03-01 09:11:58.703158 71.89 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703165 8.38 1 +2013-03-01 09:11:58.703165 8.38 1 +2013-03-01 09:11:58.703165 8.38 1 +2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.70319 90.09 1 +2013-03-01 09:11:58.70319 90.09 1 +2013-03-01 09:11:58.70319 90.09 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703193 28.42 1 +2013-03-01 09:11:58.703193 28.42 1 +2013-03-01 09:11:58.703193 28.42 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703196 0.08 1 +2013-03-01 09:11:58.703196 0.08 1 +2013-03-01 09:11:58.703196 0.08 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.7032 12.72 1 +2013-03-01 09:11:58.7032 12.72 1 +2013-03-01 09:11:58.703201 35.15 1 +2013-03-01 09:11:58.703201 35.15 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703214 31.35 1 +2013-03-01 09:11:58.703214 31.35 1 +2013-03-01 09:11:58.703214 31.35 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703217 23.57 1 +2013-03-01 09:11:58.703217 23.57 1 +2013-03-01 09:11:58.703217 23.57 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703228 4.19 1 +2013-03-01 09:11:58.703228 4.19 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703232 61.96 1 +2013-03-01 09:11:58.703232 61.96 1 +2013-03-01 09:11:58.703232 61.96 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.70324 5.1 1 +2013-03-01 09:11:58.70324 5.1 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703244 4.47 1 +2013-03-01 09:11:58.703244 4.47 1 +2013-03-01 09:11:58.703244 4.47 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703246 45.94 1 +2013-03-01 09:11:58.703246 45.94 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703249 19.42 1 +2013-03-01 09:11:58.703249 19.42 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.703251 68.98 1 +2013-03-01 09:11:58.703251 68.98 1 +2013-03-01 09:11:58.703251 68.98 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703256 23.78 1 +2013-03-01 09:11:58.703256 23.78 1 +2013-03-01 09:11:58.703256 23.78 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703269 43.84 1 +2013-03-01 09:11:58.703269 43.84 1 +2013-03-01 09:11:58.703269 43.84 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.703271 61.16 1 +2013-03-01 09:11:58.703271 61.16 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703273 10.94 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703275 20.57 1 +2013-03-01 09:11:58.703275 20.57 1 +2013-03-01 09:11:58.703275 20.57 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703287 0.89 1 +2013-03-01 09:11:58.703287 0.89 1 +2013-03-01 09:11:58.703287 0.89 1 +2013-03-01 09:11:58.703288 60.57 1 +2013-03-01 09:11:58.703288 60.57 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703299 9 1 +2013-03-01 09:11:58.703299 9 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703307 31.28 1 +2013-03-01 09:11:58.703307 31.28 1 +2013-03-01 09:11:58.703307 31.28 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703323 36.22 1 +2013-03-01 09:11:58.703323 36.22 1 +2013-03-01 09:11:58.703323 36.22 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 3-0-58a982694ba2b1e34de82b1de54936a0 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 3-0-58a982694ba2b1e34de82b1de54936a0 new file mode 100644 index 0000000000000000000000000000000000000000..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 new file mode 100644 index 0000000000000000000000000000000000000000..d02ca48857b5fe7f2549b11e50d88240084d4bd4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 @@ -0,0 +1,474 @@ +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703092 54.02 1 +2013-03-01 09:11:58.703092 54.02 1 +2013-03-01 09:11:58.703092 54.02 1 +2013-03-01 09:11:58.703096 87.84 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703246 72.87 1 +2013-03-01 09:11:58.703246 72.87 1 +2013-03-01 09:11:58.703248 81.28 1 +2013-03-01 09:11:58.703248 81.28 1 +2013-03-01 09:11:58.703248 81.28 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703258 21.21 1 +2013-03-01 09:11:58.703258 21.21 1 +2013-03-01 09:11:58.703258 21.21 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703262 78.56 1 +2013-03-01 09:11:58.703262 78.56 1 +2013-03-01 09:11:58.703262 78.56 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703273 30.49 1 +2013-03-01 09:11:58.703273 30.49 1 +2013-03-01 09:11:58.703273 30.49 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703278 69.42 1 +2013-03-01 09:11:58.703278 69.42 1 +2013-03-01 09:11:58.703278 69.42 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.70331 69.26 1 +2013-03-01 09:11:58.70331 69.26 1 +2013-03-01 09:11:58.70331 69.26 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703318 85.62 1 +2013-03-01 09:11:58.703318 85.62 1 +2013-03-01 09:11:58.703318 85.62 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703325 65.81 1 +2013-03-01 09:11:58.703325 65.81 1 +2013-03-01 09:11:58.703325 65.81 1 +2013-03-01 09:11:58.703325 65.81 1 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-0-6642a21d87e0401ba1a668ea8b244f0c b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-0-6642a21d87e0401ba1a668ea8b244f0c new file mode 100644 index 0000000000000000000000000000000000000000..119dd71df14263a26965fe39e79461dda538b0ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-0-6642a21d87e0401ba1a668ea8b244f0c @@ -0,0 +1,1049 @@ + 65560 + 65560 + 65560 +alice allen 65662 +alice allen 65662 +alice allen 65662 +alice brown 65696 +alice carson 65559 +alice davidson 65547 +alice falkner 65669 +alice garcia 65613 +alice hernandez 65737 +alice hernandez 65737 +alice johnson 65739 +alice king 65660 +alice king 65660 +alice king 65660 +alice laertes 65669 +alice laertes 65669 +alice miller 65590 +alice nixon 65586 +alice nixon 65586 +alice nixon 65586 +alice ovid 65737 +alice polk 65548 +alice quirinius 65636 +alice quirinius 65636 +alice robinson 65606 +alice robinson 65606 +alice steinbeck 65578 +alice steinbeck 65578 +alice steinbeck 65578 +alice underhill 65750 +alice van buren 65562 +alice xylophone 65578 +alice xylophone 65578 +alice xylophone 65578 +alice zipper 65553 +alice zipper 65553 +alice zipper 65553 +bob brown 65584 +bob brown 65584 +bob brown 65584 +bob carson 65713 +bob davidson 65664 +bob davidson 65664 +bob davidson 65664 +bob ellison 65591 +bob ellison 65591 +bob ellison 65591 +bob ellison 65591 +bob falkner 65789 +bob garcia 65585 +bob garcia 65585 +bob garcia 65585 +bob garcia 65585 +bob garcia 65585 +bob hernandez 65557 +bob ichabod 65549 +bob king 65715 +bob king 65715 +bob king 65715 +bob laertes 65602 +bob laertes 65602 +bob miller 65608 +bob ovid 65564 +bob ovid 65564 +bob ovid 65564 +bob ovid 65564 +bob polk 65594 +bob quirinius 65700 +bob steinbeck 65637 +bob van buren 65778 +bob white 65543 +bob white 65543 +bob xylophone 65574 +bob xylophone 65574 +bob young 65556 +bob zipper 65559 +bob zipper 65559 +bob zipper 65559 +calvin allen 65669 +calvin brown 65537 +calvin brown 65537 +calvin brown 65537 +calvin carson 65637 +calvin davidson 65541 +calvin davidson 65541 +calvin ellison 65667 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin garcia 65664 +calvin hernandez 65578 +calvin johnson 65731 +calvin laertes 65570 +calvin laertes 65570 +calvin nixon 65654 +calvin nixon 65654 +calvin nixon 65654 +calvin ovid 65554 +calvin ovid 65554 +calvin ovid 65554 +calvin ovid 65554 +calvin polk 65731 +calvin quirinius 65741 +calvin quirinius 65741 +calvin robinson 65581 +calvin steinbeck 65680 +calvin steinbeck 65680 +calvin steinbeck 65680 +calvin thompson 65560 +calvin thompson 65560 +calvin underhill 65732 +calvin van buren 65552 +calvin van buren 65552 +calvin white 65553 +calvin white 65553 +calvin xylophone 65575 +calvin xylophone 65575 +calvin xylophone 65575 +calvin young 65574 +calvin young 65574 +calvin zipper 65669 +calvin zipper 65669 +david allen 65588 +david allen 65588 +david brown 65637 +david brown 65637 +david davidson 65559 +david davidson 65559 +david davidson 65559 +david davidson 65559 +david ellison 65634 +david ellison 65634 +david ellison 65634 +david hernandez 65763 +david ichabod 65699 +david ichabod 65699 +david laertes 65762 +david nixon 65536 +david ovid 65623 +david ovid 65623 +david quirinius 65697 +david quirinius 65697 +david quirinius 65697 +david robinson 65762 +david robinson 65762 +david thompson 65550 +david underhill 65602 +david underhill 65602 +david underhill 65602 +david van buren 65625 +david van buren 65625 +david white 65678 +david xylophone 65537 +david xylophone 65537 +david xylophone 65537 +david young 65551 +david young 65551 +ethan allen 65747 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan carson 65742 +ethan ellison 65714 +ethan ellison 65714 +ethan falkner 65577 +ethan falkner 65577 +ethan garcia 65736 +ethan hernandez 65618 +ethan johnson 65536 +ethan king 65614 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan miller 65712 +ethan nixon 65766 +ethan ovid 65697 +ethan polk 65589 +ethan polk 65589 +ethan polk 65589 +ethan polk 65589 +ethan quirinius 65542 +ethan quirinius 65542 +ethan quirinius 65542 +ethan robinson 65547 +ethan robinson 65547 +ethan underhill 65570 +ethan van buren 65572 +ethan white 65677 +ethan white 65677 +ethan xylophone 65595 +ethan zipper 65593 +ethan zipper 65593 +fred davidson 65552 +fred davidson 65552 +fred davidson 65552 +fred ellison 65548 +fred ellison 65548 +fred ellison 65548 +fred falkner 65637 +fred falkner 65637 +fred falkner 65637 +fred hernandez 65541 +fred ichabod 65572 +fred ichabod 65572 +fred johnson 65758 +fred king 65694 +fred king 65694 +fred laertes 65769 +fred miller 65536 +fred nixon 65560 +fred nixon 65560 +fred nixon 65560 +fred nixon 65560 +fred polk 65603 +fred polk 65603 +fred polk 65603 +fred polk 65603 +fred quirinius 65697 +fred quirinius 65697 +fred robinson 65623 +fred steinbeck 65544 +fred steinbeck 65544 +fred steinbeck 65544 +fred underhill 65629 +fred van buren 65537 +fred van buren 65537 +fred van buren 65537 +fred van buren 65537 +fred white 65589 +fred young 65594 +fred young 65594 +fred zipper 65553 +gabriella allen 65646 +gabriella allen 65646 +gabriella brown 65704 +gabriella brown 65704 +gabriella carson 65586 +gabriella davidson 65565 +gabriella ellison 65706 +gabriella ellison 65706 +gabriella falkner 65623 +gabriella falkner 65623 +gabriella falkner 65623 +gabriella garcia 65571 +gabriella hernandez 65587 +gabriella hernandez 65587 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella king 65651 +gabriella king 65651 +gabriella laertes 65781 +gabriella miller 65646 +gabriella ovid 65556 +gabriella ovid 65556 +gabriella polk 65701 +gabriella polk 65701 +gabriella steinbeck 65582 +gabriella steinbeck 65582 +gabriella thompson 65682 +gabriella thompson 65682 +gabriella thompson 65682 +gabriella van buren 65581 +gabriella van buren 65581 +gabriella white 65638 +gabriella young 65699 +gabriella young 65699 +gabriella zipper 65540 +gabriella zipper 65540 +holly allen 65596 +holly brown 65599 +holly brown 65599 +holly falkner 65720 +holly hernandez 65602 +holly hernandez 65602 +holly hernandez 65602 +holly hernandez 65602 +holly ichabod 65711 +holly ichabod 65711 +holly ichabod 65711 +holly johnson 65655 +holly johnson 65655 +holly johnson 65655 +holly king 65549 +holly king 65549 +holly laertes 65664 +holly miller 65653 +holly nixon 65539 +holly nixon 65539 +holly polk 65743 +holly polk 65743 +holly robinson 65564 +holly thompson 65538 +holly thompson 65538 +holly thompson 65538 +holly underhill 65634 +holly underhill 65634 +holly underhill 65634 +holly underhill 65634 +holly van buren 65727 +holly white 65536 +holly white 65536 +holly xylophone 65544 +holly young 65606 +holly young 65606 +holly zipper 65607 +holly zipper 65607 +irene allen 65556 +irene brown 65633 +irene brown 65633 +irene brown 65633 +irene carson 65590 +irene ellison 65659 +irene ellison 65659 +irene falkner 65620 +irene falkner 65620 +irene garcia 65660 +irene garcia 65660 +irene garcia 65660 +irene ichabod 65645 +irene ichabod 65645 +irene johnson 65583 +irene laertes 65664 +irene laertes 65664 +irene laertes 65664 +irene miller 65730 +irene nixon 65631 +irene nixon 65631 +irene nixon 65631 +irene ovid 65691 +irene ovid 65691 +irene ovid 65691 +irene polk 65551 +irene polk 65551 +irene polk 65551 +irene polk 65551 +irene polk 65551 +irene quirinius 65724 +irene quirinius 65724 +irene quirinius 65724 +irene robinson 65554 +irene steinbeck 65683 +irene thompson 65688 +irene underhill 65591 +irene underhill 65591 +irene van buren 65579 +irene van buren 65579 +irene xylophone 65775 +jessica brown 65588 +jessica carson 65553 +jessica carson 65553 +jessica carson 65553 +jessica davidson 65549 +jessica davidson 65549 +jessica davidson 65549 +jessica davidson 65549 +jessica ellison 65567 +jessica ellison 65567 +jessica falkner 65584 +jessica garcia 65676 +jessica garcia 65676 +jessica ichabod 65704 +jessica johnson 65607 +jessica johnson 65607 +jessica miller 65733 +jessica nixon 65590 +jessica nixon 65590 +jessica ovid 65582 +jessica ovid 65582 +jessica polk 65637 +jessica quirinius 65562 +jessica quirinius 65562 +jessica quirinius 65562 +jessica quirinius 65562 +jessica robinson 65576 +jessica thompson 65581 +jessica thompson 65581 +jessica underhill 65656 +jessica underhill 65656 +jessica underhill 65656 +jessica van buren 65615 +jessica white 65544 +jessica white 65544 +jessica white 65544 +jessica white 65544 +jessica white 65544 +jessica xylophone 65562 +jessica young 65623 +jessica young 65623 +jessica zipper 65600 +jessica zipper 65600 +jessica zipper 65600 +katie allen 65542 +katie brown 65590 +katie davidson 65619 +katie ellison 65675 +katie ellison 65675 +katie falkner 65728 +katie garcia 65625 +katie garcia 65625 +katie hernandez 65550 +katie ichabod 65658 +katie ichabod 65658 +katie ichabod 65658 +katie king 65629 +katie king 65629 +katie king 65629 +katie miller 65541 +katie miller 65541 +katie nixon 65669 +katie ovid 65681 +katie polk 65746 +katie polk 65746 +katie robinson 65697 +katie van buren 65643 +katie van buren 65643 +katie white 65620 +katie white 65620 +katie xylophone 65585 +katie young 65644 +katie young 65644 +katie young 65644 +katie zipper 65568 +katie zipper 65568 +luke allen 65547 +luke allen 65547 +luke allen 65547 +luke allen 65547 +luke allen 65547 +luke brown 65719 +luke davidson 65656 +luke davidson 65656 +luke ellison 65582 +luke ellison 65582 +luke ellison 65582 +luke falkner 65589 +luke falkner 65589 +luke garcia 65687 +luke garcia 65687 +luke ichabod 65629 +luke ichabod 65629 +luke johnson 65545 +luke johnson 65545 +luke johnson 65545 +luke laertes 65608 +luke laertes 65608 +luke laertes 65608 +luke laertes 65608 +luke laertes 65608 +luke miller 65752 +luke ovid 65569 +luke ovid 65569 +luke polk 65645 +luke polk 65645 +luke quirinius 65655 +luke robinson 65634 +luke robinson 65634 +luke thompson 65626 +luke underhill 65553 +luke underhill 65553 +luke underhill 65553 +luke van buren 65678 +luke white 65693 +luke xylophone 65597 +luke zipper 65641 +mike allen 65706 +mike brown 65654 +mike carson 65698 +mike carson 65698 +mike carson 65698 +mike davidson 65658 +mike davidson 65658 +mike ellison 65598 +mike ellison 65598 +mike ellison 65598 +mike ellison 65598 +mike ellison 65598 +mike falkner 65609 +mike garcia 65571 +mike garcia 65571 +mike garcia 65571 +mike hernandez 65548 +mike hernandez 65548 +mike ichabod 65621 +mike king 65563 +mike king 65563 +mike king 65563 +mike king 65563 +mike king 65563 +mike king 65563 +mike miller 65549 +mike nixon 65619 +mike nixon 65619 +mike polk 65619 +mike polk 65619 +mike polk 65619 +mike quirinius 65717 +mike steinbeck 65550 +mike steinbeck 65550 +mike steinbeck 65550 +mike steinbeck 65550 +mike van buren 65620 +mike van buren 65620 +mike white 65648 +mike white 65648 +mike white 65648 +mike white 65648 +mike young 65545 +mike young 65545 +mike young 65545 +mike zipper 65552 +mike zipper 65552 +mike zipper 65552 +nick allen 65641 +nick allen 65641 +nick brown 65724 +nick davidson 65601 +nick ellison 65691 +nick ellison 65691 +nick falkner 65583 +nick falkner 65583 +nick garcia 65695 +nick garcia 65695 +nick garcia 65695 +nick ichabod 65572 +nick ichabod 65572 +nick ichabod 65572 +nick johnson 65585 +nick johnson 65585 +nick laertes 65624 +nick miller 65757 +nick nixon 65650 +nick ovid 65719 +nick polk 65716 +nick quirinius 65588 +nick quirinius 65588 +nick robinson 65547 +nick robinson 65547 +nick steinbeck 65689 +nick thompson 65610 +nick underhill 65619 +nick van buren 65603 +nick xylophone 65644 +nick young 65654 +nick young 65654 +nick zipper 65757 +nick zipper 65757 +oscar allen 65644 +oscar brown 65614 +oscar carson 65537 +oscar carson 65537 +oscar carson 65537 +oscar carson 65537 +oscar carson 65537 +oscar davidson 65556 +oscar ellison 65630 +oscar ellison 65630 +oscar falkner 65692 +oscar garcia 65751 +oscar hernandez 65683 +oscar hernandez 65683 +oscar ichabod 65536 +oscar ichabod 65536 +oscar ichabod 65536 +oscar ichabod 65536 +oscar johnson 65645 +oscar johnson 65645 +oscar king 65541 +oscar king 65541 +oscar king 65541 +oscar laertes 65625 +oscar laertes 65625 +oscar laertes 65625 +oscar laertes 65625 +oscar nixon 65596 +oscar ovid 65536 +oscar ovid 65536 +oscar ovid 65536 +oscar polk 65541 +oscar polk 65541 +oscar quirinius 65541 +oscar quirinius 65541 +oscar quirinius 65541 +oscar quirinius 65541 +oscar robinson 65537 +oscar robinson 65537 +oscar robinson 65537 +oscar robinson 65537 +oscar steinbeck 65709 +oscar thompson 65542 +oscar thompson 65542 +oscar thompson 65542 +oscar thompson 65542 +oscar underhill 65626 +oscar van buren 65581 +oscar van buren 65581 +oscar van buren 65581 +oscar white 65552 +oscar white 65552 +oscar white 65552 +oscar white 65552 +oscar xylophone 65773 +oscar xylophone 65773 +oscar xylophone 65773 +oscar zipper 65568 +oscar zipper 65568 +oscar zipper 65568 +priscilla brown 65670 +priscilla brown 65670 +priscilla brown 65670 +priscilla carson 65658 +priscilla carson 65658 +priscilla carson 65658 +priscilla ichabod 65627 +priscilla ichabod 65627 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla king 65646 +priscilla nixon 65564 +priscilla nixon 65564 +priscilla ovid 65541 +priscilla ovid 65541 +priscilla polk 65747 +priscilla quirinius 65672 +priscilla thompson 65654 +priscilla underhill 65715 +priscilla underhill 65715 +priscilla van buren 65607 +priscilla van buren 65607 +priscilla van buren 65607 +priscilla white 65652 +priscilla xylophone 65538 +priscilla xylophone 65538 +priscilla xylophone 65538 +priscilla young 65585 +priscilla young 65585 +priscilla zipper 65622 +priscilla zipper 65622 +quinn allen 65657 +quinn allen 65657 +quinn brown 65691 +quinn brown 65691 +quinn brown 65691 +quinn davidson 65549 +quinn davidson 65549 +quinn davidson 65549 +quinn davidson 65549 +quinn ellison 65705 +quinn ellison 65705 +quinn garcia 65568 +quinn garcia 65568 +quinn garcia 65568 +quinn garcia 65568 +quinn ichabod 65564 +quinn king 65558 +quinn king 65558 +quinn laertes 65542 +quinn laertes 65542 +quinn laertes 65542 +quinn nixon 65659 +quinn ovid 65699 +quinn quirinius 65747 +quinn robinson 65627 +quinn steinbeck 65578 +quinn steinbeck 65578 +quinn thompson 65643 +quinn thompson 65643 +quinn underhill 65549 +quinn underhill 65549 +quinn underhill 65549 +quinn van buren 65725 +quinn young 65647 +quinn zipper 65579 +quinn zipper 65579 +rachel allen 65661 +rachel allen 65661 +rachel brown 65586 +rachel brown 65586 +rachel brown 65586 +rachel brown 65586 +rachel brown 65586 +rachel carson 65677 +rachel carson 65677 +rachel davidson 65755 +rachel ellison 65761 +rachel falkner 65616 +rachel falkner 65616 +rachel falkner 65616 +rachel falkner 65616 +rachel johnson 65658 +rachel king 65604 +rachel king 65604 +rachel laertes 65562 +rachel laertes 65562 +rachel ovid 65721 +rachel ovid 65721 +rachel polk 65686 +rachel quirinius 65787 +rachel robinson 65544 +rachel robinson 65544 +rachel robinson 65544 +rachel thompson 65648 +rachel thompson 65648 +rachel thompson 65648 +rachel underhill 65667 +rachel white 65615 +rachel white 65615 +rachel young 65727 +rachel zipper 65757 +rachel zipper 65757 +sarah carson 65679 +sarah carson 65679 +sarah carson 65679 +sarah ellison 65611 +sarah falkner 65606 +sarah falkner 65606 +sarah garcia 65563 +sarah garcia 65563 +sarah garcia 65563 +sarah ichabod 65667 +sarah ichabod 65667 +sarah johnson 65659 +sarah johnson 65659 +sarah johnson 65659 +sarah johnson 65659 +sarah king 65650 +sarah king 65650 +sarah miller 65557 +sarah ovid 65550 +sarah robinson 65677 +sarah robinson 65677 +sarah steinbeck 65721 +sarah white 65622 +sarah white 65622 +sarah xylophone 65678 +sarah young 65595 +sarah zipper 65550 +tom brown 65593 +tom brown 65593 +tom carson 65539 +tom carson 65539 +tom carson 65539 +tom davidson 65780 +tom ellison 65578 +tom ellison 65578 +tom ellison 65578 +tom falkner 65574 +tom falkner 65574 +tom hernandez 65575 +tom hernandez 65575 +tom ichabod 65588 +tom johnson 65536 +tom johnson 65536 +tom king 65576 +tom laertes 65617 +tom laertes 65617 +tom miller 65594 +tom miller 65594 +tom miller 65594 +tom nixon 65672 +tom ovid 65628 +tom polk 65652 +tom polk 65652 +tom quirinius 65563 +tom quirinius 65563 +tom robinson 65626 +tom robinson 65626 +tom robinson 65626 +tom robinson 65626 +tom steinbeck 65666 +tom van buren 65621 +tom van buren 65621 +tom van buren 65621 +tom white 65548 +tom young 65544 +tom young 65544 +tom zipper 65789 +ulysses brown 65735 +ulysses carson 65602 +ulysses carson 65602 +ulysses carson 65602 +ulysses carson 65602 +ulysses davidson 65750 +ulysses ellison 65575 +ulysses garcia 65666 +ulysses hernandez 65651 +ulysses hernandez 65651 +ulysses hernandez 65651 +ulysses ichabod 65551 +ulysses ichabod 65551 +ulysses johnson 65776 +ulysses king 65649 +ulysses laertes 65691 +ulysses laertes 65691 +ulysses laertes 65691 +ulysses miller 65610 +ulysses miller 65610 +ulysses nixon 65603 +ulysses ovid 65656 +ulysses polk 65563 +ulysses polk 65563 +ulysses polk 65563 +ulysses polk 65563 +ulysses quirinius 65786 +ulysses robinson 65744 +ulysses steinbeck 65611 +ulysses steinbeck 65611 +ulysses thompson 65788 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses van buren 65684 +ulysses white 65654 +ulysses white 65654 +ulysses xylophone 65623 +ulysses xylophone 65623 +ulysses xylophone 65623 +ulysses young 65675 +ulysses young 65675 +ulysses young 65675 +victor allen 65684 +victor allen 65684 +victor brown 65550 +victor brown 65550 +victor brown 65550 +victor brown 65550 +victor davidson 65579 +victor davidson 65579 +victor davidson 65579 +victor ellison 65641 +victor ellison 65641 +victor hernandez 65571 +victor hernandez 65571 +victor hernandez 65571 +victor hernandez 65571 +victor hernandez 65571 +victor johnson 65606 +victor johnson 65606 +victor johnson 65606 +victor king 65721 +victor king 65721 +victor laertes 65638 +victor laertes 65638 +victor miller 65570 +victor nixon 65709 +victor nixon 65709 +victor ovid 65649 +victor polk 65625 +victor quirinius 65620 +victor quirinius 65620 +victor robinson 65596 +victor robinson 65596 +victor steinbeck 65618 +victor steinbeck 65618 +victor steinbeck 65618 +victor thompson 65548 +victor van buren 65664 +victor van buren 65664 +victor white 65548 +victor white 65548 +victor xylophone 65549 +victor xylophone 65549 +victor xylophone 65549 +victor xylophone 65549 +victor xylophone 65549 +victor young 65628 +victor zipper 65743 +wendy allen 65628 +wendy allen 65628 +wendy allen 65628 +wendy brown 65580 +wendy brown 65580 +wendy ellison 65545 +wendy ellison 65545 +wendy falkner 65595 +wendy falkner 65595 +wendy falkner 65595 +wendy garcia 65659 +wendy garcia 65659 +wendy garcia 65659 +wendy garcia 65659 +wendy hernandez 65650 +wendy ichabod 65730 +wendy king 65586 +wendy king 65586 +wendy king 65586 +wendy laertes 65566 +wendy laertes 65566 +wendy laertes 65566 +wendy miller 65582 +wendy miller 65582 +wendy nixon 65611 +wendy nixon 65611 +wendy ovid 65589 +wendy ovid 65589 +wendy polk 65656 +wendy polk 65656 +wendy quirinius 65766 +wendy quirinius 65766 +wendy robinson 65622 +wendy robinson 65622 +wendy robinson 65622 +wendy steinbeck 65612 +wendy thompson 65650 +wendy thompson 65650 +wendy underhill 65662 +wendy underhill 65662 +wendy underhill 65662 +wendy van buren 65680 +wendy van buren 65680 +wendy white 65705 +wendy xylophone 65687 +wendy xylophone 65687 +wendy young 65674 +wendy young 65674 +xavier allen 65611 +xavier allen 65611 +xavier allen 65611 +xavier brown 65600 +xavier brown 65600 +xavier brown 65600 +xavier carson 65731 +xavier carson 65731 +xavier davidson 65644 +xavier davidson 65644 +xavier davidson 65644 +xavier ellison 65541 +xavier ellison 65541 +xavier garcia 65672 +xavier hernandez 65541 +xavier hernandez 65541 +xavier hernandez 65541 +xavier ichabod 65597 +xavier ichabod 65597 +xavier johnson 65654 +xavier johnson 65654 +xavier king 65590 +xavier king 65590 +xavier laertes 65743 +xavier ovid 65788 +xavier polk 65587 +xavier polk 65587 +xavier polk 65587 +xavier polk 65587 +xavier quirinius 65599 +xavier quirinius 65599 +xavier quirinius 65599 +xavier quirinius 65599 +xavier thompson 65608 +xavier underhill 65710 +xavier white 65703 +xavier white 65703 +xavier xylophone 65572 +xavier zipper 65561 +yuri allen 65565 +yuri allen 65565 +yuri brown 65538 +yuri brown 65538 +yuri carson 65670 +yuri carson 65670 +yuri ellison 65570 +yuri ellison 65570 +yuri falkner 65658 +yuri falkner 65658 +yuri garcia 65639 +yuri hernandez 65706 +yuri johnson 65587 +yuri johnson 65587 +yuri johnson 65587 +yuri king 65721 +yuri laertes 65637 +yuri laertes 65637 +yuri nixon 65635 +yuri nixon 65635 +yuri polk 65607 +yuri polk 65607 +yuri polk 65607 +yuri quirinius 65544 +yuri quirinius 65544 +yuri quirinius 65544 +yuri steinbeck 65592 +yuri steinbeck 65592 +yuri thompson 65676 +yuri underhill 65718 +yuri underhill 65718 +yuri white 65659 +yuri xylophone 65714 +zach allen 65667 +zach brown 65559 +zach brown 65559 +zach brown 65559 +zach brown 65559 +zach brown 65559 +zach carson 65572 +zach ellison 65748 +zach falkner 65620 +zach falkner 65620 +zach garcia 65544 +zach garcia 65544 +zach garcia 65544 +zach garcia 65544 +zach ichabod 65599 +zach ichabod 65599 +zach king 65556 +zach king 65556 +zach king 65556 +zach miller 65584 +zach miller 65584 +zach miller 65584 +zach ovid 65578 +zach ovid 65578 +zach ovid 65578 +zach ovid 65578 +zach quirinius 65691 +zach robinson 65599 +zach steinbeck 65602 +zach steinbeck 65602 +zach thompson 65636 +zach thompson 65636 +zach underhill 65573 +zach white 65733 +zach xylophone 65542 +zach xylophone 65542 +zach young 65576 +zach zipper 65579 +zach zipper 65579 +zach zipper 65579 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-1-2bf20f39e6ffef258858f7943a974e7e b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-1-2bf20f39e6ffef258858f7943a974e7e new file mode 100644 index 0000000000000000000000000000000000000000..657e81a94f4c5b1006e58307fa543f14952df902 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-1-2bf20f39e6ffef258858f7943a974e7e @@ -0,0 +1,1049 @@ + 9.220000267028809 + 43.72999954223633 + 89.52999877929688 +alice allen 2.7899999618530273 +alice allen 21.450000762939453 +alice allen 73.62999725341797 +alice brown 71.30999755859375 +alice carson 39.029998779296875 +alice davidson 70.3499984741211 +alice falkner 90.25 +alice garcia 48.45000076293945 +alice hernandez 88.16999816894531 +alice hernandez 90.55999755859375 +alice johnson 47.359999895095825 +alice king 19.139999389648438 +alice king 23.170000076293945 +alice king 52.22999954223633 +alice laertes 68.94999694824219 +alice laertes 69.52999877929688 +alice miller 68.95999908447266 +alice nixon 40.0 +alice nixon 48.150001525878906 +alice nixon 79.83000183105469 +alice ovid 9.039999961853027 +alice polk 62.900001525878906 +alice quirinius 37.13999938964844 +alice quirinius 62.29999923706055 +alice robinson 3.934999942779541 +alice robinson 56.099998474121094 +alice steinbeck 38.619998931884766 +alice steinbeck 63.40999794006348 +alice steinbeck 92.37000274658203 +alice underhill 93.17499923706055 +alice van buren 38.939998626708984 +alice xylophone 13.816667238871256 +alice xylophone 43.15999984741211 +alice xylophone 78.20999908447266 +alice zipper 26.43000030517578 +alice zipper 42.47999954223633 +alice zipper 89.93000030517578 +bob brown 8.069999694824219 +bob brown 70.93000030517578 +bob brown 93.08999633789062 +bob carson 50.09000015258789 +bob davidson 1.2899999618530273 +bob davidson 71.93000030517578 +bob davidson 74.72000122070312 +bob ellison 41.34000015258789 +bob ellison 56.584999084472656 +bob ellison 75.02999877929688 +bob ellison 80.30000305175781 +bob falkner 16.989999771118164 +bob garcia 4.460000038146973 +bob garcia 5.400000095367432 +bob garcia 43.46500015258789 +bob garcia 80.30000305175781 +bob garcia 87.56999969482422 +bob hernandez 55.51333363850912 +bob ichabod 82.55999755859375 +bob king 8.789999961853027 +bob king 12.539999961853027 +bob king 39.0099983215332 +bob laertes 0.7900000214576721 +bob laertes 10.670000076293945 +bob miller 61.91999816894531 +bob ovid 46.86000061035156 +bob ovid 62.849998474121094 +bob ovid 88.77999877929688 +bob ovid 97.08999633789062 +bob polk 7.980000019073486 +bob quirinius 48.09499931335449 +bob steinbeck 9.699999809265137 +bob van buren 33.66999816894531 +bob white 45.34000015258789 +bob white 45.349998474121094 +bob xylophone 27.6299991607666 +bob xylophone 29.359999656677246 +bob young 35.16999816894531 +bob zipper 3.819999933242798 +bob zipper 32.07500076293945 +bob zipper 52.63999938964844 +calvin allen 63.119998931884766 +calvin brown 28.110000610351562 +calvin brown 85.9000015258789 +calvin brown 90.19999694824219 +calvin carson 59.42500114440918 +calvin davidson 20.40500020980835 +calvin davidson 86.54000091552734 +calvin ellison 26.489999771118164 +calvin falkner 2.9700000286102295 +calvin falkner 56.040000915527344 +calvin falkner 56.33000183105469 +calvin falkner 80.5999984741211 +calvin falkner 93.61000061035156 +calvin falkner 94.30999755859375 +calvin garcia 41.849998474121094 +calvin hernandez 33.869998931884766 +calvin johnson 66.61000061035156 +calvin laertes 23.1299991607666 +calvin laertes 62.670000076293945 +calvin nixon 9.8100004196167 +calvin nixon 41.20000076293945 +calvin nixon 69.73999786376953 +calvin ovid 69.95999908447266 +calvin ovid 71.26000213623047 +calvin ovid 79.12000274658203 +calvin ovid 84.72000122070312 +calvin polk 65.72000122070312 +calvin quirinius 29.540000915527344 +calvin quirinius 60.98499870300293 +calvin robinson 40.439998626708984 +calvin steinbeck 15.220000267028809 +calvin steinbeck 22.850000381469727 +calvin steinbeck 52.06666819254557 +calvin thompson 8.90999984741211 +calvin thompson 89.22500228881836 +calvin underhill 59.70000076293945 +calvin van buren 34.209999084472656 +calvin van buren 64.0 +calvin white 35.864999771118164 +calvin white 90.69000244140625 +calvin xylophone 21.700000762939453 +calvin xylophone 25.420000076293945 +calvin xylophone 56.810001373291016 +calvin young 39.810001373291016 +calvin young 70.27999941507976 +calvin zipper 9.1899995803833 +calvin zipper 95.37999725341797 +david allen 51.005001068115234 +david allen 51.25 +david brown 32.56499910354614 +david brown 93.63999938964844 +david davidson 1.0800000429153442 +david davidson 54.17499876022339 +david davidson 62.720001220703125 +david davidson 74.1500015258789 +david ellison 47.689998626708984 +david ellison 62.587501525878906 +david ellison 85.2300033569336 +david hernandez 75.0833346048991 +david ichabod 35.763334115346275 +david ichabod 82.55000305175781 +david laertes 76.70999908447266 +david nixon 34.72999954223633 +david ovid 43.915000915527344 +david ovid 58.89999961853027 +david quirinius 23.5649995803833 +david quirinius 29.239999771118164 +david quirinius 79.97000122070312 +david robinson 37.703334172566734 +david robinson 70.55999755859375 +david thompson 43.619998931884766 +david underhill 49.719998359680176 +david underhill 60.36499881744385 +david underhill 60.46666717529297 +david van buren 25.500000476837158 +david van buren 50.27999973297119 +david white 45.189998626708984 +david xylophone 30.465000867843628 +david xylophone 30.465000867843628 +david xylophone 54.34000015258789 +david young 10.25 +david young 19.310000777244568 +ethan allen 23.790000518163044 +ethan brown 15.630000114440918 +ethan brown 22.93666648864746 +ethan brown 27.78000044822693 +ethan brown 52.19333346684774 +ethan brown 73.18000030517578 +ethan brown 82.30000305175781 +ethan carson 57.635000228881836 +ethan ellison 0.2800000011920929 +ethan ellison 81.47000122070312 +ethan falkner 50.02000045776367 +ethan falkner 59.43000030517578 +ethan garcia 26.44499921798706 +ethan hernandez 32.30333264668783 +ethan johnson 90.05000305175781 +ethan king 4.349999904632568 +ethan laertes 54.75 +ethan laertes 54.87999868392944 +ethan laertes 59.209999084472656 +ethan laertes 75.31500053405762 +ethan laertes 76.94499969482422 +ethan laertes 80.4749984741211 +ethan laertes 95.06999969482422 +ethan miller 25.3700008392334 +ethan nixon 50.88999938964844 +ethan ovid 58.4950008392334 +ethan polk 2.3499999046325684 +ethan polk 21.31999969482422 +ethan polk 23.440000534057617 +ethan polk 59.869998931884766 +ethan quirinius 13.650000214576721 +ethan quirinius 70.94500160217285 +ethan quirinius 88.18000030517578 +ethan robinson 67.94000244140625 +ethan robinson 80.58500289916992 +ethan underhill 55.630001068115234 +ethan van buren 37.85499954223633 +ethan white 58.70666631062826 +ethan white 63.41999816894531 +ethan xylophone 57.11000061035156 +ethan zipper 5.914999961853027 +ethan zipper 97.51000213623047 +fred davidson 26.435000896453857 +fred davidson 28.144000816345216 +fred davidson 78.30999755859375 +fred ellison 46.65999984741211 +fred ellison 65.40666516621907 +fred ellison 71.98499870300293 +fred falkner 25.144999980926514 +fred falkner 37.62000068028768 +fred falkner 75.125 +fred hernandez 55.9900016784668 +fred ichabod 37.06999969482422 +fred ichabod 75.83499908447266 +fred johnson 96.08999633789062 +fred king 10.220000267028809 +fred king 68.40500068664551 +fred laertes 42.68750047683716 +fred miller 70.02999877929688 +fred nixon 30.589999198913574 +fred nixon 32.71666653951009 +fred nixon 70.5199966430664 +fred nixon 93.02999877929688 +fred polk 23.959999084472656 +fred polk 47.31999969482422 +fred polk 63.97999954223633 +fred polk 90.12000274658203 +fred quirinius 15.300000190734863 +fred quirinius 27.40999984741211 +fred robinson 64.42000007629395 +fred steinbeck 21.239999771118164 +fred steinbeck 21.30000066757202 +fred steinbeck 65.44333394368489 +fred underhill 85.36999893188477 +fred van buren 22.37499976158142 +fred van buren 44.49000096321106 +fred van buren 45.94999980926514 +fred van buren 56.88999938964844 +fred white 42.329999923706055 +fred young 46.79999923706055 +fred young 72.69999885559082 +fred zipper 50.14000129699707 +gabriella allen 28.040000438690186 +gabriella allen 79.64500045776367 +gabriella brown 26.164999961853027 +gabriella brown 84.83000183105469 +gabriella carson 42.7599983215332 +gabriella davidson 18.946666717529297 +gabriella ellison 48.08000183105469 +gabriella ellison 71.54000091552734 +gabriella falkner 31.609999974568684 +gabriella falkner 51.720001220703125 +gabriella falkner 87.61000061035156 +gabriella garcia 43.0099983215332 +gabriella hernandez 62.22666517893473 +gabriella hernandez 92.9800033569336 +gabriella ichabod 10.729999542236328 +gabriella ichabod 26.639999389648438 +gabriella ichabod 66.36000061035156 +gabriella ichabod 71.12999725341797 +gabriella ichabod 83.76666514078777 +gabriella king 20.670000076293945 +gabriella king 62.89999961853027 +gabriella laertes 62.62499809265137 +gabriella miller 26.043334086736042 +gabriella ovid 44.78000068664551 +gabriella ovid 92.4000015258789 +gabriella polk 35.68000030517578 +gabriella polk 90.22500228881836 +gabriella steinbeck 46.45000076293945 +gabriella steinbeck 66.86999893188477 +gabriella thompson 72.34500122070312 +gabriella thompson 75.73666636149089 +gabriella thompson 94.25 +gabriella van buren 38.935001373291016 +gabriella van buren 48.349998474121094 +gabriella white 55.18000030517578 +gabriella young 32.16333262125651 +gabriella young 59.709999084472656 +gabriella zipper 36.2599983215332 +gabriella zipper 91.62999725341797 +holly allen 63.435001373291016 +holly brown 68.51666641235352 +holly brown 86.08666737874348 +holly falkner 83.82666524251302 +holly hernandez 21.190000534057617 +holly hernandez 24.790000915527344 +holly hernandez 28.649999300638836 +holly hernandez 50.22999954223633 +holly ichabod 48.86333433787028 +holly ichabod 73.77000045776367 +holly ichabod 82.91499710083008 +holly johnson 23.447500228881836 +holly johnson 64.36000061035156 +holly johnson 65.62000274658203 +holly king 35.34499979019165 +holly king 42.310001373291016 +holly laertes 70.59666697184245 +holly miller 35.86000061035156 +holly nixon 43.82499885559082 +holly nixon 78.80749893188477 +holly polk 30.365000247955322 +holly polk 75.96499919891357 +holly robinson 82.70499801635742 +holly thompson 0.07999999821186066 +holly thompson 65.52499842643738 +holly thompson 86.69000244140625 +holly underhill 42.54999923706055 +holly underhill 53.02000045776367 +holly underhill 56.13333384195963 +holly underhill 65.84000015258789 +holly van buren 48.893333435058594 +holly white 26.5633331934611 +holly white 40.5 +holly xylophone 49.55666716893514 +holly young 41.698571750095915 +holly young 75.20999908447266 +holly zipper 79.72999827067058 +holly zipper 81.08666610717773 +irene allen 29.75999927520752 +irene brown 28.596666653951008 +irene brown 47.189998626708984 +irene brown 49.46666622161865 +irene carson 86.64999898274739 +irene ellison 38.255001068115234 +irene ellison 45.71333376566569 +irene falkner 22.079999923706055 +irene falkner 83.44666544596355 +irene garcia 38.93499994277954 +irene garcia 42.84666601816813 +irene garcia 58.43000030517578 +irene ichabod 60.7079984664917 +irene ichabod 64.58000183105469 +irene johnson 26.165000438690186 +irene laertes 21.02999997138977 +irene laertes 40.04499816894531 +irene laertes 47.04333241780599 +irene miller 65.44000244140625 +irene nixon 46.03999996185303 +irene nixon 46.96666653951009 +irene nixon 67.09499931335449 +irene ovid 35.130001068115234 +irene ovid 42.535000801086426 +irene ovid 79.75 +irene polk 0.9800000190734863 +irene polk 35.17500114440918 +irene polk 48.94666576385498 +irene polk 49.6 +irene polk 51.885000228881836 +irene quirinius 33.78000005086263 +irene quirinius 42.610000133514404 +irene quirinius 53.4800001780192 +irene robinson 92.19499969482422 +irene steinbeck 73.28000068664551 +irene thompson 46.27375066280365 +irene underhill 24.75 +irene underhill 57.349998474121094 +irene van buren 50.8799991607666 +irene van buren 74.5625 +irene xylophone 83.53499984741211 +jessica brown 26.185000479221344 +jessica carson 56.22999954223633 +jessica carson 62.20000076293945 +jessica carson 62.2400016784668 +jessica davidson 50.01666768391927 +jessica davidson 63.59499931335449 +jessica davidson 69.26666514078777 +jessica davidson 94.53333282470703 +jessica ellison 11.180000305175781 +jessica ellison 64.2060001373291 +jessica falkner 61.57333437601725 +jessica garcia 38.55250036716461 +jessica garcia 57.00999975204468 +jessica ichabod 32.63250035047531 +jessica johnson 9.5600004196167 +jessica johnson 51.959999084472656 +jessica miller 77.83999633789062 +jessica nixon 40.72249960899353 +jessica nixon 90.06999969482422 +jessica ovid 37.45250064134598 +jessica ovid 59.68000030517578 +jessica polk 49.68000030517578 +jessica quirinius 25.65750002861023 +jessica quirinius 37.64200019836426 +jessica quirinius 54.25500011444092 +jessica quirinius 58.019999186197914 +jessica robinson 42.66333484649658 +jessica thompson 30.40666675567627 +jessica thompson 43.87500023841858 +jessica underhill 43.33333269755045 +jessica underhill 45.639999866485596 +jessica underhill 57.584999084472656 +jessica van buren 67.00000047683716 +jessica white 6.170000106096268 +jessica white 63.32500076293945 +jessica white 65.1450023651123 +jessica white 73.93000030517578 +jessica white 96.62000274658203 +jessica xylophone 69.87500190734863 +jessica young 11.1899995803833 +jessica young 43.369998931884766 +jessica zipper 42.43833335240682 +jessica zipper 46.7450008392334 +jessica zipper 56.97999954223633 +katie allen 55.47666549682617 +katie brown 31.699999809265137 +katie davidson 93.22000122070312 +katie ellison 48.31999933719635 +katie ellison 64.08499892552693 +katie falkner 51.665000915527344 +katie garcia 57.71000099182129 +katie garcia 61.21000051498413 +katie hernandez 41.150000381469724 +katie ichabod 44.243333180745445 +katie ichabod 51.800000508626304 +katie ichabod 69.18799896240235 +katie king 39.83000183105469 +katie king 46.80333296457926 +katie king 51.85000038146973 +katie miller 31.399999618530273 +katie miller 74.77999877929688 +katie nixon 23.190000534057617 +katie ovid 67.94500160217285 +katie polk 26.62750005722046 +katie polk 33.9350004196167 +katie robinson 13.890000343322754 +katie van buren 44.434998512268066 +katie van buren 65.41999816894531 +katie white 37.96500015258789 +katie white 59.223333517710365 +katie xylophone 39.30000019073486 +katie young 36.660000801086426 +katie young 67.78333282470703 +katie young 72.76666577657063 +katie zipper 23.766667087872822 +katie zipper 58.75 +luke allen 50.959999084472656 +luke allen 53.36666742960612 +luke allen 54.63249969482422 +luke allen 57.670000076293945 +luke allen 70.39500045776367 +luke brown 49.595000982284546 +luke davidson 7.050000190734863 +luke davidson 18.87000036239624 +luke ellison 16.25 +luke ellison 32.9519996881485 +luke ellison 71.93500137329102 +luke falkner 21.71999979019165 +luke falkner 31.81250023841858 +luke garcia 18.65499973297119 +luke garcia 41.2300001780192 +luke ichabod 41.25750005245209 +luke ichabod 73.55000114440918 +luke johnson 31.670000076293945 +luke johnson 32.84499979019165 +luke johnson 39.54500102996826 +luke laertes 11.819999694824219 +luke laertes 21.184999227523804 +luke laertes 21.993332862854004 +luke laertes 26.696666717529297 +luke laertes 45.9900016784668 +luke miller 52.350000858306885 +luke ovid 23.804999828338623 +luke ovid 64.30000305175781 +luke polk 41.02499961853027 +luke polk 58.4566650390625 +luke quirinius 40.41999816894531 +luke robinson 48.559998750686646 +luke robinson 56.76499938964844 +luke thompson 78.04333368937175 +luke underhill 34.0166662534078 +luke underhill 47.28999996185303 +luke underhill 59.32000160217285 +luke van buren 59.91999944051107 +luke white 74.19599990844726 +luke xylophone 64.77999925613403 +luke zipper 30.434999465942383 +mike allen 30.539999961853027 +mike brown 69.86833318074544 +mike carson 30.25333309173584 +mike carson 61.33799934387207 +mike carson 89.375 +mike davidson 32.55333391825358 +mike davidson 66.74333318074544 +mike ellison 35.905999755859376 +mike ellison 39.82499885559082 +mike ellison 58.56399993896484 +mike ellison 64.52999877929688 +mike ellison 66.93749856948853 +mike falkner 48.53750002384186 +mike garcia 51.02999973297119 +mike garcia 67.93000030517578 +mike garcia 70.8499984741211 +mike hernandez 37.900001525878906 +mike hernandez 59.45000076293945 +mike ichabod 64.7699966430664 +mike king 36.17800045013428 +mike king 41.69500136375427 +mike king 49.57000017166138 +mike king 59.654998779296875 +mike king 71.57000122070312 +mike king 78.50999927520752 +mike miller 29.570000171661377 +mike nixon 45.029999828338624 +mike nixon 48.429999669392906 +mike polk 30.864000129699708 +mike polk 46.95499873161316 +mike polk 79.55500030517578 +mike quirinius 85.0699971516927 +mike steinbeck 24.267500042915344 +mike steinbeck 43.52500021457672 +mike steinbeck 61.426666577657066 +mike steinbeck 68.46000022888184 +mike van buren 27.639999389648438 +mike van buren 56.16333134969076 +mike white 34.8924994468689 +mike white 43.5566660563151 +mike white 53.689998626708984 +mike white 77.54499864578247 +mike young 34.3319993019104 +mike young 52.8100004196167 +mike young 55.64333359400431 +mike zipper 56.86666742960612 +mike zipper 63.3149995803833 +mike zipper 83.91999816894531 +nick allen 57.086001586914065 +nick allen 60.15400094985962 +nick brown 42.939998626708984 +nick davidson 63.07499885559082 +nick ellison 45.34000015258789 +nick ellison 65.88500022888184 +nick falkner 41.87999868392944 +nick falkner 64.05666732788086 +nick garcia 34.34499979019165 +nick garcia 51.08666737874349 +nick garcia 62.88600044250488 +nick ichabod 20.253333409627277 +nick ichabod 53.635000228881836 +nick ichabod 77.36000061035156 +nick johnson 20.114999175071716 +nick johnson 81.91666666666667 +nick laertes 91.56666819254558 +nick miller 71.5500005086263 +nick nixon 77.04249954223633 +nick ovid 74.62666702270508 +nick polk 39.27500009536743 +nick quirinius 60.79499816894531 +nick quirinius 67.44999694824219 +nick robinson 31.672499418258667 +nick robinson 57.66999816894531 +nick steinbeck 59.15999984741211 +nick thompson 18.88666645685832 +nick underhill 43.009998893737794 +nick van buren 34.720000902811684 +nick xylophone 75.3499984741211 +nick young 0.27000001072883606 +nick young 47.813334147135414 +nick zipper 46.22333272298177 +nick zipper 52.54333209991455 +oscar allen 37.396666844685875 +oscar brown 13.100000381469727 +oscar carson 31.91333230336507 +oscar carson 41.77333331108093 +oscar carson 57.3149995803833 +oscar carson 73.59500122070312 +oscar carson 95.44000244140625 +oscar davidson 75.18000030517578 +oscar ellison 34.04499959945679 +oscar ellison 34.04499959945679 +oscar falkner 61.72000050544739 +oscar garcia 67.4800033569336 +oscar hernandez 41.63333400090536 +oscar hernandez 47.93999986648559 +oscar ichabod 45.839999516805015 +oscar ichabod 68.62000274658203 +oscar ichabod 72.18249797821045 +oscar ichabod 76.69000244140625 +oscar johnson 23.880000114440918 +oscar johnson 65.04000091552734 +oscar king 36.69500017166138 +oscar king 49.7049994468689 +oscar king 67.98399925231934 +oscar laertes 43.616665522257485 +oscar laertes 44.755000591278076 +oscar laertes 45.26666831970215 +oscar laertes 53.710001945495605 +oscar nixon 36.56999937693278 +oscar ovid 45.89999961853027 +oscar ovid 46.93999934196472 +oscar ovid 55.277999591827395 +oscar polk 42.31999969482422 +oscar polk 63.900001525878906 +oscar quirinius 63.81500053405762 +oscar quirinius 66.28428527287075 +oscar quirinius 70.24000295003255 +oscar quirinius 81.26249980926514 +oscar robinson 11.34000015258789 +oscar robinson 47.845001220703125 +oscar robinson 59.74333349863688 +oscar robinson 63.346666971842446 +oscar steinbeck 42.49999976158142 +oscar thompson 38.23500061035156 +oscar thompson 51.469999154408775 +oscar thompson 60.029999542236325 +oscar thompson 63.079999923706055 +oscar underhill 66.97666676839192 +oscar van buren 24.085000872612 +oscar van buren 61.880001068115234 +oscar van buren 72.9533322652181 +oscar white 44.72333272298177 +oscar white 46.60999870300293 +oscar white 54.7599983215332 +oscar white 60.85500144958496 +oscar xylophone 34.946666399637856 +oscar xylophone 39.8299994468689 +oscar xylophone 57.119998931884766 +oscar zipper 28.499999046325684 +oscar zipper 47.46750068664551 +oscar zipper 59.1933339436849 +priscilla brown 47.40400066375732 +priscilla brown 77.1479995727539 +priscilla brown 80.5199966430664 +priscilla carson 7.960000038146973 +priscilla carson 28.480000153183937 +priscilla carson 45.92750144004822 +priscilla ichabod 38.95666758219401 +priscilla ichabod 62.32999928792318 +priscilla johnson 44.04499912261963 +priscilla johnson 50.53750038146973 +priscilla johnson 55.98333485921224 +priscilla johnson 59.64499855041504 +priscilla johnson 89.1500015258789 +priscilla king 50.44666735331217 +priscilla nixon 44.32222270965576 +priscilla nixon 45.267999792099 +priscilla ovid 44.78333361943563 +priscilla ovid 52.72999954223633 +priscilla polk 34.89399948120117 +priscilla quirinius 35.609999895095825 +priscilla thompson 35.16249918937683 +priscilla underhill 68.22000122070312 +priscilla underhill 73.97200012207031 +priscilla van buren 50.47000026702881 +priscilla van buren 51.39500045776367 +priscilla van buren 53.541999435424806 +priscilla white 50.47599992752075 +priscilla xylophone 0.15000000596046448 +priscilla xylophone 41.106666485468544 +priscilla xylophone 63.9574990272522 +priscilla young 0.2900000065565109 +priscilla young 19.866666316986084 +priscilla zipper 32.084999322891235 +priscilla zipper 43.90333366394043 +quinn allen 47.90333382288615 +quinn allen 83.33000183105469 +quinn brown 24.280000686645508 +quinn brown 53.98666508992513 +quinn brown 66.82500171661377 +quinn davidson 40.666666666666664 +quinn davidson 54.095001220703125 +quinn davidson 79.78333282470703 +quinn davidson 92.13000106811523 +quinn ellison 52.714999198913574 +quinn ellison 63.352500438690186 +quinn garcia 20.19000056385994 +quinn garcia 54.60000038146973 +quinn garcia 59.010000824928284 +quinn garcia 68.98999881744385 +quinn ichabod 48.60499930381775 +quinn king 61.27333450317383 +quinn king 81.46000289916992 +quinn laertes 32.08000040054321 +quinn laertes 44.45666694641113 +quinn laertes 49.85499858856201 +quinn nixon 72.2471422467913 +quinn ovid 34.423333168029785 +quinn quirinius 53.165000915527344 +quinn robinson 32.624999046325684 +quinn steinbeck 24.802499771118164 +quinn steinbeck 55.477500915527344 +quinn thompson 50.500000381469725 +quinn thompson 55.68600006103516 +quinn underhill 39.66600060462952 +quinn underhill 41.47666676839193 +quinn underhill 56.580000162124634 +quinn van buren 49.40333207448324 +quinn young 55.59000142415365 +quinn zipper 11.359999974568685 +quinn zipper 48.45000123977661 +rachel allen 45.940001249313354 +rachel allen 85.97999954223633 +rachel brown 33.01999984184901 +rachel brown 34.08250021934509 +rachel brown 37.999999046325684 +rachel brown 41.75000019868215 +rachel brown 53.679999669392906 +rachel carson 43.32400016784668 +rachel carson 66.2233320871989 +rachel davidson 14.220000267028809 +rachel ellison 17.549999833106995 +rachel falkner 56.883334159851074 +rachel falkner 57.5199998219808 +rachel falkner 58.80666637420654 +rachel falkner 70.69428443908691 +rachel johnson 36.22499990463257 +rachel king 50.970001220703125 +rachel king 83.53750133514404 +rachel laertes 42.29857151848929 +rachel laertes 71.65999984741211 +rachel ovid 42.25333329041799 +rachel ovid 47.01749947667122 +rachel polk 64.90333239237468 +rachel quirinius 53.2624990940094 +rachel robinson 40.712857246398926 +rachel robinson 53.092498898506165 +rachel robinson 64.94999694824219 +rachel thompson 24.555000439286232 +rachel thompson 31.460000038146973 +rachel thompson 46.804000282287596 +rachel underhill 47.22333272298177 +rachel white 39.87999979654948 +rachel white 41.83428575311388 +rachel young 75.7966677347819 +rachel zipper 45.794999519983925 +rachel zipper 56.909999179840085 +sarah carson 24.576666196187336 +sarah carson 36.33750060200691 +sarah carson 43.65749907493591 +sarah ellison 37.054999351501465 +sarah falkner 48.58285754067557 +sarah falkner 62.36500072479248 +sarah garcia 33.38000011444092 +sarah garcia 35.513333002726235 +sarah garcia 64.31333287556966 +sarah ichabod 36.10599975585937 +sarah ichabod 45.830000162124634 +sarah johnson 26.464999675750732 +sarah johnson 40.9300012588501 +sarah johnson 43.44000196456909 +sarah johnson 64.24333318074544 +sarah king 49.06999909877777 +sarah king 63.01333363850912 +sarah miller 41.709999084472656 +sarah ovid 63.682499408721924 +sarah robinson 39.196666399637856 +sarah robinson 66.88999938964844 +sarah steinbeck 66.89000034332275 +sarah white 41.42599945068359 +sarah white 52.95249938964844 +sarah xylophone 68.31999969482422 +sarah young 35.92750024795532 +sarah zipper 53.697500705718994 +tom brown 38.37000020345052 +tom brown 44.68000049591065 +tom carson 27.994999766349792 +tom carson 54.25250005722046 +tom carson 62.790000915527344 +tom davidson 38.679999113082886 +tom ellison 33.68600053787232 +tom ellison 46.00666618347168 +tom ellison 67.79666646321614 +tom falkner 55.61800079345703 +tom falkner 58.82500012715658 +tom hernandez 50.52250051498413 +tom hernandez 50.52250051498413 +tom ichabod 24.98399963378906 +tom johnson 34.83750009536743 +tom johnson 73.72399978637695 +tom king 69.98000106811523 +tom laertes 41.97285750934056 +tom laertes 70.40333429972331 +tom miller 43.885000586509705 +tom miller 57.10500144958496 +tom miller 76.20499992370605 +tom nixon 62.43000030517578 +tom ovid 38.096666971842446 +tom polk 51.26750087738037 +tom polk 68.22666676839192 +tom quirinius 37.720001220703125 +tom quirinius 53.20399913787842 +tom robinson 43.44333299001058 +tom robinson 54.637142998831614 +tom robinson 59.34250068664551 +tom robinson 99.1500015258789 +tom steinbeck 51.883334477742515 +tom van buren 28.380000829696655 +tom van buren 35.64999930063883 +tom van buren 54.59000015258789 +tom white 51.970001220703125 +tom young 44.7319995880127 +tom young 53.894999980926514 +tom zipper 55.44000116984049 +ulysses brown 48.72666708628336 +ulysses carson 38.742000579833984 +ulysses carson 45.513333002726235 +ulysses carson 48.75249934196472 +ulysses carson 74.64600067138672 +ulysses davidson 63.20857129778181 +ulysses ellison 68.52666759490967 +ulysses garcia 58.77250051498413 +ulysses hernandez 32.371999168395995 +ulysses hernandez 50.57000102996826 +ulysses hernandez 61.39999961853027 +ulysses ichabod 19.1299991607666 +ulysses ichabod 83.06666692097981 +ulysses johnson 51.485000451405845 +ulysses king 46.98333422342936 +ulysses laertes 29.046666741371155 +ulysses laertes 32.88599967956543 +ulysses laertes 60.12399845123291 +ulysses miller 44.552857535226 +ulysses miller 71.39249873161316 +ulysses nixon 51.300000286102296 +ulysses ovid 29.360000610351562 +ulysses polk 40.74399948120117 +ulysses polk 48.9800017674764 +ulysses polk 57.86249828338623 +ulysses polk 81.21333312988281 +ulysses quirinius 68.41500091552734 +ulysses robinson 69.53999853134155 +ulysses steinbeck 44.61833381652832 +ulysses steinbeck 48.362499713897705 +ulysses thompson 45.063334465026855 +ulysses underhill 30.829999764760334 +ulysses underhill 41.43857192993164 +ulysses underhill 44.08333269755045 +ulysses underhill 55.470001220703125 +ulysses underhill 58.9471435546875 +ulysses underhill 68.1900007724762 +ulysses underhill 78.83333333333333 +ulysses van buren 72.38428633553642 +ulysses white 36.17250043153763 +ulysses white 39.084000778198245 +ulysses xylophone 27.519999504089355 +ulysses xylophone 47.65999937057495 +ulysses xylophone 50.29999923706055 +ulysses young 23.308333079020183 +ulysses young 34.6339994430542 +ulysses young 88.06999969482422 +victor allen 49.43800010681152 +victor allen 56.7299998147147 +victor brown 40.80600037574768 +victor brown 63.5024995803833 +victor brown 71.03500080108643 +victor brown 81.71999931335449 +victor davidson 44.70333290100098 +victor davidson 59.070000076293944 +victor davidson 67.27199935913086 +victor ellison 31.28999964396159 +victor ellison 42.54999923706055 +victor hernandez 44.41333452860514 +victor hernandez 47.20249938964844 +victor hernandez 47.73333215713501 +victor hernandez 51.04999923706055 +victor hernandez 59.2399995803833 +victor johnson 54.868000626564026 +victor johnson 55.22999954223633 +victor johnson 57.41000175476074 +victor king 38.27999997138977 +victor king 49.993333180745445 +victor laertes 40.63500006993612 +victor laertes 80.5999984741211 +victor miller 71.00000190734863 +victor nixon 38.393332640329994 +victor nixon 52.920000076293945 +victor ovid 53.260000228881836 +victor polk 3.0 +victor quirinius 57.81666644414266 +victor quirinius 59.39999923706055 +victor robinson 24.614999771118164 +victor robinson 74.5049991607666 +victor steinbeck 28.862000381946565 +victor steinbeck 36.61000006539481 +victor steinbeck 43.09000015258789 +victor thompson 42.67599925994873 +victor van buren 44.669999877611794 +victor van buren 45.121999740600586 +victor white 53.67999887466431 +victor white 54.45000012715658 +victor xylophone 12.160000324249268 +victor xylophone 26.0 +victor xylophone 31.769999821980793 +victor xylophone 52.31499926249186 +victor xylophone 69.2899996439616 +victor young 64.25833320617676 +victor zipper 65.24999904632568 +wendy allen 34.04999955495199 +wendy allen 36.88199939727783 +wendy allen 44.96000012755394 +wendy brown 45.97833283742269 +wendy brown 52.73857225690569 +wendy ellison 42.91333246231079 +wendy ellison 53.56000073750814 +wendy falkner 47.602500438690186 +wendy falkner 64.9099988937378 +wendy falkner 77.5999984741211 +wendy garcia 37.38571425846645 +wendy garcia 48.76666768391927 +wendy garcia 53.225000858306885 +wendy garcia 63.93999926249186 +wendy hernandez 36.195000648498535 +wendy ichabod 10.56499981880188 +wendy king 37.57500076293945 +wendy king 53.44333457946777 +wendy king 56.319997787475586 +wendy laertes 38.39249920845032 +wendy laertes 60.19999885559082 +wendy laertes 65.30624961853027 +wendy miller 44.273332595825195 +wendy miller 53.5675014257431 +wendy nixon 54.995998764038085 +wendy nixon 64.28250026702881 +wendy ovid 43.80499863624573 +wendy ovid 61.64600105285645 +wendy polk 26.784999758005142 +wendy polk 35.21599998474121 +wendy quirinius 28.75666618347168 +wendy quirinius 60.70000092188517 +wendy robinson 42.5799994468689 +wendy robinson 42.90799944400787 +wendy robinson 43.426000237464905 +wendy steinbeck 43.42333388328552 +wendy thompson 40.352857317243306 +wendy thompson 75.93666712443034 +wendy underhill 24.459999561309814 +wendy underhill 33.440000693003334 +wendy underhill 45.51625019311905 +wendy van buren 43.30333296457926 +wendy van buren 65.58666737874348 +wendy white 39.015000104904175 +wendy xylophone 42.42500034968058 +wendy xylophone 53.981666247049965 +wendy young 27.929999828338623 +wendy young 59.609999656677246 +xavier allen 49.24500061571598 +xavier allen 67.39000034332275 +xavier allen 70.29800033569336 +xavier brown 19.772500306367874 +xavier brown 58.87000111171177 +xavier brown 74.6200008392334 +xavier carson 47.3199987411499 +xavier carson 61.52250027656555 +xavier davidson 48.41999936103821 +xavier davidson 52.70666758219401 +xavier davidson 52.78166747093201 +xavier ellison 41.84999942779541 +xavier ellison 62.80200090408325 +xavier garcia 35.0600004568696 +xavier hernandez 47.45200023651123 +xavier hernandez 49.676666259765625 +xavier hernandez 53.446667432785034 +xavier ichabod 56.70625042915344 +xavier ichabod 60.54799928665161 +xavier johnson 41.16333262125651 +xavier johnson 53.85333331425985 +xavier king 42.72800064086914 +xavier king 66.05333455403645 +xavier laertes 38.47999954223633 +xavier ovid 48.89250057935715 +xavier polk 37.05500018596649 +xavier polk 46.82666703065237 +xavier polk 55.385000705718994 +xavier polk 55.65000057220459 +xavier quirinius 59.62499964237213 +xavier quirinius 60.055998992919925 +xavier quirinius 62.52000045776367 +xavier quirinius 65.2933349609375 +xavier thompson 40.244998931884766 +xavier underhill 26.27800006866455 +xavier white 47.8671429497855 +xavier white 63.38428551810129 +xavier xylophone 49.072500228881836 +xavier zipper 8.204999923706055 +yuri allen 53.61250066757202 +yuri allen 64.86833254496257 +yuri brown 46.57500044504801 +yuri brown 66.75250005722046 +yuri carson 39.40750050544739 +yuri carson 49.01600036621094 +yuri ellison 27.49000017642975 +yuri ellison 70.5933329264323 +yuri falkner 47.23285675048828 +yuri falkner 62.807999801635745 +yuri garcia 43.967499017715454 +yuri hernandez 31.94000039100647 +yuri johnson 21.40666739145915 +yuri johnson 34.02333414554596 +yuri johnson 65.7750015258789 +yuri king 49.47333272298177 +yuri laertes 42.070000648498535 +yuri laertes 60.7549991607666 +yuri nixon 49.87142838750567 +yuri nixon 59.945000330607094 +yuri polk 37.56249952316284 +yuri polk 47.583333333333336 +yuri polk 72.60888735453288 +yuri quirinius 18.62000060081482 +yuri quirinius 51.217501401901245 +yuri quirinius 67.24000072479248 +yuri steinbeck 55.757999420166016 +yuri steinbeck 75.87999725341797 +yuri thompson 36.93499946594238 +yuri underhill 51.533334732055664 +yuri underhill 62.31888887617323 +yuri white 44.34999983651297 +yuri xylophone 25.117499828338623 +zach allen 25.92333350578944 +zach brown 38.3799991607666 +zach brown 47.404998779296875 +zach brown 54.30600090026856 +zach brown 58.970001220703125 +zach brown 65.22499942779541 +zach carson 60.783999633789065 +zach ellison 36.211428437914165 +zach falkner 41.225714683532715 +zach falkner 65.99499940872192 +zach garcia 42.8885714326586 +zach garcia 46.8870005607605 +zach garcia 47.5049991607666 +zach garcia 66.09399967193603 +zach ichabod 40.10166613260905 +zach ichabod 53.16749954223633 +zach king 39.137500405311584 +zach king 48.2825003862381 +zach king 61.18999965985616 +zach miller 44.82800054550171 +zach miller 48.52428477151053 +zach miller 53.593332608540855 +zach ovid 35.19399921447039 +zach ovid 38.35833342870077 +zach ovid 43.87200012207031 +zach ovid 83.01999918619792 +zach quirinius 42.638333320617676 +zach robinson 82.04999923706055 +zach steinbeck 55.86599960327148 +zach steinbeck 67.81428473336356 +zach thompson 29.303333282470703 +zach thompson 46.48999913533529 +zach underhill 48.681429045540945 +zach white 66.60250091552734 +zach xylophone 41.875 +zach xylophone 57.2416664759318 +zach young 73.5999984741211 +zach zipper 58.1480016708374 +zach zipper 60.1825008392334 +zach zipper 62.794999837875366 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-2-16239d2b069789ba99fbac50c4f0724f b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-2-16239d2b069789ba99fbac50c4f0724f new file mode 100644 index 0000000000000000000000000000000000000000..6cfa5ad413fa4b0afb21712257a1959319134ff6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-2-16239d2b069789ba99fbac50c4f0724f @@ -0,0 +1,1049 @@ + 65560.0 + 65718.0 + 65740.0 +alice allen 65662.0 +alice allen 65720.0 +alice allen 65758.0 +alice brown 65696.0 +alice carson 65559.0 +alice davidson 65547.0 +alice falkner 65669.0 +alice garcia 65613.0 +alice hernandez 65737.0 +alice hernandez 65784.0 +alice johnson 65739.0 +alice king 65660.0 +alice king 65734.0 +alice king 65738.0 +alice laertes 65669.0 +alice laertes 65671.0 +alice miller 65590.0 +alice nixon 65586.0 +alice nixon 65595.0 +alice nixon 65604.0 +alice ovid 65737.0 +alice polk 65548.0 +alice quirinius 65636.0 +alice quirinius 65728.0 +alice robinson 65606.0 +alice robinson 65789.0 +alice steinbeck 65578.0 +alice steinbeck 65673.0 +alice steinbeck 65786.0 +alice underhill 65750.0 +alice van buren 65562.0 +alice xylophone 65578.0 +alice xylophone 65585.0 +alice xylophone 65599.0 +alice zipper 65553.0 +alice zipper 65662.0 +alice zipper 65766.0 +bob brown 65584.0 +bob brown 65777.0 +bob brown 65783.0 +bob carson 65713.0 +bob davidson 65664.0 +bob davidson 65693.0 +bob davidson 65768.0 +bob ellison 65591.0 +bob ellison 65624.0 +bob ellison 65721.0 +bob ellison 65760.0 +bob falkner 65789.0 +bob garcia 65585.0 +bob garcia 65598.0 +bob garcia 65673.0 +bob garcia 65754.0 +bob garcia 65782.0 +bob hernandez 65557.0 +bob ichabod 65549.0 +bob king 65715.0 +bob king 65757.0 +bob king 65783.0 +bob laertes 65602.0 +bob laertes 65663.0 +bob miller 65608.0 +bob ovid 65564.0 +bob ovid 65619.0 +bob ovid 65686.0 +bob ovid 65726.0 +bob polk 65594.0 +bob quirinius 65700.0 +bob steinbeck 65637.0 +bob van buren 65778.0 +bob white 65543.0 +bob white 65605.0 +bob xylophone 65574.0 +bob xylophone 65666.0 +bob young 65556.0 +bob zipper 65559.0 +bob zipper 65633.0 +bob zipper 65739.0 +calvin allen 65669.0 +calvin brown 65537.0 +calvin brown 65580.0 +calvin brown 65677.0 +calvin carson 65637.0 +calvin davidson 65541.0 +calvin davidson 65564.0 +calvin ellison 65667.0 +calvin falkner 65573.0 +calvin falkner 65596.0 +calvin falkner 65738.0 +calvin falkner 65762.0 +calvin falkner 65778.0 +calvin falkner 65784.0 +calvin garcia 65664.0 +calvin hernandez 65578.0 +calvin johnson 65731.0 +calvin laertes 65570.0 +calvin laertes 65684.0 +calvin nixon 65654.0 +calvin nixon 65724.0 +calvin nixon 65749.0 +calvin ovid 65554.0 +calvin ovid 65643.0 +calvin ovid 65663.0 +calvin ovid 65715.0 +calvin polk 65731.0 +calvin quirinius 65741.0 +calvin quirinius 65769.0 +calvin robinson 65581.0 +calvin steinbeck 65680.0 +calvin steinbeck 65762.0 +calvin steinbeck 65779.0 +calvin thompson 65560.0 +calvin thompson 65640.0 +calvin underhill 65732.0 +calvin van buren 65552.0 +calvin van buren 65771.0 +calvin white 65553.0 +calvin white 65561.0 +calvin xylophone 65575.0 +calvin xylophone 65614.5 +calvin xylophone 65713.0 +calvin young 65574.0 +calvin young 65746.0 +calvin zipper 65669.0 +calvin zipper 65739.0 +david allen 65588.0 +david allen 65617.0 +david brown 65637.0 +david brown 65760.0 +david davidson 65559.0 +david davidson 65756.0 +david davidson 65778.0 +david davidson 65779.0 +david ellison 65634.0 +david ellison 65724.0 +david ellison 65724.0 +david hernandez 65763.0 +david ichabod 65699.0 +david ichabod 65715.0 +david laertes 65762.0 +david nixon 65536.0 +david ovid 65623.0 +david ovid 65628.0 +david quirinius 65697.0 +david quirinius 65759.0 +david quirinius 65779.0 +david robinson 65762.0 +david robinson 65775.0 +david thompson 65550.0 +david underhill 65602.0 +david underhill 65662.0 +david underhill 65751.0 +david van buren 65625.0 +david van buren 65634.0 +david white 65678.0 +david xylophone 65537.0 +david xylophone 65670.0 +david xylophone 65764.0 +david young 65551.0 +david young 65694.0 +ethan allen 65747.0 +ethan brown 65539.0 +ethan brown 65617.0 +ethan brown 65685.0 +ethan brown 65685.0 +ethan brown 65722.0 +ethan brown 65733.0 +ethan carson 65742.0 +ethan ellison 65714.0 +ethan ellison 65732.0 +ethan falkner 65577.0 +ethan falkner 65614.0 +ethan garcia 65736.0 +ethan hernandez 65630.5 +ethan johnson 65536.0 +ethan king 65614.0 +ethan laertes 65562.0 +ethan laertes 65597.0 +ethan laertes 65628.0 +ethan laertes 65643.0 +ethan laertes 65680.0 +ethan laertes 65745.0 +ethan laertes 65760.0 +ethan miller 65712.0 +ethan nixon 65766.0 +ethan ovid 65697.0 +ethan polk 65589.0 +ethan polk 65615.0 +ethan polk 65622.0 +ethan polk 65622.0 +ethan quirinius 65542.0 +ethan quirinius 65591.0 +ethan quirinius 65706.0 +ethan robinson 65547.0 +ethan robinson 65659.0 +ethan underhill 65570.0 +ethan van buren 65572.0 +ethan white 65677.0 +ethan white 65788.0 +ethan xylophone 65595.0 +ethan zipper 65593.0 +ethan zipper 65680.0 +fred davidson 65552.0 +fred davidson 65595.0 +fred davidson 65721.0 +fred ellison 65548.0 +fred ellison 65691.0 +fred ellison 65771.0 +fred falkner 65637.0 +fred falkner 65648.0 +fred falkner 65783.0 +fred hernandez 65541.0 +fred ichabod 65572.0 +fred ichabod 65789.0 +fred johnson 65758.0 +fred king 65694.0 +fred king 65745.0 +fred laertes 65769.0 +fred miller 65536.0 +fred nixon 65560.0 +fred nixon 65612.0 +fred nixon 65703.0 +fred nixon 65705.0 +fred polk 65603.0 +fred polk 65656.0 +fred polk 65701.0 +fred polk 65706.0 +fred quirinius 65697.0 +fred quirinius 65782.0 +fred robinson 65623.0 +fred steinbeck 65544.0 +fred steinbeck 65651.0 +fred steinbeck 65755.0 +fred underhill 65629.0 +fred van buren 65537.0 +fred van buren 65561.0 +fred van buren 65745.0 +fred van buren 65789.0 +fred white 65589.0 +fred young 65594.0 +fred young 65773.0 +fred zipper 65553.0 +gabriella allen 65646.0 +gabriella allen 65677.0 +gabriella brown 65704.0 +gabriella brown 65753.0 +gabriella carson 65586.0 +gabriella davidson 65565.0 +gabriella ellison 65706.0 +gabriella ellison 65716.0 +gabriella falkner 65623.0 +gabriella falkner 65711.0 +gabriella falkner 65767.0 +gabriella garcia 65571.0 +gabriella hernandez 65587.0 +gabriella hernandez 65717.0 +gabriella ichabod 65559.0 +gabriella ichabod 65633.0 +gabriella ichabod 65702.0 +gabriella ichabod 65712.0 +gabriella ichabod 65717.0 +gabriella king 65651.0 +gabriella king 65657.0 +gabriella laertes 65781.0 +gabriella miller 65646.0 +gabriella ovid 65556.0 +gabriella ovid 65583.0 +gabriella polk 65701.0 +gabriella polk 65790.0 +gabriella steinbeck 65582.0 +gabriella steinbeck 65653.0 +gabriella thompson 65682.0 +gabriella thompson 65755.0 +gabriella thompson 65766.0 +gabriella van buren 65581.0 +gabriella van buren 65644.0 +gabriella white 65638.0 +gabriella young 65699.0 +gabriella young 65774.0 +gabriella zipper 65540.0 +gabriella zipper 65754.0 +holly allen 65596.0 +holly brown 65599.0 +holly brown 65619.0 +holly falkner 65720.0 +holly hernandez 65602.0 +holly hernandez 65686.0 +holly hernandez 65750.0 +holly hernandez 65788.0 +holly ichabod 65711.0 +holly ichabod 65749.0 +holly ichabod 65752.0 +holly johnson 65655.0 +holly johnson 65662.0 +holly johnson 65755.0 +holly king 65549.0 +holly king 65648.0 +holly laertes 65664.0 +holly miller 65653.0 +holly nixon 65539.0 +holly nixon 65658.0 +holly polk 65743.0 +holly polk 65751.0 +holly robinson 65564.0 +holly thompson 65538.0 +holly thompson 65578.0 +holly thompson 65713.0 +holly underhill 65634.0 +holly underhill 65654.0 +holly underhill 65721.0 +holly underhill 65759.0 +holly van buren 65727.0 +holly white 65536.0 +holly white 65602.0 +holly xylophone 65544.0 +holly young 65606.0 +holly young 65765.0 +holly zipper 65607.0 +holly zipper 65755.0 +irene allen 65556.0 +irene brown 65633.0 +irene brown 65664.0 +irene brown 65765.0 +irene carson 65590.0 +irene ellison 65659.0 +irene ellison 65696.0 +irene falkner 65620.0 +irene falkner 65661.0 +irene garcia 65660.0 +irene garcia 65711.0 +irene garcia 65787.0 +irene ichabod 65645.0 +irene ichabod 65722.0 +irene johnson 65583.0 +irene laertes 65664.0 +irene laertes 65710.0 +irene laertes 65722.0 +irene miller 65730.0 +irene nixon 65631.0 +irene nixon 65643.0 +irene nixon 65653.0 +irene ovid 65691.0 +irene ovid 65734.0 +irene ovid 65753.0 +irene polk 65551.0 +irene polk 65575.0 +irene polk 65579.0 +irene polk 65595.0 +irene polk 65610.0 +irene quirinius 65724.0 +irene quirinius 65769.0 +irene quirinius 65773.0 +irene robinson 65554.0 +irene steinbeck 65683.0 +irene thompson 65688.0 +irene underhill 65591.0 +irene underhill 65707.5 +irene van buren 65579.0 +irene van buren 65589.0 +irene xylophone 65775.0 +jessica brown 65588.0 +jessica carson 65553.0 +jessica carson 65672.0 +jessica carson 65747.0 +jessica davidson 65549.0 +jessica davidson 65606.0 +jessica davidson 65675.0 +jessica davidson 65727.0 +jessica ellison 65567.0 +jessica ellison 65663.0 +jessica falkner 65584.0 +jessica garcia 65676.0 +jessica garcia 65789.0 +jessica ichabod 65704.0 +jessica johnson 65607.0 +jessica johnson 65720.0 +jessica miller 65733.0 +jessica nixon 65590.0 +jessica nixon 65774.0 +jessica ovid 65582.0 +jessica ovid 65751.0 +jessica polk 65637.0 +jessica quirinius 65562.0 +jessica quirinius 65608.0 +jessica quirinius 65712.0 +jessica quirinius 65716.0 +jessica robinson 65576.0 +jessica thompson 65581.0 +jessica thompson 65675.0 +jessica underhill 65656.0 +jessica underhill 65702.0 +jessica underhill 65783.0 +jessica van buren 65615.0 +jessica white 65544.0 +jessica white 65570.0 +jessica white 65594.0 +jessica white 65673.0 +jessica white 65779.0 +jessica xylophone 65562.0 +jessica young 65623.0 +jessica young 65711.0 +jessica zipper 65600.0 +jessica zipper 65657.0 +jessica zipper 65778.0 +katie allen 65542.0 +katie brown 65590.0 +katie davidson 65619.0 +katie ellison 65675.0 +katie ellison 65699.0 +katie falkner 65728.0 +katie garcia 65625.0 +katie garcia 65747.0 +katie hernandez 65550.0 +katie ichabod 65658.0 +katie ichabod 65726.0 +katie ichabod 65757.0 +katie king 65629.0 +katie king 65647.0 +katie king 65776.0 +katie miller 65541.0 +katie miller 65661.0 +katie nixon 65669.0 +katie ovid 65681.0 +katie polk 65746.0 +katie polk 65784.0 +katie robinson 65697.0 +katie van buren 65643.0 +katie van buren 65730.0 +katie white 65620.0 +katie white 65719.0 +katie xylophone 65585.0 +katie young 65644.0 +katie young 65746.0 +katie young 65764.0 +katie zipper 65568.0 +katie zipper 65733.0 +luke allen 65547.0 +luke allen 65552.0 +luke allen 65576.0 +luke allen 65681.0 +luke allen 65776.0 +luke brown 65719.0 +luke davidson 65656.0 +luke davidson 65791.0 +luke ellison 65582.0 +luke ellison 65664.0 +luke ellison 65779.0 +luke falkner 65589.0 +luke falkner 65618.0 +luke garcia 65687.0 +luke garcia 65778.0 +luke ichabod 65629.0 +luke ichabod 65654.0 +luke johnson 65545.0 +luke johnson 65716.0 +luke johnson 65718.0 +luke laertes 65608.0 +luke laertes 65657.0 +luke laertes 65685.0 +luke laertes 65730.0 +luke laertes 65756.0 +luke miller 65752.0 +luke ovid 65569.0 +luke ovid 65693.0 +luke polk 65645.0 +luke polk 65658.0 +luke quirinius 65655.0 +luke robinson 65634.0 +luke robinson 65772.0 +luke thompson 65626.0 +luke underhill 65553.0 +luke underhill 65571.0 +luke underhill 65651.0 +luke van buren 65678.0 +luke white 65693.0 +luke xylophone 65597.0 +luke zipper 65641.0 +mike allen 65706.0 +mike brown 65654.0 +mike carson 65698.0 +mike carson 65700.0 +mike carson 65751.0 +mike davidson 65658.0 +mike davidson 65759.0 +mike ellison 65598.0 +mike ellison 65606.0 +mike ellison 65718.0 +mike ellison 65738.0 +mike ellison 65760.0 +mike falkner 65609.0 +mike garcia 65571.0 +mike garcia 65600.0 +mike garcia 65770.0 +mike hernandez 65548.0 +mike hernandez 65672.0 +mike ichabod 65621.0 +mike king 65563.0 +mike king 65586.0 +mike king 65591.0 +mike king 65642.0 +mike king 65769.0 +mike king 65776.0 +mike miller 65549.0 +mike nixon 65619.0 +mike nixon 65704.0 +mike polk 65619.0 +mike polk 65658.0 +mike polk 65704.0 +mike quirinius 65717.0 +mike steinbeck 65550.0 +mike steinbeck 65564.0 +mike steinbeck 65573.0 +mike steinbeck 65749.0 +mike van buren 65620.0 +mike van buren 65770.0 +mike white 65648.0 +mike white 65685.0 +mike white 65769.0 +mike white 65778.0 +mike young 65545.0 +mike young 65581.0 +mike young 65736.0 +mike zipper 65552.0 +mike zipper 65695.0 +mike zipper 65779.0 +nick allen 65641.0 +nick allen 65786.0 +nick brown 65724.0 +nick davidson 65601.0 +nick ellison 65691.0 +nick ellison 65745.0 +nick falkner 65583.0 +nick falkner 65676.0 +nick garcia 65712.0 +nick garcia 65720.0 +nick garcia 65723.0 +nick ichabod 65572.0 +nick ichabod 65681.0 +nick ichabod 65737.0 +nick johnson 65585.0 +nick johnson 65784.0 +nick laertes 65624.0 +nick miller 65757.0 +nick nixon 65650.0 +nick ovid 65719.0 +nick polk 65716.0 +nick quirinius 65588.0 +nick quirinius 65723.0 +nick robinson 65547.0 +nick robinson 65675.0 +nick steinbeck 65689.0 +nick thompson 65610.0 +nick underhill 65619.0 +nick van buren 65603.0 +nick xylophone 65644.0 +nick young 65654.0 +nick young 65660.0 +nick zipper 65757.0 +nick zipper 65765.0 +oscar allen 65644.0 +oscar brown 65614.0 +oscar carson 65537.0 +oscar carson 65548.0 +oscar carson 65549.0 +oscar carson 65624.0 +oscar carson 65697.0 +oscar davidson 65556.0 +oscar ellison 65630.0 +oscar ellison 65630.0 +oscar falkner 65692.0 +oscar garcia 65751.0 +oscar hernandez 65683.0 +oscar hernandez 65707.0 +oscar ichabod 65536.0 +oscar ichabod 65562.0 +oscar ichabod 65637.0 +oscar ichabod 65763.0 +oscar johnson 65645.0 +oscar johnson 65778.0 +oscar king 65541.0 +oscar king 65550.0 +oscar king 65787.0 +oscar laertes 65625.0 +oscar laertes 65690.0 +oscar laertes 65756.0 +oscar laertes 65790.0 +oscar nixon 65596.0 +oscar ovid 65536.0 +oscar ovid 65615.0 +oscar ovid 65665.5 +oscar polk 65541.0 +oscar polk 65643.0 +oscar quirinius 65541.0 +oscar quirinius 65560.0 +oscar quirinius 65689.0 +oscar quirinius 65720.0 +oscar robinson 65537.0 +oscar robinson 65658.0 +oscar robinson 65687.0 +oscar robinson 65782.0 +oscar steinbeck 65709.0 +oscar thompson 65542.0 +oscar thompson 65681.0 +oscar thompson 65727.0 +oscar thompson 65738.0 +oscar underhill 65626.0 +oscar van buren 65581.0 +oscar van buren 65635.0 +oscar van buren 65705.0 +oscar white 65552.0 +oscar white 65564.0 +oscar white 65671.0 +oscar white 65735.0 +oscar xylophone 65773.0 +oscar xylophone 65773.0 +oscar xylophone 65775.0 +oscar zipper 65568.0 +oscar zipper 65740.0 +oscar zipper 65777.0 +priscilla brown 65670.0 +priscilla brown 65690.0 +priscilla brown 65749.0 +priscilla carson 65658.0 +priscilla carson 65687.0 +priscilla carson 65755.0 +priscilla ichabod 65627.0 +priscilla ichabod 65759.0 +priscilla johnson 65543.0 +priscilla johnson 65668.0 +priscilla johnson 65674.5 +priscilla johnson 65681.0 +priscilla johnson 65755.0 +priscilla king 65646.0 +priscilla nixon 65564.0 +priscilla nixon 65600.0 +priscilla ovid 65541.0 +priscilla ovid 65790.0 +priscilla polk 65747.0 +priscilla quirinius 65672.0 +priscilla thompson 65654.0 +priscilla underhill 65715.0 +priscilla underhill 65729.0 +priscilla van buren 65607.0 +priscilla van buren 65685.0 +priscilla van buren 65749.0 +priscilla white 65652.0 +priscilla xylophone 65538.0 +priscilla xylophone 65763.0 +priscilla xylophone 65774.0 +priscilla young 65585.0 +priscilla young 65658.0 +priscilla zipper 65622.0 +priscilla zipper 65726.0 +quinn allen 65657.0 +quinn allen 65708.0 +quinn brown 65691.0 +quinn brown 65700.0 +quinn brown 65733.0 +quinn davidson 65549.0 +quinn davidson 65714.0 +quinn davidson 65776.0 +quinn davidson 65779.0 +quinn ellison 65705.0 +quinn ellison 65778.0 +quinn garcia 65568.0 +quinn garcia 65604.0 +quinn garcia 65610.0 +quinn garcia 65773.0 +quinn ichabod 65609.0 +quinn king 65558.0 +quinn king 65649.0 +quinn laertes 65542.0 +quinn laertes 65560.0 +quinn laertes 65627.0 +quinn nixon 65659.0 +quinn ovid 65699.0 +quinn quirinius 65747.0 +quinn robinson 65627.0 +quinn steinbeck 65578.0 +quinn steinbeck 65763.0 +quinn thompson 65643.0 +quinn thompson 65774.0 +quinn underhill 65549.0 +quinn underhill 65694.0 +quinn underhill 65767.0 +quinn van buren 65725.0 +quinn young 65647.0 +quinn zipper 65579.0 +quinn zipper 65693.0 +rachel allen 65661.0 +rachel allen 65709.0 +rachel brown 65586.0 +rachel brown 65587.0 +rachel brown 65587.0 +rachel brown 65610.0 +rachel brown 65693.0 +rachel carson 65677.0 +rachel carson 65682.0 +rachel davidson 65755.0 +rachel ellison 65761.0 +rachel falkner 65616.0 +rachel falkner 65681.0 +rachel falkner 65693.0 +rachel falkner 65764.0 +rachel johnson 65658.0 +rachel king 65604.0 +rachel king 65643.0 +rachel laertes 65562.0 +rachel laertes 65624.0 +rachel ovid 65721.0 +rachel ovid 65736.0 +rachel polk 65686.0 +rachel quirinius 65787.0 +rachel robinson 65544.0 +rachel robinson 65717.0 +rachel robinson 65724.0 +rachel thompson 65648.0 +rachel thompson 65662.0 +rachel thompson 65733.0 +rachel underhill 65667.0 +rachel white 65615.0 +rachel white 65717.0 +rachel young 65727.0 +rachel zipper 65757.0 +rachel zipper 65785.0 +sarah carson 65616.0 +sarah carson 65693.0 +sarah carson 65694.0 +sarah ellison 65611.0 +sarah falkner 65606.0 +sarah falkner 65680.0 +sarah garcia 65563.0 +sarah garcia 65638.0 +sarah garcia 65661.0 +sarah ichabod 65667.0 +sarah ichabod 65671.0 +sarah johnson 65659.0 +sarah johnson 65716.0 +sarah johnson 65731.0 +sarah johnson 65751.0 +sarah king 65650.0 +sarah king 65699.0 +sarah miller 65557.0 +sarah ovid 65550.0 +sarah robinson 65677.0 +sarah robinson 65763.0 +sarah steinbeck 65721.0 +sarah white 65622.0 +sarah white 65747.0 +sarah xylophone 65678.0 +sarah young 65595.0 +sarah zipper 65550.0 +tom brown 65593.0 +tom brown 65675.0 +tom carson 65539.0 +tom carson 65624.0 +tom carson 65780.0 +tom davidson 65780.0 +tom ellison 65578.0 +tom ellison 65670.0 +tom ellison 65756.0 +tom falkner 65574.0 +tom falkner 65625.0 +tom hernandez 65575.0 +tom hernandez 65632.0 +tom ichabod 65588.0 +tom johnson 65536.0 +tom johnson 65789.0 +tom king 65576.0 +tom laertes 65617.0 +tom laertes 65701.0 +tom miller 65594.0 +tom miller 65603.0 +tom miller 65704.0 +tom nixon 65672.0 +tom ovid 65628.0 +tom polk 65652.0 +tom polk 65742.0 +tom quirinius 65563.0 +tom quirinius 65783.0 +tom robinson 65626.0 +tom robinson 65632.0 +tom robinson 65691.0 +tom robinson 65758.0 +tom steinbeck 65666.0 +tom van buren 65621.0 +tom van buren 65652.0 +tom van buren 65669.0 +tom white 65548.0 +tom young 65544.0 +tom young 65546.0 +tom zipper 65789.0 +ulysses brown 65735.0 +ulysses carson 65602.0 +ulysses carson 65643.0 +ulysses carson 65703.0 +ulysses carson 65716.0 +ulysses davidson 65750.0 +ulysses ellison 65575.0 +ulysses garcia 65666.0 +ulysses hernandez 65651.0 +ulysses hernandez 65702.0 +ulysses hernandez 65786.0 +ulysses ichabod 65551.0 +ulysses ichabod 65566.0 +ulysses johnson 65776.0 +ulysses king 65649.0 +ulysses laertes 65691.0 +ulysses laertes 65711.0 +ulysses laertes 65781.0 +ulysses miller 65610.0 +ulysses miller 65637.0 +ulysses nixon 65603.0 +ulysses ovid 65656.0 +ulysses polk 65563.0 +ulysses polk 65580.0 +ulysses polk 65612.0 +ulysses polk 65777.0 +ulysses quirinius 65786.0 +ulysses robinson 65744.0 +ulysses steinbeck 65611.0 +ulysses steinbeck 65680.0 +ulysses thompson 65788.0 +ulysses underhill 65570.0 +ulysses underhill 65616.0 +ulysses underhill 65620.0 +ulysses underhill 65623.0 +ulysses underhill 65641.0 +ulysses underhill 65713.0 +ulysses underhill 65785.0 +ulysses van buren 65684.0 +ulysses white 65654.0 +ulysses white 65675.0 +ulysses xylophone 65623.0 +ulysses xylophone 65636.0 +ulysses xylophone 65781.0 +ulysses young 65675.0 +ulysses young 65736.0 +ulysses young 65748.0 +victor allen 65684.0 +victor allen 65707.0 +victor brown 65550.0 +victor brown 65555.0 +victor brown 65622.0 +victor brown 65673.0 +victor davidson 65579.0 +victor davidson 65628.0 +victor davidson 65783.0 +victor ellison 65641.0 +victor ellison 65782.0 +victor hernandez 65571.0 +victor hernandez 65659.0 +victor hernandez 65708.0 +victor hernandez 65735.0 +victor hernandez 65775.0 +victor johnson 65606.0 +victor johnson 65607.0 +victor johnson 65607.0 +victor king 65721.0 +victor king 65743.0 +victor laertes 65638.0 +victor laertes 65644.0 +victor miller 65570.0 +victor nixon 65709.0 +victor nixon 65791.0 +victor ovid 65649.0 +victor polk 65625.0 +victor quirinius 65620.0 +victor quirinius 65651.0 +victor robinson 65596.0 +victor robinson 65673.0 +victor steinbeck 65618.0 +victor steinbeck 65661.0 +victor steinbeck 65686.0 +victor thompson 65548.0 +victor van buren 65664.0 +victor van buren 65774.0 +victor white 65548.0 +victor white 65601.0 +victor xylophone 65549.0 +victor xylophone 65618.0 +victor xylophone 65644.0 +victor xylophone 65677.0 +victor xylophone 65755.0 +victor young 65628.0 +victor zipper 65743.0 +wendy allen 65628.0 +wendy allen 65711.0 +wendy allen 65782.0 +wendy brown 65580.0 +wendy brown 65657.0 +wendy ellison 65545.0 +wendy ellison 65603.0 +wendy falkner 65595.0 +wendy falkner 65604.0 +wendy falkner 65635.0 +wendy garcia 65659.0 +wendy garcia 65746.0 +wendy garcia 65747.0 +wendy garcia 65777.0 +wendy hernandez 65650.0 +wendy ichabod 65730.0 +wendy king 65586.0 +wendy king 65664.0 +wendy king 65670.0 +wendy laertes 65566.0 +wendy laertes 65683.0 +wendy laertes 65727.0 +wendy miller 65582.0 +wendy miller 65626.0 +wendy nixon 65611.0 +wendy nixon 65746.0 +wendy ovid 65589.0 +wendy ovid 65643.0 +wendy polk 65656.0 +wendy polk 65692.0 +wendy quirinius 65766.0 +wendy quirinius 65767.0 +wendy robinson 65622.0 +wendy robinson 65715.0 +wendy robinson 65774.0 +wendy steinbeck 65612.0 +wendy thompson 65650.0 +wendy thompson 65737.0 +wendy underhill 65662.0 +wendy underhill 65758.0 +wendy underhill 65775.0 +wendy van buren 65680.0 +wendy van buren 65699.0 +wendy white 65705.0 +wendy xylophone 65687.0 +wendy xylophone 65773.0 +wendy young 65674.0 +wendy young 65685.0 +xavier allen 65611.0 +xavier allen 65618.0 +xavier allen 65771.0 +xavier brown 65600.0 +xavier brown 65704.0 +xavier brown 65723.0 +xavier carson 65731.0 +xavier carson 65758.0 +xavier davidson 65644.0 +xavier davidson 65664.0 +xavier davidson 65755.0 +xavier ellison 65541.0 +xavier ellison 65622.0 +xavier garcia 65672.0 +xavier hernandez 65541.0 +xavier hernandez 65544.0 +xavier hernandez 65766.0 +xavier ichabod 65597.0 +xavier ichabod 65663.0 +xavier johnson 65655.0 +xavier johnson 65744.0 +xavier king 65590.0 +xavier king 65601.0 +xavier laertes 65743.0 +xavier ovid 65788.0 +xavier polk 65587.0 +xavier polk 65653.0 +xavier polk 65675.0 +xavier polk 65696.0 +xavier quirinius 65599.0 +xavier quirinius 65650.0 +xavier quirinius 65656.0 +xavier quirinius 65737.0 +xavier thompson 65608.0 +xavier underhill 65710.0 +xavier white 65703.0 +xavier white 65732.0 +xavier xylophone 65572.0 +xavier zipper 65561.0 +yuri allen 65565.0 +yuri allen 65682.0 +yuri brown 65538.0 +yuri brown 65688.0 +yuri carson 65670.0 +yuri carson 65769.0 +yuri ellison 65570.0 +yuri ellison 65581.0 +yuri falkner 65658.0 +yuri falkner 65681.0 +yuri garcia 65639.0 +yuri hernandez 65706.0 +yuri johnson 65587.0 +yuri johnson 65697.0 +yuri johnson 65712.0 +yuri king 65721.0 +yuri laertes 65637.0 +yuri laertes 65773.0 +yuri nixon 65635.0 +yuri nixon 65740.0 +yuri polk 65607.0 +yuri polk 65713.0 +yuri polk 65742.0 +yuri quirinius 65544.0 +yuri quirinius 65617.0 +yuri quirinius 65695.0 +yuri steinbeck 65592.0 +yuri steinbeck 65679.0 +yuri thompson 65676.0 +yuri underhill 65718.0 +yuri underhill 65750.0 +yuri white 65659.0 +yuri xylophone 65714.0 +zach allen 65667.0 +zach brown 65559.0 +zach brown 65588.0 +zach brown 65691.0 +zach brown 65759.0 +zach brown 65762.0 +zach carson 65572.0 +zach ellison 65748.0 +zach falkner 65620.0 +zach falkner 65627.0 +zach garcia 65544.0 +zach garcia 65623.0 +zach garcia 65629.0 +zach garcia 65764.5 +zach ichabod 65599.0 +zach ichabod 65612.0 +zach king 65556.0 +zach king 65702.0 +zach king 65773.0 +zach miller 65583.0 +zach miller 65665.0 +zach miller 65719.0 +zach ovid 65578.0 +zach ovid 65669.0 +zach ovid 65703.0 +zach ovid 65784.0 +zach quirinius 65691.0 +zach robinson 65599.0 +zach steinbeck 65602.0 +zach steinbeck 65695.0 +zach thompson 65636.0 +zach thompson 65696.0 +zach underhill 65573.0 +zach white 65733.0 +zach xylophone 65542.0 +zach xylophone 65780.0 +zach young 65576.0 +zach zipper 65579.0 +zach zipper 65649.0 +zach zipper 65676.0 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-3-d90b27fca067b0b3c48d873b3ef32af7 b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-3-d90b27fca067b0b3c48d873b3ef32af7 new file mode 100644 index 0000000000000000000000000000000000000000..072a8a891a836edb7072af8c22e829210a861e46 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-3-d90b27fca067b0b3c48d873b3ef32af7 @@ -0,0 +1,1049 @@ +65536 +65536 +65536 +65536 +65536 +65536 +65537 +65537 +65537 +65537 +65538 +65538 +65538 +65539 +65539 +65539 +65540 +65541 +65541 +65541 +65541 +65541 +65541 +65541 +65541 +65542 +65542 +65542 +65542 +65543 +65543 +65544 +65544 +65544 +65544 +65544 +65544 +65544 +65544 +65545 +65545 +65545 +65547 +65547 +65547 +65547 +65548 +65548 +65548 +65548 +65548 +65548 +65548 +65549 +65549 +65549 +65549 +65549 +65549 +65549 +65549 +65550 +65550 +65550 +65550 +65550 +65550 +65550 +65551 +65551 +65551 +65552 +65552 +65552 +65552 +65552 +65553 +65553 +65553 +65553 +65554 +65554 +65555 +65556 +65556 +65556 +65557 +65557 +65558 +65559 +65559 +65559 +65559 +65560 +65560 +65560 +65560 +65561 +65562 +65562 +65562 +65562 +65562 +65562 +65563 +65563 +65563 +65563 +65563 +65564 +65564 +65564 +65564 +65564 +65564 +65564 +65564 +65565 +65565 +65566 +65566 +65567 +65568 +65568 +65568 +65569 +65570 +65570 +65570 +65570 +65570 +65571 +65571 +65571 +65572 +65572 +65572 +65572 +65572 +65573 +65573 +65573 +65574 +65574 +65574 +65575 +65575 +65575 +65575 +65576 +65576 +65576 +65576 +65577 +65578 +65578 +65578 +65578 +65578 +65578 +65579 +65579 +65579 +65579 +65579 +65580 +65580 +65580 +65581 +65581 +65581 +65581 +65581 +65582 +65582 +65582 +65582 +65582 +65583 +65583 +65583 +65583 +65584 +65584 +65584 +65584 +65585 +65585 +65585 +65585 +65585 +65586 +65586 +65586 +65586 +65586 +65587 +65587 +65587 +65587 +65588 +65588 +65588 +65588 +65588 +65589 +65589 +65589 +65589 +65589 +65590 +65590 +65590 +65590 +65590 +65591 +65591 +65591 +65591 +65592 +65593 +65593 +65594 +65594 +65594 +65594 +65595 +65595 +65595 +65595 +65595 +65595 +65596 +65596 +65596 +65596 +65596 +65597 +65597 +65597 +65598 +65599 +65599 +65599 +65600 +65600 +65600 +65600 +65601 +65601 +65601 +65602 +65602 +65602 +65602 +65603 +65603 +65603 +65603 +65603 +65604 +65604 +65604 +65605 +65606 +65606 +65606 +65606 +65606 +65607 +65607 +65607 +65607 +65607 +65607 +65608 +65608 +65608 +65608 +65609 +65610 +65610 +65610 +65610 +65610 +65610 +65611 +65611 +65611 +65612 +65612 +65612 +65614 +65614 +65614 +65615 +65615 +65615 +65616 +65616 +65617 +65617 +65617 +65617 +65618 +65618 +65618 +65618 +65618 +65619 +65619 +65619 +65619 +65619 +65619 +65620 +65620 +65620 +65620 +65620 +65620 +65621 +65621 +65622 +65622 +65622 +65622 +65622 +65622 +65623 +65623 +65623 +65623 +65623 +65623 +65623 +65624 +65624 +65624 +65624 +65625 +65625 +65625 +65625 +65626 +65626 +65626 +65626 +65627 +65627 +65627 +65627 +65628 +65628 +65628 +65628 +65628 +65628 +65629 +65629 +65629 +65630 +65630 +65631 +65632 +65632 +65632 +65633 +65633 +65633 +65633 +65634 +65634 +65634 +65634 +65635 +65635 +65635 +65636 +65636 +65636 +65636 +65637 +65637 +65637 +65637 +65637 +65637 +65637 +65638 +65638 +65638 +65639 +65640 +65641 +65641 +65641 +65641 +65642 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65644 +65644 +65644 +65644 +65644 +65645 +65645 +65645 +65646 +65646 +65646 +65647 +65647 +65648 +65648 +65648 +65648 +65649 +65649 +65649 +65650 +65650 +65650 +65650 +65650 +65650 +65650 +65651 +65651 +65651 +65651 +65651 +65651 +65652 +65652 +65652 +65653 +65653 +65653 +65653 +65654 +65654 +65654 +65654 +65654 +65654 +65654 +65654 +65654 +65655 +65655 +65656 +65656 +65656 +65656 +65656 +65656 +65657 +65657 +65657 +65657 +65657 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65659 +65659 +65659 +65659 +65659 +65659 +65659 +65659 +65660 +65660 +65660 +65661 +65661 +65661 +65661 +65661 +65662 +65662 +65662 +65662 +65662 +65662 +65663 +65663 +65663 +65663 +65664 +65664 +65664 +65664 +65664 +65664 +65664 +65665 +65666 +65666 +65667 +65667 +65667 +65667 +65667 +65667 +65668 +65669 +65669 +65669 +65669 +65669 +65669 +65670 +65670 +65670 +65670 +65670 +65671 +65671 +65671 +65672 +65672 +65672 +65672 +65672 +65672 +65673 +65673 +65673 +65673 +65673 +65674 +65674 +65674 +65674 +65675 +65675 +65675 +65675 +65675 +65675 +65675 +65675 +65676 +65676 +65676 +65677 +65677 +65677 +65677 +65677 +65677 +65678 +65678 +65678 +65678 +65679 +65679 +65680 +65680 +65680 +65680 +65680 +65680 +65680 +65681 +65681 +65681 +65681 +65681 +65682 +65682 +65682 +65683 +65683 +65683 +65683 +65684 +65684 +65684 +65684 +65685 +65685 +65685 +65685 +65685 +65685 +65686 +65686 +65686 +65687 +65687 +65687 +65687 +65688 +65688 +65689 +65689 +65690 +65690 +65691 +65691 +65691 +65691 +65691 +65691 +65691 +65691 +65692 +65692 +65693 +65693 +65693 +65693 +65693 +65693 +65694 +65694 +65694 +65695 +65695 +65695 +65695 +65695 +65696 +65696 +65696 +65696 +65697 +65697 +65697 +65697 +65697 +65697 +65698 +65698 +65698 +65699 +65699 +65699 +65699 +65699 +65699 +65700 +65700 +65700 +65701 +65701 +65701 +65702 +65702 +65702 +65702 +65702 +65703 +65703 +65703 +65703 +65703 +65704 +65704 +65704 +65704 +65704 +65704 +65705 +65705 +65705 +65705 +65706 +65706 +65706 +65706 +65706 +65706 +65707 +65707 +65708 +65708 +65709 +65709 +65709 +65710 +65711 +65711 +65711 +65711 +65711 +65711 +65712 +65712 +65712 +65712 +65712 +65713 +65713 +65713 +65713 +65713 +65714 +65714 +65714 +65715 +65715 +65715 +65715 +65715 +65716 +65716 +65716 +65716 +65716 +65716 +65717 +65717 +65717 +65717 +65717 +65718 +65718 +65718 +65718 +65719 +65719 +65719 +65719 +65720 +65720 +65720 +65720 +65720 +65720 +65721 +65721 +65721 +65721 +65721 +65721 +65721 +65722 +65722 +65722 +65722 +65723 +65723 +65724 +65724 +65724 +65724 +65724 +65724 +65725 +65726 +65726 +65726 +65726 +65727 +65727 +65727 +65727 +65727 +65728 +65728 +65729 +65730 +65730 +65730 +65730 +65731 +65731 +65731 +65731 +65732 +65732 +65732 +65733 +65733 +65733 +65733 +65733 +65733 +65734 +65734 +65735 +65735 +65735 +65736 +65736 +65736 +65736 +65737 +65737 +65737 +65737 +65737 +65738 +65738 +65738 +65738 +65739 +65739 +65739 +65740 +65740 +65740 +65741 +65742 +65742 +65742 +65743 +65743 +65743 +65743 +65744 +65744 +65745 +65745 +65745 +65745 +65746 +65746 +65746 +65746 +65747 +65747 +65747 +65747 +65747 +65747 +65747 +65748 +65748 +65749 +65749 +65749 +65749 +65749 +65750 +65750 +65750 +65750 +65750 +65751 +65751 +65751 +65751 +65751 +65752 +65752 +65753 +65753 +65754 +65754 +65755 +65755 +65755 +65755 +65755 +65755 +65755 +65755 +65755 +65756 +65756 +65756 +65756 +65756 +65757 +65757 +65757 +65757 +65757 +65758 +65758 +65758 +65758 +65758 +65758 +65759 +65759 +65759 +65759 +65759 +65760 +65760 +65760 +65760 +65760 +65761 +65762 +65762 +65762 +65762 +65762 +65763 +65763 +65763 +65763 +65763 +65764 +65764 +65764 +65765 +65765 +65765 +65766 +65766 +65766 +65766 +65766 +65767 +65767 +65767 +65768 +65769 +65769 +65769 +65769 +65769 +65769 +65769 +65770 +65770 +65771 +65771 +65771 +65772 +65773 +65773 +65773 +65773 +65773 +65773 +65773 +65773 +65774 +65774 +65774 +65774 +65774 +65774 +65775 +65775 +65775 +65775 +65775 +65775 +65776 +65776 +65776 +65776 +65776 +65776 +65776 +65777 +65777 +65777 +65777 +65777 +65777 +65778 +65778 +65778 +65778 +65778 +65778 +65778 +65778 +65778 +65779 +65779 +65779 +65779 +65779 +65779 +65779 +65780 +65780 +65780 +65781 +65781 +65781 +65782 +65782 +65782 +65782 +65782 +65783 +65783 +65783 +65783 +65783 +65783 +65783 +65784 +65784 +65784 +65784 +65784 +65785 +65785 +65786 +65786 +65786 +65786 +65786 +65787 +65787 +65787 +65787 +65787 +65788 +65788 +65788 +65788 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65790 +65790 +65790 +65791 +65791 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-4-f2e4d659b65a833e9281b6786d3d55c1 b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-4-f2e4d659b65a833e9281b6786d3d55c1 new file mode 100644 index 0000000000000000000000000000000000000000..9cc7e7ea6c2b28c237132a04a1b425f36e5a9686 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-4-f2e4d659b65a833e9281b6786d3d55c1 @@ -0,0 +1,1049 @@ + 24.37875 + 27.900000000000002 + 43.64 +alice allen 16.919999999999998 +alice allen 20.39 +alice allen 23.59 +alice brown 6.91 +alice carson 41.74 +alice davidson 26.346000000000004 +alice falkner 32.166666666666664 +alice garcia 15.412 +alice hernandez 19.958181818181817 +alice hernandez 30.482857142857142 +alice johnson 25.51 +alice king 13.085 +alice king 25.616666666666664 +alice king 42.335 +alice laertes 20.549999999999997 +alice laertes 30.436 +alice miller 29.563333333333333 +alice nixon 19.28666666666667 +alice nixon 24.5625 +alice nixon 32.99 +alice ovid 31.35 +alice polk 17.863999999999997 +alice quirinius 19.032857142857143 +alice quirinius 23.9425 +alice robinson 23.338750000000005 +alice robinson 39.85 +alice steinbeck 22.862000000000002 +alice steinbeck 26.328000000000003 +alice steinbeck 27.08777777777778 +alice underhill 24.032222222222224 +alice van buren 19.642000000000003 +alice xylophone 24.438000000000002 +alice xylophone 28.739999999999995 +alice xylophone 30.0825 +alice zipper 26.3 +alice zipper 28.735000000000003 +alice zipper 31.05545454545455 +bob brown 12.902222222222223 +bob brown 13.945 +bob brown 33.843333333333334 +bob carson 28.627999999999997 +bob davidson 19.8525 +bob davidson 23.482 +bob davidson 24.67 +bob ellison 16.315714285714286 +bob ellison 18.4 +bob ellison 26.913999999999998 +bob ellison 27.59 +bob falkner 9.27 +bob garcia 11.63 +bob garcia 22.221249999999998 +bob garcia 23.59636363636364 +bob garcia 26.88857142857143 +bob garcia 28.715000000000003 +bob hernandez 37.23 +bob ichabod 28.33875 +bob king 8.615 +bob king 19.77 +bob king 26.7325 +bob laertes 21.33 +bob laertes 37.88 +bob miller 25.495 +bob ovid 25.675 +bob ovid 25.83 +bob ovid 28.37875 +bob ovid 32.5025 +bob polk 9.74 +bob quirinius 34.57 +bob steinbeck 9.725 +bob van buren 29.552857142857142 +bob white 17.685 +bob white 29.46285714285715 +bob xylophone 17.03 +bob xylophone 33.24 +bob young 19.824 +bob zipper 24.095 +bob zipper 33.36 +bob zipper 34.99 +calvin allen 21.3 +calvin brown 20.808 +calvin brown 24.16 +calvin brown 24.636666666666667 +calvin carson 22.815 +calvin davidson 22.116666666666664 +calvin davidson 22.364 +calvin ellison 24.92 +calvin falkner 18.343999999999998 +calvin falkner 19.56 +calvin falkner 22.946000000000005 +calvin falkner 23.327777777777776 +calvin falkner 23.974999999999998 +calvin falkner 33.382 +calvin garcia 17.285 +calvin hernandez 12.663333333333334 +calvin johnson 24.898571428571433 +calvin laertes 28.105 +calvin laertes 28.362000000000002 +calvin nixon 26.784285714285716 +calvin nixon 27.36 +calvin nixon 32.282 +calvin ovid 22.063333333333336 +calvin ovid 22.81500000000001 +calvin ovid 25.495714285714286 +calvin ovid 30.926666666666666 +calvin polk 27.820000000000004 +calvin quirinius 16.28 +calvin quirinius 25.552500000000002 +calvin robinson 31.814999999999998 +calvin steinbeck 12.85 +calvin steinbeck 14.939999999999998 +calvin steinbeck 17.535 +calvin thompson 28.592857142857145 +calvin thompson 40.79 +calvin underhill 24.062 +calvin van buren 26.525 +calvin van buren 28.865 +calvin white 28.256249999999998 +calvin white 43.275 +calvin xylophone 24.13111111111111 +calvin xylophone 25.27 +calvin xylophone 36.455 +calvin young 19.06 +calvin young 21.455999999999996 +calvin zipper 10.674999999999999 +calvin zipper 26.012857142857143 +david allen 25.134285714285713 +david allen 41.72333333333333 +david brown 8.52 +david brown 28.968181818181815 +david davidson 17.63 +david davidson 26.563333333333336 +david davidson 30.7325 +david davidson 33.33 +david ellison 23.79909090909091 +david ellison 24.74888888888889 +david ellison 26.198571428571427 +david hernandez 27.766 +david ichabod 16.66 +david ichabod 19.538 +david laertes 24.587500000000002 +david nixon 26.01375 +david ovid 24.131428571428575 +david ovid 32.72 +david quirinius 16.5 +david quirinius 25.08 +david quirinius 29.415 +david robinson 22.2175 +david robinson 30.99 +david thompson 25.38 +david underhill 1.17 +david underhill 21.546666666666667 +david underhill 28.26 +david van buren 26.45833333333334 +david van buren 35.7825 +david white 15.833333333333334 +david xylophone 10.71 +david xylophone 26.341428571428565 +david xylophone 33.224000000000004 +david young 9.64 +david young 21.22 +ethan allen 22.68 +ethan brown 19.37 +ethan brown 21.58666666666667 +ethan brown 21.799999999999997 +ethan brown 29.099999999999998 +ethan brown 32.43666666666667 +ethan brown 39.84 +ethan carson 24.15666666666667 +ethan ellison 27.80777777777778 +ethan ellison 48.71 +ethan falkner 17.993333333333336 +ethan falkner 26.775000000000002 +ethan garcia 19.15 +ethan hernandez 25.081111111111113 +ethan johnson 32.81875 +ethan king 19.51 +ethan laertes 16.463 +ethan laertes 17.625999999999998 +ethan laertes 25.020714285714288 +ethan laertes 26.697142857142858 +ethan laertes 28.14 +ethan laertes 29.668571428571425 +ethan laertes 36.589999999999996 +ethan miller 24.326666666666664 +ethan nixon 34.78666666666667 +ethan ovid 20.642857142857142 +ethan polk 6.98 +ethan polk 12.756666666666666 +ethan polk 30.324 +ethan polk 40.46 +ethan quirinius 23.419999999999998 +ethan quirinius 24.36 +ethan quirinius 29.068 +ethan robinson 24.463750000000005 +ethan robinson 31.630000000000003 +ethan underhill 19.86 +ethan van buren 22.241999999999997 +ethan white 31.3175 +ethan white 32.87 +ethan xylophone 30.996000000000002 +ethan zipper 22.728333333333335 +ethan zipper 29.66 +fred davidson 30.116666666666667 +fred davidson 33.55200000000001 +fred davidson 39.37 +fred ellison 16.72 +fred ellison 17.462 +fred ellison 35.1 +fred falkner 14.51 +fred falkner 27.207000000000004 +fred falkner 27.887500000000003 +fred hernandez 36.045 +fred ichabod 29.017000000000003 +fred ichabod 30.405000000000005 +fred johnson 16.9925 +fred king 20.024 +fred king 32.54666666666667 +fred laertes 25.610000000000003 +fred miller 25.92 +fred nixon 14.915 +fred nixon 21.830000000000002 +fred nixon 24.4125 +fred nixon 31.360000000000003 +fred polk 18.698 +fred polk 19.743000000000002 +fred polk 20.96 +fred polk 31.11 +fred quirinius 20.085 +fred quirinius 33.9 +fred robinson 22.502 +fred steinbeck 21.123749999999998 +fred steinbeck 25.572 +fred steinbeck 30.81 +fred underhill 29.198888888888884 +fred van buren 21.34 +fred van buren 23.285 +fred van buren 26.520000000000003 +fred van buren 33.6 +fred white 21.41 +fred young 16.876250000000002 +fred young 20.996666666666666 +fred zipper 23.627499999999998 +gabriella allen 24.113333333333333 +gabriella allen 28.4725 +gabriella brown 29.963333333333335 +gabriella brown 30.65222222222222 +gabriella carson 16.6325 +gabriella davidson 34.52 +gabriella ellison 20.18 +gabriella ellison 29.62 +gabriella falkner 14.37 +gabriella falkner 17.738333333333333 +gabriella falkner 28.61 +gabriella garcia 39.025 +gabriella hernandez 20.818333333333335 +gabriella hernandez 24.601666666666663 +gabriella ichabod 10.4925 +gabriella ichabod 20.686666666666667 +gabriella ichabod 23.185 +gabriella ichabod 23.43 +gabriella ichabod 27.44636363636364 +gabriella king 13.645 +gabriella king 22.23 +gabriella laertes 23.735 +gabriella miller 17.165 +gabriella ovid 22.884545454545453 +gabriella ovid 25.29 +gabriella polk 20.38714285714286 +gabriella polk 25.832000000000004 +gabriella steinbeck 6.226666666666667 +gabriella steinbeck 29.683333333333337 +gabriella thompson 25.565454545454546 +gabriella thompson 29.031 +gabriella thompson 29.122500000000006 +gabriella van buren 24.353 +gabriella van buren 34.21666666666667 +gabriella white 36.5175 +gabriella young 21.28142857142857 +gabriella young 21.32 +gabriella zipper 21.798461538461545 +gabriella zipper 28.676666666666666 +holly allen 27.18 +holly brown 22.76 +holly brown 30.950000000000003 +holly falkner 29.666666666666668 +holly hernandez 19.875 +holly hernandez 23.7075 +holly hernandez 24.5 +holly hernandez 26.50333333333333 +holly ichabod 23.262857142857143 +holly ichabod 25.85090909090909 +holly ichabod 29.521666666666665 +holly johnson 18.939999999999998 +holly johnson 23.2625 +holly johnson 26.49285714285714 +holly king 20.61333333333333 +holly king 30.95888888888889 +holly laertes 17.509999999999998 +holly miller 40.8975 +holly nixon 27.775714285714287 +holly nixon 30.642500000000002 +holly polk 21.02 +holly polk 24.446666666666665 +holly robinson 26.083750000000006 +holly thompson 18.801428571428573 +holly thompson 23.91 +holly thompson 29.97125 +holly underhill 18.19 +holly underhill 22.22888888888889 +holly underhill 22.813333333333336 +holly underhill 30.613999999999997 +holly van buren 20.113333333333333 +holly white 25.284999999999997 +holly white 41.0125 +holly xylophone 26.88571428571429 +holly young 30.8425 +holly young 33.24333333333334 +holly zipper 27.784000000000002 +holly zipper 28.384285714285713 +irene allen 34.605000000000004 +irene brown 18.740000000000002 +irene brown 28.974999999999998 +irene brown 32.230000000000004 +irene carson 25.665833333333335 +irene ellison 10.225000000000001 +irene ellison 26.119999999999997 +irene falkner 9.94 +irene falkner 19.41 +irene garcia 9.790000000000001 +irene garcia 19.666666666666668 +irene garcia 21.22666666666667 +irene ichabod 20.956666666666667 +irene ichabod 24.488333333333333 +irene johnson 25.34 +irene laertes 15.85 +irene laertes 21.573333333333334 +irene laertes 22.041999999999998 +irene miller 34.994285714285716 +irene nixon 22.52 +irene nixon 32.485 +irene nixon 33.165 +irene ovid 17.73 +irene ovid 22.96 +irene ovid 30.92 +irene polk 5.35 +irene polk 25.535 +irene polk 33.76 +irene polk 35.05 +irene polk 45.14 +irene quirinius 38.36 +irene quirinius 41.864999999999995 +irene quirinius 42.0 +irene robinson 30.86 +irene steinbeck 15.08 +irene thompson 28.419999999999998 +irene underhill 27.977999999999998 +irene underhill 28.438 +irene van buren 26.93625 +irene van buren 27.797999999999995 +irene xylophone 29.10454545454546 +jessica brown 38.325 +jessica carson 16.038 +jessica carson 29.668333333333337 +jessica carson 33.06 +jessica davidson 18.926 +jessica davidson 26.2975 +jessica davidson 27.611428571428572 +jessica davidson 29.86 +jessica ellison 26.873333333333335 +jessica ellison 27.123333333333335 +jessica falkner 21.75142857142858 +jessica garcia 16.939090909090908 +jessica garcia 26.48 +jessica ichabod 28.971666666666664 +jessica johnson 21.601428571428574 +jessica johnson 24.42 +jessica miller 26.90571428571429 +jessica nixon 19.15 +jessica nixon 27.025000000000002 +jessica ovid 30.72285714285714 +jessica ovid 30.895 +jessica polk 27.912857142857145 +jessica quirinius 17.05 +jessica quirinius 21.529999999999998 +jessica quirinius 25.16 +jessica quirinius 26.347999999999995 +jessica robinson 24.322857142857142 +jessica thompson 28.658000000000005 +jessica thompson 30.873636363636365 +jessica underhill 14.6725 +jessica underhill 25.831666666666667 +jessica underhill 31.345000000000002 +jessica van buren 19.575 +jessica white 18.35 +jessica white 19.175 +jessica white 20.812 +jessica white 26.0 +jessica white 29.307142857142857 +jessica xylophone 22.26 +jessica young 27.9525 +jessica young 37.61333333333334 +jessica zipper 7.03 +jessica zipper 15.794999999999998 +jessica zipper 19.95 +katie allen 27.283846153846152 +katie brown 24.156666666666666 +katie davidson 13.498000000000001 +katie ellison 19.2 +katie ellison 24.888571428571428 +katie falkner 28.959999999999997 +katie garcia 28.287142857142857 +katie garcia 36.196666666666665 +katie hernandez 25.14428571428572 +katie ichabod 19.363333333333333 +katie ichabod 20.458571428571428 +katie ichabod 28.924999999999997 +katie king 21.64125 +katie king 21.855 +katie king 22.895 +katie miller 16.263333333333335 +katie miller 30.274285714285718 +katie nixon 25.022499999999997 +katie ovid 24.055000000000003 +katie polk 21.296666666666667 +katie polk 32.03 +katie robinson 36.26 +katie van buren 28.332 +katie van buren 31.408000000000005 +katie white 23.48 +katie white 26.236666666666665 +katie xylophone 32.415 +katie young 18.209999999999997 +katie young 22.88125 +katie young 28.39888888888889 +katie zipper 10.285 +katie zipper 27.495 +luke allen 9.42 +luke allen 21.374615384615385 +luke allen 25.32 +luke allen 27.174999999999997 +luke allen 35.434 +luke brown 25.08 +luke davidson 28.205 +luke davidson 28.790000000000003 +luke ellison 7.8 +luke ellison 16.04 +luke ellison 23.426666666666666 +luke falkner 18.0 +luke falkner 22.19 +luke garcia 29.619999999999997 +luke garcia 32.722 +luke ichabod 21.150000000000002 +luke ichabod 32.78142857142857 +luke johnson 21.58666666666667 +luke johnson 23.03 +luke johnson 23.054 +luke laertes 20.264 +luke laertes 33.72 +luke laertes 39.8 +luke laertes 41.36 +luke laertes 42.254999999999995 +luke miller 20.054444444444446 +luke ovid 19.819999999999997 +luke ovid 30.832857142857147 +luke polk 24.348750000000003 +luke polk 26.57625 +luke quirinius 38.07 +luke robinson 30.119999999999994 +luke robinson 30.31375 +luke thompson 29.026874999999997 +luke underhill 21.735714285714288 +luke underhill 22.175 +luke underhill 26.785714285714285 +luke van buren 17.072222222222223 +luke white 29.063333333333333 +luke xylophone 28.994 +luke zipper 33.995 +mike allen 32.78 +mike brown 27.592222222222222 +mike carson 28.8675 +mike carson 29.88 +mike carson 32.07142857142857 +mike davidson 21.240000000000002 +mike davidson 46.31 +mike ellison 20.5275 +mike ellison 21.99 +mike ellison 24.36 +mike ellison 24.511111111111113 +mike ellison 27.703333333333337 +mike falkner 40.335 +mike garcia 24.3525 +mike garcia 24.582 +mike garcia 35.12 +mike hernandez 8.783333333333333 +mike hernandez 19.40666666666667 +mike ichabod 29.120000000000005 +mike king 14.256666666666668 +mike king 17.889999999999997 +mike king 20.493333333333336 +mike king 23.86 +mike king 26.081 +mike king 30.974 +mike miller 29.275 +mike nixon 17.306 +mike nixon 25.572 +mike polk 18.96 +mike polk 23.75142857142857 +mike polk 33.42 +mike quirinius 19.37375 +mike steinbeck 14.155 +mike steinbeck 19.305833333333332 +mike steinbeck 20.721249999999998 +mike steinbeck 31.75 +mike van buren 15.520000000000001 +mike van buren 25.828333333333333 +mike white 19.13111111111111 +mike white 22.4025 +mike white 24.7725 +mike white 35.235 +mike young 1.5 +mike young 24.679 +mike young 34.02833333333333 +mike zipper 17.97 +mike zipper 26.247333333333337 +mike zipper 44.169999999999995 +nick allen 23.744999999999997 +nick allen 36.93 +nick brown 27.669999999999998 +nick davidson 31.97285714285714 +nick ellison 23.061666666666667 +nick ellison 27.676666666666666 +nick falkner 22.555714285714284 +nick falkner 27.46 +nick garcia 17.465 +nick garcia 18.854 +nick garcia 33.60333333333333 +nick ichabod 19.231428571428573 +nick ichabod 27.645000000000003 +nick ichabod 35.836666666666666 +nick johnson 5.58 +nick johnson 25.274 +nick laertes 26.57857142857143 +nick miller 22.208333333333332 +nick nixon 16.107499999999998 +nick ovid 31.350000000000005 +nick polk 35.70333333333334 +nick quirinius 20.753333333333334 +nick quirinius 30.573333333333334 +nick robinson 21.48 +nick robinson 23.185 +nick steinbeck 19.56555555555556 +nick thompson 31.474999999999998 +nick underhill 38.24 +nick van buren 20.77375 +nick xylophone 30.909999999999997 +nick young 10.725000000000001 +nick young 24.95 +nick zipper 16.185000000000002 +nick zipper 34.72 +oscar allen 24.645 +oscar brown 39.55 +oscar carson 21.893333333333334 +oscar carson 22.868 +oscar carson 27.4875 +oscar carson 28.09428571428571 +oscar carson 30.373333333333335 +oscar davidson 9.046666666666667 +oscar ellison 24.185000000000002 +oscar ellison 30.1675 +oscar falkner 19.295 +oscar garcia 22.495833333333334 +oscar hernandez 16.6825 +oscar hernandez 25.736 +oscar ichabod 17.64 +oscar ichabod 21.11 +oscar ichabod 23.508000000000003 +oscar ichabod 30.392222222222227 +oscar johnson 19.9375 +oscar johnson 21.114444444444445 +oscar king 24.590000000000003 +oscar king 26.675 +oscar king 39.6 +oscar laertes 14.975 +oscar laertes 15.525 +oscar laertes 22.6 +oscar laertes 41.6 +oscar nixon 25.4025 +oscar ovid 24.854285714285712 +oscar ovid 25.309 +oscar ovid 29.63 +oscar polk 21.235999999999997 +oscar polk 21.27 +oscar quirinius 24.200000000000003 +oscar quirinius 24.391428571428573 +oscar quirinius 27.83285714285714 +oscar quirinius 27.853333333333328 +oscar robinson 12.3625 +oscar robinson 12.545 +oscar robinson 20.234 +oscar robinson 28.071666666666673 +oscar steinbeck 31.101111111111113 +oscar thompson 19.4875 +oscar thompson 19.975714285714286 +oscar thompson 21.1425 +oscar thompson 21.166363636363638 +oscar underhill 27.644 +oscar van buren 25.843333333333334 +oscar van buren 29.073333333333334 +oscar van buren 29.682727272727274 +oscar white 19.0775 +oscar white 23.483333333333334 +oscar white 24.705000000000002 +oscar white 28.0075 +oscar xylophone 30.020000000000003 +oscar xylophone 30.46833333333333 +oscar xylophone 33.64 +oscar zipper 21.69 +oscar zipper 23.478 +oscar zipper 31.36 +priscilla brown 14.222 +priscilla brown 27.044999999999998 +priscilla brown 31.14769230769231 +priscilla carson 14.33 +priscilla carson 18.951428571428572 +priscilla carson 27.084999999999997 +priscilla ichabod 28.160999999999994 +priscilla ichabod 49.46 +priscilla johnson 8.365 +priscilla johnson 18.176666666666666 +priscilla johnson 25.02666666666667 +priscilla johnson 26.918333333333337 +priscilla johnson 30.695999999999998 +priscilla king 19.747142857142855 +priscilla nixon 29.035555555555554 +priscilla nixon 30.27333333333333 +priscilla ovid 13.591999999999999 +priscilla ovid 35.879999999999995 +priscilla polk 23.12 +priscilla quirinius 21.826666666666668 +priscilla thompson 20.44 +priscilla underhill 28.23 +priscilla underhill 34.33200000000001 +priscilla van buren 18.122857142857143 +priscilla van buren 20.16 +priscilla van buren 26.447999999999997 +priscilla white 26.37769230769231 +priscilla xylophone 13.95 +priscilla xylophone 20.596666666666668 +priscilla xylophone 27.22 +priscilla young 29.19 +priscilla young 46.28 +priscilla zipper 11.64 +priscilla zipper 31.159999999999997 +quinn allen 26.347272727272728 +quinn allen 26.85833333333333 +quinn brown 26.822857142857146 +quinn brown 30.406000000000006 +quinn brown 41.53 +quinn davidson 17.375714285714288 +quinn davidson 20.22666666666667 +quinn davidson 25.6375 +quinn davidson 30.173333333333332 +quinn ellison 23.052 +quinn ellison 40.565 +quinn garcia 20.544 +quinn garcia 24.104999999999997 +quinn garcia 25.174 +quinn garcia 28.446000000000005 +quinn ichabod 15.12 +quinn king 12.73 +quinn king 15.12125 +quinn laertes 17.29 +quinn laertes 28.221666666666668 +quinn laertes 32.96 +quinn nixon 26.034000000000002 +quinn ovid 28.71 +quinn quirinius 8.61 +quinn robinson 16.852 +quinn steinbeck 30.093333333333334 +quinn steinbeck 49.21 +quinn thompson 7.365 +quinn thompson 33.43125 +quinn underhill 24.045 +quinn underhill 27.905454545454543 +quinn underhill 31.21 +quinn van buren 27.807692307692314 +quinn young 30.56 +quinn zipper 18.31 +quinn zipper 21.380000000000003 +rachel allen 32.501666666666665 +rachel allen 46.57 +rachel brown 23.08 +rachel brown 23.880000000000003 +rachel brown 24.43 +rachel brown 34.11 +rachel brown 35.345 +rachel carson 27.468125 +rachel carson 37.446666666666665 +rachel davidson 22.75 +rachel ellison 22.848333333333333 +rachel falkner 18.78125 +rachel falkner 28.876250000000002 +rachel falkner 29.577777777777776 +rachel falkner 31.831249999999997 +rachel johnson 31.108000000000004 +rachel king 17.4175 +rachel king 30.873749999999998 +rachel laertes 17.470000000000002 +rachel laertes 33.51 +rachel ovid 3.03 +rachel ovid 15.38 +rachel polk 18.564285714285713 +rachel quirinius 31.692500000000003 +rachel robinson 0.6 +rachel robinson 23.953333333333333 +rachel robinson 37.645 +rachel thompson 11.96 +rachel thompson 29.484 +rachel thompson 38.43 +rachel underhill 27.55333333333333 +rachel white 23.511428571428574 +rachel white 33.7 +rachel young 24.85166666666667 +rachel zipper 22.85 +rachel zipper 37.382 +sarah carson 10.38 +sarah carson 22.639 +sarah carson 44.92 +sarah ellison 16.36 +sarah falkner 29.34875 +sarah falkner 29.64125 +sarah garcia 11.296666666666667 +sarah garcia 20.723333333333333 +sarah garcia 24.115 +sarah ichabod 26.948333333333327 +sarah ichabod 33.80428571428571 +sarah johnson 18.3925 +sarah johnson 23.087500000000002 +sarah johnson 26.57857142857143 +sarah johnson 37.01 +sarah king 9.556666666666667 +sarah king 25.6125 +sarah miller 19.14875 +sarah ovid 29.205 +sarah robinson 11.326666666666668 +sarah robinson 35.809999999999995 +sarah steinbeck 23.26 +sarah white 21.75111111111111 +sarah white 26.850000000000005 +sarah xylophone 33.40571428571429 +sarah young 30.66 +sarah zipper 29.521666666666672 +tom brown 16.38 +tom brown 23.645 +tom carson 23.630000000000003 +tom carson 31.935 +tom carson 41.83 +tom davidson 30.404285714285717 +tom ellison 27.056 +tom ellison 27.401999999999997 +tom ellison 29.812 +tom falkner 15.901999999999997 +tom falkner 25.49857142857143 +tom hernandez 11.418000000000001 +tom hernandez 30.705000000000002 +tom ichabod 14.83 +tom johnson 30.748571428571434 +tom johnson 37.086666666666666 +tom king 17.923333333333332 +tom laertes 19.201666666666668 +tom laertes 22.276666666666667 +tom miller 17.9925 +tom miller 19.791666666666668 +tom miller 19.9225 +tom nixon 25.70625 +tom ovid 29.66 +tom polk 27.0975 +tom polk 28.646666666666672 +tom quirinius 37.68333333333333 +tom quirinius 38.28 +tom robinson 18.07 +tom robinson 19.094 +tom robinson 27.34125 +tom robinson 31.135714285714283 +tom steinbeck 32.70333333333333 +tom van buren 20.723333333333333 +tom van buren 24.8525 +tom van buren 31.631666666666664 +tom white 25.646000000000004 +tom young 3.12 +tom young 19.588333333333335 +tom zipper 23.317272727272726 +ulysses brown 16.196666666666665 +ulysses carson 16.3475 +ulysses carson 22.448181818181823 +ulysses carson 28.258 +ulysses carson 32.10833333333333 +ulysses davidson 37.775 +ulysses ellison 30.517000000000003 +ulysses garcia 32.92 +ulysses hernandez 13.877500000000001 +ulysses hernandez 20.856666666666666 +ulysses hernandez 21.32625 +ulysses ichabod 3.29 +ulysses ichabod 24.629999999999995 +ulysses johnson 32.208333333333336 +ulysses king 25.29111111111111 +ulysses laertes 14.936666666666667 +ulysses laertes 25.89 +ulysses laertes 26.63 +ulysses miller 2.36 +ulysses miller 26.403333333333336 +ulysses nixon 34.4575 +ulysses ovid 23.810000000000002 +ulysses polk 22.4075 +ulysses polk 26.778000000000002 +ulysses polk 38.73166666666667 +ulysses polk 47.68 +ulysses quirinius 33.07833333333333 +ulysses robinson 17.386666666666667 +ulysses steinbeck 22.2675 +ulysses steinbeck 24.904000000000003 +ulysses thompson 22.687142857142856 +ulysses underhill 6.66 +ulysses underhill 22.539 +ulysses underhill 24.853333333333335 +ulysses underhill 27.314 +ulysses underhill 29.424999999999997 +ulysses underhill 32.905 +ulysses underhill 41.653333333333336 +ulysses van buren 21.868181818181817 +ulysses white 15.296666666666667 +ulysses white 28.343333333333334 +ulysses xylophone 24.718 +ulysses xylophone 30.205 +ulysses xylophone 35.61 +ulysses young 21.56 +ulysses young 32.28125 +ulysses young 37.275 +victor allen 23.548000000000002 +victor allen 24.759999999999998 +victor brown 22.10181818181818 +victor brown 23.73 +victor brown 25.427272727272726 +victor brown 26.218571428571433 +victor davidson 20.55 +victor davidson 22.21666666666667 +victor davidson 29.778 +victor ellison 13.0775 +victor ellison 33.666 +victor hernandez 10.896 +victor hernandez 18.922 +victor hernandez 24.908888888888892 +victor hernandez 27.426666666666666 +victor hernandez 35.6675 +victor johnson 20.02 +victor johnson 27.070000000000004 +victor johnson 29.0775 +victor king 18.066666666666666 +victor king 21.488 +victor laertes 26.77777777777778 +victor laertes 28.095000000000002 +victor miller 5.3100000000000005 +victor nixon 21.395714285714288 +victor nixon 28.33 +victor ovid 35.225 +victor polk 21.990000000000002 +victor quirinius 24.62833333333333 +victor quirinius 29.742500000000003 +victor robinson 14.575 +victor robinson 25.92 +victor steinbeck 26.136666666666667 +victor steinbeck 26.485 +victor steinbeck 34.745999999999995 +victor thompson 18.735 +victor van buren 27.758333333333336 +victor van buren 37.38333333333333 +victor white 24.607999999999997 +victor white 30.66 +victor xylophone 2.775 +victor xylophone 8.356666666666667 +victor xylophone 24.259999999999998 +victor xylophone 25.636666666666667 +victor xylophone 31.610000000000003 +victor young 22.264444444444443 +victor zipper 39.84 +wendy allen 3.4 +wendy allen 24.695000000000004 +wendy allen 29.912 +wendy brown 28.22 +wendy brown 36.74 +wendy ellison 17.549999999999997 +wendy ellison 22.720000000000002 +wendy falkner 13.765 +wendy falkner 24.424444444444443 +wendy falkner 27.86733333333333 +wendy garcia 12.3 +wendy garcia 22.396666666666665 +wendy garcia 26.8325 +wendy garcia 28.596666666666664 +wendy hernandez 21.111428571428572 +wendy ichabod 4.44 +wendy king 23.654285714285713 +wendy king 29.325714285714287 +wendy king 34.21666666666667 +wendy laertes 31.160714285714285 +wendy laertes 31.46666666666667 +wendy laertes 39.22 +wendy miller 12.73 +wendy miller 30.343333333333334 +wendy nixon 19.92714285714286 +wendy nixon 29.675714285714285 +wendy ovid 21.193749999999998 +wendy ovid 28.49846153846154 +wendy polk 20.94 +wendy polk 22.999999999999996 +wendy quirinius 21.05 +wendy quirinius 26.8425 +wendy robinson 8.39 +wendy robinson 24.05 +wendy robinson 26.974285714285713 +wendy steinbeck 26.765 +wendy thompson 24.14 +wendy thompson 28.995384615384616 +wendy underhill 23.118333333333336 +wendy underhill 25.581666666666667 +wendy underhill 32.985 +wendy van buren 25.151666666666667 +wendy van buren 27.077142857142857 +wendy white 24.4025 +wendy xylophone 22.85181818181818 +wendy xylophone 26.96 +wendy young 4.83 +wendy young 21.325 +xavier allen 19.133333333333333 +xavier allen 26.11466666666667 +xavier allen 34.58 +xavier brown 2.63 +xavier brown 24.764285714285712 +xavier brown 30.166666666666668 +xavier carson 29.006666666666664 +xavier carson 32.106 +xavier davidson 14.094999999999999 +xavier davidson 15.906666666666666 +xavier davidson 27.353333333333335 +xavier ellison 22.174166666666668 +xavier ellison 35.01 +xavier garcia 30.357500000000005 +xavier hernandez 19.87 +xavier hernandez 20.805 +xavier hernandez 33.497499999999995 +xavier ichabod 12.34 +xavier ichabod 26.166249999999998 +xavier johnson 20.33222222222222 +xavier johnson 22.503333333333334 +xavier king 1.3 +xavier king 31.348571428571425 +xavier laertes 7.420000000000001 +xavier ovid 25.576 +xavier polk 11.094285714285714 +xavier polk 19.93 +xavier polk 23.63125 +xavier polk 30.194 +xavier quirinius 13.776666666666666 +xavier quirinius 22.27 +xavier quirinius 24.977692307692312 +xavier quirinius 34.95 +xavier thompson 16.47 +xavier underhill 1.31 +xavier white 19.331666666666667 +xavier white 34.68 +xavier xylophone 21.09625 +xavier zipper 14.89 +yuri allen 18.490000000000002 +yuri allen 22.689999999999998 +yuri brown 15.502857142857142 +yuri brown 22.934285714285714 +yuri carson 27.139999999999997 +yuri carson 35.27 +yuri ellison 10.52 +yuri ellison 25.2025 +yuri falkner 24.633076923076924 +yuri falkner 28.52 +yuri garcia 25.545 +yuri hernandez 16.35 +yuri johnson 19.9525 +yuri johnson 27.636000000000003 +yuri johnson 39.92 +yuri king 15.450000000000001 +yuri laertes 0.41000000000000003 +yuri laertes 33.15 +yuri nixon 27.795 +yuri nixon 39.145 +yuri polk 0.8 +yuri polk 9.705 +yuri polk 25.513333333333332 +yuri quirinius 16.29 +yuri quirinius 19.254999999999995 +yuri quirinius 37.878 +yuri steinbeck 27.6275 +yuri steinbeck 48.89 +yuri thompson 23.330000000000002 +yuri underhill 20.504444444444445 +yuri underhill 21.66 +yuri white 31.205 +yuri xylophone 18.790000000000003 +zach allen 13.06 +zach brown 19.985 +zach brown 26.52333333333333 +zach brown 34.66 +zach brown 34.972857142857144 +zach brown 37.45399999999999 +zach carson 26.195999999999998 +zach ellison 17.55 +zach falkner 3.42 +zach falkner 16.18 +zach garcia 20.062 +zach garcia 25.935 +zach garcia 28.974285714285717 +zach garcia 35.449999999999996 +zach ichabod 10.59 +zach ichabod 31.691999999999997 +zach king 6.81 +zach king 20.817 +zach king 32.542500000000004 +zach miller 13.23 +zach miller 26.30666666666667 +zach miller 26.73 +zach ovid 21.122500000000002 +zach ovid 26.983999999999998 +zach ovid 33.15 +zach ovid 40.59 +zach quirinius 13.38 +zach robinson 20.451999999999998 +zach steinbeck 20.358333333333334 +zach steinbeck 29.65 +zach thompson 16.45 +zach thompson 21.430000000000003 +zach underhill 31.438333333333333 +zach white 23.111428571428572 +zach xylophone 21.221428571428568 +zach xylophone 23.156666666666666 +zach young 24.72666666666667 +zach zipper 19.878888888888884 +zach zipper 34.84571428571429 +zach zipper 35.36 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf2-0-96659fde37d7a38ea15b367b47f59ce2 b/sql/hive/src/test/resources/golden/windowing_udaf2-0-96659fde37d7a38ea15b367b47f59ce2 new file mode 100644 index 0000000000000000000000000000000000000000..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf2-1-b4bdee4908b1cb8e240c549ae5cfe4c0 b/sql/hive/src/test/resources/golden/windowing_udaf2-1-b4bdee4908b1cb8e240c549ae5cfe4c0 new file mode 100644 index 0000000000000000000000000000000000000000..17c31c0f04592723f2c63800cc48bacc46b92340 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf2-1-b4bdee4908b1cb8e240c549ae5cfe4c0 @@ -0,0 +1 @@ +130091 130091 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-0-f498cccf82480be03022d2a36f87651e b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-0-f498cccf82480be03022d2a36f87651e new file mode 100644 index 0000000000000000000000000000000000000000..31b1f85a5eb5aa93dd90f2dbd09a2498a28ea159 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-0-f498cccf82480be03022d2a36f87651e @@ -0,0 +1,1049 @@ + 4294967354 + 4294967416 + 4294967457 +alice allen 4294967487 +alice allen 4294967525 +alice allen 4294967531 +alice brown 4294967355 +alice carson 4294967370 +alice davidson 4294967517 +alice falkner 4294967316 +alice garcia 4294967369 +alice hernandez 4294967299 +alice hernandez 4294967314 +alice johnson 4294967424 +alice king 4294967387 +alice king 4294967516 +alice king 4294967546 +alice laertes 4294967519 +alice laertes 8589934835 +alice miller 4294967324 +alice nixon 4294967410 +alice nixon 4294967413 +alice nixon 4294967484 +alice ovid 8589934726 +alice polk 4294967366 +alice quirinius 4294967505 +alice quirinius 4294967549 +alice robinson 4294967445 +alice robinson 4294967502 +alice steinbeck 4294967364 +alice steinbeck 4294967474 +alice steinbeck 4294967549 +alice underhill 4294967441 +alice van buren 4294967428 +alice xylophone 4294967363 +alice xylophone 4294967519 +alice xylophone 8589934832 +alice zipper 4294967380 +alice zipper 4294967520 +alice zipper 8589935026 +bob brown 4294967422 +bob brown 4294967427 +bob brown 4294967431 +bob carson 4294967408 +bob davidson 4294967354 +bob davidson 4294967435 +bob davidson 4294967504 +bob ellison 4294967344 +bob ellison 4294967362 +bob ellison 4294967436 +bob ellison 4294967530 +bob falkner 8589934966 +bob garcia 4294967369 +bob garcia 4294967435 +bob garcia 4294967439 +bob garcia 8589934707 +bob garcia 8589934867 +bob hernandez 4294967500 +bob ichabod 4294967424 +bob king 4294967297 +bob king 4294967539 +bob king 8589934870 +bob laertes 4294967380 +bob laertes 4294967472 +bob miller 4294967349 +bob ovid 4294967395 +bob ovid 4294967400 +bob ovid 4294967401 +bob ovid 4294967512 +bob polk 4294967337 +bob quirinius 4294967346 +bob steinbeck 4294967342 +bob van buren 4294967422 +bob white 4294967362 +bob white 4294967493 +bob xylophone 4294967407 +bob xylophone 4294967465 +bob young 4294967413 +bob zipper 4294967299 +bob zipper 8589934723 +bob zipper 8589934840 +calvin allen 12884902208 +calvin brown 4294967411 +calvin brown 4294967437 +calvin brown 4294967530 +calvin carson 8589934876 +calvin davidson 4294967468 +calvin davidson 8589934837 +calvin ellison 4294967480 +calvin falkner 4294967300 +calvin falkner 4294967305 +calvin falkner 4294967345 +calvin falkner 8589934749 +calvin falkner 8589934840 +calvin falkner 8589934978 +calvin garcia 8589934927 +calvin hernandez 12884902173 +calvin johnson 4294967546 +calvin laertes 4294967431 +calvin laertes 4294967499 +calvin nixon 4294967300 +calvin nixon 4294967412 +calvin nixon 4294967488 +calvin ovid 4294967329 +calvin ovid 4294967349 +calvin ovid 8589934723 +calvin ovid 8589934835 +calvin polk 8589934962 +calvin quirinius 4294967521 +calvin quirinius 4294967532 +calvin robinson 4294967326 +calvin steinbeck 4294967474 +calvin steinbeck 4294967505 +calvin steinbeck 8589934722 +calvin thompson 4294967513 +calvin thompson 8589934700 +calvin underhill 4294967478 +calvin van buren 4294967300 +calvin van buren 4294967508 +calvin white 4294967304 +calvin white 8589934924 +calvin xylophone 4294967376 +calvin xylophone 8589934599 +calvin xylophone 8589934866 +calvin young 4294967342 +calvin young 8589934817 +calvin zipper 12884902359 +calvin zipper 17179869649 +david allen 4294967371 +david allen 4294967381 +david brown 8589934762 +david brown 12884902420 +david davidson 4294967522 +david davidson 8589934819 +david davidson 12884902188 +david davidson 12884902327 +david ellison 4294967463 +david ellison 8589934777 +david ellison 12884902263 +david hernandez 4294967324 +david ichabod 4294967487 +david ichabod 12884902220 +david laertes 12884902107 +david nixon 4294967381 +david ovid 4294967396 +david ovid 4294967443 +david quirinius 4294967457 +david quirinius 4294967530 +david quirinius 12884902194 +david robinson 4294967465 +david robinson 17179869575 +david thompson 4294967361 +david underhill 4294967384 +david underhill 8589934942 +david underhill 12884902357 +david van buren 4294967309 +david van buren 8589934901 +david white 4294967428 +david xylophone 4294967479 +david xylophone 4294967480 +david xylophone 8589934856 +david young 4294967296 +david young 4294967305 +ethan allen 4294967351 +ethan brown 4294967320 +ethan brown 4294967331 +ethan brown 4294967403 +ethan brown 4294967420 +ethan brown 8589934797 +ethan brown 8589934805 +ethan carson 4294967352 +ethan ellison 4294967514 +ethan ellison 8589934887 +ethan falkner 4294967318 +ethan falkner 4294967461 +ethan garcia 4294967310 +ethan hernandez 4294967349 +ethan johnson 8589934738 +ethan king 8589934731 +ethan laertes 4294967422 +ethan laertes 4294967531 +ethan laertes 8589934767 +ethan laertes 8589934806 +ethan laertes 8589934830 +ethan laertes 8589934995 +ethan laertes 12884902063 +ethan miller 4294967352 +ethan nixon 8589935019 +ethan ovid 8589934909 +ethan polk 4294967329 +ethan polk 4294967382 +ethan polk 4294967479 +ethan polk 8589935021 +ethan quirinius 4294967348 +ethan quirinius 4294967501 +ethan quirinius 8589934695 +ethan robinson 4294967353 +ethan robinson 8589935019 +ethan underhill 8589934897 +ethan van buren 4294967511 +ethan white 4294967427 +ethan white 8589934975 +ethan xylophone 8589934956 +ethan zipper 4294967462 +ethan zipper 12884902348 +fred davidson 8589934724 +fred davidson 8589934850 +fred davidson 12884902468 +fred ellison 4294967393 +fred ellison 8589934797 +fred ellison 8589934978 +fred falkner 4294967547 +fred falkner 12884902170 +fred falkner 17179869760 +fred hernandez 8589934833 +fred ichabod 8589934853 +fred ichabod 12884902455 +fred johnson 8589934904 +fred king 8589934651 +fred king 8589934951 +fred laertes 8589934883 +fred miller 12884902228 +fred nixon 4294967297 +fred nixon 4294967375 +fred nixon 4294967514 +fred nixon 12884902182 +fred polk 4294967332 +fred polk 4294967458 +fred polk 4294967507 +fred polk 8589934944 +fred quirinius 8589934894 +fred quirinius 12884902335 +fred robinson 8589934904 +fred steinbeck 4294967329 +fred steinbeck 4294967411 +fred steinbeck 4294967472 +fred underhill 4294967387 +fred van buren 8589934830 +fred van buren 12884902319 +fred van buren 12884902382 +fred van buren 17179869836 +fred white 8589934763 +fred young 4294967485 +fred young 8589934832 +fred zipper 12884902371 +gabriella allen 4294967405 +gabriella allen 12884902509 +gabriella brown 4294967403 +gabriella brown 4294967543 +gabriella carson 8589934950 +gabriella davidson 4294967507 +gabriella ellison 4294967393 +gabriella ellison 12884902284 +gabriella falkner 4294967378 +gabriella falkner 4294967523 +gabriella falkner 12884902338 +gabriella garcia 4294967419 +gabriella hernandez 4294967462 +gabriella hernandez 4294967481 +gabriella ichabod 4294967337 +gabriella ichabod 8589934740 +gabriella ichabod 8589934797 +gabriella ichabod 8589934818 +gabriella ichabod 17179869508 +gabriella king 4294967393 +gabriella king 8589934906 +gabriella laertes 4294967410 +gabriella miller 8589934768 +gabriella ovid 4294967522 +gabriella ovid 8589934895 +gabriella polk 4294967302 +gabriella polk 8589934868 +gabriella steinbeck 4294967435 +gabriella steinbeck 4294967500 +gabriella thompson 4294967412 +gabriella thompson 8589934814 +gabriella thompson 12884902318 +gabriella van buren 4294967470 +gabriella van buren 8589934783 +gabriella white 4294967335 +gabriella young 4294967431 +gabriella young 8589934980 +gabriella zipper 4294967510 +gabriella zipper 8589934792 +holly allen 12884901926 +holly brown 8589934722 +holly brown 8589934857 +holly falkner 8589934849 +holly hernandez 8589934749 +holly hernandez 8589934805 +holly hernandez 8589935056 +holly hernandez 12884902485 +holly ichabod 4294967329 +holly ichabod 8589934754 +holly ichabod 8589934981 +holly johnson 4294967535 +holly johnson 12884902194 +holly johnson 17179869874 +holly king 8589934785 +holly king 8589934939 +holly laertes 12884902333 +holly miller 8589934823 +holly nixon 4294967383 +holly nixon 8589934744 +holly polk 4294967434 +holly polk 8589934782 +holly robinson 12884902369 +holly thompson 4294967339 +holly thompson 12884902395 +holly thompson 17179869547 +holly underhill 8589934913 +holly underhill 8589934924 +holly underhill 12884902376 +holly underhill 12884902412 +holly van buren 4294967539 +holly white 17179869548 +holly white 17179869900 +holly xylophone 8589934846 +holly young 4294967500 +holly young 8589934932 +holly zipper 4294967509 +holly zipper 17179869531 +irene allen 12884902413 +irene brown 4294967428 +irene brown 8589934934 +irene brown 12884902207 +irene carson 8589934797 +irene ellison 8589934732 +irene ellison 8589934773 +irene falkner 4294967404 +irene falkner 4294967548 +irene garcia 4294967323 +irene garcia 8589934887 +irene garcia 12884902479 +irene ichabod 4294967509 +irene ichabod 8589934860 +irene johnson 8589934990 +irene laertes 4294967481 +irene laertes 12884902196 +irene laertes 17179869632 +irene miller 4294967387 +irene nixon 4294967538 +irene nixon 12884902129 +irene nixon 12884902324 +irene ovid 8589934764 +irene ovid 8589934886 +irene ovid 8589934903 +irene polk 4294967465 +irene polk 4294967521 +irene polk 8589934672 +irene polk 8589934842 +irene polk 17179869877 +irene quirinius 8589934875 +irene quirinius 12884902269 +irene quirinius 17179869628 +irene robinson 8589934676 +irene steinbeck 4294967549 +irene thompson 4294967479 +irene underhill 8589934694 +irene underhill 12884902077 +irene van buren 8589934932 +irene van buren 12884902202 +irene xylophone 8589934901 +jessica brown 8589934867 +jessica carson 4294967508 +jessica carson 8589934740 +jessica carson 17179869819 +jessica davidson 4294967384 +jessica davidson 8589934864 +jessica davidson 12884902256 +jessica davidson 12884902321 +jessica ellison 4294967316 +jessica ellison 12884902128 +jessica falkner 8589934980 +jessica garcia 4294967540 +jessica garcia 21474837337 +jessica ichabod 8589934816 +jessica johnson 8589935006 +jessica johnson 12884902222 +jessica miller 8589934898 +jessica nixon 8589934742 +jessica nixon 12884902240 +jessica ovid 8589934830 +jessica ovid 12884902307 +jessica polk 21474837163 +jessica quirinius 8589934701 +jessica quirinius 8589934872 +jessica quirinius 12884902159 +jessica quirinius 12884902276 +jessica robinson 4294967542 +jessica thompson 8589934698 +jessica thompson 12884902232 +jessica underhill 8589934810 +jessica underhill 8589934878 +jessica underhill 17179869479 +jessica van buren 8589934726 +jessica white 12884902155 +jessica white 12884902281 +jessica white 12884902296 +jessica white 12884902314 +jessica white 17179869676 +jessica xylophone 17179869697 +jessica young 17179869859 +jessica young 17179869861 +jessica zipper 4294967372 +jessica zipper 8589934727 +jessica zipper 17179869778 +katie allen 8589934791 +katie brown 17179869660 +katie davidson 12884902181 +katie ellison 12884902184 +katie ellison 12884902355 +katie falkner 8589934911 +katie garcia 8589934683 +katie garcia 12884902046 +katie hernandez 8589934812 +katie ichabod 8589934795 +katie ichabod 8589934862 +katie ichabod 8589934869 +katie king 4294967339 +katie king 4294967421 +katie king 8589934826 +katie miller 8589934829 +katie miller 12884902267 +katie nixon 21474837149 +katie ovid 4294967519 +katie polk 8589934726 +katie polk 12884902291 +katie robinson 17179869645 +katie van buren 8589934722 +katie van buren 17179869441 +katie white 4294967306 +katie white 8589934885 +katie xylophone 12884902193 +katie young 8589934819 +katie young 8589935024 +katie young 12884902058 +katie zipper 4294967354 +katie zipper 12884902310 +luke allen 8589934864 +luke allen 8589934931 +luke allen 8589935059 +luke allen 12884902257 +luke allen 12884902322 +luke brown 8589934779 +luke davidson 4294967354 +luke davidson 12884902360 +luke ellison 12884902183 +luke ellison 21474836998 +luke ellison 21474837060 +luke falkner 8589934772 +luke falkner 17179869561 +luke garcia 4294967304 +luke garcia 21474837157 +luke ichabod 12884902150 +luke ichabod 12884902366 +luke johnson 4294967527 +luke johnson 8589934812 +luke johnson 12884902161 +luke laertes 8589935027 +luke laertes 12884902031 +luke laertes 12884902184 +luke laertes 12884902213 +luke laertes 12884902378 +luke miller 8589934826 +luke ovid 4294967492 +luke ovid 8589934913 +luke polk 8589934837 +luke polk 12884902340 +luke quirinius 8589934855 +luke robinson 4294967307 +luke robinson 17179869711 +luke thompson 4294967521 +luke underhill 8589934829 +luke underhill 12884902299 +luke underhill 21474837138 +luke van buren 8589934852 +luke white 12884902418 +luke xylophone 8589934804 +luke zipper 4294967353 +mike allen 17179869750 +mike brown 17179869735 +mike carson 4294967477 +mike carson 8589934803 +mike carson 17179869855 +mike davidson 12884902377 +mike davidson 17179869841 +mike ellison 8589934833 +mike ellison 12884902165 +mike ellison 12884902513 +mike ellison 17179869587 +mike ellison 17179869824 +mike falkner 4294967301 +mike garcia 4294967398 +mike garcia 8589934800 +mike garcia 12884902292 +mike hernandez 8589934824 +mike hernandez 12884902281 +mike ichabod 4294967494 +mike king 4294967347 +mike king 4294967400 +mike king 12884902363 +mike king 12884902475 +mike king 17179869528 +mike king 17179869592 +mike miller 17179869705 +mike nixon 12884902293 +mike nixon 17179869708 +mike polk 17179869752 +mike polk 21474837097 +mike polk 21474837344 +mike quirinius 12884902240 +mike steinbeck 8589934653 +mike steinbeck 12884902273 +mike steinbeck 12884902301 +mike steinbeck 17179869903 +mike van buren 8589934942 +mike van buren 12884902402 +mike white 12884902485 +mike white 17179869676 +mike white 21474836928 +mike white 25769804626 +mike young 8589934704 +mike young 8589934878 +mike young 17179869685 +mike zipper 4294967501 +mike zipper 17179869582 +mike zipper 25769804400 +nick allen 8589934664 +nick allen 8589934860 +nick brown 21474836962 +nick davidson 4294967357 +nick ellison 12884902066 +nick ellison 17179869779 +nick falkner 8589935020 +nick falkner 12884902433 +nick garcia 8589934885 +nick garcia 17179869635 +nick garcia 17179869681 +nick ichabod 12884902193 +nick ichabod 12884902223 +nick ichabod 12884902252 +nick johnson 17179869591 +nick johnson 17179869702 +nick laertes 8589934919 +nick miller 12884902419 +nick nixon 8589934910 +nick ovid 12884902267 +nick polk 17179869712 +nick quirinius 4294967296 +nick quirinius 12884902183 +nick robinson 17179869506 +nick robinson 17179869731 +nick steinbeck 4294967355 +nick thompson 8589934922 +nick underhill 25769804624 +nick van buren 8589934635 +nick xylophone 12884902279 +nick young 12884902399 +nick young 21474837140 +nick zipper 12884902300 +nick zipper 17179869849 +oscar allen 17179869779 +oscar brown 12884902062 +oscar carson 12884902232 +oscar carson 17179869663 +oscar carson 17179869779 +oscar carson 21474837066 +oscar carson 21474837089 +oscar davidson 17179869895 +oscar ellison 4294967304 +oscar ellison 8589934740 +oscar falkner 4294967526 +oscar garcia 21474837156 +oscar hernandez 4294967343 +oscar hernandez 8589935049 +oscar ichabod 8589934837 +oscar ichabod 21474836952 +oscar ichabod 21474837021 +oscar ichabod 25769804491 +oscar johnson 12884902182 +oscar johnson 30064772044 +oscar king 12884902159 +oscar king 17179869738 +oscar king 17179869834 +oscar laertes 4294967550 +oscar laertes 8589934727 +oscar laertes 12884902043 +oscar laertes 12884902478 +oscar nixon 17179869458 +oscar ovid 12884902128 +oscar ovid 12884902240 +oscar ovid 25769804460 +oscar polk 21474836829 +oscar polk 21474837063 +oscar quirinius 8589934728 +oscar quirinius 17179869698 +oscar quirinius 21474837051 +oscar quirinius 25769804521 +oscar robinson 8589934656 +oscar robinson 12884902249 +oscar robinson 21474837105 +oscar robinson 25769804694 +oscar steinbeck 4294967548 +oscar thompson 8589934776 +oscar thompson 12884902164 +oscar thompson 12884902317 +oscar thompson 17179869884 +oscar underhill 8589934895 +oscar van buren 4294967500 +oscar van buren 8589934984 +oscar van buren 21474837205 +oscar white 4294967454 +oscar white 8589934826 +oscar white 21474836931 +oscar white 21474837305 +oscar xylophone 12884902193 +oscar xylophone 12884902307 +oscar xylophone 17179869593 +oscar zipper 8589934865 +oscar zipper 8589934874 +oscar zipper 8589934911 +priscilla brown 8589934848 +priscilla brown 8589935013 +priscilla brown 17179869801 +priscilla carson 12884902145 +priscilla carson 21474836880 +priscilla carson 30064772126 +priscilla ichabod 4294967547 +priscilla ichabod 17179869756 +priscilla johnson 4294967468 +priscilla johnson 8589934667 +priscilla johnson 17179869667 +priscilla johnson 17179869787 +priscilla johnson 25769804279 +priscilla king 12884902153 +priscilla nixon 12884902188 +priscilla nixon 25769804766 +priscilla ovid 12884902234 +priscilla ovid 30064772049 +priscilla polk 17179869480 +priscilla quirinius 12884902171 +priscilla thompson 25769804637 +priscilla underhill 4294967333 +priscilla underhill 17179869740 +priscilla van buren 12884902324 +priscilla van buren 21474837167 +priscilla van buren 21474837343 +priscilla white 4294967419 +priscilla xylophone 8589934792 +priscilla xylophone 12884902245 +priscilla xylophone 12884902287 +priscilla young 21474836992 +priscilla young 34359739656 +priscilla zipper 12884902296 +priscilla zipper 12884902537 +quinn allen 4294967542 +quinn allen 17179869552 +quinn brown 12884902251 +quinn brown 17179869401 +quinn brown 17179869626 +quinn davidson 8589934992 +quinn davidson 17179869690 +quinn davidson 25769804455 +quinn davidson 30064771771 +quinn ellison 12884902376 +quinn ellison 34359739559 +quinn garcia 8589934828 +quinn garcia 12884902387 +quinn garcia 12884902460 +quinn garcia 21474837066 +quinn ichabod 30064772171 +quinn king 4294967458 +quinn king 4294967538 +quinn laertes 8589935080 +quinn laertes 17179869711 +quinn laertes 21474837142 +quinn nixon 17179869672 +quinn ovid 17179869695 +quinn quirinius 21474836827 +quinn robinson 12884902445 +quinn steinbeck 17179869739 +quinn steinbeck 21474836905 +quinn thompson 17179869645 +quinn thompson 25769804317 +quinn underhill 8589934815 +quinn underhill 12884902185 +quinn underhill 30064771762 +quinn van buren 4294967362 +quinn young 8589934731 +quinn zipper 12884902453 +quinn zipper 17179869841 +rachel allen 8589934882 +rachel allen 12884902208 +rachel brown 8589934768 +rachel brown 12884902075 +rachel brown 17179869910 +rachel brown 17179869911 +rachel brown 21474837280 +rachel carson 8589934728 +rachel carson 17179869970 +rachel davidson 30064771666 +rachel ellison 4294967423 +rachel falkner 4294967348 +rachel falkner 12884902482 +rachel falkner 21474837331 +rachel falkner 25769804739 +rachel johnson 38654707197 +rachel king 12884902157 +rachel king 30064771759 +rachel laertes 17179869678 +rachel laertes 25769804379 +rachel ovid 12884902055 +rachel ovid 17179869857 +rachel polk 12884902391 +rachel quirinius 17179869456 +rachel robinson 17179869499 +rachel robinson 17179869703 +rachel robinson 25769804290 +rachel thompson 17179869910 +rachel thompson 21474836989 +rachel thompson 21474837392 +rachel underhill 8589934862 +rachel white 17179869585 +rachel white 21474837039 +rachel young 17179869708 +rachel zipper 4294967434 +rachel zipper 21474837228 +sarah carson 4294967319 +sarah carson 17179869688 +sarah carson 30064772084 +sarah ellison 4294967542 +sarah falkner 17179869797 +sarah falkner 21474837349 +sarah garcia 8589934733 +sarah garcia 8589934858 +sarah garcia 17179869599 +sarah ichabod 12884902196 +sarah ichabod 12884902401 +sarah johnson 12884902455 +sarah johnson 21474836981 +sarah johnson 21474837145 +sarah johnson 25769804480 +sarah king 12884902453 +sarah king 21474837191 +sarah miller 8589934958 +sarah ovid 21474837184 +sarah robinson 21474837237 +sarah robinson 21474837389 +sarah steinbeck 21474837313 +sarah white 17179869905 +sarah white 25769804341 +sarah xylophone 12884902207 +sarah young 21474837319 +sarah zipper 25769804616 +tom brown 8589934894 +tom brown 21474837024 +tom carson 4294967388 +tom carson 12884902278 +tom carson 21474836983 +tom davidson 8589934895 +tom ellison 12884902192 +tom ellison 17179869965 +tom ellison 25769804262 +tom falkner 12884902272 +tom falkner 17179869815 +tom hernandez 4294967296 +tom hernandez 12884902109 +tom ichabod 17179869628 +tom johnson 25769804829 +tom johnson 30064771891 +tom king 12884902390 +tom laertes 12884902181 +tom laertes 12884902236 +tom miller 12884901992 +tom miller 17179869647 +tom miller 21474837107 +tom nixon 17179869677 +tom ovid 12884902279 +tom polk 8589934748 +tom polk 8589934892 +tom quirinius 12884902174 +tom quirinius 21474836986 +tom robinson 8589934753 +tom robinson 12884902203 +tom robinson 12884902358 +tom robinson 21474836952 +tom steinbeck 8589934912 +tom van buren 8589934823 +tom van buren 12884902122 +tom van buren 25769804641 +tom white 21474837076 +tom young 4294967535 +tom young 21474837038 +tom zipper 30064772355 +ulysses brown 8589934991 +ulysses carson 8589934789 +ulysses carson 21474837258 +ulysses carson 25769804457 +ulysses carson 34359739082 +ulysses davidson 12884902216 +ulysses ellison 17179869551 +ulysses garcia 12884902382 +ulysses hernandez 12884902210 +ulysses hernandez 12884902276 +ulysses hernandez 17179869748 +ulysses ichabod 4294967353 +ulysses ichabod 12884902217 +ulysses johnson 21474837122 +ulysses king 8589934995 +ulysses laertes 8589934801 +ulysses laertes 21474837354 +ulysses laertes 25769804499 +ulysses miller 21474837284 +ulysses miller 30064771926 +ulysses nixon 17179869288 +ulysses ovid 17179869754 +ulysses polk 8589934855 +ulysses polk 8589934862 +ulysses polk 12884902420 +ulysses polk 17179869479 +ulysses quirinius 17179869659 +ulysses robinson 4294967531 +ulysses steinbeck 8589935027 +ulysses steinbeck 21474837100 +ulysses thompson 12884902194 +ulysses underhill 8589934760 +ulysses underhill 8589934799 +ulysses underhill 12884902240 +ulysses underhill 17179869759 +ulysses underhill 17179869760 +ulysses underhill 17179869939 +ulysses underhill 21474837264 +ulysses van buren 8589934938 +ulysses white 25769804453 +ulysses white 30064772086 +ulysses xylophone 8589935029 +ulysses xylophone 12884902249 +ulysses xylophone 25769804765 +ulysses young 4294967427 +ulysses young 17179869391 +ulysses young 30064771844 +victor allen 8589934793 +victor allen 12884902264 +victor brown 4294967455 +victor brown 17179869657 +victor brown 21474837426 +victor brown 30064771922 +victor davidson 17179869715 +victor davidson 17179869872 +victor davidson 25769804287 +victor ellison 17179869611 +victor ellison 17179869709 +victor hernandez 8589934847 +victor hernandez 12884902463 +victor hernandez 17179869647 +victor hernandez 17179869720 +victor hernandez 25769804310 +victor johnson 17179869652 +victor johnson 21474837148 +victor johnson 25769804771 +victor king 8589934917 +victor king 25769804714 +victor laertes 12884902188 +victor laertes 21474837186 +victor miller 21474837170 +victor nixon 8589934778 +victor nixon 12884902261 +victor ovid 12884902350 +victor polk 17179869376 +victor quirinius 21474837074 +victor quirinius 21474837279 +victor robinson 21474836948 +victor robinson 21474837097 +victor steinbeck 12884902162 +victor steinbeck 17179869721 +victor steinbeck 21474836916 +victor thompson 25769804395 +victor van buren 21474837010 +victor van buren 25769804601 +victor white 8589934816 +victor white 30064771798 +victor xylophone 17179869560 +victor xylophone 25769804719 +victor xylophone 25769804760 +victor xylophone 34359739093 +victor xylophone 34359739095 +victor young 21474837052 +victor zipper 12884902345 +wendy allen 21474837127 +wendy allen 25769804525 +wendy allen 25769804732 +wendy brown 12884902342 +wendy brown 21474836889 +wendy ellison 12884902392 +wendy ellison 21474836763 +wendy falkner 8589934926 +wendy falkner 17179869470 +wendy falkner 25769804816 +wendy garcia 17179869439 +wendy garcia 17179869732 +wendy garcia 30064771654 +wendy garcia 30064771704 +wendy hernandez 17179869752 +wendy ichabod 17179869547 +wendy king 17179869612 +wendy king 21474837301 +wendy king 30064772042 +wendy laertes 8589934872 +wendy laertes 12884902469 +wendy laertes 21474837084 +wendy miller 17179869661 +wendy miller 17179869682 +wendy nixon 12884902521 +wendy nixon 21474836846 +wendy ovid 21474837025 +wendy ovid 38654706512 +wendy polk 8589934960 +wendy polk 21474837144 +wendy quirinius 12884902263 +wendy quirinius 17179869652 +wendy robinson 21474837104 +wendy robinson 25769804321 +wendy robinson 25769804728 +wendy steinbeck 12884902299 +wendy thompson 17179869494 +wendy thompson 21474837072 +wendy underhill 17179869898 +wendy underhill 21474837064 +wendy underhill 25769804845 +wendy van buren 25769804447 +wendy van buren 25769804679 +wendy white 17179869866 +wendy xylophone 17179869596 +wendy xylophone 25769804554 +wendy young 4294967313 +wendy young 25769804562 +xavier allen 12884902364 +xavier allen 17179869960 +xavier allen 21474836864 +xavier brown 8589934824 +xavier brown 17179869646 +xavier brown 25769804653 +xavier carson 17179869770 +xavier carson 21474837445 +xavier davidson 30064772118 +xavier davidson 34359739403 +xavier davidson 38654706539 +xavier ellison 34359739490 +xavier ellison 34359739559 +xavier garcia 21474837142 +xavier hernandez 21474837012 +xavier hernandez 25769804421 +xavier hernandez 38654707021 +xavier ichabod 12884902315 +xavier ichabod 17179869567 +xavier johnson 8589934922 +xavier johnson 38654707066 +xavier king 12884902272 +xavier king 21474836962 +xavier laertes 17179869795 +xavier ovid 17179869597 +xavier polk 12884902254 +xavier polk 17179869581 +xavier polk 17179869743 +xavier polk 34359739344 +xavier quirinius 12884902240 +xavier quirinius 21474836996 +xavier quirinius 25769804437 +xavier quirinius 25769804456 +xavier thompson 17179869822 +xavier underhill 8589934813 +xavier white 12884902262 +xavier white 12884902366 +xavier xylophone 17179869722 +xavier zipper 12884902377 +yuri allen 8589935035 +yuri allen 12884902279 +yuri brown 8589934912 +yuri brown 12884902319 +yuri carson 21474837146 +yuri carson 25769804245 +yuri ellison 25769804504 +yuri ellison 25769804568 +yuri falkner 25769804699 +yuri falkner 42949674720 +yuri garcia 4294967362 +yuri hernandez 21474837117 +yuri johnson 21474837002 +yuri johnson 21474837165 +yuri johnson 25769804545 +yuri king 30064772090 +yuri laertes 30064772076 +yuri laertes 34359739328 +yuri nixon 12884902232 +yuri nixon 12884902265 +yuri polk 12884902362 +yuri polk 21474837245 +yuri polk 25769804539 +yuri quirinius 12884902198 +yuri quirinius 17179869606 +yuri quirinius 30064771819 +yuri steinbeck 4294967535 +yuri steinbeck 8589934657 +yuri thompson 12884902467 +yuri underhill 17179869566 +yuri underhill 17179869715 +yuri white 34359739045 +yuri xylophone 12884902412 +zach allen 17179869908 +zach brown 21474836879 +zach brown 21474836891 +zach brown 21474837040 +zach brown 21474837073 +zach brown 30064771852 +zach carson 21474837185 +zach ellison 8589934898 +zach falkner 17179869807 +zach falkner 25769804634 +zach garcia 17179869536 +zach garcia 21474837142 +zach garcia 30064772246 +zach garcia 34359739192 +zach ichabod 17179869613 +zach ichabod 17179869838 +zach king 17179869700 +zach king 21474837427 +zach king 34359739578 +zach miller 4294967391 +zach miller 12884902310 +zach miller 17179869709 +zach ovid 17179869731 +zach ovid 21474837032 +zach ovid 21474837127 +zach ovid 30064771625 +zach quirinius 34359739151 +zach robinson 21474836938 +zach steinbeck 17179869667 +zach steinbeck 25769804623 +zach thompson 12884902354 +zach thompson 17179869659 +zach underhill 12884902149 +zach white 25769804490 +zach xylophone 12884902198 +zach xylophone 21474837163 +zach young 17179869687 +zach zipper 17179869708 +zach zipper 17179869834 +zach zipper 21474837369 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-1-6378faf36ffd3f61e61cee6c0cb70e6 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-1-6378faf36ffd3f61e61cee6c0cb70e6 new file mode 100644 index 0000000000000000000000000000000000000000..1436509e4ec17c2ed6854d8e75650d23a323c582 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-1-6378faf36ffd3f61e61cee6c0cb70e6 @@ -0,0 +1,1049 @@ + 9.220000267028809 + 43.72999954223633 + 89.52999877929688 +alice allen 2.7899999618530273 +alice allen 21.450000762939453 +alice allen 73.62999725341797 +alice brown 71.30999755859375 +alice carson 39.029998779296875 +alice davidson 70.3499984741211 +alice falkner 90.25 +alice garcia 48.45000076293945 +alice hernandez 88.16999816894531 +alice hernandez 90.55999755859375 +alice johnson 4.46999979019165 +alice king 19.139999389648438 +alice king 23.170000076293945 +alice king 52.22999954223633 +alice laertes 68.94999694824219 +alice laertes 69.52999877929688 +alice miller 68.95999908447266 +alice nixon 40.0 +alice nixon 48.150001525878906 +alice nixon 79.83000183105469 +alice ovid 9.039999961853027 +alice polk 62.900001525878906 +alice quirinius 37.13999938964844 +alice quirinius 62.29999923706055 +alice robinson 5.079999923706055 +alice robinson 56.099998474121094 +alice steinbeck 38.619998931884766 +alice steinbeck 55.5099983215332 +alice steinbeck 92.37000274658203 +alice underhill 98.18000030517578 +alice van buren 38.939998626708984 +alice xylophone 33.58000183105469 +alice xylophone 43.15999984741211 +alice xylophone 78.20999908447266 +alice zipper 26.43000030517578 +alice zipper 42.47999954223633 +alice zipper 89.93000030517578 +bob brown 8.069999694824219 +bob brown 70.93000030517578 +bob brown 93.08999633789062 +bob carson 50.09000015258789 +bob davidson 1.2899999618530273 +bob davidson 53.93000030517578 +bob davidson 74.72000122070312 +bob ellison 41.34000015258789 +bob ellison 65.0199966430664 +bob ellison 75.02999877929688 +bob ellison 80.30000305175781 +bob falkner 16.989999771118164 +bob garcia 4.460000038146973 +bob garcia 5.400000095367432 +bob garcia 45.59000015258789 +bob garcia 80.30000305175781 +bob garcia 87.56999969482422 +bob hernandez 22.68000030517578 +bob ichabod 82.55999755859375 +bob king 8.789999961853027 +bob king 12.539999961853027 +bob king 39.0099983215332 +bob laertes 0.7900000214576721 +bob laertes 10.670000076293945 +bob miller 61.91999816894531 +bob ovid 46.86000061035156 +bob ovid 62.849998474121094 +bob ovid 88.77999877929688 +bob ovid 97.08999633789062 +bob polk 7.980000019073486 +bob quirinius 46.099998474121094 +bob steinbeck 9.699999809265137 +bob van buren 33.66999816894531 +bob white 45.34000015258789 +bob white 45.349998474121094 +bob xylophone 19.690000534057617 +bob xylophone 107.93000221252441 +bob young 35.16999816894531 +bob zipper 1.25 +bob zipper 3.819999933242798 +bob zipper 34.349998474121094 +calvin allen 63.119998931884766 +calvin brown 28.110000610351562 +calvin brown 85.9000015258789 +calvin brown 90.19999694824219 +calvin carson 80.2300033569336 +calvin davidson 31.770000457763672 +calvin davidson 85.51000213623047 +calvin ellison 26.489999771118164 +calvin falkner 2.9700000286102295 +calvin falkner 56.040000915527344 +calvin falkner 56.33000183105469 +calvin falkner 80.5999984741211 +calvin falkner 93.61000061035156 +calvin falkner 94.30999755859375 +calvin garcia 41.849998474121094 +calvin hernandez 33.869998931884766 +calvin johnson 66.61000061035156 +calvin laertes 23.1299991607666 +calvin laertes 50.310001373291016 +calvin nixon 9.8100004196167 +calvin nixon 41.20000076293945 +calvin nixon 69.73999786376953 +calvin ovid 69.95999908447266 +calvin ovid 71.26000213623047 +calvin ovid 79.12000274658203 +calvin ovid 84.72000122070312 +calvin polk 65.72000122070312 +calvin quirinius 29.540000915527344 +calvin quirinius 53.02000045776367 +calvin robinson 40.439998626708984 +calvin steinbeck 15.220000267028809 +calvin steinbeck 22.850000381469727 +calvin steinbeck 93.30000305175781 +calvin thompson 8.90999984741211 +calvin thompson 93.7300033569336 +calvin underhill 59.70000076293945 +calvin van buren 34.209999084472656 +calvin van buren 64.0 +calvin white 50.279998779296875 +calvin white 90.69000244140625 +calvin xylophone 21.700000762939453 +calvin xylophone 25.420000076293945 +calvin xylophone 56.810001373291016 +calvin young 24.489999771118164 +calvin young 39.810001373291016 +calvin zipper 9.1899995803833 +calvin zipper 95.37999725341797 +david allen 51.25 +david allen 64.87000274658203 +david brown 3.2100000381469727 +david brown 93.63999938964844 +david davidson 1.0800000429153442 +david davidson 62.720001220703125 +david davidson 74.1500015258789 +david davidson 95.80999755859375 +david ellison 47.689998626708984 +david ellison 85.2300033569336 +david ellison 94.1500015258789 +david hernandez 99.91000366210938 +david ichabod 5.28000020980835 +david ichabod 82.55000305175781 +david laertes 76.70999908447266 +david nixon 50.31999969482422 +david ovid 25.110000610351562 +david ovid 61.70000076293945 +david quirinius 20.639999389648438 +david quirinius 29.239999771118164 +david quirinius 79.97000122070312 +david robinson 25.280000686645508 +david robinson 161.1199951171875 +david thompson 80.89999771118164 +david underhill 8.319999694824219 +david underhill 88.7699966430664 +david underhill 97.55999755859375 +david van buren 83.56999969482422 +david van buren 115.43999862670898 +david white 45.189998626708984 +david xylophone 8.069999694824219 +david xylophone 54.34000015258789 +david xylophone 72.9800033569336 +david young 10.25 +david young 35.650001525878906 +ethan allen 32.75 +ethan brown 7.110000133514404 +ethan brown 10.09000015258789 +ethan brown 15.630000114440918 +ethan brown 61.86000061035156 +ethan brown 73.18000030517578 +ethan brown 82.30000305175781 +ethan carson 76.33000183105469 +ethan ellison 0.2800000011920929 +ethan ellison 81.47000122070312 +ethan falkner 50.02000045776367 +ethan falkner 59.43000030517578 +ethan garcia 43.189998626708984 +ethan hernandez 49.779998779296875 +ethan johnson 90.05000305175781 +ethan king 4.349999904632568 +ethan laertes 15.449999809265137 +ethan laertes 54.75 +ethan laertes 59.209999084472656 +ethan laertes 70.38999938964844 +ethan laertes 80.70999908447266 +ethan laertes 95.06999969482422 +ethan laertes 96.29000091552734 +ethan miller 25.3700008392334 +ethan nixon 37.779998779296875 +ethan ovid 57.290000915527344 +ethan polk 2.3499999046325684 +ethan polk 21.31999969482422 +ethan polk 23.440000534057617 +ethan polk 122.71999740600586 +ethan quirinius 3.859999895095825 +ethan quirinius 51.84000015258789 +ethan quirinius 97.23999786376953 +ethan robinson 67.94000244140625 +ethan robinson 78.62000274658203 +ethan underhill 55.630001068115234 +ethan van buren 36.70000076293945 +ethan white 60.849998474121094 +ethan white 63.41999816894531 +ethan xylophone 57.11000061035156 +ethan zipper 2.9200000762939453 +ethan zipper 97.51000213623047 +fred davidson 18.860000610351562 +fred davidson 37.2400016784668 +fred davidson 78.30999755859375 +fred ellison 31.179998874664307 +fred ellison 48.59000015258789 +fred ellison 96.77999877929688 +fred falkner 10.289999961853027 +fred falkner 72.04000091552734 +fred falkner 85.0 +fred hernandez 55.9900016784668 +fred ichabod 47.359999656677246 +fred ichabod 81.31999969482422 +fred johnson 96.08999633789062 +fred king 48.369998931884766 +fred king 72.13999843597412 +fred laertes 57.63999938964844 +fred miller 46.970001220703125 +fred nixon 28.690000534057617 +fred nixon 38.04999923706055 +fred nixon 70.5199966430664 +fred nixon 93.02999877929688 +fred polk 23.959999084472656 +fred polk 39.18000030517578 +fred polk 47.31999969482422 +fred polk 90.12000274658203 +fred quirinius 15.300000190734863 +fred quirinius 29.399999618530273 +fred robinson 89.02999877929688 +fred steinbeck 32.22999954223633 +fred steinbeck 41.310001373291016 +fred steinbeck 91.05000305175781 +fred underhill 90.7699966430664 +fred van buren 1.0199999809265137 +fred van buren 21.940000534057617 +fred van buren 52.869998931884766 +fred van buren 83.58000183105469 +fred white 37.79999923706055 +fred young 46.79999923706055 +fred young 97.70999908447266 +fred zipper 29.020000457763672 +gabriella allen 46.27000045776367 +gabriella allen 64.22000122070312 +gabriella brown 15.260000228881836 +gabriella brown 84.83000183105469 +gabriella carson 42.7599983215332 +gabriella davidson 6.550000190734863 +gabriella ellison 48.08000183105469 +gabriella ellison 71.54000091552734 +gabriella falkner 10.170000076293945 +gabriella falkner 51.720001220703125 +gabriella falkner 87.61000061035156 +gabriella garcia 43.0099983215332 +gabriella hernandez 76.91999816894531 +gabriella hernandez 92.9800033569336 +gabriella ichabod 10.729999542236328 +gabriella ichabod 26.639999389648438 +gabriella ichabod 66.36000061035156 +gabriella ichabod 71.12999725341797 +gabriella ichabod 90.3499984741211 +gabriella king 20.670000076293945 +gabriella king 80.45999908447266 +gabriella laertes 65.37999725341797 +gabriella miller 50.83000183105469 +gabriella ovid 77.7400016784668 +gabriella ovid 92.4000015258789 +gabriella polk 35.68000030517578 +gabriella polk 88.05000305175781 +gabriella steinbeck 46.45000076293945 +gabriella steinbeck 78.63999938964844 +gabriella thompson 73.31999969482422 +gabriella thompson 88.36000061035156 +gabriella thompson 94.25 +gabriella van buren 69.80000305175781 +gabriella van buren 70.05999755859375 +gabriella white 55.18000030517578 +gabriella young 9.25 +gabriella young 59.709999084472656 +gabriella zipper 36.2599983215332 +gabriella zipper 91.62999725341797 +holly allen 44.56999969482422 +holly brown 77.80999755859375 +holly brown 78.7300033569336 +holly falkner 80.73999786376953 +holly hernandez 20.81999969482422 +holly hernandez 21.190000534057617 +holly hernandez 24.790000915527344 +holly hernandez 30.25 +holly ichabod 83.2699966430664 +holly ichabod 84.69000244140625 +holly ichabod 90.51000213623047 +holly johnson 36.95000076293945 +holly johnson 64.36000061035156 +holly johnson 65.62000274658203 +holly king 42.310001373291016 +holly king 55.38999938964844 +holly laertes 52.5 +holly miller 50.400001525878906 +holly nixon 53.779998779296875 +holly nixon 88.0199966430664 +holly polk 3.619999885559082 +holly polk 98.30999755859375 +holly robinson 69.31999969482422 +holly thompson 0.07999999821186066 +holly thompson 86.69000244140625 +holly thompson 145.93999481201172 +holly underhill 42.54999923706055 +holly underhill 50.40999984741211 +holly underhill 79.95999908447266 +holly underhill 96.68000030517578 +holly van buren 68.80999755859375 +holly white 7.960000038146973 +holly white 32.91999816894531 +holly xylophone 93.11000061035156 +holly young 60.220001220703125 +holly young 66.16999816894531 +holly zipper 99.12999725341797 +holly zipper 99.29000091552734 +irene allen 38.849998474121094 +irene brown 4.789999961853027 +irene brown 53.939998626708984 +irene brown 87.66999816894531 +irene carson 94.54000091552734 +irene ellison 45.2400016784668 +irene ellison 50.08000183105469 +irene falkner 22.079999923706055 +irene falkner 99.91999816894531 +irene garcia 15.369999885559082 +irene garcia 58.43000030517578 +irene garcia 86.93000030517578 +irene ichabod 41.439998626708984 +irene ichabod 99.62000274658203 +irene johnson 5.880000114440918 +irene laertes 9.569999694824219 +irene laertes 42.66999816894531 +irene laertes 44.43000030517578 +irene miller 65.44000244140625 +irene nixon 15.100000381469727 +irene nixon 29.780000686645508 +irene nixon 42.560001373291016 +irene ovid 5.239999771118164 +irene ovid 35.130001068115234 +irene ovid 79.75 +irene polk 0.9800000190734863 +irene polk 24.020000457763672 +irene polk 42.2400016784668 +irene polk 47.08000183105469 +irene polk 95.83999633789062 +irene quirinius 12.899999618530273 +irene quirinius 58.86000061035156 +irene quirinius 70.0 +irene robinson 94.2699966430664 +irene steinbeck 94.33000183105469 +irene thompson 78.30000305175781 +irene underhill 28.309999465942383 +irene underhill 57.349998474121094 +irene van buren 54.439998626708984 +irene van buren 54.9900016784668 +irene xylophone 74.19000244140625 +jessica brown 51.290000915527344 +jessica carson 25.549999237060547 +jessica carson 31.860000610351562 +jessica carson 62.20000076293945 +jessica davidson 33.54999923706055 +jessica davidson 49.77000045776367 +jessica davidson 95.33999633789062 +jessica davidson 99.20999908447266 +jessica ellison 11.180000305175781 +jessica ellison 22.780000686645508 +jessica falkner 99.6500015258789 +jessica garcia 5.539999961853027 +jessica garcia 87.92999941110611 +jessica ichabod 59.15999984741211 +jessica johnson 9.5600004196167 +jessica johnson 40.79999923706055 +jessica miller 151.0199966430664 +jessica nixon 77.0999984741211 +jessica nixon 90.06999969482422 +jessica ovid 71.68000030517578 +jessica ovid 119.9000015258789 +jessica polk 49.68000030517578 +jessica quirinius 22.940000534057617 +jessica quirinius 32.470001220703125 +jessica quirinius 35.619998931884766 +jessica quirinius 46.869998931884766 +jessica robinson 112.36000442504883 +jessica thompson 38.33000183105469 +jessica thompson 89.55000305175781 +jessica underhill 26.079999923706055 +jessica underhill 45.41999816894531 +jessica underhill 46.209999084472656 +jessica van buren 9.739999771118164 +jessica white 11.550000190734863 +jessica white 36.58000183105469 +jessica white 73.93000030517578 +jessica white 74.30000305175781 +jessica white 96.62000274658203 +jessica xylophone 53.060001373291016 +jessica young 11.1899995803833 +jessica young 43.369998931884766 +jessica zipper 6.630000114440918 +jessica zipper 12.020000457763672 +jessica zipper 92.43999862670898 +katie allen 64.66999816894531 +katie brown 27.719999313354492 +katie davidson 170.84000396728516 +katie ellison 3.609999895095825 +katie ellison 80.97000122070312 +katie falkner 18.5 +katie garcia 24.729999542236328 +katie garcia 84.4000015258789 +katie hernandez 38.61999988555908 +katie ichabod 30.709999084472656 +katie ichabod 39.97999954223633 +katie ichabod 43.16999816894531 +katie king 39.34000015258789 +katie king 39.83000183105469 +katie king 97.80999755859375 +katie miller 31.399999618530273 +katie miller 74.77999877929688 +katie nixon 121.3700008392334 +katie ovid 50.65999984741211 +katie polk 11.680000305175781 +katie polk 40.2400016784668 +katie robinson 13.890000343322754 +katie van buren 17.739999771118164 +katie van buren 52.529998779296875 +katie white 1.309999942779541 +katie white 34.72999954223633 +katie xylophone 14.130000114440918 +katie young 31.010000228881836 +katie young 72.51000213623047 +katie young 97.56999969482422 +katie zipper 18.93000030517578 +katie zipper 58.75 +luke allen 15.180000305175781 +luke allen 50.959999084472656 +luke allen 66.61000061035156 +luke allen 89.55000305175781 +luke allen 99.38999938964844 +luke brown 51.790000915527344 +luke davidson 7.050000190734863 +luke davidson 28.950000762939453 +luke ellison 1.8700000047683716 +luke ellison 16.25 +luke ellison 87.83000183105469 +luke falkner 32.25 +luke falkner 39.60000038146973 +luke garcia 13.350000381469727 +luke garcia 30.3700008392334 +luke ichabod 8.449999809265137 +luke ichabod 97.87000274658203 +luke johnson 11.149999618530273 +luke johnson 14.4399995803833 +luke johnson 31.670000076293945 +luke laertes 0.5199999809265137 +luke laertes 4.800000190734863 +luke laertes 11.819999694824219 +luke laertes 16.690000534057617 +luke laertes 45.9900016784668 +luke miller 97.6500015258789 +luke ovid 38.04999923706055 +luke ovid 159.68000030517578 +luke polk 46.880001068115234 +luke polk 95.27999877929688 +luke quirinius 40.41999816894531 +luke robinson 55.099998474121094 +luke robinson 65.69999694824219 +luke thompson 94.37999725341797 +luke underhill 59.68000030517578 +luke underhill 95.52999877929688 +luke underhill 96.94000244140625 +luke van buren 148.62999725341797 +luke white 67.12000274658203 +luke xylophone 48.279998779296875 +luke zipper 24.829999923706055 +mike allen 48.53999900817871 +mike brown 48.22999954223633 +mike carson 20.06999969482422 +mike carson 47.56999969482422 +mike carson 81.66000366210938 +mike davidson 27.309999465942383 +mike davidson 54.83000183105469 +mike ellison 28.559999465942383 +mike ellison 37.099998474121094 +mike ellison 62.13999938964844 +mike ellison 79.37999725341797 +mike ellison 85.73999786376953 +mike falkner 16.479999542236328 +mike garcia 70.8499984741211 +mike garcia 75.83000183105469 +mike garcia 79.20999908447266 +mike hernandez 37.900001525878906 +mike hernandez 59.45000076293945 +mike ichabod 64.7699966430664 +mike king 38.790000915527344 +mike king 62.7400016784668 +mike king 78.26000213623047 +mike king 84.2300033569336 +mike king 85.0999984741211 +mike king 94.68000030517578 +mike miller 3.9600000381469727 +mike nixon 60.119998931884766 +mike nixon 92.95999908447266 +mike polk 12.449999809265137 +mike polk 27.06999969482422 +mike polk 99.68000030517578 +mike quirinius 89.37999725341797 +mike steinbeck 5.849999904632568 +mike steinbeck 85.13999938964844 +mike steinbeck 93.07000207901001 +mike steinbeck 97.45999908447266 +mike van buren 80.83999633789062 +mike van buren 114.56999969482422 +mike white 9.569999694824219 +mike white 28.889999389648438 +mike white 32.0099983215332 +mike white 91.87999725341797 +mike young 7.820000171661377 +mike young 74.58999633789062 +mike young 83.54000091552734 +mike zipper 26.729999542236328 +mike zipper 83.91999816894531 +mike zipper 97.38999938964844 +nick allen 21.830000400543213 +nick allen 35.08000183105469 +nick brown 42.5099983215332 +nick davidson 49.439998626708984 +nick ellison 9.680000305175781 +nick ellison 89.01000213623047 +nick falkner 10.130000114440918 +nick falkner 88.47000122070312 +nick garcia 13.9399995803833 +nick garcia 26.389999389648438 +nick garcia 46.43000030517578 +nick ichabod 23.450000762939453 +nick ichabod 47.59000015258789 +nick ichabod 74.41999816894531 +nick johnson 3.9700000286102295 +nick johnson 94.08000183105469 +nick laertes 96.25 +nick miller 82.97000122070312 +nick nixon 96.37999725341797 +nick ovid 87.98999786376953 +nick polk 59.27000141143799 +nick quirinius 67.44999694824219 +nick quirinius 81.16999816894531 +nick robinson 57.66999816894531 +nick robinson 60.709999084472656 +nick steinbeck 97.83000183105469 +nick thompson 11.90999984741211 +nick underhill 20.809999465942383 +nick van buren 51.290000915527344 +nick xylophone 103.45999908447266 +nick young 0.27000001072883606 +nick young 24.799999237060547 +nick zipper 56.619998931884766 +nick zipper 119.0199966430664 +oscar allen 18.6299991607666 +oscar brown 13.100000381469727 +oscar carson 6.869999885559082 +oscar carson 55.20000076293945 +oscar carson 78.9800033569336 +oscar carson 87.4800033569336 +oscar carson 98.51000213623047 +oscar davidson 64.45999908447266 +oscar ellison 57.88999938964844 +oscar ellison 107.7100019454956 +oscar falkner 98.4800033569336 +oscar garcia 67.4800033569336 +oscar hernandez 95.4800033569336 +oscar hernandez 125.92999649047852 +oscar ichabod 3.3299999237060547 +oscar ichabod 33.52000045776367 +oscar ichabod 71.80000305175781 +oscar ichabod 76.69000244140625 +oscar johnson 16.09000015258789 +oscar johnson 139.69000244140625 +oscar king 19.059999465942383 +oscar king 25.8799991607666 +oscar king 59.5 +oscar laertes 5.510000228881836 +oscar laertes 8.420000076293945 +oscar laertes 9.260000228881836 +oscar laertes 27.1200008392334 +oscar nixon 41.619998931884766 +oscar ovid 37.13999938964844 +oscar ovid 82.23999786376953 +oscar ovid 91.52999877929688 +oscar polk 30.610000610351562 +oscar polk 63.900001525878906 +oscar quirinius 41.45000076293945 +oscar quirinius 65.43000030517578 +oscar quirinius 113.35000228881836 +oscar quirinius 139.10000610351562 +oscar robinson 11.34000015258789 +oscar robinson 42.849998474121094 +oscar robinson 74.52999877929688 +oscar robinson 131.31999969482422 +oscar steinbeck 29.59000015258789 +oscar thompson 31.90999984741211 +oscar thompson 41.34000015258789 +oscar thompson 60.529998779296875 +oscar thompson 70.88999938964844 +oscar underhill 87.4000015258789 +oscar van buren 2.180000066757202 +oscar van buren 61.880001068115234 +oscar van buren 91.77999877929688 +oscar white 19.0 +oscar white 28.450000762939453 +oscar white 51.849998474121094 +oscar white 59.83000183105469 +oscar xylophone 21.799999237060547 +oscar xylophone 57.119998931884766 +oscar xylophone 57.22999954223633 +oscar zipper 13.989999771118164 +oscar zipper 32.88999938964844 +oscar zipper 39.81999969482422 +priscilla brown 70.23999786376953 +priscilla brown 80.5199966430664 +priscilla brown 104.63999938964844 +priscilla carson 7.960000038146973 +priscilla carson 79.80999946594238 +priscilla carson 85.43000316619873 +priscilla ichabod 80.04000091552734 +priscilla ichabod 92.61000061035156 +priscilla johnson 61.939998626708984 +priscilla johnson 67.9800033569336 +priscilla johnson 68.32999992370605 +priscilla johnson 91.4800033569336 +priscilla johnson 92.48000144958496 +priscilla king 43.91999816894531 +priscilla nixon 95.80999755859375 +priscilla nixon 107.69000244140625 +priscilla ovid 52.72999954223633 +priscilla ovid 125.73999643325806 +priscilla polk 15.149999618530273 +priscilla quirinius 9.710000038146973 +priscilla thompson 9.800000190734863 +priscilla underhill 35.720001220703125 +priscilla underhill 68.22000122070312 +priscilla van buren 68.88999938964844 +priscilla van buren 91.61000061035156 +priscilla van buren 170.5500030517578 +priscilla white 78.27999877929688 +priscilla xylophone 0.15000000596046448 +priscilla xylophone 21.489999771118164 +priscilla xylophone 59.61000061035156 +priscilla young 0.4300000071525574 +priscilla young 4.320000171661377 +priscilla zipper 18.6299991607666 +priscilla zipper 25.670000076293945 +quinn allen 54.72999954223633 +quinn allen 83.33000183105469 +quinn brown 24.280000686645508 +quinn brown 52.439998626708984 +quinn brown 80.58000183105469 +quinn davidson 61.57999849319458 +quinn davidson 67.18000030517578 +quinn davidson 83.4000015258789 +quinn davidson 95.11000061035156 +quinn ellison 19.280000686645508 +quinn ellison 30.649999618530273 +quinn garcia 40.97999954223633 +quinn garcia 59.9900016784668 +quinn garcia 74.0199966430664 +quinn garcia 172.8499984741211 +quinn ichabod 36.790000915527344 +quinn king 74.62000274658203 +quinn king 86.2300033569336 +quinn laertes 4.710000038146973 +quinn laertes 41.290000915527344 +quinn laertes 76.5199966430664 +quinn nixon 86.64000129699707 +quinn ovid 52.500000953674316 +quinn quirinius 32.18000030517578 +quinn robinson 38.64999866485596 +quinn steinbeck 8.449999809265137 +quinn steinbeck 66.51000213623047 +quinn thompson 74.9399995803833 +quinn thompson 76.27999877929688 +quinn underhill 17.15999984741211 +quinn underhill 79.4800033569336 +quinn underhill 140.92000198364258 +quinn van buren 82.5199966430664 +quinn young 45.060001373291016 +quinn zipper 22.25 +quinn zipper 58.0 +rachel allen 15.8100004196167 +rachel allen 74.44999694824219 +rachel brown 2.9600000381469727 +rachel brown 30.809999465942383 +rachel brown 33.36000061035156 +rachel brown 34.40999984741211 +rachel brown 52.16999816894531 +rachel carson 37.599998474121094 +rachel carson 98.95999908447266 +rachel davidson 4.920000076293945 +rachel ellison 10.600000381469727 +rachel falkner 46.150001525878906 +rachel falkner 80.91999816894531 +rachel falkner 88.80000305175781 +rachel falkner 99.23999786376953 +rachel johnson 62.22999954223633 +rachel king 36.220001220703125 +rachel king 59.45000076293945 +rachel laertes 44.220001220703125 +rachel laertes 45.45000076293945 +rachel ovid 0.6000000238418579 +rachel ovid 1.0800000429153442 +rachel polk 89.27999877929688 +rachel quirinius 12.4399995803833 +rachel robinson 4.570000171661377 +rachel robinson 30.360000610351562 +rachel robinson 64.94999694824219 +rachel thompson 0.5600000023841858 +rachel thompson 4.170000076293945 +rachel thompson 58.52000045776367 +rachel underhill 48.45000076293945 +rachel white 43.709999084472656 +rachel white 94.72000122070312 +rachel young 43.130001068115234 +rachel zipper 7.059999942779541 +rachel zipper 72.18000030517578 +sarah carson 1.909999966621399 +sarah carson 14.210000038146973 +sarah carson 78.88999938964844 +sarah ellison 16.989999771118164 +sarah falkner 90.27999877929688 +sarah falkner 99.36000061035156 +sarah garcia 41.290000915527344 +sarah garcia 58.010000228881836 +sarah garcia 153.8800048828125 +sarah ichabod 81.31999969482422 +sarah ichabod 97.26000213623047 +sarah johnson 16.239999771118164 +sarah johnson 45.099998474121094 +sarah johnson 73.87999725341797 +sarah johnson 77.66000366210938 +sarah king 41.869998931884766 +sarah king 48.25 +sarah miller 41.709999084472656 +sarah ovid 60.02000045776367 +sarah robinson 33.83000183105469 +sarah robinson 66.88999938964844 +sarah steinbeck 40.16999816894531 +sarah white 37.849998474121094 +sarah white 89.80999755859375 +sarah xylophone 68.31999969482422 +sarah young 45.560001373291016 +sarah zipper 83.08000183105469 +tom brown 8.609999656677246 +tom brown 12.319999694824219 +tom carson 5.440000057220459 +tom carson 16.079999923706055 +tom carson 18.889999389648438 +tom davidson 170.0 +tom ellison 76.73999786376953 +tom ellison 98.2300033569336 +tom ellison 155.99999618530273 +tom falkner 60.130001068115234 +tom falkner 88.22000122070312 +tom hernandez 41.36000061035156 +tom hernandez 81.63999938964844 +tom ichabod 103.29000282287598 +tom johnson 14.920000076293945 +tom johnson 43.56999969482422 +tom king 15.75 +tom laertes 43.310001373291016 +tom laertes 64.6500015258789 +tom miller 21.229999542236328 +tom miller 68.25 +tom miller 139.04000091552734 +tom nixon 153.83999633789062 +tom ovid 8.670000076293945 +tom polk 38.29999923706055 +tom polk 54.43000030517578 +tom quirinius 10.1899995803833 +tom quirinius 75.31999969482422 +tom robinson 90.69000244140625 +tom robinson 98.72000122070312 +tom robinson 99.1500015258789 +tom robinson 123.5199966430664 +tom steinbeck 26.489999771118164 +tom van buren 3.2799999713897705 +tom van buren 40.779998779296875 +tom van buren 63.5099983215332 +tom white 40.040000915527344 +tom young 22.850000381469727 +tom young 84.30999755859375 +tom zipper 122.78000259399414 +ulysses brown 72.79000091552734 +ulysses carson 77.41999816894531 +ulysses carson 79.54000091552734 +ulysses carson 146.7100067138672 +ulysses carson 220.18000030517578 +ulysses davidson 55.16999816894531 +ulysses ellison 96.7300033569336 +ulysses garcia 89.80000305175781 +ulysses hernandez 35.16999816894531 +ulysses hernandez 54.470001220703125 +ulysses hernandez 68.25 +ulysses ichabod 19.1299991607666 +ulysses ichabod 98.56999969482422 +ulysses johnson 102.5999984741211 +ulysses king 74.19000244140625 +ulysses laertes 1.9199999570846558 +ulysses laertes 24.860000610351562 +ulysses laertes 50.1899995803833 +ulysses miller 2.9600000381469727 +ulysses miller 76.27999877929688 +ulysses nixon 80.95999908447266 +ulysses ovid 29.360000610351562 +ulysses polk 8.710000038146973 +ulysses polk 60.060001373291016 +ulysses polk 65.0199966430664 +ulysses polk 97.10000038146973 +ulysses quirinius 112.56999969482422 +ulysses robinson 104.85999870300293 +ulysses steinbeck 32.40999984741211 +ulysses steinbeck 74.0 +ulysses thompson 198.83000564575195 +ulysses underhill 14.119999885559082 +ulysses underhill 22.360000610351562 +ulysses underhill 35.88999938964844 +ulysses underhill 57.369998931884766 +ulysses underhill 81.58000183105469 +ulysses underhill 88.4800033569336 +ulysses underhill 99.66999816894531 +ulysses van buren 95.52999877929688 +ulysses white 59.54999923706055 +ulysses white 170.0800018310547 +ulysses xylophone 39.689998626708984 +ulysses xylophone 54.099998474121094 +ulysses xylophone 57.3100004196167 +ulysses young 14.930000305175781 +ulysses young 32.52000045776367 +ulysses young 114.55999946594238 +victor allen 44.27000045776367 +victor allen 89.5 +victor brown 59.34000015258789 +victor brown 77.88999938964844 +victor brown 90.37999725341797 +victor brown 91.97000122070312 +victor davidson 60.2599983215332 +victor davidson 66.5999984741211 +victor davidson 98.54999923706055 +victor ellison 17.8700008392334 +victor ellison 68.8499984741211 +victor hernandez 19.030000686645508 +victor hernandez 59.619998931884766 +victor hernandez 69.87999725341797 +victor hernandez 71.3499984741211 +victor hernandez 74.5199966430664 +victor johnson 18.200000762939453 +victor johnson 42.89000141620636 +victor johnson 72.55999755859375 +victor king 47.880001068115234 +victor king 66.66999816894531 +victor laertes 62.91999816894531 +victor laertes 67.58999633789062 +victor miller 22.1200008392334 +victor nixon 34.029998779296875 +victor nixon 68.5 +victor ovid 125.84000015258789 +victor polk 17.210000038146973 +victor quirinius 50.70000076293945 +victor quirinius 134.4000015258789 +victor robinson 51.560001373291016 +victor robinson 58.66999816894531 +victor steinbeck 12.460000038146973 +victor steinbeck 46.09000015258789 +victor steinbeck 52.720001220703125 +victor thompson 58.65999984741211 +victor van buren 34.970001220703125 +victor van buren 41.68000030517578 +victor white 5.670000076293945 +victor white 135.02999687194824 +victor xylophone 10.09000015258789 +victor xylophone 11.220000267028809 +victor xylophone 28.5 +victor xylophone 62.38999938964844 +victor xylophone 76.0999984741211 +victor young 88.55000305175781 +victor zipper 26.289999961853027 +wendy allen 56.06999969482422 +wendy allen 93.96999740600586 +wendy allen 220.7900003194809 +wendy brown 27.8700008392334 +wendy brown 50.2599983215332 +wendy ellison 94.66000366210938 +wendy ellison 124.93999481201172 +wendy falkner 22.010000228881836 +wendy falkner 97.68000030517578 +wendy falkner 141.36000061035156 +wendy garcia 30.6200008392334 +wendy garcia 57.25 +wendy garcia 82.1500015258789 +wendy garcia 133.3400001525879 +wendy hernandez 48.11000061035156 +wendy ichabod 13.149999618530273 +wendy king 45.189998626708984 +wendy king 63.33000183105469 +wendy king 183.75 +wendy laertes 46.619998931884766 +wendy laertes 70.37999725341797 +wendy laertes 79.98999786376953 +wendy miller 1.2699999809265137 +wendy miller 12.420000076293945 +wendy nixon 45.91999816894531 +wendy nixon 60.2599983215332 +wendy ovid 86.62999725341797 +wendy ovid 95.33000183105469 +wendy polk 32.369998931884766 +wendy polk 42.04000073671341 +wendy quirinius 12.15999984741211 +wendy quirinius 14.300000190734863 +wendy robinson 26.469999313354492 +wendy robinson 71.06999969482422 +wendy robinson 117.02000045776367 +wendy steinbeck 120.67000007629395 +wendy thompson 67.34000015258789 +wendy thompson 85.76000213623047 +wendy underhill 68.04000091552734 +wendy underhill 79.19000244140625 +wendy underhill 89.77999877929688 +wendy van buren 57.459999084472656 +wendy van buren 92.81999969482422 +wendy white 73.68000030517578 +wendy xylophone 76.69999694824219 +wendy xylophone 90.60000038146973 +wendy young 8.449999809265137 +wendy young 33.7599983215332 +xavier allen 45.68000030517578 +xavier allen 83.93000030517578 +xavier allen 98.22000122070312 +xavier brown 7.789999961853027 +xavier brown 90.7300033569336 +xavier brown 96.2300033569336 +xavier carson 20.790000915527344 +xavier carson 94.68000030517578 +xavier davidson 15.920000076293945 +xavier davidson 82.41000366210938 +xavier davidson 106.5199966430664 +xavier ellison 12.850000381469727 +xavier ellison 77.97000122070312 +xavier garcia 70.04000091552734 +xavier hernandez 6.670000076293945 +xavier hernandez 38.56999969482422 +xavier hernandez 67.26000213623047 +xavier ichabod 4.71999979019165 +xavier ichabod 71.19000244140625 +xavier johnson 27.299999237060547 +xavier johnson 203.65999794006348 +xavier king 8.569999694824219 +xavier king 87.22000122070312 +xavier laertes 15.899999618530273 +xavier ovid 112.91000366210938 +xavier polk 13.869999885559082 +xavier polk 61.209999084472656 +xavier polk 72.62000274658203 +xavier polk 76.93000030517578 +xavier quirinius 62.52000045776367 +xavier quirinius 83.01000022888184 +xavier quirinius 89.55000305175781 +xavier quirinius 97.14999961853027 +xavier thompson 9.930000305175781 +xavier underhill 47.27000045776367 +xavier white 59.20000171661377 +xavier white 75.29000091552734 +xavier xylophone 79.41999816894531 +xavier zipper 8.449999809265137 +yuri allen 52.849998474121094 +yuri allen 94.98999977111816 +yuri brown 75.19000244140625 +yuri brown 84.02999877929688 +yuri carson 6.289999961853027 +yuri carson 91.16000366210938 +yuri ellison 1.1200000047683716 +yuri ellison 98.82999801635742 +yuri falkner 39.6299991607666 +yuri falkner 86.0 +yuri garcia 27.65999984741211 +yuri hernandez 2.069999933242798 +yuri johnson 0.12999999523162842 +yuri johnson 39.900001525878906 +yuri johnson 48.220001220703125 +yuri king 69.59000015258789 +yuri laertes 37.59000015258789 +yuri laertes 61.95000076293945 +yuri nixon 2.200000047683716 +yuri nixon 82.81000328063965 +yuri polk 26.760000228881836 +yuri polk 28.790000915527344 +yuri polk 105.11999702453613 +yuri quirinius 10.260000228881836 +yuri quirinius 54.310001373291016 +yuri quirinius 57.93000030517578 +yuri steinbeck 17.790000915527344 +yuri steinbeck 75.87999725341797 +yuri thompson 14.920000076293945 +yuri underhill 23.770000457763672 +yuri underhill 83.87000274658203 +yuri white 34.58000183105469 +yuri xylophone 20.3799991607666 +zach allen 65.43000030517578 +zach brown 48.0099983215332 +zach brown 49.119998931884766 +zach brown 57.08000183105469 +zach brown 67.37999725341797 +zach brown 100.46000289916992 +zach carson 95.86999893188477 +zach ellison 6.840000152587891 +zach falkner 9.130000114440918 +zach falkner 91.41999816894531 +zach garcia 32.20000076293945 +zach garcia 84.37999725341797 +zach garcia 106.86999893188477 +zach garcia 167.62000274658203 +zach ichabod 64.25 +zach ichabod 106.69000244140625 +zach king 46.18000030517578 +zach king 70.51000213623047 +zach king 86.93000030517578 +zach miller 2.5999999046325684 +zach miller 21.280000686645508 +zach miller 53.27000045776367 +zach ovid 0.10000000149011612 +zach ovid 23.06999969482422 +zach ovid 92.55000305175781 +zach ovid 94.33999633789062 +zach quirinius 39.209999084472656 +zach robinson 122.81000137329102 +zach steinbeck 85.48999786376953 +zach steinbeck 90.05000305175781 +zach thompson 71.5 +zach thompson 91.63999938964844 +zach underhill 86.22000122070312 +zach white 70.52999877929688 +zach xylophone 43.84999942779541 +zach xylophone 71.01000213623047 +zach young 71.31999969482422 +zach zipper 52.60000133514404 +zach zipper 85.87000274658203 +zach zipper 94.43000030517578 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-2-5f0eab306ea3c22b11ace9b542a7ee56 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-2-5f0eab306ea3c22b11ace9b542a7ee56 new file mode 100644 index 0000000000000000000000000000000000000000..e55bede9242e58c3170ce3c7b1a370ca8deb1aa7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-2-5f0eab306ea3c22b11ace9b542a7ee56 @@ -0,0 +1,1049 @@ + 257.04000091552734 + 261.16000175476074 + 284.2699966430664 +alice allen 73.62999725341797 +alice allen 195.0099983215332 +alice allen 196.729998588562 +alice brown 245.52000427246094 +alice carson 424.63000106811523 +alice davidson 319.00999450683594 +alice falkner 90.25 +alice garcia 174.36999893188477 +alice hernandez 185.6699981689453 +alice hernandez 380.1699981689453 +alice johnson 367.2900071144104 +alice king 58.78999900817871 +alice king 294.1199951171875 +alice king 371.23999404907227 +alice laertes 68.94999694824219 +alice laertes 258.3999938964844 +alice miller 154.19000244140625 +alice nixon 209.91000366210938 +alice nixon 246.36000442504883 +alice nixon 260.19000244140625 +alice ovid 49.8199987411499 +alice polk 148.63999938964844 +alice quirinius 239.81999588012695 +alice quirinius 301.4100036621094 +alice robinson 140.47999572753906 +alice robinson 266.4900016784668 +alice steinbeck 169.76000213623047 +alice steinbeck 186.70999908447266 +alice steinbeck 446.8099937438965 +alice underhill 98.18000030517578 +alice van buren 112.42000198364258 +alice xylophone 78.20999908447266 +alice xylophone 91.22000122070312 +alice xylophone 413.1199951171875 +alice zipper 89.93000030517578 +alice zipper 279.54000091552734 +alice zipper 293.25000381469727 +bob brown 188.89999389648438 +bob brown 228.80999946594238 +bob brown 247.37999725341797 +bob carson 207.67000198364258 +bob davidson 53.93000030517578 +bob davidson 113.83999919891357 +bob davidson 259.0899963378906 +bob ellison 65.0199966430664 +bob ellison 80.30000305175781 +bob ellison 243.86000061035156 +bob ellison 245.02999877929688 +bob falkner 208.82000160217285 +bob garcia 33.410000801086426 +bob garcia 87.56999969482422 +bob garcia 120.17999649047852 +bob garcia 148.65999841690063 +bob garcia 178.87000274658203 +bob hernandez 337.23999977111816 +bob ichabod 82.55999755859375 +bob king 114.11000156402588 +bob king 134.81999588012695 +bob king 152.7699956893921 +bob laertes 42.89999961853027 +bob laertes 393.99999433755875 +bob miller 146.1500015258789 +bob ovid 62.849998474121094 +bob ovid 88.77999877929688 +bob ovid 97.08999633789062 +bob ovid 102.93000030517578 +bob polk 261.4599976539612 +bob quirinius 298.7199897766113 +bob steinbeck 103.01999950408936 +bob van buren 174.89999771118164 +bob white 194.25 +bob white 347.7799949645996 +bob xylophone 19.690000534057617 +bob xylophone 191.52999687194824 +bob young 78.17999649047852 +bob zipper 132.86000061035156 +bob zipper 139.6900042295456 +bob zipper 295.59000039100647 +calvin allen 255.68000411987305 +calvin brown 85.9000015258789 +calvin brown 238.02000427246094 +calvin brown 275.8699951171875 +calvin carson 80.2300033569336 +calvin davidson 31.770000457763672 +calvin davidson 181.76000213623047 +calvin ellison 188.0300006866455 +calvin falkner 93.61000061035156 +calvin falkner 94.30999755859375 +calvin falkner 125.91999816894531 +calvin falkner 137.1699981689453 +calvin falkner 140.99999594688416 +calvin falkner 168.81999969482422 +calvin garcia 307.439998626709 +calvin hernandez 303.4599952697754 +calvin johnson 152.8300018310547 +calvin laertes 150.69999885559082 +calvin laertes 216.81000137329102 +calvin nixon 131.57999801635742 +calvin nixon 143.3699951171875 +calvin nixon 196.34000301361084 +calvin ovid 69.95999908447266 +calvin ovid 176.13999938964844 +calvin ovid 176.3800048828125 +calvin ovid 248.65999603271484 +calvin polk 147.04000091552734 +calvin quirinius 226.66999435424805 +calvin quirinius 266.7100009918213 +calvin robinson 289.7900047302246 +calvin steinbeck 92.05000305175781 +calvin steinbeck 118.15000057220459 +calvin steinbeck 333.6000003814697 +calvin thompson 93.7300033569336 +calvin thompson 249.56000137329102 +calvin underhill 208.3400001525879 +calvin van buren 136.51000213623047 +calvin van buren 347.0999946594238 +calvin white 90.69000244140625 +calvin white 112.15999984741211 +calvin xylophone 25.420000076293945 +calvin xylophone 237.71999740600586 +calvin xylophone 315.2099952697754 +calvin young 222.96000289916992 +calvin young 243.3199977874756 +calvin zipper 95.37999725341797 +calvin zipper 531.3600015640259 +david allen 202.43000030517578 +david allen 302.4399948120117 +david brown 93.63999938964844 +david brown 258.05999851226807 +david davidson 74.1500015258789 +david davidson 95.80999755859375 +david davidson 106.50000202655792 +david davidson 149.94000244140625 +david ellison 85.2300033569336 +david ellison 94.1500015258789 +david ellison 208.3900032043457 +david hernandez 99.91000366210938 +david ichabod 82.55000305175781 +david ichabod 320.47999143600464 +david laertes 250.1699981689453 +david nixon 174.58999633789062 +david ovid 198.21000289916992 +david ovid 230.47999954223633 +david quirinius 29.239999771118164 +david quirinius 79.97000122070312 +david quirinius 180.92999649047852 +david robinson 147.65999603271484 +david robinson 168.7100009918213 +david thompson 41.88999938964844 +david underhill 97.55999755859375 +david underhill 277.5999984741211 +david underhill 369.4600009918213 +david van buren 83.56999969482422 +david van buren 289.189998626709 +david white 124.6099967956543 +david xylophone 135.70000076293945 +david xylophone 237.06000137329102 +david xylophone 338.20999908447266 +david young 172.49000549316406 +david young 184.9800033569336 +ethan allen 240.42000198364258 +ethan brown 61.86000061035156 +ethan brown 73.18000030517578 +ethan brown 105.29000043869019 +ethan brown 177.8300018310547 +ethan brown 185.98999691009521 +ethan brown 284.729998588562 +ethan carson 265.22999572753906 +ethan ellison 166.5 +ethan ellison 244.99000671505928 +ethan falkner 59.43000030517578 +ethan falkner 196.17000198364258 +ethan garcia 271.5999946594238 +ethan hernandez 264.50999450683594 +ethan johnson 90.05000305175781 +ethan king 36.49000024795532 +ethan laertes 95.06999969482422 +ethan laertes 96.29000091552734 +ethan laertes 189.66000270843506 +ethan laertes 192.79999923706055 +ethan laertes 249.04000091552734 +ethan laertes 249.76000213623047 +ethan laertes 369.9599952697754 +ethan miller 314.5599994659424 +ethan nixon 493.03000259399414 +ethan ovid 57.290000915527344 +ethan polk 2.3499999046325684 +ethan polk 59.869998931884766 +ethan polk 219.6599998474121 +ethan polk 263.8600025177002 +ethan quirinius 97.23999786376953 +ethan quirinius 111.70999908447266 +ethan quirinius 317.69000363349915 +ethan robinson 78.62000274658203 +ethan robinson 149.5800018310547 +ethan underhill 231.25000381469727 +ethan van buren 152.60000228881836 +ethan white 155.81999969482422 +ethan white 235.55999755859375 +ethan xylophone 414.61000061035156 +ethan zipper 97.51000213623047 +ethan zipper 297.20999908447266 +fred davidson 78.30999755859375 +fred davidson 105.8499984741211 +fred davidson 220.56000137329102 +fred ellison 56.489999771118164 +fred ellison 96.77999877929688 +fred ellison 199.52000045776367 +fred falkner 66.77999973297119 +fred falkner 85.0 +fred falkner 169.91000366210938 +fred hernandez 117.85000228881836 +fred ichabod 81.31999969482422 +fred ichabod 202.45000457763672 +fred johnson 96.08999633789062 +fred king 140.24999618530273 +fred king 343.82000064849854 +fred laertes 57.63999938964844 +fred miller 176.18000030517578 +fred nixon 28.690000534057617 +fred nixon 187.40999603271484 +fred nixon 246.77000045776367 +fred nixon 338.34999084472656 +fred polk 90.12000274658203 +fred polk 323.1899948120117 +fred polk 357.19000244140625 +fred polk 496.16999435424805 +fred quirinius 218.82999801635742 +fred quirinius 224.12000179290771 +fred robinson 89.02999877929688 +fred steinbeck 32.22999954223633 +fred steinbeck 91.05000305175781 +fred steinbeck 231.92000198364258 +fred underhill 183.31999969482422 +fred van buren 83.58000183105469 +fred van buren 318.38000106811523 +fred van buren 346.1400008201599 +fred van buren 391.9999942779541 +fred white 187.38000106811523 +fred young 97.70999908447266 +fred young 141.22999954223633 +fred zipper 163.89999771118164 +gabriella allen 274.8599967956543 +gabriella allen 283.5 +gabriella brown 163.89999961853027 +gabriella brown 465.0 +gabriella carson 147.78999710083008 +gabriella davidson 263.64000415802 +gabriella ellison 71.54000091552734 +gabriella ellison 188.55999755859375 +gabriella falkner 51.720001220703125 +gabriella falkner 87.61000061035156 +gabriella falkner 162.21999835968018 +gabriella garcia 43.0099983215332 +gabriella hernandez 190.5500030517578 +gabriella hernandez 267.4700012207031 +gabriella ichabod 71.12999725341797 +gabriella ichabod 90.3499984741211 +gabriella ichabod 148.6999969482422 +gabriella ichabod 175.70000076293945 +gabriella ichabod 285.72999572753906 +gabriella king 166.75000190734863 +gabriella king 177.6999969482422 +gabriella laertes 65.37999725341797 +gabriella miller 148.4800033569336 +gabriella ovid 92.4000015258789 +gabriella ovid 137.82999801635742 +gabriella polk 244.07000350952148 +gabriella polk 282.00000762939453 +gabriella steinbeck 272.2799987792969 +gabriella steinbeck 461.060001373291 +gabriella thompson 88.36000061035156 +gabriella thompson 94.25 +gabriella thompson 158.80999755859375 +gabriella van buren 146.0800018310547 +gabriella van buren 151.63999938964844 +gabriella white 138.72000122070312 +gabriella young 30.739999771118164 +gabriella young 146.62999725341797 +gabriella zipper 91.62999725341797 +gabriella zipper 357.5099983215332 +holly allen 44.56999969482422 +holly brown 173.64999389648438 +holly brown 174.2100067138672 +holly falkner 166.22999572753906 +holly hernandez 180.0800018310547 +holly hernandez 248.71000480651855 +holly hernandez 336.87000274658203 +holly hernandez 523.2800025939941 +holly ichabod 179.37000274658203 +holly ichabod 180.18000030517578 +holly ichabod 184.66000366210938 +holly johnson 64.36000061035156 +holly johnson 145.61000061035156 +holly johnson 157.12999725341797 +holly king 275.2299995422363 +holly king 288.52000427246094 +holly laertes 246.21000289916992 +holly miller 290.21999740600586 +holly nixon 177.39999389648438 +holly nixon 228.58999633789062 +holly polk 98.30999755859375 +holly polk 307.0799951553345 +holly robinson 219.27999877929688 +holly thompson 75.41999816894531 +holly thompson 86.69000244140625 +holly thompson 523.360002592206 +holly underhill 96.68000030517578 +holly underhill 163.54000091552734 +holly underhill 187.47000122070312 +holly underhill 328.0099983215332 +holly van buren 161.7699966430664 +holly white 122.98999786376953 +holly white 335.93999576568604 +holly xylophone 191.34000396728516 +holly young 60.220001220703125 +holly young 297.20999908447266 +holly zipper 99.12999725341797 +holly zipper 99.29000091552734 +irene allen 234.6400032043457 +irene brown 4.789999961853027 +irene brown 176.4499969482422 +irene brown 338.2099952697754 +irene carson 292.0 +irene ellison 201.06000137329102 +irene ellison 230.79000091552734 +irene falkner 99.91999816894531 +irene falkner 210.11000061035156 +irene garcia 40.78999996185303 +irene garcia 86.93000030517578 +irene garcia 183.02000045776367 +irene ichabod 99.62000274658203 +irene ichabod 281.96999740600586 +irene johnson 243.59999752044678 +irene laertes 112.54000091552734 +irene laertes 227.45000076293945 +irene laertes 246.53000259399414 +irene miller 395.9100036621094 +irene nixon 29.780000686645508 +irene nixon 199.45999908447266 +irene nixon 261.46000480651855 +irene ovid 158.97000122070312 +irene ovid 339.94000244140625 +irene ovid 362.82000732421875 +irene polk 95.83999633789062 +irene polk 183.43000411987305 +irene polk 258.7100033760071 +irene polk 284.6300048828125 +irene polk 507.2400016784668 +irene quirinius 157.5800018310547 +irene quirinius 250.61000061035156 +irene quirinius 431.6499996185303 +irene robinson 191.72999572753906 +irene steinbeck 94.33000183105469 +irene thompson 256.0 +irene underhill 135.55999755859375 +irene underhill 327.0299892425537 +irene van buren 54.439998626708984 +irene van buren 193.71000289916992 +irene xylophone 168.5 +jessica brown 422.5299949645996 +jessica carson 103.66000366210938 +jessica carson 144.92000198364258 +jessica carson 259.1099967956543 +jessica davidson 95.33999633789062 +jessica davidson 99.20999908447266 +jessica davidson 137.17000198364258 +jessica davidson 227.79999923706055 +jessica ellison 207.35000228881836 +jessica ellison 237.4300022125244 +jessica falkner 99.6500015258789 +jessica garcia 174.70999908447266 +jessica garcia 185.62000179290771 +jessica ichabod 124.59000015258789 +jessica johnson 272.0500030517578 +jessica johnson 294.2899990081787 +jessica miller 77.83999633789062 +jessica nixon 77.0999984741211 +jessica nixon 90.06999969482422 +jessica ovid 71.68000030517578 +jessica ovid 309.44000244140625 +jessica polk 472.2099952697754 +jessica quirinius 35.619998931884766 +jessica quirinius 192.7000026702881 +jessica quirinius 208.6500015258789 +jessica quirinius 370.0599937438965 +jessica robinson 254.5300064086914 +jessica thompson 115.9000015258789 +jessica thompson 180.60000610351562 +jessica underhill 199.10999870300293 +jessica underhill 234.29000091552734 +jessica underhill 257.09000396728516 +jessica van buren 9.739999771118164 +jessica white 96.62000274658203 +jessica white 166.54000091552734 +jessica white 240.52999877929688 +jessica white 432.17999362945557 +jessica white 497.6400032043457 +jessica xylophone 385.4799995422363 +jessica young 47.410000801086426 +jessica young 240.6500015258789 +jessica zipper 323.4199962615967 +jessica zipper 344.8399953842163 +jessica zipper 480.06999588012695 +katie allen 312.9700012207031 +katie brown 573.4599933624268 +katie davidson 96.91000366210938 +katie ellison 163.52999877929688 +katie ellison 384.4699947834015 +katie falkner 125.57000160217285 +katie garcia 84.4000015258789 +katie garcia 160.28999710083008 +katie hernandez 257.9600028991699 +katie ichabod 187.63999557495117 +katie ichabod 274.97999572753906 +katie ichabod 362.9200019836426 +katie king 97.80999755859375 +katie king 169.56999969482422 +katie king 314.1999969482422 +katie miller 31.399999618530273 +katie miller 228.40999603271484 +katie nixon 23.190000534057617 +katie ovid 207.1200065612793 +katie polk 143.2599983215332 +katie polk 247.02000045776367 +katie robinson 83.84999942779541 +katie van buren 297.0300064086914 +katie van buren 464.54999351501465 +katie white 344.1700019836426 +katie white 465.8599934577942 +katie xylophone 175.89999675750732 +katie young 31.010000228881836 +katie young 72.51000213623047 +katie young 97.56999969482422 +katie zipper 101.9000015258789 +katie zipper 314.75 +luke allen 89.55000305175781 +luke allen 133.4800033569336 +luke allen 210.8800048828125 +luke allen 392.0300064086914 +luke allen 420.6299934387207 +luke brown 129.20999908447266 +luke davidson 28.950000762939453 +luke davidson 106.41000080108643 +luke ellison 42.09000027179718 +luke ellison 136.52000427246094 +luke ellison 187.51000213623047 +luke falkner 172.8799991607666 +luke falkner 216.0199966430664 +luke garcia 50.94000053405762 +luke garcia 345.1200008392334 +luke ichabod 67.90000057220459 +luke ichabod 97.87000274658203 +luke johnson 59.00999927520752 +luke johnson 105.32000160217285 +luke johnson 187.2899990081787 +luke laertes 105.42000198364258 +luke laertes 147.14999723434448 +luke laertes 158.86000061035156 +luke laertes 167.01999855041504 +luke laertes 281.19999504089355 +luke miller 97.6500015258789 +luke ovid 186.53000259399414 +luke ovid 340.1300048828125 +luke polk 95.27999877929688 +luke polk 277.6700019836426 +luke quirinius 115.83999633789062 +luke robinson 137.33999633789062 +luke robinson 145.23999786376953 +luke thompson 94.37999725341797 +luke underhill 96.94000244140625 +luke underhill 194.73999786376953 +luke underhill 372.6899948120117 +luke van buren 193.93999862670898 +luke white 67.12000274658203 +luke xylophone 102.37999725341797 +luke zipper 223.54000282287598 +mike allen 79.60999870300293 +mike brown 202.81999588012695 +mike carson 81.66000366210938 +mike carson 105.02999877929688 +mike carson 405.4499931335449 +mike davidson 137.74999809265137 +mike davidson 393.17999267578125 +mike ellison 79.37999725341797 +mike ellison 85.73999786376953 +mike ellison 127.15999603271484 +mike ellison 228.07999992370605 +mike ellison 263.8899955749512 +mike falkner 254.50000381469727 +mike garcia 70.8499984741211 +mike garcia 173.63999938964844 +mike garcia 177.5199966430664 +mike hernandez 59.45000076293945 +mike hernandez 327.6900062561035 +mike ichabod 64.7699966430664 +mike king 78.26000213623047 +mike king 84.2300033569336 +mike king 94.68000030517578 +mike king 133.5900001525879 +mike king 134.87999725341797 +mike king 173.45999908447266 +mike miller 57.890000343322754 +mike nixon 92.95999908447266 +mike nixon 203.68999862670898 +mike polk 32.140000343322754 +mike polk 99.68000030517578 +mike polk 306.61000061035156 +mike quirinius 89.37999725341797 +mike steinbeck 85.13999938964844 +mike steinbeck 97.45999908447266 +mike steinbeck 153.86000204086304 +mike steinbeck 221.21999502182007 +mike van buren 80.83999633789062 +mike van buren 174.21000289916992 +mike white 91.87999725341797 +mike white 341.80999755859375 +mike white 341.86000061035156 +mike white 389.20000076293945 +mike young 74.58999633789062 +mike young 83.54000091552734 +mike young 112.19000101089478 +mike zipper 86.98999786376953 +mike zipper 97.38999938964844 +mike zipper 174.61000061035156 +nick allen 173.32000207901 +nick allen 257.7300033569336 +nick brown 192.45000076293945 +nick davidson 258.9799919128418 +nick ellison 183.34000396728516 +nick ellison 193.02000427246094 +nick falkner 10.130000114440918 +nick falkner 182.72000122070312 +nick garcia 142.65999603271484 +nick garcia 183.7699966430664 +nick garcia 277.8299951553345 +nick ichabod 110.43999862670898 +nick ichabod 112.54999923706055 +nick ichabod 241.68999481201172 +nick johnson 192.56000518798828 +nick johnson 325.9499976634979 +nick laertes 96.25 +nick miller 82.97000122070312 +nick nixon 96.37999725341797 +nick ovid 184.3699951171875 +nick polk 199.57000064849854 +nick quirinius 174.80999755859375 +nick quirinius 243.8300018310547 +nick robinson 129.65999603271484 +nick robinson 216.54999923706055 +nick steinbeck 97.83000183105469 +nick thompson 205.4900016784668 +nick underhill 166.42000007629395 +nick van buren 222.6500015258789 +nick xylophone 75.3499984741211 +nick young 332.23999786376953 +nick young 346.41000083088875 +nick zipper 222.9199981689453 +nick zipper 529.7199974060059 +oscar allen 246.42999839782715 +oscar brown 274.01000022888184 +oscar carson 78.9800033569336 +oscar carson 87.4800033569336 +oscar carson 98.51000213623047 +oscar carson 203.86000442504883 +oscar carson 321.82000064849854 +oscar davidson 361.6699981689453 +oscar ellison 146.44000244140625 +oscar ellison 234.32000160217285 +oscar falkner 98.4800033569336 +oscar garcia 231.04000091552734 +oscar hernandez 85.48999786376953 +oscar hernandez 95.4800033569336 +oscar ichabod 71.80000305175781 +oscar ichabod 123.78000068664551 +oscar ichabod 173.31000518798828 +oscar ichabod 251.22000122070312 +oscar johnson 146.27000427246094 +oscar johnson 260.1600036621094 +oscar king 124.2699966430664 +oscar king 249.5399990081787 +oscar king 284.8599910736084 +oscar laertes 15.640000343322754 +oscar laertes 254.8499984741211 +oscar laertes 261.41000175476074 +oscar laertes 261.8400020599365 +oscar nixon 41.619998931884766 +oscar ovid 82.23999786376953 +oscar ovid 187.76000213623047 +oscar ovid 260.6100044250488 +oscar polk 63.900001525878906 +oscar polk 252.71000289916992 +oscar quirinius 73.4800033569336 +oscar quirinius 165.3800048828125 +oscar quirinius 244.2699966430664 +oscar quirinius 248.75 +oscar robinson 93.31999969482422 +oscar robinson 163.55999755859375 +oscar robinson 191.8300018310547 +oscar robinson 315.1999912261963 +oscar steinbeck 376.6899948120117 +oscar thompson 131.1400032043457 +oscar thompson 148.01000213623047 +oscar thompson 325.42000579833984 +oscar thompson 545.7399940490723 +oscar underhill 87.4000015258789 +oscar van buren 61.880001068115234 +oscar van buren 188.8699951171875 +oscar van buren 209.53000235557556 +oscar white 129.73999786376953 +oscar white 148.9800033569336 +oscar white 275.1500015258789 +oscar white 303.8599910736084 +oscar xylophone 115.22999954223633 +oscar xylophone 319.75000381469727 +oscar xylophone 475.3300018310547 +oscar zipper 109.53999710083008 +oscar zipper 214.40999603271484 +oscar zipper 214.6500015258789 +priscilla brown 77.56999969482422 +priscilla brown 165.5199966430664 +priscilla brown 408.4499969482422 +priscilla carson 168.8300018310547 +priscilla carson 195.7900047302246 +priscilla carson 207.5300006866455 +priscilla ichabod 92.61000061035156 +priscilla ichabod 206.16000366210938 +priscilla johnson 89.1500015258789 +priscilla johnson 156.4600067138672 +priscilla johnson 158.88000106811523 +priscilla johnson 190.61000061035156 +priscilla johnson 211.01000022888184 +priscilla king 371.9299964904785 +priscilla nixon 95.80999755859375 +priscilla nixon 278.87999725341797 +priscilla ovid 96.27000284194946 +priscilla ovid 198.3400001525879 +priscilla polk 252.5800018310547 +priscilla quirinius 131.8499994277954 +priscilla thompson 230.36000156402588 +priscilla underhill 143.56999969482422 +priscilla underhill 354.37000274658203 +priscilla van buren 82.72000122070312 +priscilla van buren 145.61000061035156 +priscilla van buren 183.72000122070312 +priscilla white 78.27999877929688 +priscilla xylophone 21.489999771118164 +priscilla xylophone 159.26000213623047 +priscilla xylophone 406.1000007688999 +priscilla young 163.2900013923645 +priscilla young 260.59000366926193 +priscilla zipper 311.399995803833 +priscilla zipper 327.97999572753906 +quinn allen 257.94000244140625 +quinn allen 365.2299995422363 +quinn brown 80.58000183105469 +quinn brown 80.81999969482422 +quinn brown 198.71000289916992 +quinn davidson 83.4000015258789 +quinn davidson 95.11000061035156 +quinn davidson 154.79000091552734 +quinn davidson 227.13999938964844 +quinn ellison 237.17000007629395 +quinn ellison 361.14000129699707 +quinn garcia 92.33000183105469 +quinn garcia 148.63999938964844 +quinn garcia 226.78999710083008 +quinn garcia 246.70000076293945 +quinn ichabod 89.63999938964844 +quinn king 74.62000274658203 +quinn king 86.2300033569336 +quinn laertes 112.36000061035156 +quinn laertes 243.6900019645691 +quinn laertes 265.51000213623047 +quinn nixon 149.3300018310547 +quinn ovid 393.2099943161011 +quinn quirinius 266.8200035095215 +quinn robinson 247.6400032043457 +quinn steinbeck 144.81999969482422 +quinn steinbeck 213.65999507904053 +quinn thompson 156.51000213623047 +quinn thompson 274.1599931716919 +quinn underhill 248.3000030517578 +quinn underhill 252.61999130249023 +quinn underhill 321.9799976348877 +quinn van buren 82.5199966430664 +quinn young 90.97999954223633 +quinn zipper 58.0 +quinn zipper 249.38999938964844 +rachel allen 15.8100004196167 +rachel allen 151.80999755859375 +rachel brown 193.5800018310547 +rachel brown 312.0800018310547 +rachel brown 347.7999954223633 +rachel brown 423.98999214172363 +rachel brown 437.64999771118164 +rachel carson 98.95999908447266 +rachel carson 385.3799934387207 +rachel davidson 396.38999938964844 +rachel ellison 299.12000465393066 +rachel falkner 88.80000305175781 +rachel falkner 99.23999786376953 +rachel falkner 172.54999542236328 +rachel falkner 233.55999755859375 +rachel johnson 197.92999649047852 +rachel king 36.220001220703125 +rachel king 219.8400001525879 +rachel laertes 97.17000198364258 +rachel laertes 109.5999984741211 +rachel ovid 80.20999872684479 +rachel ovid 260.18999683856964 +rachel polk 89.27999877929688 +rachel quirinius 205.1400022506714 +rachel robinson 254.1300015449524 +rachel robinson 286.0400047302246 +rachel robinson 332.4199981689453 +rachel thompson 137.73000198602676 +rachel thompson 213.31000137329102 +rachel thompson 380.85999488830566 +rachel underhill 175.6099967956543 +rachel white 94.72000122070312 +rachel white 196.18000030517578 +rachel young 230.6400032043457 +rachel zipper 148.9000015258789 +rachel zipper 238.98000192642212 +sarah carson 175.62000274658203 +sarah carson 307.70000088214874 +sarah carson 386.8999948501587 +sarah ellison 161.80999946594238 +sarah falkner 99.36000061035156 +sarah falkner 281.62000274658203 +sarah garcia 73.6500015258789 +sarah garcia 153.73000144958496 +sarah garcia 312.8899955749512 +sarah ichabod 81.31999969482422 +sarah ichabod 97.26000213623047 +sarah johnson 140.37999725341797 +sarah johnson 177.57000732421875 +sarah johnson 248.4499969482422 +sarah johnson 309.1800022125244 +sarah king 216.75 +sarah king 268.5399932861328 +sarah miller 222.31000518798828 +sarah ovid 146.25000381469727 +sarah robinson 143.43000030517578 +sarah robinson 310.75 +sarah steinbeck 208.72000122070312 +sarah white 140.22999572753906 +sarah white 181.86000061035156 +sarah xylophone 68.31999969482422 +sarah young 185.80999755859375 +sarah zipper 168.22000122070312 +tom brown 181.1000051498413 +tom brown 404.3500061035156 +tom carson 142.60999822616577 +tom carson 299.57999992370605 +tom carson 592.3499927520752 +tom davidson 180.61000061035156 +tom ellison 98.2300033569336 +tom ellison 154.58999633789062 +tom ellison 173.02999877929688 +tom falkner 88.22000122070312 +tom falkner 139.11000442504883 +tom hernandez 81.63999938964844 +tom hernandez 263.67000579833984 +tom ichabod 214.0699977874756 +tom johnson 405.95000076293945 +tom johnson 438.9099922180176 +tom king 218.18000030517578 +tom laertes 244.37000274658203 +tom laertes 473.0999984741211 +tom miller 68.25 +tom miller 85.59000015258789 +tom miller 127.56999969482422 +tom nixon 85.02999877929688 +tom ovid 217.32000160217285 +tom polk 188.87000274658203 +tom polk 206.52000045776367 +tom quirinius 120.27000427246094 +tom quirinius 232.63000202178955 +tom robinson 90.69000244140625 +tom robinson 98.72000122070312 +tom robinson 99.1500015258789 +tom robinson 209.5399932861328 +tom steinbeck 277.7100009918213 +tom van buren 40.779998779296875 +tom van buren 217.70000076293945 +tom van buren 375.2099964618683 +tom white 223.4700050354004 +tom young 174.36000061035156 +tom young 304.8199977874756 +tom zipper 213.7900047302246 +ulysses brown 247.1500015258789 +ulysses carson 77.41999816894531 +ulysses carson 79.54000091552734 +ulysses carson 150.93000030517578 +ulysses carson 162.24000549316406 +ulysses davidson 414.7100009918213 +ulysses ellison 96.7300033569336 +ulysses garcia 89.80000305175781 +ulysses hernandez 106.29999542236328 +ulysses hernandez 134.44000244140625 +ulysses hernandez 160.22000122070312 +ulysses ichabod 98.56999969482422 +ulysses ichabod 309.34999656677246 +ulysses johnson 152.47000122070312 +ulysses king 244.7100067138672 +ulysses laertes 138.4400042295456 +ulysses laertes 173.55999755859375 +ulysses laertes 256.91999912261963 +ulysses miller 76.27999877929688 +ulysses miller 417.67000102996826 +ulysses nixon 174.56999969482422 +ulysses ovid 130.13000106811523 +ulysses polk 123.9399995803833 +ulysses polk 149.95999908447266 +ulysses polk 205.2400016784668 +ulysses polk 237.5699920654297 +ulysses quirinius 330.4700012207031 +ulysses robinson 79.48999786376953 +ulysses steinbeck 144.8300018310547 +ulysses steinbeck 155.66000366210938 +ulysses thompson 159.92000579833984 +ulysses underhill 81.58000183105469 +ulysses underhill 88.4800033569336 +ulysses underhill 99.66999816894531 +ulysses underhill 135.55999755859375 +ulysses underhill 189.1099977493286 +ulysses underhill 289.6800003051758 +ulysses underhill 385.60000228881836 +ulysses van buren 95.52999877929688 +ulysses white 188.8300018310547 +ulysses white 305.79000091552734 +ulysses xylophone 54.099998474121094 +ulysses xylophone 205.2099952697754 +ulysses xylophone 251.94000148773193 +ulysses young 100.77000045776367 +ulysses young 275.8300018310547 +ulysses young 522.1700019836426 +victor allen 220.1699981689453 +victor allen 222.10000228881836 +victor brown 77.88999938964844 +victor brown 90.37999725341797 +victor brown 91.97000122070312 +victor brown 455.25000381469727 +victor davidson 149.06000137329102 +victor davidson 291.48000717163086 +victor davidson 321.25 +victor ellison 314.37000274658203 +victor ellison 442.50000190734863 +victor hernandez 69.87999725341797 +victor hernandez 99.85000038146973 +victor hernandez 143.02000045776367 +victor hernandez 160.38999938964844 +victor hernandez 391.2999954223633 +victor johnson 145.18000030517578 +victor johnson 190.27000045776367 +victor johnson 308.1900006532669 +victor king 108.10000228881836 +victor king 310.5 +victor laertes 145.42999267578125 +victor laertes 214.72999572753906 +victor miller 173.76000022888184 +victor nixon 68.5 +victor nixon 269.5899963378906 +victor ovid 151.39999771118164 +victor polk 175.8799991607666 +victor quirinius 65.55000305175781 +victor quirinius 168.5500030517578 +victor robinson 177.9100022315979 +victor robinson 204.09999084472656 +victor steinbeck 52.720001220703125 +victor steinbeck 220.45999908447266 +victor steinbeck 309.4900064468384 +victor thompson 58.65999984741211 +victor van buren 206.77999877929688 +victor van buren 222.44000244140625 +victor white 156.36999893188477 +victor white 167.2699966430664 +victor xylophone 158.36999893188477 +victor xylophone 161.54000091552734 +victor xylophone 234.76000308990479 +victor xylophone 267.82999420166016 +victor xylophone 314.95000076293945 +victor young 88.55000305175781 +victor zipper 192.92999649047852 +wendy allen 56.06999969482422 +wendy allen 66.16000306606293 +wendy allen 267.3199996948242 +wendy brown 453.53000259399414 +wendy brown 525.5100040435791 +wendy ellison 193.95000457763672 +wendy ellison 260.9099998474121 +wendy falkner 77.36000061035156 +wendy falkner 97.68000030517578 +wendy falkner 128.30999565124512 +wendy garcia 4.409999847412109 +wendy garcia 76.72000122070312 +wendy garcia 189.42999839782715 +wendy garcia 265.5900001525879 +wendy hernandez 48.11000061035156 +wendy ichabod 104.3700008392334 +wendy king 156.89999771118164 +wendy king 183.31999969482422 +wendy king 403.27000427246094 +wendy laertes 79.98999786376953 +wendy laertes 165.0999984741211 +wendy laertes 365.0 +wendy miller 72.9500002861023 +wendy miller 313.8300037384033 +wendy nixon 45.91999816894531 +wendy nixon 60.2599983215332 +wendy ovid 95.33000183105469 +wendy ovid 180.36000061035156 +wendy polk 386.7400016784668 +wendy polk 443.3400018811226 +wendy quirinius 152.04999828338623 +wendy quirinius 240.23999977111816 +wendy robinson 71.06999969482422 +wendy robinson 249.35000610351562 +wendy robinson 391.4699993133545 +wendy steinbeck 92.11000061035156 +wendy thompson 136.35000228881836 +wendy thompson 183.1500015258789 +wendy underhill 318.6500015258789 +wendy underhill 320.75000190734863 +wendy underhill 328.2300033569336 +wendy van buren 57.459999084472656 +wendy van buren 92.81999969482422 +wendy white 171.36000061035156 +wendy xylophone 153.62999725341797 +wendy xylophone 223.94999885559082 +wendy young 40.22000026702881 +wendy young 513.8299942016602 +xavier allen 102.97000122070312 +xavier allen 168.3300018310547 +xavier allen 197.45999908447266 +xavier brown 55.20000076293945 +xavier brown 90.7300033569336 +xavier brown 96.2300033569336 +xavier carson 193.63999938964844 +xavier carson 265.1600036621094 +xavier davidson 63.349998474121094 +xavier davidson 264.27000427246094 +xavier davidson 288.1999988555908 +xavier ellison 138.42000198364258 +xavier ellison 262.6300048828125 +xavier garcia 148.66000366210938 +xavier hernandez 122.13999938964844 +xavier hernandez 164.97000122070312 +xavier hernandez 306.25 +xavier ichabod 211.84000635147095 +xavier ichabod 244.50000762939453 +xavier johnson 56.53999900817871 +xavier johnson 89.0999984741211 +xavier king 87.22000122070312 +xavier king 151.22999572753906 +xavier laertes 183.65999794006348 +xavier ovid 398.2100067138672 +xavier polk 72.62000274658203 +xavier polk 76.93000030517578 +xavier polk 261.5100030899048 +xavier polk 318.01000213623047 +xavier quirinius 22.1200008392334 +xavier quirinius 89.55000305175781 +xavier quirinius 246.2400016784668 +xavier quirinius 402.2100009918213 +xavier thompson 283.9400005340576 +xavier underhill 120.45000076293945 +xavier white 138.02999591827393 +xavier white 172.06999969482422 +xavier xylophone 79.41999816894531 +xavier zipper 373.67999935150146 +yuri allen 52.849998474121094 +yuri allen 417.3700008392334 +yuri brown 170.52000427246094 +yuri brown 180.70999908447266 +yuri carson 188.99000549316406 +yuri carson 537.6500015258789 +yuri ellison 86.91999816894531 +yuri ellison 376.32999646663666 +yuri falkner 152.99000358581543 +yuri falkner 181.06999969482422 +yuri garcia 274.6800003051758 +yuri hernandez 153.46999764442444 +yuri johnson 197.28000259399414 +yuri johnson 236.0800018310547 +yuri johnson 258.1899985074997 +yuri king 551.9899978637695 +yuri laertes 37.59000015258789 +yuri laertes 253.4799976348877 +yuri nixon 95.54999732971191 +yuri nixon 248.9700005054474 +yuri polk 82.33999633789062 +yuri polk 275.3200035095215 +yuri polk 305.6399974822998 +yuri quirinius 112.97000122070312 +yuri quirinius 148.27999877929688 +yuri quirinius 449.1699924468994 +yuri steinbeck 292.94000244140625 +yuri steinbeck 357.5 +yuri thompson 428.03999519348145 +yuri underhill 83.87000274658203 +yuri underhill 350.7999897003174 +yuri white 132.09000396728516 +yuri xylophone 107.07000160217285 +zach allen 65.43000030517578 +zach brown 135.6999969482422 +zach brown 247.04999542236328 +zach brown 256.8000030517578 +zach brown 362.38000106811523 +zach brown 418.75 +zach carson 291.7700004577637 +zach ellison 135.149995803833 +zach falkner 91.41999816894531 +zach falkner 196.41999912261963 +zach garcia 84.37999725341797 +zach garcia 160.70000457763672 +zach garcia 167.7599983215332 +zach garcia 205.36999893188477 +zach ichabod 116.2699966430664 +zach ichabod 151.18000030517578 +zach king 127.63000106811523 +zach king 182.2699966430664 +zach king 269.0999984741211 +zach miller 199.71000289916992 +zach miller 220.73999977111816 +zach miller 264.0600047111511 +zach ovid 92.55000305175781 +zach ovid 94.33999633789062 +zach ovid 105.94999847561121 +zach ovid 136.04000091552734 +zach quirinius 103.11000061035156 +zach robinson 76.72000122070312 +zach steinbeck 85.48999786376953 +zach steinbeck 182.87000274658203 +zach thompson 116.93999862670898 +zach thompson 319.9499969482422 +zach underhill 86.22000122070312 +zach white 70.52999877929688 +zach xylophone 227.52000427246094 +zach xylophone 286.45000076293945 +zach young 313.00999450683594 +zach zipper 85.87000274658203 +zach zipper 94.43000030517578 +zach zipper 139.38999938964844 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-3-6f104992e0050576085064815de43194 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-3-6f104992e0050576085064815de43194 new file mode 100644 index 0000000000000000000000000000000000000000..ae2a1e9dd7d3fdc41cbccd31ff4b06c98b8ec011 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-3-6f104992e0050576085064815de43194 @@ -0,0 +1,1049 @@ + 43.52666695912679 + 48.735000133514404 + 57.96666590372721 +alice allen 39.21833221117655 +alice allen 42.813999557495116 +alice allen 49.1824996471405 +alice brown 59.68166707456112 +alice carson 46.703333189090095 +alice davidson 59.51833279927572 +alice falkner 64.48333247502644 +alice garcia 55.114999771118164 +alice hernandez 49.61333228151003 +alice hernandez 69.70166667302449 +alice johnson 49.25166805585226 +alice king 20.052499681711197 +alice king 49.019999186197914 +alice king 56.0733331044515 +alice laertes 35.24999898672104 +alice laertes 68.85333251953125 +alice miller 55.542000198364256 +alice nixon 44.010000785191856 +alice nixon 50.900000762939456 +alice nixon 64.78333409627278 +alice ovid 24.90999937057495 +alice polk 47.426666259765625 +alice quirinius 52.80166610081991 +alice quirinius 52.94833393891653 +alice robinson 44.41500027974447 +alice robinson 55.04249954223633 +alice steinbeck 37.735000928243004 +alice steinbeck 41.02800045013428 +alice steinbeck 53.83499916394552 +alice underhill 52.64500021934509 +alice van buren 48.27666727701823 +alice xylophone 28.047500252723694 +alice xylophone 37.82199954986572 +alice xylophone 59.75166606903076 +alice zipper 48.875000635782875 +alice zipper 51.365000108877815 +alice zipper 89.93000030517578 +bob brown 38.134999910990395 +bob brown 56.20666662851969 +bob brown 77.51166598002116 +bob carson 52.77200050354004 +bob davidson 28.945000171661377 +bob davidson 37.946666399637856 +bob davidson 53.90333271026611 +bob ellison 51.383999824523926 +bob ellison 57.30333344141642 +bob ellison 58.53666559855143 +bob ellison 66.7400016784668 +bob falkner 39.053333600362144 +bob garcia 16.705000400543213 +bob garcia 37.16499960422516 +bob garcia 51.17999919255575 +bob garcia 52.37666575113932 +bob garcia 67.39166768391927 +bob hernandez 53.261999893188474 +bob ichabod 43.96999979019165 +bob king 38.03666718800863 +bob king 38.19249892234802 +bob king 67.40999794006348 +bob laertes 21.449999809265137 +bob laertes 50.37166612346967 +bob miller 41.906000471115114 +bob ovid 27.836666425069172 +bob ovid 39.3833335240682 +bob ovid 60.90749979019165 +bob ovid 63.069999313354494 +bob polk 41.88333296775818 +bob quirinius 54.504998207092285 +bob steinbeck 34.33999983469645 +bob van buren 58.29999923706055 +bob white 30.22333288192749 +bob white 41.44000005722046 +bob xylophone 12.163333415985107 +bob xylophone 47.88249921798706 +bob young 27.38499927520752 +bob zipper 27.93800084590912 +bob zipper 59.11800007820129 +bob zipper 66.43000030517578 +calvin allen 71.51000118255615 +calvin brown 50.44166612625122 +calvin brown 53.625000953674316 +calvin brown 67.48333485921223 +calvin carson 62.17750120162964 +calvin davidson 14.03000009059906 +calvin davidson 43.640000343322754 +calvin ellison 52.52750015258789 +calvin falkner 24.016666332880657 +calvin falkner 46.87333329518636 +calvin falkner 47.53666607538859 +calvin falkner 56.47499918937683 +calvin falkner 57.336666107177734 +calvin falkner 72.25 +calvin garcia 46.484999338785805 +calvin hernandez 43.90199909210205 +calvin johnson 76.41500091552734 +calvin laertes 39.09249973297119 +calvin laertes 47.323333422342934 +calvin nixon 30.113332668940227 +calvin nixon 47.7533327738444 +calvin nixon 49.08500075340271 +calvin ovid 41.924999713897705 +calvin ovid 49.10499978065491 +calvin ovid 62.26499938964844 +calvin ovid 62.27999989191691 +calvin polk 52.95333353678385 +calvin quirinius 53.34200019836426 +calvin quirinius 54.831998634338376 +calvin robinson 60.470001220703125 +calvin steinbeck 35.644999980926514 +calvin steinbeck 53.75800037384033 +calvin steinbeck 56.11000084877014 +calvin thompson 42.355000257492065 +calvin thompson 65.91666666666667 +calvin underhill 47.41199951171875 +calvin van buren 31.591666102409363 +calvin van buren 43.18200063705444 +calvin white 45.27500128746033 +calvin white 56.079999923706055 +calvin xylophone 20.394999980926514 +calvin xylophone 40.59999958674113 +calvin xylophone 54.56500005722046 +calvin young 42.246666272481285 +calvin young 64.49000072479248 +calvin zipper 57.49500036239624 +calvin zipper 57.59000039100647 +david allen 40.14999977747599 +david allen 54.545000076293945 +david brown 35.4516666730245 +david brown 62.83999938964844 +david davidson 35.50000067551931 +david davidson 52.75250005722046 +david davidson 74.1500015258789 +david davidson 95.80999755859375 +david ellison 43.43166727821032 +david ellison 52.18750047683716 +david ellison 72.58400039672851 +david hernandez 64.47600173950195 +david ichabod 29.518332719802856 +david ichabod 34.6100010573864 +david laertes 61.127999496459964 +david nixon 53.60249900817871 +david ovid 37.977500796318054 +david ovid 41.58999983469645 +david quirinius 24.96250009536743 +david quirinius 45.23249912261963 +david quirinius 52.282000350952146 +david robinson 42.17750024795532 +david robinson 62.54666519165039 +david thompson 41.88999938964844 +david underhill 48.143333752950035 +david underhill 62.53499941031138 +david underhill 97.55999755859375 +david van buren 34.84749984741211 +david van buren 51.61666679382324 +david white 62.30499839782715 +david xylophone 33.92500019073486 +david xylophone 53.142000222206114 +david xylophone 72.77166684468587 +david young 30.13666756947835 +david young 51.540000915527344 +ethan allen 53.442500591278076 +ethan brown 7.110000133514404 +ethan brown 41.260000228881836 +ethan brown 41.495000084241234 +ethan brown 41.83833312988281 +ethan brown 46.497499227523804 +ethan brown 63.17750072479248 +ethan carson 64.31999969482422 +ethan ellison 46.72599983215332 +ethan ellison 61.24750167876482 +ethan falkner 36.90250104665756 +ethan falkner 52.71000099182129 +ethan garcia 32.771666407585144 +ethan hernandez 56.239999008178714 +ethan johnson 82.38333384195964 +ethan king 8.399999856948853 +ethan laertes 37.93200054168701 +ethan laertes 40.4883329073588 +ethan laertes 53.396667132774986 +ethan laertes 53.81666628519694 +ethan laertes 55.987499713897705 +ethan laertes 66.36999956766765 +ethan laertes 68.83400039672851 +ethan miller 56.519999504089355 +ethan nixon 50.13333353648583 +ethan ovid 37.51333363850912 +ethan polk 2.3499999046325684 +ethan polk 44.147999954223636 +ethan polk 54.91499996185303 +ethan polk 60.340000788370766 +ethan quirinius 40.21750020980835 +ethan quirinius 47.125000298023224 +ethan quirinius 57.69000013669332 +ethan robinson 41.5060001373291 +ethan robinson 55.0640007019043 +ethan underhill 68.01250076293945 +ethan van buren 43.55250072479248 +ethan white 46.40999913215637 +ethan white 53.03200073242188 +ethan xylophone 70.29333432515462 +ethan zipper 46.92999982833862 +ethan zipper 66.04500198364258 +fred davidson 26.487499618902802 +fred davidson 44.99333477020264 +fred davidson 53.93666648864746 +fred ellison 22.2599999109904 +fred ellison 48.047999954223634 +fred ellison 63.423333485921226 +fred falkner 20.035000324249268 +fred falkner 44.243999004364014 +fred falkner 62.130001068115234 +fred hernandez 37.42199997901916 +fred ichabod 46.56999969482422 +fred ichabod 58.275001525878906 +fred johnson 54.63333257039388 +fred king 46.30499919255575 +fred king 61.48500037193298 +fred laertes 26.203333059946697 +fred miller 43.46400032043457 +fred nixon 28.690000534057617 +fred nixon 32.77999955415726 +fred nixon 52.03799936771393 +fred nixon 60.468332290649414 +fred polk 26.81166632970174 +fred polk 42.48999996185303 +fred polk 46.95200061798096 +fred polk 60.465998840332034 +fred quirinius 43.466000366210935 +fred quirinius 45.79799957275391 +fred robinson 62.42833296457926 +fred steinbeck 32.22999954223633 +fred steinbeck 48.73800039291382 +fred steinbeck 65.91750144958496 +fred underhill 72.94250011444092 +fred van buren 41.28250002861023 +fred van buren 41.5283338278532 +fred van buren 50.871665954589844 +fred van buren 81.77000045776367 +fred white 34.897499561309814 +fred young 58.56666692097982 +fred young 70.61499977111816 +fred zipper 33.90500068664551 +gabriella allen 51.041666666666664 +gabriella allen 55.13999938964844 +gabriella brown 54.63333320617676 +gabriella brown 72.33333396911621 +gabriella carson 49.26333236694336 +gabriella davidson 52.72800083160401 +gabriella ellison 54.68999989827474 +gabriella ellison 71.54000091552734 +gabriella falkner 20.834000015258788 +gabriella falkner 46.348333517710365 +gabriella falkner 48.58500099182129 +gabriella garcia 24.78999964396159 +gabriella hernandez 57.61000029246012 +gabriella hernandez 65.11166667938232 +gabriella ichabod 33.78749895095825 +gabriella ichabod 38.9950008392334 +gabriella ichabod 48.15499925613403 +gabriella ichabod 49.739999008178714 +gabriella ichabod 52.789999643961586 +gabriella king 35.58200044631958 +gabriella king 49.63000059723854 +gabriella laertes 47.81000010172526 +gabriella miller 62.17666753133138 +gabriella ovid 45.94333267211914 +gabriella ovid 50.435001373291016 +gabriella polk 42.58000100851059 +gabriella polk 72.87000179290771 +gabriella steinbeck 65.42000102996826 +gabriella steinbeck 72.0499997138977 +gabriella thompson 49.897499561309814 +gabriella thompson 52.84250023961067 +gabriella thompson 57.23249959945679 +gabriella van buren 39.31000053882599 +gabriella van buren 57.920000076293945 +gabriella white 49.85333410898844 +gabriella young 15.369999885559082 +gabriella young 49.04999907811483 +gabriella zipper 42.82499901453654 +gabriella zipper 59.58499972025553 +holly allen 29.50499963760376 +holly brown 55.284998178482056 +holly brown 55.79833386838436 +holly falkner 40.07666663328806 +holly hernandez 46.40500044822693 +holly hernandez 56.14500045776367 +holly hernandez 56.160000483194985 +holly hernandez 58.95166748017073 +holly ichabod 67.19250011444092 +holly ichabod 68.44500064849854 +holly ichabod 90.09000015258789 +holly johnson 42.795000076293945 +holly johnson 55.76999855041504 +holly johnson 66.11333338419597 +holly king 48.788000869750974 +holly king 64.25 +holly laertes 50.40750074386597 +holly miller 45.60666608810425 +holly nixon 58.096665700276695 +holly nixon 69.59249877929688 +holly polk 41.59666601816813 +holly polk 44.64799900054932 +holly robinson 48.405999755859376 +holly thompson 34.605000495910645 +holly thompson 38.36749941110611 +holly thompson 65.99833394338687 +holly underhill 46.52600040435791 +holly underhill 47.92666663726171 +holly underhill 62.41600036621094 +holly underhill 79.95999908447266 +holly van buren 58.63333225250244 +holly white 31.232499361038208 +holly white 61.494998931884766 +holly xylophone 70.32500044504802 +holly young 54.05000114440918 +holly young 57.103333473205566 +holly zipper 50.59500014781952 +holly zipper 67.81000137329102 +irene allen 53.364000701904295 +irene brown 4.789999961853027 +irene brown 51.65399932861328 +irene brown 87.66999816894531 +irene carson 59.2433336575826 +irene ellison 36.446667989095054 +irene ellison 53.85000038146973 +irene falkner 47.429999995231626 +irene falkner 61.20666694641113 +irene garcia 15.369999885559082 +irene garcia 48.28000005086263 +irene garcia 55.643333435058594 +irene ichabod 40.868333299954735 +irene ichabod 64.45666694641113 +irene johnson 44.37999935150147 +irene laertes 27.625 +irene laertes 46.25 +irene laertes 49.17500019073486 +irene miller 75.87500063578288 +irene nixon 29.780000686645508 +irene nixon 30.070000807444256 +irene nixon 36.34333356221517 +irene ovid 27.21500023206075 +irene ovid 55.6175012588501 +irene ovid 60.353999328613284 +irene polk 47.438334147135414 +irene polk 47.77666505177816 +irene polk 51.74200067520142 +irene polk 52.12200088500977 +irene polk 52.184000205993655 +irene quirinius 52.958333015441895 +irene quirinius 55.29000053405762 +irene quirinius 78.79000091552734 +irene robinson 61.398332595825195 +irene steinbeck 64.34000142415364 +irene thompson 41.92250097543001 +irene underhill 27.72499966621399 +irene underhill 50.783331871032715 +irene van buren 46.65999984741211 +irene van buren 49.71000099182129 +irene xylophone 61.220001220703125 +jessica brown 63.449999491373696 +jessica carson 41.78999951481819 +jessica carson 47.52000069618225 +jessica carson 51.83000183105469 +jessica davidson 34.938333332538605 +jessica davidson 45.91000066200892 +jessica davidson 51.082499504089355 +jessica davidson 64.3099988301595 +jessica ellison 32.53000124295553 +jessica ellison 35.265999984741214 +jessica falkner 54.933334032694496 +jessica garcia 32.575000286102295 +jessica garcia 60.584999084472656 +jessica ichabod 46.704999923706055 +jessica johnson 55.73199977874756 +jessica johnson 72.14000066121419 +jessica miller 55.27799835205078 +jessica nixon 58.53999900817871 +jessica nixon 90.06999969482422 +jessica ovid 36.47500014305115 +jessica ovid 63.03000005086263 +jessica polk 69.52999877929688 +jessica quirinius 19.01333288351695 +jessica quirinius 41.630000829696655 +jessica quirinius 42.58750009536743 +jessica quirinius 47.72999986012777 +jessica robinson 64.81000061035157 +jessica thompson 45.83666737874349 +jessica thompson 57.540000915527344 +jessica underhill 45.360000133514404 +jessica underhill 51.40999889373779 +jessica underhill 64.27250099182129 +jessica van buren 9.739999771118164 +jessica white 38.38999891281128 +jessica white 40.75400023460388 +jessica white 61.89800128936768 +jessica white 62.34749984741211 +jessica white 65.7680004119873 +jessica xylophone 50.808332761128746 +jessica young 18.400000254313152 +jessica young 49.04400033950806 +jessica zipper 35.039999643961586 +jessica zipper 52.78999869028727 +jessica zipper 55.51499891281128 +katie allen 56.10800056457519 +katie brown 48.86833222707113 +katie davidson 96.91000366210938 +katie ellison 31.106667200724285 +katie ellison 38.356666127840676 +katie falkner 17.24333318074544 +katie garcia 53.429999033610024 +katie garcia 53.933334002892174 +katie hernandez 72.71666717529297 +katie ichabod 39.97999954223633 +katie ichabod 54.99599914550781 +katie ichabod 59.41999944051107 +katie king 44.88599967956543 +katie king 60.2549991607666 +katie king 67.27499961853027 +katie miller 31.399999618530273 +katie miller 50.57666619618734 +katie nixon 23.190000534057617 +katie ovid 52.96000158786774 +katie polk 51.029998779296875 +katie polk 54.93600006103516 +katie robinson 13.890000343322754 +katie van buren 52.78999908765157 +katie van buren 53.21750092506409 +katie white 59.799998950958255 +katie white 64.70499992370605 +katie xylophone 53.54499959945679 +katie young 31.010000228881836 +katie young 37.55500102043152 +katie young 49.87499872843424 +katie zipper 29.79666694998741 +katie zipper 50.95000076293945 +luke allen 43.757998657226565 +luke allen 53.18000030517578 +luke allen 62.09749984741211 +luke allen 70.2933349609375 +luke allen 89.55000305175781 +luke brown 45.95999972025553 +luke davidson 28.950000762939453 +luke davidson 53.20500040054321 +luke ellison 5.159999907016754 +luke ellison 18.630000392595928 +luke ellison 76.88000106811523 +luke falkner 6.175000190734863 +luke falkner 54.0049991607666 +luke garcia 15.320000424981117 +luke garcia 25.47000026702881 +luke ichabod 33.950000286102295 +luke ichabod 57.17500114440918 +luke johnson 14.4399995803833 +luke johnson 37.23333263397217 +luke johnson 52.660000801086426 +luke laertes 11.819999694824219 +luke laertes 23.5 +luke laertes 43.71999907493591 +luke laertes 45.9900016784668 +luke laertes 57.85499906539917 +luke miller 67.85000038146973 +luke ovid 38.04999923706055 +luke ovid 79.64200134277344 +luke polk 55.106666564941406 +luke polk 70.18999862670898 +luke quirinius 26.016666491826374 +luke robinson 61.256665547688804 +luke robinson 72.61999893188477 +luke thompson 47.203332940737404 +luke underhill 49.244999408721924 +luke underhill 59.32999897003174 +luke underhill 72.18333307902019 +luke van buren 64.646666208903 +luke white 67.12000274658203 +luke xylophone 46.74333190917969 +luke zipper 36.16000008583069 +mike allen 10.709999859333038 +mike brown 61.0674991607666 +mike carson 50.90249824523926 +mike carson 52.30200061798096 +mike carson 52.51499938964844 +mike davidson 29.733333269755047 +mike davidson 47.396666526794434 +mike ellison 37.807498931884766 +mike ellison 44.339999198913574 +mike ellison 45.89199924468994 +mike ellison 47.91249990463257 +mike ellison 55.295000076293945 +mike falkner 16.479999542236328 +mike garcia 38.98999913533529 +mike garcia 54.91333325703939 +mike garcia 57.096666971842446 +mike hernandez 59.45000076293945 +mike hernandez 62.446667989095054 +mike ichabod 54.69666544596354 +mike king 38.790000915527344 +mike king 43.2050017118454 +mike king 46.85666592915853 +mike king 46.945000648498535 +mike king 53.382500648498535 +mike king 94.68000030517578 +mike miller 3.9600000381469727 +mike nixon 53.894999742507935 +mike nixon 92.95999908447266 +mike polk 12.449999809265137 +mike polk 39.040000319480896 +mike polk 71.40500068664551 +mike quirinius 89.37999725341797 +mike steinbeck 34.05499875545502 +mike steinbeck 45.909999465942384 +mike steinbeck 51.286667346954346 +mike steinbeck 59.292500019073486 +mike van buren 43.13999819755554 +mike van buren 49.59000015258789 +mike white 30.53999964396159 +mike white 34.61499961217245 +mike white 48.08999943733215 +mike white 53.967501163482666 +mike young 10.484999895095825 +mike young 47.070000648498535 +mike young 74.58999633789062 +mike zipper 29.029999288419884 +mike zipper 77.96200027465821 +mike zipper 91.57500076293945 +nick allen 29.146666367848713 +nick allen 64.4325008392334 +nick brown 49.429999669392906 +nick davidson 34.36599922180176 +nick ellison 49.345001220703125 +nick ellison 89.01000213623047 +nick falkner 7.820000171661377 +nick falkner 45.025000631809235 +nick garcia 23.9499994913737 +nick garcia 33.49333349863688 +nick garcia 64.33499908447266 +nick ichabod 30.945000171661377 +nick ichabod 56.27499961853027 +nick ichabod 59.24249863624573 +nick johnson 32.3674995303154 +nick johnson 74.30666859944661 +nick laertes 38.38749980926514 +nick miller 82.97000122070312 +nick nixon 70.01333173116048 +nick ovid 56.82999897003174 +nick polk 33.929999669392906 +nick quirinius 58.91199932098389 +nick quirinius 81.16999816894531 +nick robinson 34.53999948501587 +nick robinson 59.80499839782715 +nick steinbeck 57.25333340962728 +nick thompson 51.3725004196167 +nick underhill 55.47333335876465 +nick van buren 74.21666717529297 +nick xylophone 51.82000001271566 +nick young 0.27000001072883606 +nick young 41.02499961853027 +nick zipper 47.72999954223633 +nick zipper 61.917999267578125 +oscar allen 35.21600015163422 +oscar brown 38.03999948501587 +oscar carson 38.09600009918213 +oscar carson 51.34333419799805 +oscar carson 56.9925012588501 +oscar carson 74.00500106811523 +oscar carson 98.51000213623047 +oscar davidson 65.88750076293945 +oscar ellison 50.507500410079956 +oscar ellison 66.57000096638997 +oscar falkner 64.42000198364258 +oscar garcia 66.36333465576172 +oscar hernandez 42.089999516805015 +oscar hernandez 51.21199997067451 +oscar ichabod 25.300000190734863 +oscar ichabod 41.173332850138344 +oscar ichabod 53.44666830698649 +oscar ichabod 71.80000305175781 +oscar johnson 39.9366668065389 +oscar johnson 44.30500118434429 +oscar king 30.59749937057495 +oscar king 46.149999141693115 +oscar king 49.65999984741211 +oscar laertes 5.510000228881836 +oscar laertes 19.099999745686848 +oscar laertes 39.36250039935112 +oscar laertes 45.340000788370766 +oscar nixon 41.619998931884766 +oscar ovid 45.09000015258789 +oscar ovid 82.23999786376953 +oscar ovid 83.47750091552734 +oscar polk 42.54333368937174 +oscar polk 56.8033332824707 +oscar quirinius 52.94500160217285 +oscar quirinius 65.34666570027669 +oscar quirinius 67.00000127156575 +oscar quirinius 79.4800033569336 +oscar robinson 32.16999944051107 +oscar robinson 38.880001068115234 +oscar robinson 63.9433339436849 +oscar robinson 65.80500030517578 +oscar steinbeck 45.73599967956543 +oscar thompson 40.33599853515625 +oscar thompson 47.860000928243004 +oscar thompson 60.529998779296875 +oscar thompson 60.59333292643229 +oscar underhill 43.980000764131546 +oscar van buren 2.180000066757202 +oscar van buren 53.34999942779541 +oscar van buren 61.880001068115234 +oscar white 38.0633331934611 +oscar white 42.584999084472656 +oscar white 55.179999033610024 +oscar white 74.4900016784668 +oscar xylophone 41.3133331934611 +oscar xylophone 55.5049991607666 +oscar xylophone 67.54500007629395 +oscar zipper 15.680000305175781 +oscar zipper 24.019999504089355 +oscar zipper 39.81999969482422 +priscilla brown 42.88666502634684 +priscilla brown 63.242499351501465 +priscilla brown 77.56999969482422 +priscilla carson 44.799999713897705 +priscilla carson 49.663333892822266 +priscilla carson 78.57333374023438 +priscilla ichabod 56.83666737874349 +priscilla ichabod 58.48666636149088 +priscilla johnson 34.416667779286705 +priscilla johnson 52.890000343322754 +priscilla johnson 53.72666883468628 +priscilla johnson 61.939998626708984 +priscilla johnson 89.1500015258789 +priscilla king 34.30750045180321 +priscilla nixon 27.734999656677246 +priscilla nixon 60.90999984741211 +priscilla ovid 48.13500142097473 +priscilla ovid 66.35999870300293 +priscilla polk 15.149999618530273 +priscilla quirinius 18.606666564941406 +priscilla thompson 48.87000131607056 +priscilla underhill 40.05500078201294 +priscilla underhill 49.54999961853027 +priscilla van buren 42.62666734059652 +priscilla van buren 61.54000017642975 +priscilla van buren 72.80500030517578 +priscilla white 43.177499771118164 +priscilla xylophone 21.489999771118164 +priscilla xylophone 40.144999124109745 +priscilla xylophone 59.61000061035156 +priscilla young 31.610000324249267 +priscilla young 53.71000152826309 +priscilla zipper 18.8799991607666 +priscilla zipper 25.670000076293945 +quinn allen 56.77749991416931 +quinn allen 76.47250080108643 +quinn brown 23.536666870117188 +quinn brown 31.829999446868896 +quinn brown 50.388000297546384 +quinn davidson 41.42499923706055 +quinn davidson 45.90250015258789 +quinn davidson 48.16000032424927 +quinn davidson 71.51000022888184 +quinn ellison 50.6766668955485 +quinn ellison 62.32666842142741 +quinn garcia 39.98599967956543 +quinn garcia 43.27000045776367 +quinn garcia 74.31999969482422 +quinn garcia 92.33000183105469 +quinn ichabod 44.81999969482422 +quinn king 50.99666786193848 +quinn king 74.62000274658203 +quinn laertes 5.884999990463257 +quinn laertes 49.1379997253418 +quinn laertes 56.18000030517578 +quinn nixon 74.66500091552734 +quinn ovid 1.2100000381469727 +quinn quirinius 46.15500068664551 +quinn robinson 44.96249985694885 +quinn steinbeck 24.06999921798706 +quinn steinbeck 41.750000953674316 +quinn thompson 23.744999408721924 +quinn thompson 73.64500045776367 +quinn underhill 41.83333269755045 +quinn underhill 49.63749885559082 +quinn underhill 83.85000228881836 +quinn van buren 54.36333338419596 +quinn young 45.489999771118164 +quinn zipper 22.25 +quinn zipper 33.355000019073486 +rachel allen 15.8100004196167 +rachel allen 71.57666524251302 +rachel brown 2.9600000381469727 +rachel brown 33.022499561309814 +rachel brown 42.442498207092285 +rachel brown 59.21999931335449 +rachel brown 64.52666727701823 +rachel carson 61.17999776204427 +rachel carson 69.85333315531413 +rachel davidson 42.292500495910645 +rachel ellison 10.600000381469727 +rachel falkner 23.615000784397125 +rachel falkner 50.46250069141388 +rachel falkner 54.669999837875366 +rachel falkner 72.96999740600586 +rachel johnson 61.76249885559082 +rachel king 22.005000591278076 +rachel king 66.46500015258789 +rachel laertes 39.025001525878906 +rachel laertes 45.45000076293945 +rachel ovid 0.6000000238418579 +rachel ovid 1.0800000429153442 +rachel polk 78.08499908447266 +rachel quirinius 51.12000131607056 +rachel robinson 30.360000610351562 +rachel robinson 37.683333237965904 +rachel robinson 39.199998219807945 +rachel thompson 0.5600000023841858 +rachel thompson 38.81000010172526 +rachel thompson 49.772499561309814 +rachel underhill 48.45000076293945 +rachel white 42.83999991416931 +rachel white 58.609999656677246 +rachel young 43.130001068115234 +rachel zipper 7.059999942779541 +rachel zipper 49.676667173703514 +sarah carson 54.022500067949295 +sarah carson 54.18333212534586 +sarah carson 87.81000137329102 +sarah ellison 16.989999771118164 +sarah falkner 73.06500005722046 +sarah falkner 99.36000061035156 +sarah garcia 38.43250036239624 +sarah garcia 64.97333272298177 +sarah garcia 73.6500015258789 +sarah ichabod 57.46000003814697 +sarah ichabod 81.31999969482422 +sarah johnson 34.98499917984009 +sarah johnson 45.099998474121094 +sarah johnson 74.42249870300293 +sarah johnson 74.72000122070312 +sarah king 41.869998931884766 +sarah king 48.25 +sarah miller 41.53499984741211 +sarah ovid 33.38000011444092 +sarah robinson 33.83000183105469 +sarah robinson 66.88999938964844 +sarah steinbeck 31.023332993189495 +sarah white 45.974998474121094 +sarah white 61.54666709899902 +sarah xylophone 61.60666529337565 +sarah young 45.560001373291016 +sarah zipper 60.69000053405762 +tom brown 40.08000135421753 +tom brown 55.44499969482422 +tom carson 5.440000057220459 +tom carson 26.32499885559082 +tom carson 31.476666768391926 +tom davidson 53.00749921798706 +tom ellison 67.32666714986165 +tom ellison 76.73999786376953 +tom ellison 77.29499816894531 +tom falkner 60.130001068115234 +tom falkner 88.22000122070312 +tom hernandez 41.36000061035156 +tom hernandez 81.63999938964844 +tom ichabod 42.08666547139486 +tom johnson 33.75999959309896 +tom johnson 53.47666549682617 +tom king 40.0 +tom laertes 32.05000114440918 +tom laertes 43.46000099182129 +tom miller 21.229999542236328 +tom miller 43.37666702270508 +tom miller 44.41333325703939 +tom nixon 46.35333251953125 +tom ovid 43.04499912261963 +tom polk 38.29999923706055 +tom polk 54.45000076293945 +tom quirinius 19.82000058889389 +tom quirinius 22.580000400543213 +tom robinson 66.16999816894531 +tom robinson 74.18666712443034 +tom robinson 80.77000045776367 +tom robinson 98.72000122070312 +tom steinbeck 44.999999046325684 +tom van buren 31.103334546089172 +tom van buren 40.779998779296875 +tom van buren 63.5099983215332 +tom white 49.06500053405762 +tom young 54.16999912261963 +tom young 78.54999923706055 +tom zipper 48.666666984558105 +ulysses brown 72.79000091552734 +ulysses carson 40.28500175476074 +ulysses carson 71.55000305175781 +ulysses carson 77.41999816894531 +ulysses carson 79.54000091552734 +ulysses davidson 41.90166711807251 +ulysses ellison 96.7300033569336 +ulysses garcia 89.80000305175781 +ulysses hernandez 21.339999516805012 +ulysses hernandez 54.470001220703125 +ulysses hernandez 80.11000061035156 +ulysses ichabod 19.1299991607666 +ulysses ichabod 98.56999969482422 +ulysses johnson 42.55000019073486 +ulysses king 81.57000223795573 +ulysses laertes 1.9199999570846558 +ulysses laertes 11.890000343322754 +ulysses laertes 53.599998474121094 +ulysses miller 40.43499946594238 +ulysses miller 47.78200054168701 +ulysses nixon 74.60333251953125 +ulysses ovid 30.940000534057617 +ulysses polk 8.710000038146973 +ulysses polk 39.17500019073486 +ulysses polk 65.0199966430664 +ulysses polk 68.41333389282227 +ulysses quirinius 65.93499946594238 +ulysses robinson 79.48999786376953 +ulysses steinbeck 32.40999984741211 +ulysses steinbeck 43.93499994277954 +ulysses thompson 79.96000289916992 +ulysses underhill 17.85000006357829 +ulysses underhill 42.5533332824707 +ulysses underhill 46.60000157356262 +ulysses underhill 47.84000049829483 +ulysses underhill 51.85000133514404 +ulysses underhill 67.77999877929688 +ulysses underhill 99.66999816894531 +ulysses van buren 69.89999961853027 +ulysses white 45.01500183343887 +ulysses white 71.3933334350586 +ulysses xylophone 27.880000591278076 +ulysses xylophone 39.689998626708984 +ulysses xylophone 54.099998474121094 +ulysses young 32.52000045776367 +ulysses young 39.81333382924398 +ulysses young 80.7933349609375 +victor allen 44.27000045776367 +victor allen 57.994998931884766 +victor brown 56.84499931335449 +victor brown 59.34000015258789 +victor brown 90.37999725341797 +victor brown 91.97000122070312 +victor davidson 52.07000160217285 +victor davidson 54.239999771118164 +victor davidson 68.8033332824707 +victor ellison 45.45750088989735 +victor ellison 58.42999839782715 +victor hernandez 23.164999961853027 +victor hernandez 57.98499870300293 +victor hernandez 59.619998931884766 +victor hernandez 71.42499923706055 +victor hernandez 80.19499969482422 +victor johnson 1.5800000429153442 +victor johnson 46.7450008392334 +victor johnson 72.59000015258789 +victor king 37.559998989105225 +victor king 47.880001068115234 +victor laertes 50.0099983215332 +victor laertes 70.13999938964844 +victor miller 22.1200008392334 +victor nixon 34.029998779296875 +victor nixon 37.08500003814697 +victor ovid 18.815000653266907 +victor polk 3.0 +victor quirinius 26.450000405311584 +victor quirinius 33.080001533031464 +victor robinson 4.590000152587891 +victor robinson 41.21999931335449 +victor steinbeck 41.82500123977661 +victor steinbeck 46.09000015258789 +victor steinbeck 52.720001220703125 +victor thompson 45.346666971842446 +victor van buren 34.970001220703125 +victor van buren 46.57333246866862 +victor white 5.670000076293945 +victor white 74.16999816894531 +victor xylophone 11.220000267028809 +victor xylophone 28.954999923706055 +victor xylophone 34.010000228881836 +victor xylophone 43.179999351501465 +victor xylophone 62.38999938964844 +victor young 70.91000175476074 +victor zipper 48.795000076293945 +wendy allen 0.6100000143051147 +wendy allen 52.64999961853027 +wendy allen 56.06999969482422 +wendy brown 51.874999046325684 +wendy brown 66.73999913533528 +wendy ellison 27.014999389648438 +wendy ellison 94.66000366210938 +wendy falkner 14.425000190734863 +wendy falkner 77.36000061035156 +wendy falkner 85.68000030517578 +wendy garcia 4.409999847412109 +wendy garcia 20.390000343322754 +wendy garcia 38.42500060796738 +wendy garcia 57.25 +wendy hernandez 48.11000061035156 +wendy ichabod 13.149999618530273 +wendy king 33.234999656677246 +wendy king 74.97999954223633 +wendy king 87.94000244140625 +wendy laertes 49.01999855041504 +wendy laertes 54.750000635782875 +wendy laertes 79.98999786376953 +wendy miller 1.2699999809265137 +wendy miller 47.41500186920166 +wendy nixon 30.17999916151166 +wendy nixon 45.91999816894531 +wendy ovid 85.26000213623047 +wendy ovid 86.62999725341797 +wendy polk 43.679999669392906 +wendy polk 54.65333424011866 +wendy quirinius 12.15999984741211 +wendy quirinius 14.300000190734863 +wendy robinson 58.815001487731934 +wendy robinson 71.06999969482422 +wendy robinson 71.77999877929688 +wendy steinbeck 77.31500053405762 +wendy thompson 58.09000015258789 +wendy thompson 85.76000213623047 +wendy underhill 41.23333231608073 +wendy underhill 49.33500003814697 +wendy underhill 81.56000137329102 +wendy van buren 57.459999084472656 +wendy van buren 91.43500137329102 +wendy white 73.68000030517578 +wendy xylophone 31.149999618530273 +wendy xylophone 76.81499862670898 +wendy young 8.449999809265137 +wendy young 33.7599983215332 +xavier allen 45.68000030517578 +xavier allen 52.255000591278076 +xavier allen 83.93000030517578 +xavier brown 7.789999961853027 +xavier brown 77.15500259399414 +xavier brown 80.35000228881836 +xavier carson 20.790000915527344 +xavier carson 55.30000019073486 +xavier davidson 15.920000076293945 +xavier davidson 58.46999931335449 +xavier davidson 82.41000366210938 +xavier ellison 16.614999771118164 +xavier ellison 77.97000122070312 +xavier garcia 49.415000915527344 +xavier hernandez 6.670000076293945 +xavier hernandez 23.054999828338623 +xavier hernandez 67.26000213623047 +xavier ichabod 4.71999979019165 +xavier ichabod 71.19000244140625 +xavier johnson 27.299999237060547 +xavier johnson 89.0999984741211 +xavier king 22.729999542236328 +xavier king 87.22000122070312 +xavier laertes 24.050000190734863 +xavier ovid 58.08000183105469 +xavier polk 13.869999885559082 +xavier polk 58.98750066757202 +xavier polk 72.62000274658203 +xavier polk 76.93000030517578 +xavier quirinius 22.1200008392334 +xavier quirinius 58.24666786193848 +xavier quirinius 62.52000045776367 +xavier quirinius 89.55000305175781 +xavier thompson 9.930000305175781 +xavier underhill 47.27000045776367 +xavier white 8.369999885559082 +xavier white 75.29000091552734 +xavier xylophone 79.41999816894531 +xavier zipper 8.449999809265137 +yuri allen 30.6299991607666 +yuri allen 52.849998474121094 +yuri brown 75.19000244140625 +yuri brown 84.02999877929688 +yuri carson 6.289999961853027 +yuri carson 91.16000366210938 +yuri ellison 1.1200000047683716 +yuri ellison 86.91999816894531 +yuri falkner 6.739999771118164 +yuri falkner 80.8650016784668 +yuri garcia 27.65999984741211 +yuri hernandez 2.069999933242798 +yuri johnson 0.12999999523162842 +yuri johnson 39.900001525878906 +yuri johnson 48.220001220703125 +yuri king 22.270000457763672 +yuri laertes 10.15999984741211 +yuri laertes 37.59000015258789 +yuri nixon 2.200000047683716 +yuri nixon 17.3700008392334 +yuri polk 26.760000228881836 +yuri polk 28.790000915527344 +yuri polk 82.33999633789062 +yuri quirinius 10.260000228881836 +yuri quirinius 38.69000053405762 +yuri quirinius 57.93000030517578 +yuri steinbeck 56.064998626708984 +yuri steinbeck 75.87999725341797 +yuri thompson 14.920000076293945 +yuri underhill 23.770000457763672 +yuri underhill 83.87000274658203 +yuri white 34.58000183105469 +yuri xylophone 20.3799991607666 +zach allen 65.43000030517578 +zach brown 48.0099983215332 +zach brown 57.08000183105469 +zach brown 58.24999809265137 +zach brown 67.37999725341797 +zach brown 75.7300033569336 +zach carson 67.78500175476074 +zach ellison 6.840000152587891 +zach falkner 50.274999141693115 +zach falkner 91.41999816894531 +zach garcia 32.20000076293945 +zach garcia 35.79999923706055 +zach garcia 69.97000122070312 +zach garcia 84.37999725341797 +zach ichabod 36.88999938964844 +zach ichabod 64.25 +zach king 31.864999771118164 +zach king 46.18000030517578 +zach king 86.93000030517578 +zach miller 2.5999999046325684 +zach miller 21.280000686645508 +zach miller 53.27000045776367 +zach ovid 0.10000000149011612 +zach ovid 23.06999969482422 +zach ovid 92.55000305175781 +zach ovid 94.33999633789062 +zach quirinius 39.209999084472656 +zach robinson 76.72000122070312 +zach steinbeck 85.48999786376953 +zach steinbeck 90.05000305175781 +zach thompson 53.59000015258789 +zach thompson 71.5 +zach underhill 86.22000122070312 +zach white 70.52999877929688 +zach xylophone 29.40999984741211 +zach xylophone 71.01000213623047 +zach young 71.31999969482422 +zach zipper 7.539999961853027 +zach zipper 85.87000274658203 +zach zipper 94.43000030517578 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-4-cd2e3d2344810cb3ba843d4c01c81d7e b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-4-cd2e3d2344810cb3ba843d4c01c81d7e new file mode 100644 index 0000000000000000000000000000000000000000..ee1c26e331a1e52098aa7fa9a3ee0ac39fca6389 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-4-cd2e3d2344810cb3ba843d4c01c81d7e @@ -0,0 +1,1049 @@ + 17.601666666666667 + 30.72 + 33.07833333333334 +alice allen 23.081666666666667 +alice allen 23.263333333333332 +alice allen 31.38 +alice brown 11.518333333333333 +alice carson 31.99166666666667 +alice davidson 26.986666666666665 +alice falkner 30.513333333333335 +alice garcia 17.13 +alice hernandez 28.126666666666665 +alice hernandez 28.69666666666667 +alice johnson 28.30333333333333 +alice king 23.451666666666668 +alice king 25.20333333333333 +alice king 30.613999999999997 +alice laertes 23.633333333333336 +alice laertes 30.16428571428571 +alice miller 32.50833333333333 +alice nixon 25.278333333333336 +alice nixon 35.345 +alice nixon 36.458333333333336 +alice ovid 26.04714285714286 +alice polk 23.185 +alice quirinius 20.388333333333335 +alice quirinius 23.064999999999998 +alice robinson 30.296666666666667 +alice robinson 33.15 +alice steinbeck 27.894999999999996 +alice steinbeck 31.47833333333334 +alice steinbeck 36.089999999999996 +alice underhill 22.513333333333335 +alice van buren 36.32000000000001 +alice xylophone 27.355000000000004 +alice xylophone 30.505714285714284 +alice xylophone 30.613999999999997 +alice zipper 20.113333333333333 +alice zipper 28.058333333333337 +alice zipper 29.705000000000002 +bob brown 13.546666666666669 +bob brown 16.50333333333333 +bob brown 27.59 +bob carson 25.781666666666666 +bob davidson 18.073333333333334 +bob davidson 23.573333333333334 +bob davidson 31.894999999999996 +bob ellison 16.493333333333336 +bob ellison 17.889999999999997 +bob ellison 25.84142857142857 +bob ellison 33.07833333333334 +bob falkner 21.783333333333335 +bob garcia 16.492857142857144 +bob garcia 23.088333333333335 +bob garcia 26.42833333333333 +bob garcia 32.709999999999994 +bob garcia 33.91833333333334 +bob hernandez 22.303333333333338 +bob ichabod 23.336666666666662 +bob king 20.591666666666665 +bob king 21.244999999999997 +bob king 33.15333333333333 +bob laertes 21.240000000000002 +bob laertes 28.88 +bob miller 32.158750000000005 +bob ovid 21.83666666666667 +bob ovid 23.678571428571427 +bob ovid 25.12333333333333 +bob ovid 31.46 +bob polk 16.3475 +bob quirinius 28.465714285714284 +bob steinbeck 26.99 +bob van buren 26.127142857142854 +bob white 23.021666666666672 +bob white 23.582857142857144 +bob xylophone 26.18166666666667 +bob xylophone 27.995 +bob young 25.495714285714286 +bob zipper 26.435000000000002 +bob zipper 28.384285714285713 +bob zipper 30.65833333333333 +calvin allen 20.812 +calvin brown 20.808 +calvin brown 28.26 +calvin brown 28.37875 +calvin carson 20.38714285714286 +calvin davidson 22.65142857142857 +calvin davidson 23.585714285714285 +calvin ellison 24.6875 +calvin falkner 16.073333333333334 +calvin falkner 17.054285714285715 +calvin falkner 25.2025 +calvin falkner 28.525714285714283 +calvin falkner 33.382 +calvin falkner 37.29666666666667 +calvin garcia 19.924285714285713 +calvin hernandez 21.759999999999998 +calvin johnson 24.03222222222222 +calvin laertes 24.702857142857145 +calvin laertes 28.788333333333338 +calvin nixon 15.631250000000001 +calvin nixon 25.64428571428571 +calvin nixon 29.084999999999997 +calvin ovid 16.243333333333332 +calvin ovid 25.2025 +calvin ovid 25.935555555555553 +calvin ovid 29.299999999999997 +calvin polk 29.615 +calvin quirinius 19.294999999999998 +calvin quirinius 31.64625 +calvin robinson 27.811428571428575 +calvin steinbeck 17.14 +calvin steinbeck 17.535 +calvin steinbeck 21.551666666666666 +calvin thompson 28.592857142857145 +calvin thompson 33.382 +calvin underhill 20.113333333333333 +calvin van buren 28.384285714285713 +calvin van buren 32.106 +calvin white 26.948333333333334 +calvin white 28.256249999999998 +calvin xylophone 25.33666666666667 +calvin xylophone 27.061428571428568 +calvin xylophone 28.384285714285713 +calvin young 19.331666666666667 +calvin young 24.63 +calvin zipper 16.08125 +calvin zipper 28.80555555555556 +david allen 26.21375 +david allen 26.948333333333334 +david brown 14.222 +david brown 27.1 +david davidson 21.062857142857144 +david davidson 26.12 +david davidson 30.951428571428576 +david davidson 31.96142857142857 +david ellison 22.687142857142856 +david ellison 23.435 +david ellison 25.442999999999998 +david hernandez 28.279999999999998 +david ichabod 16.3475 +david ichabod 23.751428571428573 +david laertes 29.07 +david nixon 25.535714285714285 +david ovid 26.75714285714286 +david ovid 29.912857142857142 +david quirinius 17.179999999999996 +david quirinius 20.808 +david quirinius 22.90625 +david robinson 25.274 +david robinson 25.885 +david thompson 22.19125 +david underhill 20.812 +david underhill 21.546666666666667 +david underhill 28.26 +david van buren 24.472857142857148 +david van buren 32.75125 +david white 22.861428571428576 +david xylophone 19.3325 +david xylophone 26.930000000000003 +david xylophone 30.093333333333334 +david young 21.12375 +david young 25.03857142857143 +ethan allen 27.038333333333338 +ethan brown 16.3475 +ethan brown 19.37 +ethan brown 23.73 +ethan brown 25.57166666666667 +ethan brown 26.168333333333333 +ethan brown 31.893333333333334 +ethan carson 25.655714285714286 +ethan ellison 27.807777777777776 +ethan ellison 32.244285714285716 +ethan falkner 28.287142857142857 +ethan falkner 31.411428571428576 +ethan garcia 24.153750000000002 +ethan hernandez 17.986 +ethan johnson 31.54571428571429 +ethan king 22.62 +ethan laertes 15.045555555555556 +ethan laertes 17.889999999999997 +ethan laertes 24.28 +ethan laertes 25.2025 +ethan laertes 26.56888888888889 +ethan laertes 28.37875 +ethan laertes 34.84571428571429 +ethan miller 28.592857142857145 +ethan nixon 29.86833333333333 +ethan ovid 17.85166666666667 +ethan polk 16.463 +ethan polk 18.184545454545457 +ethan polk 25.737142857142857 +ethan polk 29.895000000000003 +ethan quirinius 21.827777777777776 +ethan quirinius 23.42 +ethan quirinius 35.54333333333333 +ethan robinson 24.03222222222222 +ethan robinson 36.35333333333333 +ethan underhill 21.545555555555556 +ethan van buren 15.21142857142857 +ethan white 29.702857142857145 +ethan white 33.33 +ethan xylophone 29.521666666666665 +ethan zipper 23.994 +ethan zipper 26.765 +fred davidson 27.729999999999997 +fred davidson 29.90625 +fred davidson 30.804999999999996 +fred ellison 16.720000000000002 +fred ellison 22.17125 +fred ellison 33.15833333333334 +fred falkner 17.13142857142857 +fred falkner 26.82 +fred falkner 31.925555555555555 +fred hernandez 28.094285714285718 +fred ichabod 23.352500000000003 +fred ichabod 32.906666666666666 +fred johnson 16.9925 +fred king 20.125 +fred king 30.377142857142854 +fred laertes 26.458571428571428 +fred miller 29.66666666666667 +fred nixon 19.565555555555555 +fred nixon 21.830000000000002 +fred nixon 25.828333333333333 +fred nixon 28.094285714285718 +fred polk 21.744999999999997 +fred polk 22.301666666666666 +fred polk 24.156666666666666 +fred polk 28.217142857142857 +fred quirinius 27.495 +fred quirinius 29.615714285714283 +fred robinson 24.243636363636366 +fred steinbeck 17.91333333333333 +fred steinbeck 21.12375 +fred steinbeck 26.47125 +fred underhill 26.43166666666667 +fred van buren 16.165714285714284 +fred van buren 23.285 +fred van buren 24.875714285714288 +fred van buren 27.878333333333334 +fred white 29.498571428571427 +fred young 17.889999999999997 +fred young 19.565555555555555 +fred zipper 21.581428571428575 +gabriella allen 22.03142857142857 +gabriella allen 26.87375 +gabriella brown 29.675714285714285 +gabriella brown 32.44 +gabriella carson 28.75 +gabriella davidson 27.531250000000004 +gabriella ellison 19.82 +gabriella ellison 27.353333333333335 +gabriella falkner 17.535 +gabriella falkner 19.487500000000004 +gabriella falkner 30.964999999999996 +gabriella garcia 20.544 +gabriella hernandez 20.818333333333335 +gabriella hernandez 28.094285714285718 +gabriella ichabod 10.58 +gabriella ichabod 18.64272727272727 +gabriella ichabod 20.504444444444445 +gabriella ichabod 23.185 +gabriella ichabod 23.35375 +gabriella king 16.18 +gabriella king 27.887500000000003 +gabriella laertes 23.799090909090907 +gabriella miller 15.695714285714283 +gabriella ovid 23.515454545454546 +gabriella ovid 33.33 +gabriella polk 20.38714285714286 +gabriella polk 35.77285714285715 +gabriella steinbeck 16.18 +gabriella steinbeck 32.464999999999996 +gabriella thompson 26.477777777777778 +gabriella thompson 27.29777777777778 +gabriella thompson 30.87666666666667 +gabriella van buren 28.513333333333335 +gabriella van buren 32.41111111111111 +gabriella white 26.765 +gabriella young 24.022499999999997 +gabriella young 29.521666666666665 +gabriella zipper 21.43727272727273 +gabriella zipper 32.106 +holly allen 24.271428571428572 +holly brown 22.959 +holly brown 27.498 +holly falkner 29.66666666666667 +holly hernandez 14.179999999999998 +holly hernandez 22.396666666666665 +holly hernandez 27.434000000000005 +holly hernandez 27.887500000000003 +holly ichabod 27.87375 +holly ichabod 32.525 +holly ichabod 34.042857142857144 +holly johnson 20.808 +holly johnson 25.024285714285718 +holly johnson 30.487142857142857 +holly king 23.185 +holly king 29.008888888888894 +holly laertes 19.41 +holly miller 29.89333333333333 +holly nixon 23.162857142857142 +holly nixon 28.876250000000002 +holly polk 22.7825 +holly polk 26.297499999999996 +holly robinson 24.160000000000004 +holly thompson 19.565555555555555 +holly thompson 27.048999999999996 +holly thompson 29.70555555555556 +holly underhill 17.876250000000002 +holly underhill 27.820000000000004 +holly underhill 30.613999999999997 +holly underhill 30.708 +holly van buren 20.113333333333333 +holly white 23.185 +holly white 29.64272727272727 +holly xylophone 26.400909090909092 +holly young 27.807777777777776 +holly young 31.63 +holly zipper 27.401999999999997 +holly zipper 28.384285714285713 +irene allen 35.345 +irene brown 22.527500000000003 +irene brown 28.384285714285713 +irene brown 32.81875 +irene carson 27.10666666666667 +irene ellison 16.720000000000002 +irene ellison 28.592857142857145 +irene falkner 19.41 +irene falkner 30.564999999999998 +irene garcia 16.9925 +irene garcia 24.03222222222222 +irene garcia 26.297499999999996 +irene ichabod 27.540000000000003 +irene ichabod 29.34875 +irene johnson 25.418181818181814 +irene laertes 22.124285714285712 +irene laertes 22.200000000000003 +irene laertes 24.446666666666665 +irene miller 30.166666666666668 +irene nixon 18.922222222222224 +irene nixon 25.2025 +irene nixon 33.382 +irene ovid 28.256249999999998 +irene ovid 31.63 +irene ovid 32.088750000000005 +irene polk 24.63 +irene polk 25.296363636363637 +irene polk 27.137142857142855 +irene polk 30.65222222222222 +irene polk 33.760000000000005 +irene quirinius 27.044999999999998 +irene quirinius 33.07833333333334 +irene quirinius 41.865 +irene robinson 32.18875 +irene steinbeck 16.463 +irene thompson 25.281666666666666 +irene underhill 24.4025 +irene underhill 28.531 +irene van buren 25.532222222222224 +irene van buren 32.50142857142857 +irene xylophone 26.288181818181815 +jessica brown 28.968181818181822 +jessica carson 19.41 +jessica carson 24.854285714285716 +jessica carson 25.406363636363633 +jessica davidson 22.19625 +jessica davidson 23.888 +jessica davidson 26.297499999999996 +jessica davidson 26.825454545454537 +jessica ellison 22.07777777777778 +jessica ellison 33.33 +jessica falkner 22.637272727272727 +jessica garcia 14.749999999999996 +jessica garcia 29.675714285714285 +jessica ichabod 31.831249999999997 +jessica johnson 21.546666666666667 +jessica johnson 29.986363636363638 +jessica miller 28.735000000000003 +jessica nixon 19.13111111111111 +jessica nixon 26.244999999999997 +jessica ovid 25.274 +jessica ovid 33.181666666666665 +jessica polk 26.79222222222222 +jessica quirinius 20.38714285714286 +jessica quirinius 25.776000000000003 +jessica quirinius 28.26 +jessica quirinius 29.605000000000008 +jessica robinson 24.5625 +jessica thompson 25.736 +jessica thompson 30.87363636363636 +jessica underhill 16.400000000000002 +jessica underhill 25.529090909090915 +jessica underhill 31.63 +jessica van buren 24.446666666666665 +jessica white 20.812 +jessica white 23.26 +jessica white 27.807777777777776 +jessica white 29.031000000000006 +jessica white 30.654545454545453 +jessica xylophone 15.296666666666667 +jessica young 26.718333333333334 +jessica young 27.853749999999998 +jessica zipper 20.3575 +jessica zipper 24.446666666666665 +jessica zipper 29.276363636363637 +katie allen 29.029090909090915 +katie brown 24.156666666666666 +katie davidson 15.383749999999997 +katie ellison 20.978333333333335 +katie ellison 26.96 +katie falkner 24.5625 +katie garcia 27.807777777777776 +katie garcia 28.287142857142857 +katie hernandez 23.667272727272724 +katie ichabod 14.222 +katie ichabod 20.553749999999997 +katie ichabod 31.831249999999997 +katie king 20.05444444444445 +katie king 20.242222222222225 +katie king 23.342857142857145 +katie miller 26.21857142857143 +katie miller 27.675000000000004 +katie nixon 14.476999999999999 +katie ovid 28.37875 +katie polk 20.99 +katie polk 25.090000000000003 +katie robinson 30.65222222222222 +katie van buren 27.133636363636366 +katie van buren 29.675714285714285 +katie white 22.555714285714288 +katie white 24.463749999999997 +katie xylophone 25.74142857142857 +katie young 23.011250000000004 +katie young 26.650000000000002 +katie young 29.301428571428573 +katie zipper 28.26 +katie zipper 29.675714285714285 +luke allen 16.9925 +luke allen 20.595000000000002 +luke allen 27.54181818181818 +luke allen 27.887500000000003 +luke allen 33.07833333333334 +luke brown 29.34875 +luke davidson 27.3575 +luke davidson 31.473333333333333 +luke ellison 14.024444444444443 +luke ellison 22.555714285714288 +luke ellison 28.592857142857145 +luke falkner 21.855 +luke falkner 27.044999999999998 +luke garcia 27.887500000000003 +luke garcia 31.237000000000002 +luke ichabod 28.198571428571427 +luke ichabod 34.345000000000006 +luke johnson 21.239999999999995 +luke johnson 25.462727272727275 +luke johnson 30.188888888888894 +luke laertes 18.344 +luke laertes 20.817 +luke laertes 27.401999999999997 +luke laertes 30.72285714285714 +luke laertes 41.865 +luke miller 22.539000000000005 +luke ovid 16.615454545454543 +luke ovid 26.06625 +luke polk 28.163000000000004 +luke polk 28.840909090909097 +luke quirinius 27.077142857142857 +luke robinson 24.816363636363644 +luke robinson 27.110909090909093 +luke thompson 28.44454545454545 +luke underhill 22.175 +luke underhill 25.518888888888892 +luke underhill 27.34125 +luke van buren 16.54 +luke white 20.544 +luke xylophone 24.5625 +luke zipper 24.764285714285712 +mike allen 23.860000000000003 +mike brown 31.016363636363643 +mike carson 26.066363636363636 +mike carson 28.947142857142858 +mike carson 33.382 +mike davidson 20.544 +mike davidson 21.239999999999995 +mike ellison 18.922 +mike ellison 21.4175 +mike ellison 25.45272727272727 +mike ellison 27.26 +mike ellison 28.39888888888889 +mike falkner 29.397777777777776 +mike garcia 20.544 +mike garcia 24.582 +mike garcia 34.84571428571429 +mike hernandez 10.4925 +mike hernandez 17.7 +mike ichabod 26.772727272727273 +mike king 17.889999999999997 +mike king 19.294999999999998 +mike king 20.004285714285714 +mike king 23.197999999999997 +mike king 23.285 +mike king 27.401999999999997 +mike miller 31.587272727272726 +mike nixon 17.775555555555556 +mike nixon 27.044999999999998 +mike polk 22.175 +mike polk 23.751428571428573 +mike polk 28.095 +mike quirinius 19.13111111111111 +mike steinbeck 14.222 +mike steinbeck 18.100909090909088 +mike steinbeck 18.344 +mike steinbeck 33.760000000000005 +mike van buren 23.42 +mike van buren 25.828333333333333 +mike white 19.13111111111111 +mike white 25.755714285714284 +mike white 29.031000000000006 +mike white 30.516999999999996 +mike young 26.765 +mike young 27.766 +mike young 28.409090909090914 +mike zipper 17.306 +mike zipper 33.23 +mike zipper 41.865 +nick allen 19.331666666666667 +nick allen 32.106 +nick brown 27.578181818181818 +nick davidson 29.100000000000005 +nick ellison 24.764285714285712 +nick ellison 29.521666666666665 +nick falkner 22.555714285714288 +nick falkner 23.15888888888888 +nick garcia 21.546666666666667 +nick garcia 26.25090909090909 +nick garcia 30.166666666666668 +nick ichabod 21.855 +nick ichabod 23.479000000000003 +nick ichabod 29.100000000000005 +nick johnson 25.274 +nick johnson 29.994 +nick laertes 25.820909090909094 +nick miller 19.87888888888889 +nick nixon 17.082 +nick ovid 33.597777777777786 +nick polk 25.736 +nick quirinius 20.707500000000003 +nick quirinius 28.094285714285718 +nick robinson 22.396666666666665 +nick robinson 25.298749999999995 +nick steinbeck 21.192857142857143 +nick thompson 30.72285714285714 +nick underhill 29.345 +nick van buren 25.152727272727272 +nick xylophone 26.948333333333334 +nick young 23.751428571428573 +nick young 24.810000000000002 +nick zipper 24.854285714285716 +nick zipper 27.353333333333335 +oscar allen 18.815 +oscar brown 26.948333333333334 +oscar carson 24.764285714285712 +oscar carson 27.766 +oscar carson 28.094285714285718 +oscar carson 28.31555555555556 +oscar carson 35.22818181818182 +oscar davidson 17.535 +oscar ellison 22.121428571428574 +oscar ellison 28.735000000000003 +oscar falkner 19.294999999999998 +oscar garcia 20.62636363636364 +oscar hernandez 22.539000000000005 +oscar hernandez 23.31888888888889 +oscar ichabod 20.818333333333335 +oscar ichabod 21.546666666666667 +oscar ichabod 26.914545454545454 +oscar ichabod 28.811111111111106 +oscar johnson 22.381818181818183 +oscar johnson 24.266363636363643 +oscar king 15.296666666666667 +oscar king 25.580000000000002 +oscar king 28.37875 +oscar laertes 21.51818181818182 +oscar laertes 23.285 +oscar laertes 24.4025 +oscar laertes 25.345454545454547 +oscar nixon 18.88111111111111 +oscar ovid 24.854285714285716 +oscar ovid 25.274 +oscar ovid 33.29636363636364 +oscar polk 19.331666666666667 +oscar polk 29.34875 +oscar quirinius 22.928 +oscar quirinius 25.66727272727273 +oscar quirinius 25.970909090909092 +oscar quirinius 29.66666666666667 +oscar robinson 20.90666666666667 +oscar robinson 21.855 +oscar robinson 23.42 +oscar robinson 32.90500000000001 +oscar steinbeck 32.02818181818182 +oscar thompson 20.817 +oscar thompson 21.477000000000004 +oscar thompson 21.843636363636367 +oscar thompson 23.559000000000005 +oscar underhill 22.555714285714288 +oscar van buren 27.210000000000008 +oscar van buren 28.592857142857145 +oscar van buren 31.375454545454545 +oscar white 20.818333333333335 +oscar white 21.748 +oscar white 24.582 +oscar white 28.287142857142857 +oscar xylophone 25.845 +oscar xylophone 28.735000000000003 +oscar xylophone 30.72285714285714 +oscar zipper 24.511111111111113 +oscar zipper 25.067777777777778 +oscar zipper 26.21857142857143 +priscilla brown 14.222 +priscilla brown 27.044999999999998 +priscilla brown 30.19909090909091 +priscilla carson 18.07 +priscilla carson 20.70875 +priscilla carson 26.297499999999996 +priscilla ichabod 29.451111111111118 +priscilla ichabod 29.878888888888884 +priscilla johnson 16.9925 +priscilla johnson 22.050000000000004 +priscilla johnson 24.093000000000004 +priscilla johnson 29.200000000000003 +priscilla johnson 29.246 +priscilla king 15.536666666666669 +priscilla nixon 18.9 +priscilla nixon 30.516999999999996 +priscilla ovid 16.005000000000003 +priscilla ovid 29.88111111111111 +priscilla polk 28.018888888888892 +priscilla quirinius 23.064999999999998 +priscilla thompson 27.077142857142857 +priscilla underhill 28.160999999999994 +priscilla underhill 28.56571428571429 +priscilla van buren 20.7 +priscilla van buren 21.830000000000002 +priscilla van buren 24.665 +priscilla white 29.23375 +priscilla xylophone 19.331666666666667 +priscilla xylophone 23.185 +priscilla xylophone 27.34125 +priscilla young 26.32777777777778 +priscilla young 30.613999999999997 +priscilla zipper 13.498 +priscilla zipper 31.972727272727266 +quinn allen 28.786666666666672 +quinn allen 29.471818181818175 +quinn brown 26.314285714285713 +quinn brown 27.38 +quinn brown 28.39888888888889 +quinn davidson 20.808 +quinn davidson 22.71285714285714 +quinn davidson 23.306250000000002 +quinn davidson 27.34125 +quinn ellison 25.002857142857142 +quinn ellison 33.760000000000005 +quinn garcia 20.544 +quinn garcia 27.055999999999994 +quinn garcia 29.183333333333334 +quinn garcia 31.831249999999997 +quinn ichabod 22.101818181818185 +quinn king 17.535 +quinn king 18.035714285714285 +quinn laertes 24.511111111111113 +quinn laertes 28.876250000000002 +quinn laertes 29.202857142857145 +quinn nixon 21.75111111111111 +quinn ovid 29.64125 +quinn quirinius 18.922 +quinn robinson 23.985 +quinn steinbeck 27.077142857142857 +quinn steinbeck 28.160999999999994 +quinn thompson 21.99888888888889 +quinn thompson 34.46857142857143 +quinn underhill 20.113333333333333 +quinn underhill 23.107272727272726 +quinn underhill 26.224285714285717 +quinn van buren 23.612222222222222 +quinn young 24.5625 +quinn zipper 17.889999999999997 +quinn zipper 20.163333333333338 +rachel allen 28.446666666666665 +rachel allen 41.865 +rachel brown 20.92875 +rachel brown 26.21857142857143 +rachel brown 27.905 +rachel brown 30.166666666666668 +rachel brown 35.345 +rachel carson 28.735000000000003 +rachel carson 31.715999999999998 +rachel davidson 29.100000000000005 +rachel ellison 27.055454545454552 +rachel falkner 14.812499999999998 +rachel falkner 28.876250000000002 +rachel falkner 29.308888888888887 +rachel falkner 31.831249999999997 +rachel johnson 31.63 +rachel king 24.511111111111113 +rachel king 30.873749999999998 +rachel laertes 17.306 +rachel laertes 26.765 +rachel ovid 24.042727272727276 +rachel ovid 28.01181818181818 +rachel polk 21.12375 +rachel quirinius 29.831249999999997 +rachel robinson 10.4925 +rachel robinson 22.264444444444447 +rachel robinson 33.43125 +rachel thompson 16.720000000000002 +rachel thompson 26.905714285714286 +rachel thompson 28.876250000000002 +rachel underhill 21.75111111111111 +rachel white 22.175 +rachel white 31.580000000000002 +rachel young 28.150000000000002 +rachel zipper 22.187142857142856 +rachel zipper 33.760000000000005 +sarah carson 21.86818181818182 +sarah carson 22.175 +sarah carson 33.43125 +sarah ellison 17.535 +sarah falkner 29.34875 +sarah falkner 29.64125 +sarah garcia 10.4925 +sarah garcia 20.812 +sarah garcia 28.2175 +sarah ichabod 26.948333333333334 +sarah ichabod 33.62375 +sarah johnson 21.546666666666667 +sarah johnson 24.978000000000005 +sarah johnson 29.608000000000004 +sarah johnson 33.760000000000005 +sarah king 19.41 +sarah king 27.055999999999994 +sarah miller 24.815454545454543 +sarah ovid 28.31625 +sarah robinson 13.498 +sarah robinson 28.256249999999998 +sarah steinbeck 23.26 +sarah white 21.75111111111111 +sarah white 31.63 +sarah xylophone 21.964545454545455 +sarah young 29.335555555555555 +sarah zipper 29.521666666666665 +tom brown 22.873333333333335 +tom brown 30.415555555555557 +tom carson 22.4025 +tom carson 28.39888888888889 +tom carson 29.64125 +tom davidson 30.61142857142857 +tom ellison 23.568 +tom ellison 27.884999999999998 +tom ellison 32.02625 +tom falkner 13.72 +tom falkner 19.849999999999998 +tom hernandez 16.720000000000002 +tom hernandez 29.974285714285713 +tom ichabod 20.113333333333333 +tom johnson 27.077142857142857 +tom johnson 32.90500000000001 +tom king 21.855 +tom laertes 17.981666666666666 +tom laertes 21.80857142857143 +tom miller 18.922 +tom miller 21.239999999999995 +tom miller 22.396666666666665 +tom nixon 27.005000000000003 +tom ovid 34.84571428571429 +tom polk 29.521666666666665 +tom polk 29.805 +tom quirinius 24.764285714285712 +tom quirinius 36.46857142857143 +tom robinson 16.18 +tom robinson 18.07 +tom robinson 27.34125 +tom robinson 34.958571428571425 +tom steinbeck 30.613999999999997 +tom van buren 22.6475 +tom van buren 23.13555555555556 +tom van buren 24.4025 +tom white 27.715714285714284 +tom young 19.41 +tom young 24.63 +tom zipper 22.902 +ulysses brown 16.223333333333333 +ulysses carson 16.3475 +ulysses carson 22.057500000000005 +ulysses carson 28.256249999999998 +ulysses carson 28.27285714285714 +ulysses davidson 24.701249999999998 +ulysses ellison 29.52333333333333 +ulysses garcia 33.382 +ulysses hernandez 18.421818181818185 +ulysses hernandez 20.443749999999998 +ulysses hernandez 22.365 +ulysses ichabod 24.63 +ulysses ichabod 33.24333333333333 +ulysses johnson 33.43125 +ulysses king 27.083333333333332 +ulysses laertes 26.915000000000003 +ulysses laertes 27.305 +ulysses laertes 28.501111111111115 +ulysses miller 18.22 +ulysses miller 26.21857142857143 +ulysses nixon 30.65222222222222 +ulysses ovid 21.366666666666667 +ulysses polk 22.555714285714288 +ulysses polk 22.66625 +ulysses polk 25.11777777777778 +ulysses polk 27.141666666666666 +ulysses quirinius 33.07833333333334 +ulysses robinson 21.12375 +ulysses steinbeck 23.751428571428573 +ulysses steinbeck 25.931428571428572 +ulysses thompson 22.264444444444447 +ulysses underhill 20.812 +ulysses underhill 23.751428571428573 +ulysses underhill 25.071666666666662 +ulysses underhill 25.828333333333333 +ulysses underhill 25.865 +ulysses underhill 28.722499999999997 +ulysses underhill 35.268888888888895 +ulysses van buren 22.134999999999998 +ulysses white 15.296666666666667 +ulysses white 32.093333333333334 +ulysses xylophone 20.38714285714286 +ulysses xylophone 25.274 +ulysses xylophone 29.64125 +ulysses young 22.213333333333335 +ulysses young 22.90285714285714 +ulysses young 32.93125 +victor allen 24.82875 +victor allen 27.51 +victor brown 21.621250000000003 +victor brown 23.73 +victor brown 26.21857142857143 +victor brown 27.548571428571428 +victor davidson 22.391666666666666 +victor davidson 33.16428571428572 +victor davidson 35.197500000000005 +victor ellison 11.100000000000001 +victor ellison 30.96857142857143 +victor hernandez 10.4925 +victor hernandez 18.922 +victor hernandez 24.301250000000003 +victor hernandez 26.69857142857143 +victor hernandez 35.358333333333334 +victor johnson 16.580000000000002 +victor johnson 27.516666666666666 +victor johnson 32.106 +victor king 19.962857142857143 +victor king 33.01857142857143 +victor laertes 21.78142857142857 +victor laertes 33.10999999999999 +victor miller 21.93285714285714 +victor nixon 20.419999999999998 +victor nixon 33.69 +victor ovid 28.75857142857143 +victor polk 18.43111111111111 +victor quirinius 17.84777777777778 +victor quirinius 27.53 +victor robinson 19.37 +victor robinson 20.38714285714286 +victor steinbeck 20.818333333333335 +victor steinbeck 25.16 +victor steinbeck 30.503749999999997 +victor thompson 23.987142857142857 +victor van buren 27.009999999999998 +victor van buren 33.43125 +victor white 24.322857142857142 +victor white 28.287142857142857 +victor xylophone 11.807142857142859 +victor xylophone 13.988571428571428 +victor xylophone 16.720000000000002 +victor xylophone 19.686666666666667 +victor xylophone 37.20428571428572 +victor young 22.264444444444447 +victor zipper 24.854285714285716 +wendy allen 28.24142857142857 +wendy allen 29.675714285714285 +wendy allen 34.275 +wendy brown 22.482857142857142 +wendy brown 27.79714285714286 +wendy ellison 16.80888888888889 +wendy ellison 18.135 +wendy falkner 22.628888888888884 +wendy falkner 23.325000000000003 +wendy falkner 24.0375 +wendy garcia 19.307142857142853 +wendy garcia 21.761428571428574 +wendy garcia 24.63 +wendy garcia 24.854285714285716 +wendy hernandez 16.60875 +wendy ichabod 28.26 +wendy king 22.5 +wendy king 24.793333333333333 +wendy king 28.252857142857145 +wendy laertes 25.881428571428568 +wendy laertes 30.338333333333328 +wendy laertes 30.52857142857143 +wendy miller 15.478333333333332 +wendy miller 25.34333333333333 +wendy nixon 19.54714285714286 +wendy nixon 27.003333333333334 +wendy ovid 14.283750000000001 +wendy ovid 30.878333333333334 +wendy polk 21.69375 +wendy polk 24.63 +wendy quirinius 28.731428571428573 +wendy quirinius 29.74333333333333 +wendy robinson 16.720000000000002 +wendy robinson 23.834285714285716 +wendy robinson 29.911666666666672 +wendy steinbeck 29.272857142857145 +wendy thompson 18.17875 +wendy thompson 22.544285714285714 +wendy underhill 21.69625 +wendy underhill 27.077142857142857 +wendy underhill 30.03333333333333 +wendy van buren 28.624285714285715 +wendy van buren 29.28333333333333 +wendy white 24.4025 +wendy xylophone 16.84 +wendy xylophone 23.426666666666666 +wendy young 20.80125 +wendy young 32.693333333333335 +xavier allen 16.535 +xavier allen 17.398333333333337 +xavier allen 35.708333333333336 +xavier brown 20.787142857142857 +xavier brown 24.764285714285712 +xavier brown 31.784999999999997 +xavier carson 20.818333333333335 +xavier carson 32.106 +xavier davidson 16.862857142857145 +xavier davidson 20.53625 +xavier davidson 27.353333333333335 +xavier ellison 17.991666666666667 +xavier ellison 23.976666666666663 +xavier garcia 35.84428571428572 +xavier hernandez 22.654285714285713 +xavier hernandez 26.948333333333334 +xavier hernandez 28.075 +xavier ichabod 20.344285714285714 +xavier ichabod 20.818333333333335 +xavier johnson 15.754285714285714 +xavier johnson 19.490000000000002 +xavier king 29.246666666666666 +xavier king 29.521666666666665 +xavier laertes 19.294999999999998 +xavier ovid 28.51 +xavier polk 12.728333333333333 +xavier polk 19.37 +xavier polk 22.548333333333332 +xavier polk 28.465714285714284 +xavier quirinius 9.991428571428571 +xavier quirinius 24.156666666666666 +xavier quirinius 25.69666666666667 +xavier quirinius 25.828333333333333 +xavier thompson 23.961428571428574 +xavier underhill 21.830000000000002 +xavier white 19.331666666666667 +xavier white 35.345 +xavier xylophone 21.187142857142856 +xavier zipper 17.488333333333333 +yuri allen 15.705714285714285 +yuri allen 20.808 +yuri brown 19.53 +yuri brown 22.457142857142856 +yuri carson 25.699999999999996 +yuri carson 27.216666666666665 +yuri ellison 15.034999999999998 +yuri ellison 28.463333333333335 +yuri falkner 17.81833333333333 +yuri falkner 19.294999999999998 +yuri garcia 28.287142857142857 +yuri hernandez 32.395 +yuri johnson 25.828333333333333 +yuri johnson 27.301666666666666 +yuri johnson 29.578333333333337 +yuri king 19.921666666666663 +yuri laertes 16.18 +yuri laertes 30.519999999999996 +yuri nixon 16.383333333333333 +yuri nixon 25.828333333333333 +yuri polk 16.18 +yuri polk 20.503333333333334 +yuri polk 30.16333333333333 +yuri quirinius 20.311666666666667 +yuri quirinius 23.185 +yuri quirinius 24.828333333333333 +yuri steinbeck 19.331666666666667 +yuri steinbeck 28.50666666666667 +yuri thompson 35.27 +yuri underhill 23.042857142857144 +yuri underhill 28.786666666666665 +yuri white 30.72285714285714 +yuri xylophone 24.173333333333332 +zach allen 8.983333333333333 +zach brown 18.922 +zach brown 23.036666666666665 +zach brown 29.72666666666667 +zach brown 31.58285714285714 +zach brown 33.07833333333334 +zach carson 27.110000000000003 +zach ellison 18.168333333333333 +zach falkner 16.18 +zach falkner 30.83285714285714 +zach garcia 16.586666666666666 +zach garcia 22.53333333333333 +zach garcia 28.13166666666667 +zach garcia 34.84571428571429 +zach ichabod 17.535 +zach ichabod 30.72285714285714 +zach king 19.878333333333334 +zach king 25.643333333333334 +zach king 28.646666666666665 +zach miller 23.285 +zach miller 23.366666666666664 +zach miller 30.46833333333333 +zach ovid 23.94666666666667 +zach ovid 28.75166666666667 +zach ovid 28.763333333333335 +zach ovid 34.84571428571429 +zach quirinius 20.755 +zach robinson 21.546666666666667 +zach steinbeck 27.243333333333336 +zach steinbeck 30.073333333333334 +zach thompson 14.222 +zach thompson 24.755 +zach underhill 31.885 +zach white 20.208333333333332 +zach xylophone 10.485 +zach xylophone 20.113333333333333 +zach young 20.176666666666666 +zach zipper 21.709999999999997 +zach zipper 22.264999999999997 +zach zipper 34.01166666666667 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-5-ee44c5cdc80e1c832b702f9fb76d8145 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-5-ee44c5cdc80e1c832b702f9fb76d8145 new file mode 100644 index 0000000000000000000000000000000000000000..a9ae190825a0113f4e04c34599022e6b47c6b298 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-5-ee44c5cdc80e1c832b702f9fb76d8145 @@ -0,0 +1,1049 @@ + 65560 + 65718 + 65740 +alice allen 65662 +alice allen 65720 +alice allen 65758 +alice brown 65696 +alice carson 65559 +alice davidson 65547 +alice falkner 65669 +alice garcia 65613 +alice hernandez 65737 +alice hernandez 65784 +alice johnson 65739 +alice king 65660 +alice king 65738 +alice king 131281 +alice laertes 65669 +alice laertes 65671 +alice miller 65590 +alice nixon 65586 +alice nixon 65595 +alice nixon 65604 +alice ovid 65737 +alice polk 65548 +alice quirinius 65636 +alice quirinius 65728 +alice robinson 65606 +alice robinson 65789 +alice steinbeck 65578 +alice steinbeck 65673 +alice steinbeck 65786 +alice underhill 65750 +alice van buren 65562 +alice xylophone 65585 +alice xylophone 65599 +alice xylophone 131247 +alice zipper 65553 +alice zipper 65662 +alice zipper 65766 +bob brown 65584 +bob brown 65777 +bob brown 65783 +bob carson 65713 +bob davidson 65664 +bob davidson 65693 +bob davidson 65768 +bob ellison 65591 +bob ellison 65624 +bob ellison 65721 +bob ellison 65760 +bob falkner 65789 +bob garcia 65585 +bob garcia 65598 +bob garcia 65673 +bob garcia 65754 +bob garcia 65782 +bob hernandez 131340 +bob ichabod 65549 +bob king 65715 +bob king 65757 +bob king 65783 +bob laertes 65602 +bob laertes 131447 +bob miller 65608 +bob ovid 65564 +bob ovid 65686 +bob ovid 196959 +bob ovid 196973 +bob polk 65594 +bob quirinius 65700 +bob steinbeck 65637 +bob van buren 65778 +bob white 65543 +bob white 65605 +bob xylophone 65574 +bob xylophone 65666 +bob young 65556 +bob zipper 65559 +bob zipper 65633 +bob zipper 65739 +calvin allen 65669 +calvin brown 65537 +calvin brown 131272 +calvin brown 197027 +calvin carson 65637 +calvin davidson 65541 +calvin davidson 65564 +calvin ellison 65667 +calvin falkner 65573 +calvin falkner 65596 +calvin falkner 65778 +calvin falkner 131397 +calvin falkner 131411 +calvin falkner 131433 +calvin garcia 131212 +calvin hernandez 131251 +calvin johnson 65731 +calvin laertes 65570 +calvin laertes 65684 +calvin nixon 65654 +calvin nixon 131386 +calvin nixon 131503 +calvin ovid 65554 +calvin ovid 65643 +calvin ovid 65715 +calvin ovid 196944 +calvin polk 65731 +calvin quirinius 65741 +calvin quirinius 65769 +calvin robinson 131320 +calvin steinbeck 131271 +calvin steinbeck 131326 +calvin steinbeck 131415 +calvin thompson 65560 +calvin thompson 131244 +calvin underhill 196944 +calvin van buren 65771 +calvin van buren 131138 +calvin white 65553 +calvin white 65561 +calvin xylophone 65575 +calvin xylophone 65596 +calvin xylophone 262686 +calvin young 65746 +calvin young 131168 +calvin zipper 65669 +calvin zipper 131476 +david allen 65588 +david allen 131222 +david brown 65637 +david brown 131303 +david davidson 65756 +david davidson 65778 +david davidson 65779 +david davidson 131342 +david ellison 65724 +david ellison 65724 +david ellison 131224 +david hernandez 197083 +david ichabod 131454 +david ichabod 197085 +david laertes 65762 +david nixon 65536 +david ovid 65623 +david ovid 196766 +david quirinius 65759 +david quirinius 65779 +david quirinius 131303 +david robinson 65762 +david robinson 65775 +david thompson 65550 +david underhill 65662 +david underhill 65751 +david underhill 131198 +david van buren 65634 +david van buren 262584 +david white 65678 +david xylophone 65537 +david xylophone 131426 +david xylophone 131447 +david young 65551 +david young 131255 +ethan allen 131460 +ethan brown 65539 +ethan brown 65617 +ethan brown 65685 +ethan brown 65685 +ethan brown 65722 +ethan brown 131483 +ethan carson 197189 +ethan ellison 65714 +ethan ellison 131302 +ethan falkner 131222 +ethan falkner 131333 +ethan garcia 131507 +ethan hernandez 65618 +ethan johnson 65536 +ethan king 131280 +ethan laertes 65562 +ethan laertes 65597 +ethan laertes 65628 +ethan laertes 65680 +ethan laertes 65760 +ethan laertes 131304 +ethan laertes 328329 +ethan miller 328296 +ethan nixon 65766 +ethan ovid 65697 +ethan polk 65589 +ethan polk 65615 +ethan polk 131206 +ethan polk 197082 +ethan quirinius 65591 +ethan quirinius 196912 +ethan quirinius 196957 +ethan robinson 65547 +ethan robinson 65659 +ethan underhill 65570 +ethan van buren 131252 +ethan white 65677 +ethan white 197039 +ethan xylophone 65595 +ethan zipper 65593 +ethan zipper 131365 +fred davidson 65595 +fred davidson 65721 +fred davidson 131221 +fred ellison 65548 +fred ellison 65691 +fred ellison 65771 +fred falkner 65637 +fred falkner 131474 +fred falkner 196920 +fred hernandez 131226 +fred ichabod 131109 +fred ichabod 131520 +fred johnson 131332 +fred king 65694 +fred king 197016 +fred laertes 131354 +fred miller 65536 +fred nixon 65560 +fred nixon 65612 +fred nixon 65705 +fred nixon 196929 +fred polk 65656 +fred polk 131231 +fred polk 262645 +fred polk 262733 +fred quirinius 131486 +fred quirinius 196950 +fred robinson 65623 +fred steinbeck 65544 +fred steinbeck 65755 +fred steinbeck 131253 +fred underhill 131188 +fred van buren 65561 +fred van buren 65745 +fred van buren 131380 +fred van buren 328270 +fred white 131136 +fred young 65594 +fred young 131551 +fred zipper 196885 +gabriella allen 65677 +gabriella allen 131283 +gabriella brown 65753 +gabriella brown 197180 +gabriella carson 65586 +gabriella davidson 65565 +gabriella ellison 65706 +gabriella ellison 131505 +gabriella falkner 65767 +gabriella falkner 131183 +gabriella falkner 131397 +gabriella garcia 131127 +gabriella hernandez 131304 +gabriella hernandez 131304 +gabriella ichabod 65559 +gabriella ichabod 65712 +gabriella ichabod 131297 +gabriella ichabod 131311 +gabriella ichabod 131460 +gabriella king 65657 +gabriella king 197031 +gabriella laertes 131543 +gabriella miller 131300 +gabriella ovid 65556 +gabriella ovid 131260 +gabriella polk 65790 +gabriella polk 131425 +gabriella steinbeck 65582 +gabriella steinbeck 131248 +gabriella thompson 131528 +gabriella thompson 197181 +gabriella thompson 262632 +gabriella van buren 65644 +gabriella van buren 131238 +gabriella white 65638 +gabriella young 65699 +gabriella young 65774 +gabriella zipper 65754 +gabriella zipper 196762 +holly allen 65596 +holly brown 131315 +holly brown 131368 +holly falkner 65720 +holly hernandez 65602 +holly hernandez 65686 +holly hernandez 131387 +holly hernandez 131554 +holly ichabod 65752 +holly ichabod 131308 +holly ichabod 131473 +holly johnson 65755 +holly johnson 131240 +holly johnson 131277 +holly king 131286 +holly king 131303 +holly laertes 196950 +holly miller 131381 +holly nixon 196941 +holly nixon 328184 +holly polk 197132 +holly polk 262782 +holly robinson 131241 +holly thompson 65578 +holly thompson 65713 +holly thompson 197092 +holly underhill 65654 +holly underhill 131323 +holly underhill 131385 +holly underhill 131504 +holly van buren 131449 +holly white 131092 +holly white 262734 +holly xylophone 196792 +holly young 65765 +holly young 131229 +holly zipper 131151 +holly zipper 131545 +irene allen 131109 +irene brown 65765 +irene brown 131368 +irene brown 393929 +irene carson 262770 +irene ellison 196956 +irene ellison 196982 +irene falkner 131287 +irene falkner 197046 +irene garcia 65660 +irene garcia 131286 +irene garcia 131375 +irene ichabod 65645 +irene ichabod 131442 +irene johnson 131179 +irene laertes 131324 +irene laertes 131381 +irene laertes 131407 +irene miller 262822 +irene nixon 197105 +irene nixon 262409 +irene nixon 262565 +irene ovid 65734 +irene ovid 196935 +irene ovid 262836 +irene polk 65551 +irene polk 131189 +irene polk 131189 +irene polk 196943 +irene polk 328365 +irene quirinius 131369 +irene quirinius 196998 +irene quirinius 262855 +irene robinson 131259 +irene steinbeck 65683 +irene thompson 262719 +irene underhill 131291 +irene underhill 131386 +irene van buren 131216 +irene van buren 262539 +irene xylophone 131348 +jessica brown 393772 +jessica carson 65747 +jessica carson 131207 +jessica carson 131232 +jessica davidson 65606 +jessica davidson 65675 +jessica davidson 196917 +jessica davidson 197030 +jessica ellison 131108 +jessica ellison 196885 +jessica falkner 131270 +jessica garcia 197059 +jessica garcia 328458 +jessica ichabod 197028 +jessica johnson 131177 +jessica johnson 197085 +jessica miller 197024 +jessica nixon 131549 +jessica nixon 196682 +jessica ovid 65751 +jessica ovid 196890 +jessica polk 459409 +jessica quirinius 131222 +jessica quirinius 131248 +jessica quirinius 131294 +jessica quirinius 393878 +jessica robinson 131174 +jessica thompson 131336 +jessica thompson 196927 +jessica underhill 131218 +jessica underhill 131267 +jessica underhill 197086 +jessica van buren 65615 +jessica white 65544 +jessica white 65594 +jessica white 197012 +jessica white 262435 +jessica white 262571 +jessica xylophone 196866 +jessica young 65711 +jessica young 131183 +jessica zipper 196897 +jessica zipper 262523 +jessica zipper 262695 +katie allen 196740 +katie brown 328113 +katie davidson 131371 +katie ellison 131248 +katie ellison 197182 +katie falkner 131441 +katie garcia 131384 +katie garcia 197051 +katie hernandez 131296 +katie ichabod 131495 +katie ichabod 197131 +katie ichabod 197275 +katie king 131252 +katie king 262588 +katie king 262861 +katie miller 65661 +katie miller 262723 +katie nixon 65669 +katie ovid 65681 +katie polk 65784 +katie polk 197249 +katie robinson 131251 +katie van buren 131237 +katie van buren 197141 +katie white 262510 +katie white 262860 +katie xylophone 197034 +katie young 65644 +katie young 328173 +katie young 393859 +katie zipper 65733 +katie zipper 328287 +luke allen 65776 +luke allen 131268 +luke allen 196819 +luke allen 196855 +luke allen 328011 +luke brown 196967 +luke davidson 65656 +luke davidson 131573 +luke ellison 65582 +luke ellison 131343 +luke ellison 197118 +luke falkner 196797 +luke falkner 196837 +luke garcia 65778 +luke garcia 393974 +luke ichabod 65629 +luke ichabod 262574 +luke johnson 131302 +luke johnson 131312 +luke johnson 131361 +luke laertes 131226 +luke laertes 131504 +luke laertes 197018 +luke laertes 197153 +luke laertes 197177 +luke miller 197052 +luke ovid 65569 +luke ovid 262745 +luke polk 65658 +luke polk 262627 +luke quirinius 131233 +luke robinson 65634 +luke robinson 262569 +luke thompson 196858 +luke underhill 65651 +luke underhill 131240 +luke underhill 328248 +luke van buren 131398 +luke white 65693 +luke xylophone 131312 +luke zipper 131297 +mike allen 196928 +mike brown 197149 +mike carson 65751 +mike carson 131284 +mike carson 393711 +mike davidson 196917 +mike davidson 262912 +mike ellison 65598 +mike ellison 131366 +mike ellison 131412 +mike ellison 131509 +mike ellison 262704 +mike falkner 328183 +mike garcia 131530 +mike garcia 328305 +mike garcia 328461 +mike hernandez 131301 +mike hernandez 328384 +mike ichabod 131157 +mike king 196965 +mike king 197091 +mike king 197121 +mike king 262471 +mike king 262527 +mike king 328279 +mike miller 131317 +mike nixon 131328 +mike nixon 262653 +mike polk 131240 +mike polk 196899 +mike polk 262885 +mike quirinius 525126 +mike steinbeck 65550 +mike steinbeck 131201 +mike steinbeck 131490 +mike steinbeck 262490 +mike van buren 131548 +mike van buren 262547 +mike white 197000 +mike white 197060 +mike white 262425 +mike white 328482 +mike young 196935 +mike young 196976 +mike young 328084 +mike zipper 131147 +mike zipper 197075 +mike zipper 328517 +nick allen 131192 +nick allen 197024 +nick brown 131503 +nick davidson 262686 +nick ellison 197119 +nick ellison 197119 +nick falkner 65583 +nick falkner 328561 +nick garcia 131318 +nick garcia 262755 +nick garcia 328281 +nick ichabod 131430 +nick ichabod 196812 +nick ichabod 328593 +nick johnson 131453 +nick johnson 262597 +nick laertes 196732 +nick miller 131490 +nick nixon 262547 +nick ovid 328266 +nick polk 196852 +nick quirinius 131438 +nick quirinius 328176 +nick robinson 131326 +nick robinson 196980 +nick steinbeck 131250 +nick thompson 65610 +nick underhill 65619 +nick van buren 196795 +nick xylophone 196972 +nick young 394136 +nick young 459634 +nick zipper 262954 +nick zipper 394218 +oscar allen 262674 +oscar brown 196916 +oscar carson 131099 +oscar carson 131330 +oscar carson 196731 +oscar carson 196733 +oscar carson 196918 +oscar davidson 262554 +oscar ellison 65630 +oscar ellison 197116 +oscar falkner 197145 +oscar garcia 328305 +oscar hernandez 197022 +oscar hernandez 328315 +oscar ichabod 131302 +oscar ichabod 131309 +oscar ichabod 196760 +oscar ichabod 196874 +oscar johnson 196942 +oscar johnson 197203 +oscar king 196793 +oscar king 196944 +oscar king 328236 +oscar laertes 131208 +oscar laertes 262522 +oscar laertes 262842 +oscar laertes 328364 +oscar nixon 65596 +oscar ovid 131228 +oscar ovid 262580 +oscar ovid 393817 +oscar polk 131078 +oscar polk 131260 +oscar quirinius 131103 +oscar quirinius 196748 +oscar quirinius 196829 +oscar quirinius 262838 +oscar robinson 196874 +oscar robinson 262803 +oscar robinson 393773 +oscar robinson 394087 +oscar steinbeck 328432 +oscar thompson 196826 +oscar thompson 196992 +oscar thompson 262593 +oscar thompson 459401 +oscar underhill 131301 +oscar van buren 131134 +oscar van buren 328162 +oscar van buren 394034 +oscar white 131457 +oscar white 262345 +oscar white 328538 +oscar white 459337 +oscar xylophone 65773 +oscar xylophone 262708 +oscar xylophone 262906 +oscar zipper 196904 +oscar zipper 262512 +oscar zipper 328262 +priscilla brown 196950 +priscilla brown 328237 +priscilla brown 328624 +priscilla carson 262488 +priscilla carson 262510 +priscilla carson 262703 +priscilla ichabod 131178 +priscilla ichabod 131303 +priscilla johnson 131224 +priscilla johnson 196906 +priscilla johnson 196994 +priscilla johnson 197184 +priscilla johnson 394171 +priscilla king 262692 +priscilla nixon 262691 +priscilla nixon 394188 +priscilla ovid 65541 +priscilla ovid 197067 +priscilla polk 394009 +priscilla quirinius 131306 +priscilla thompson 196875 +priscilla underhill 197084 +priscilla underhill 262701 +priscilla van buren 65685 +priscilla van buren 131368 +priscilla van buren 196814 +priscilla white 196893 +priscilla xylophone 131473 +priscilla xylophone 262597 +priscilla xylophone 262785 +priscilla young 131392 +priscilla young 262788 +priscilla zipper 393888 +priscilla zipper 394031 +quinn allen 197095 +quinn allen 394225 +quinn brown 131470 +quinn brown 131473 +quinn brown 262642 +quinn davidson 197079 +quinn davidson 197112 +quinn davidson 262510 +quinn davidson 459427 +quinn ellison 197268 +quinn ellison 328130 +quinn garcia 65604 +quinn garcia 131321 +quinn garcia 197067 +quinn garcia 328528 +quinn ichabod 65564 +quinn king 65649 +quinn king 196879 +quinn laertes 65542 +quinn laertes 196877 +quinn laertes 262466 +quinn nixon 196837 +quinn ovid 525126 +quinn quirinius 328235 +quinn robinson 131378 +quinn steinbeck 131484 +quinn steinbeck 262528 +quinn thompson 197030 +quinn thompson 262717 +quinn underhill 262791 +quinn underhill 328146 +quinn underhill 393824 +quinn van buren 197234 +quinn young 65647 +quinn zipper 131466 +quinn zipper 262658 +rachel allen 65661 +rachel allen 196935 +rachel brown 131220 +rachel brown 328076 +rachel brown 328320 +rachel brown 393835 +rachel brown 524988 +rachel carson 131259 +rachel carson 459393 +rachel davidson 262632 +rachel ellison 393845 +rachel falkner 196947 +rachel falkner 262474 +rachel falkner 394046 +rachel falkner 525086 +rachel johnson 65658 +rachel king 131354 +rachel king 196907 +rachel laertes 131391 +rachel laertes 197105 +rachel ovid 262664 +rachel ovid 328195 +rachel polk 328389 +rachel quirinius 262779 +rachel robinson 262491 +rachel robinson 262862 +rachel robinson 590712 +rachel thompson 197034 +rachel thompson 328158 +rachel thompson 394094 +rachel underhill 197033 +rachel white 131399 +rachel white 197190 +rachel young 196967 +rachel zipper 328223 +rachel zipper 394149 +sarah carson 131379 +sarah carson 196870 +sarah carson 262491 +sarah ellison 197095 +sarah falkner 131262 +sarah falkner 328251 +sarah garcia 196963 +sarah garcia 197030 +sarah garcia 459657 +sarah ichabod 262504 +sarah ichabod 262766 +sarah johnson 131409 +sarah johnson 262783 +sarah johnson 328591 +sarah johnson 394043 +sarah king 196998 +sarah king 328416 +sarah miller 196893 +sarah ovid 131199 +sarah robinson 262868 +sarah robinson 394066 +sarah steinbeck 262650 +sarah white 197059 +sarah white 262579 +sarah xylophone 131336 +sarah young 394123 +sarah zipper 262818 +tom brown 196848 +tom brown 328268 +tom carson 197328 +tom carson 262517 +tom carson 656251 +tom davidson 262864 +tom ellison 196974 +tom ellison 328416 +tom ellison 393921 +tom falkner 393809 +tom falkner 459407 +tom hernandez 262525 +tom hernandez 328085 +tom ichabod 197048 +tom johnson 328321 +tom johnson 393865 +tom king 196951 +tom laertes 262657 +tom laertes 459805 +tom miller 131278 +tom miller 131459 +tom miller 262633 +tom nixon 262588 +tom ovid 262595 +tom polk 328470 +tom polk 328584 +tom quirinius 262597 +tom quirinius 262681 +tom robinson 196978 +tom robinson 328481 +tom robinson 459857 +tom robinson 525095 +tom steinbeck 262426 +tom van buren 131389 +tom van buren 328095 +tom van buren 328313 +tom white 328128 +tom young 131080 +tom young 393692 +tom zipper 197167 +ulysses brown 196815 +ulysses carson 131277 +ulysses carson 262450 +ulysses carson 262937 +ulysses carson 328311 +ulysses davidson 262750 +ulysses ellison 262445 +ulysses garcia 328445 +ulysses hernandez 131414 +ulysses hernandez 196871 +ulysses hernandez 394370 +ulysses ichabod 393834 +ulysses ichabod 459582 +ulysses johnson 262966 +ulysses king 131363 +ulysses laertes 262739 +ulysses laertes 328412 +ulysses laertes 328462 +ulysses miller 262661 +ulysses miller 328360 +ulysses nixon 394194 +ulysses ovid 328289 +ulysses polk 65563 +ulysses polk 197046 +ulysses polk 328294 +ulysses polk 590698 +ulysses quirinius 525643 +ulysses robinson 394160 +ulysses steinbeck 196783 +ulysses steinbeck 262778 +ulysses thompson 262607 +ulysses underhill 131214 +ulysses underhill 196937 +ulysses underhill 197027 +ulysses underhill 262623 +ulysses underhill 262623 +ulysses underhill 262648 +ulysses underhill 262836 +ulysses van buren 196944 +ulysses white 197033 +ulysses white 393988 +ulysses xylophone 262695 +ulysses xylophone 328151 +ulysses xylophone 328747 +ulysses young 196903 +ulysses young 394037 +ulysses young 459782 +victor allen 197189 +victor allen 262651 +victor brown 262544 +victor brown 262799 +victor brown 327900 +victor brown 591265 +victor davidson 197173 +victor davidson 262486 +victor davidson 328274 +victor ellison 328618 +victor ellison 393962 +victor hernandez 197041 +victor hernandez 197132 +victor hernandez 262771 +victor hernandez 328261 +victor hernandez 459902 +victor johnson 131155 +victor johnson 131169 +victor johnson 394168 +victor king 131486 +victor king 328509 +victor laertes 262573 +victor laertes 328435 +victor miller 196784 +victor nixon 196987 +victor nixon 394249 +victor ovid 196882 +victor polk 262462 +victor quirinius 65620 +victor quirinius 328301 +victor robinson 328334 +victor robinson 394031 +victor steinbeck 65661 +victor steinbeck 262560 +victor steinbeck 262750 +victor thompson 65548 +victor van buren 197173 +victor van buren 328261 +victor white 262588 +victor white 328039 +victor xylophone 131203 +victor xylophone 262596 +victor xylophone 328191 +victor xylophone 393913 +victor xylophone 459542 +victor young 131258 +victor zipper 131349 +wendy allen 131402 +wendy allen 196954 +wendy allen 328359 +wendy brown 328365 +wendy brown 459501 +wendy ellison 262718 +wendy ellison 328191 +wendy falkner 197009 +wendy falkner 262430 +wendy falkner 328177 +wendy garcia 65746 +wendy garcia 393974 +wendy garcia 459883 +wendy garcia 459926 +wendy hernandez 65650 +wendy ichabod 262665 +wendy king 262545 +wendy king 328229 +wendy king 393951 +wendy laertes 262739 +wendy laertes 262794 +wendy laertes 328315 +wendy miller 131377 +wendy miller 328161 +wendy nixon 131258 +wendy nixon 196893 +wendy ovid 196952 +wendy ovid 459594 +wendy polk 328520 +wendy polk 394310 +wendy quirinius 328703 +wendy quirinius 394360 +wendy robinson 131316 +wendy robinson 394030 +wendy robinson 459665 +wendy steinbeck 262645 +wendy thompson 262725 +wendy thompson 393865 +wendy underhill 328445 +wendy underhill 394295 +wendy underhill 460068 +wendy van buren 65699 +wendy van buren 196964 +wendy white 328135 +wendy xylophone 262894 +wendy xylophone 525344 +wendy young 197017 +wendy young 721936 +xavier allen 197025 +xavier allen 525393 +xavier allen 525839 +xavier brown 197058 +xavier brown 262626 +xavier brown 328388 +xavier carson 196990 +xavier carson 328415 +xavier davidson 65644 +xavier davidson 262745 +xavier davidson 393825 +xavier ellison 197095 +xavier ellison 328447 +xavier garcia 262590 +xavier hernandez 196847 +xavier hernandez 197077 +xavier hernandez 393838 +xavier ichabod 262600 +xavier ichabod 328157 +xavier johnson 197084 +xavier johnson 262785 +xavier king 196919 +xavier king 262774 +xavier laertes 262770 +xavier ovid 328414 +xavier polk 196844 +xavier polk 328474 +xavier polk 394013 +xavier polk 590931 +xavier quirinius 65650 +xavier quirinius 131140 +xavier quirinius 328382 +xavier quirinius 459669 +xavier thompson 393799 +xavier underhill 197012 +xavier white 196858 +xavier white 262712 +xavier xylophone 131250 +xavier zipper 394070 +yuri allen 131129 +yuri allen 459977 +yuri brown 262640 +yuri brown 393858 +yuri carson 459799 +yuri carson 591063 +yuri ellison 197085 +yuri ellison 459558 +yuri falkner 196857 +yuri falkner 525350 +yuri garcia 328378 +yuri hernandez 262588 +yuri johnson 393861 +yuri johnson 394444 +yuri johnson 525638 +yuri king 525526 +yuri laertes 131551 +yuri laertes 459611 +yuri nixon 262644 +yuri nixon 393936 +yuri polk 328197 +yuri polk 328404 +yuri polk 328481 +yuri quirinius 131092 +yuri quirinius 196898 +yuri quirinius 525159 +yuri steinbeck 394037 +yuri steinbeck 525180 +yuri thompson 459710 +yuri underhill 328325 +yuri underhill 459781 +yuri white 131252 +yuri xylophone 262809 +zach allen 394026 +zach brown 262789 +zach brown 262789 +zach brown 459521 +zach brown 459846 +zach brown 590938 +zach carson 262320 +zach ellison 262757 +zach falkner 262608 +zach falkner 262608 +zach garcia 262818 +zach garcia 328314 +zach garcia 393686 +zach garcia 394011 +zach ichabod 262518 +zach ichabod 262563 +zach king 196780 +zach king 196905 +zach king 459991 +zach miller 196923 +zach miller 393813 +zach miller 393892 +zach ovid 196876 +zach ovid 262643 +zach ovid 328023 +zach ovid 459615 +zach quirinius 262471 +zach robinson 196967 +zach steinbeck 131394 +zach steinbeck 459294 +zach thompson 131340 +zach thompson 525538 +zach underhill 131304 +zach white 65733 +zach xylophone 262810 +zach xylophone 459455 +zach young 393615 +zach zipper 197130 +zach zipper 262496 +zach zipper 393937 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-6-4d78f7b1d172d20c91f5867bc13a42a0 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-6-4d78f7b1d172d20c91f5867bc13a42a0 new file mode 100644 index 0000000000000000000000000000000000000000..b3f08818f491afedf0a0d52194ac2ec8cdea3059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-6-4d78f7b1d172d20c91f5867bc13a42a0 @@ -0,0 +1,1049 @@ +0.08 0.07999999821186066 +0.1 0.10000000149011612 +0.13 0.12999999523162842 +0.15 0.15000000596046448 +0.27 0.27000001072883606 +0.28 0.2800000011920929 +0.43 0.4300000071525574 +0.52 0.5199999809265137 +0.56 0.5600000023841858 +0.6 0.6000000238418579 +0.61 0.6100000143051147 +0.79 0.7900000214576721 +0.84 0.8399999737739563 +0.98 0.9800000190734863 +1.02 1.2899999916553497 +1.08 1.0800000429153442 +1.08 1.0800000429153442 +1.12 1.1200000047683716 +1.21 2.0000000596046448 +1.25 1.25 +1.27 1.2699999809265137 +1.29 1.2899999618530273 +1.31 1.309999942779541 +1.58 1.5800000429153442 +1.87 1.8700000047683716 +1.91 1.909999966621399 +1.92 3.1699999570846558 +2.07 2.069999933242798 +2.18 2.180000066757202 +2.2 2.200000047683716 +2.35 2.3499999046325684 +2.6 2.5999999046325684 +2.79 2.7899999618530273 +2.92 2.9200000762939453 +2.96 2.9600000381469727 +2.96 2.9600000381469727 +2.97 2.9700000286102295 +3.0 3.0 +3.21 3.340000033378601 +3.28 4.399999976158142 +3.33 3.3299999237060547 +3.61 3.609999895095825 +3.62 3.619999885559082 +3.82 3.819999933242798 +3.86 3.859999895095825 +3.96 3.9600000381469727 +3.97 3.9700000286102295 +4.17 7.7799999713897705 +4.32 4.320000171661377 +4.35 4.349999904632568 +4.41 4.409999847412109 +4.46 4.460000038146973 +4.47 4.46999979019165 +4.57 4.570000171661377 +4.59 4.590000152587891 +4.71 4.710000038146973 +4.72 4.71999979019165 +4.79 4.789999961853027 +4.8 4.800000190734863 +4.92 4.920000076293945 +5.08 5.079999923706055 +5.24 9.559999942779541 +5.28 5.28000020980835 +5.4 5.400000095367432 +5.44 5.440000057220459 +5.45 5.449999809265137 +5.51 5.510000228881836 +5.54 5.539999961853027 +5.62 5.619999885559082 +5.67 5.670000076293945 +5.85 5.849999904632568 +5.88 5.880000114440918 +6.29 6.289999961853027 +6.55 6.550000190734863 +6.57 11.160000324249268 +6.63 9.59000015258789 +6.67 6.670000076293945 +6.72 6.71999979019165 +6.74 6.739999771118164 +6.84 6.840000152587891 +6.87 6.869999885559082 +7.05 7.050000190734863 +7.06 11.769999980926514 +7.11 7.110000133514404 +7.54 7.539999961853027 +7.56 7.559999942779541 +7.79 7.789999961853027 +7.82 7.820000171661377 +7.96 7.960000038146973 +7.96 7.960000038146973 +7.98 7.980000019073486 +8.07 8.069999694824219 +8.07 8.069999694824219 +8.32 8.319999694824219 +8.37 11.339999914169312 +8.42 11.760000109672546 +8.45 8.449999809265137 +8.45 8.449999809265137 +8.45 10.319999814033508 +8.45 16.009999752044678 +8.57 8.569999694824219 +8.61 8.609999656677246 +8.67 8.670000076293945 +8.71 8.710000038146973 +8.79 8.789999961853027 +8.91 13.480000019073486 +9.04 9.039999961853027 +9.13 9.130000114440918 +9.19 15.479999542236328 +9.22 9.220000267028809 +9.25 9.25 +9.26 9.260000228881836 +9.35 12.350000381469727 +9.48 9.479999542236328 +9.56 12.480000495910645 +9.57 9.569999694824219 +9.57 9.569999694824219 +9.68 9.680000305175781 +9.7 9.699999809265137 +9.71 17.25 +9.74 9.739999771118164 +9.8 9.800000190734863 +9.81 9.8100004196167 +9.93 9.930000305175781 +10.09 10.09000015258789 +10.09 10.09000015258789 +10.13 15.640000343322754 +10.16 18.139999866485596 +10.17 14.970000267028809 +10.19 10.1899995803833 +10.2 10.199999809265137 +10.22 10.220000267028809 +10.25 18.859999656677246 +10.26 10.260000228881836 +10.29 10.289999961853027 +10.6 10.600000381469727 +10.66 10.65999984741211 +10.67 10.670000076293945 +10.73 10.729999542236328 +11.15 20.27999973297119 +11.18 13.360000371932983 +11.19 18.979999542236328 +11.22 11.220000267028809 +11.34 16.62000036239624 +11.55 11.550000190734863 +11.57 11.569999694824219 +11.68 17.080000400543213 +11.82 11.819999694824219 +11.89 11.890000343322754 +11.91 11.90999984741211 +12.02 12.020000457763672 +12.16 12.15999984741211 +12.19 20.149999618530273 +12.32 12.319999694824219 +12.42 16.27999997138977 +12.44 12.4399995803833 +12.45 16.799999713897705 +12.46 12.460000038146973 +12.5 15.460000038146973 +12.54 12.539999961853027 +12.85 12.850000381469727 +12.9 12.899999618530273 +13.01 28.47000026702881 +13.1 23.030000686645508 +13.15 20.96999979019165 +13.35 13.350000381469727 +13.87 13.869999885559082 +13.89 13.890000343322754 +13.94 13.9399995803833 +13.99 13.989999771118164 +14.13 24.22000026702881 +14.21 14.210000038146973 +14.3 29.270000457763672 +14.44 14.4399995803833 +14.84 14.84000015258789 +14.92 14.920000076293945 +14.92 25.18000030517578 +14.93 30.40999984741211 +15.1 17.700000286102295 +15.15 24.40999984741211 +15.18 26.730000495910645 +15.22 15.220000267028809 +15.26 15.260000228881836 +15.3 25.5 +15.37 15.369999885559082 +15.45 15.449999809265137 +15.63 28.110000610351562 +15.75 15.75 +15.81 15.8100004196167 +15.86 25.079999923706055 +15.9 21.34999942779541 +15.92 15.920000076293945 +16.08 22.75 +16.09 16.520000159740448 +16.24 27.809999465942383 +16.25 19.419999957084656 +16.48 16.479999542236328 +16.69 16.690000534057617 +16.99 16.989999771118164 +16.99 42.489999771118164 +17.16 21.12999987602234 +17.37 31.360000610351562 +17.74 19.049999713897705 +17.79 45.60000038146973 +17.87 18.710000813007355 +18.2 18.200000762939453 +18.5 31.350000381469727 +18.56 18.559999465942383 +18.63 26.589999198913574 +18.63 30.389999270439148 +18.86 18.96000061184168 +18.89 18.889999389648438 +18.93 18.93000030517578 +19.0 35.62000036239624 +19.03 19.030000686645508 +19.06 19.059999465942383 +19.06 19.059999465942383 +19.13 45.719998359680176 +19.14 19.139999389648438 +19.28 27.600000381469727 +19.69 36.49000024795532 +20.07 46.80000019073486 +20.38 51.72999954223633 +20.64 20.639999389648438 +20.67 31.830000400543213 +20.79 20.790000915527344 +20.81 20.809999465942383 +20.82 21.419999718666077 +20.82 26.359999656677246 +21.18 21.18000030517578 +21.19 21.190000534057617 +21.23 21.229999542236328 +21.28 29.350000381469727 +21.32 21.31999969482422 +21.45 40.510000228881836 +21.49 30.739999771118164 +21.61 37.0600004196167 +21.7 27.58000087738037 +21.8 21.799999237060547 +21.94 23.940000593662262 +22.01 28.850000381469727 +22.08 22.079999923706055 +22.12 22.1200008392334 +22.12 22.1200008392334 +22.25 22.25 +22.27 22.270000457763672 +22.36 22.360000610351562 +22.68 22.68000030517578 +22.78 47.19000053405762 +22.85 33.070000648498535 +22.85 43.980000257492065 +22.94 35.38000011444092 +23.07 23.06999969482422 +23.13 28.799999237060547 +23.17 44.58999979496002 +23.19 23.190000534057617 +23.44 23.440000534057617 +23.45 24.74000072479248 +23.6 33.16000032424927 +23.77 23.770000457763672 +23.96 23.959999084472656 +24.02 24.020000457763672 +24.28 43.310001373291016 +24.49 42.62999963760376 +24.52 32.59000015258789 +24.73 45.369998931884766 +24.79 24.790000915527344 +24.8 34.369998931884766 +24.83 36.05000019073486 +24.86 65.3700008392334 +25.11 44.170000076293945 +25.28 25.280000686645508 +25.37 48.05000114440918 +25.42 40.78999996185303 +25.55 26.62999927997589 +25.67 37.69000053405762 +25.88 61.49999952316284 +26.08 26.079999923706055 +26.39 34.959999084472656 +26.43 26.43000030517578 +26.47 31.389999389648438 +26.49 26.489999771118164 +26.49 48.56999969482422 +26.64 64.32999992370605 +26.71 36.999999046325684 +26.73 45.69000015407801 +26.76 26.760000228881836 +27.07 28.649999737739563 +27.12 32.20000076293945 +27.3 70.61000061035156 +27.31 56.579999923706055 +27.63 27.6299991607666 +27.66 27.65999984741211 +27.72 46.60999870300293 +27.87 27.8700008392334 +28.11 44.59000015258789 +28.31 52.079999923706055 +28.45 74.05000114440918 +28.5 35.36999988555908 +28.56 40.71999931335449 +28.69 28.690000534057617 +28.71 55.46999931335449 +28.79 28.790000915527344 +28.89 56.489999771118164 +28.95 33.410000801086426 +29.02 56.64999961853027 +29.24 99.85000038146973 +29.36 62.52000093460083 +29.4 72.02999925613403 +29.41 64.77999973297119 +29.54 29.540000915527344 +29.59 37.37000012397766 +29.78 66.77999973297119 +30.25 30.32999999821186 +30.36 30.360000610351562 +30.37 31.660000830888748 +30.61 30.610000610351562 +30.62 102.65000009536743 +30.63 30.6299991607666 +30.65 60.19000053405762 +30.71 49.849998474121094 +30.81 55.989999771118164 +31.01 31.010000228881836 +31.15 31.149999618530273 +31.4 31.399999618530273 +31.61 31.610000610351562 +31.67 40.46000003814697 +31.77 42.09000027179718 +31.86 31.860000610351562 +31.91 78.51999855041504 +32.01 60.47999858856201 +32.18 58.61000061035156 +32.2 53.55000019073486 +32.23 42.89999961853027 +32.25 59.83000087738037 +32.37 62.99999809265137 +32.41 32.40999984741211 +32.47 41.14000129699707 +32.52 95.0400013923645 +32.75 56.19000053405762 +32.89 80.07999992370605 +32.92 47.7599983215332 +33.36 45.27000045776367 +33.52 60.010000228881836 +33.55 63.939998507499695 +33.58 54.55000162124634 +33.67 33.66999816894531 +33.76 112.27999687194824 +33.83 59.110002517700195 +33.85 47.719998359680176 +33.87 37.48999881744385 +34.03 71.51999759674072 +34.21 71.57999920845032 +34.35 34.349998474121094 +34.41 59.20000076293945 +34.58 34.58000183105469 +34.73 34.72999954223633 +34.97 45.160000801086426 +35.0 35.0 +35.08 36.060001850128174 +35.13 39.600000858306885 +35.17 64.01999855041504 +35.17 66.52999877929688 +35.56 37.63000130653381 +35.62 80.20999872684479 +35.65 56.83000183105469 +35.68 52.20000046491623 +35.72 98.71999931335449 +35.8 79.96999931335449 +35.89 81.2599983215332 +36.22 55.20000076293945 +36.26 36.2599983215332 +36.58 64.45000267028809 +36.7 73.76000118255615 +36.79 36.790000915527344 +36.89 71.8499984741211 +36.95 36.95000076293945 +37.07 37.06999969482422 +37.1 51.039998054504395 +37.14 53.41999936103821 +37.14 61.15999984741211 +37.24 47.04000186920166 +37.59 50.94000053405762 +37.6 84.39999866485596 +37.72 57.14000117778778 +37.78 68.10999877750874 +37.8 57.94999885559082 +37.85 50.38999843597412 +37.9 77.50000238418579 +38.05 40.24999928474426 +38.05 47.859999656677246 +38.3 98.48999977111816 +38.33 112.09000301361084 +38.57 55.81999969482422 +38.62 73.99999904632568 +38.79 95.44000053405762 +38.85 97.45999908447266 +38.88 48.58000087738037 +38.94 71.3499984741211 +39.01 39.0099983215332 +39.03 57.73999959230423 +39.18 99.6599988937378 +39.21 71.79999923706055 +39.34 39.34000015258789 +39.69 55.69999837875366 +39.81 74.81000137329102 +39.82 39.81999969482422 +39.83 119.80000114440918 +39.87 62.22999954223633 +39.9 64.98000144958496 +39.98 39.97999954223633 +40.0 84.59000015258789 +40.04 101.20000076293945 +40.17 80.41999745368958 +40.24 67.9000015258789 +40.42 78.04999947547913 +40.44 117.94000101089478 +40.78 49.8199987411499 +40.8 40.79999923706055 +40.98 92.01999759674072 +41.2 58.280001163482666 +41.29 41.290000915527344 +41.29 112.87000012397766 +41.31 53.08000135421753 +41.34 53.230000495910645 +41.34 115.33999919891357 +41.36 41.36000061035156 +41.44 85.41999888420105 +41.45 91.29999923706055 +41.62 41.619998931884766 +41.68 109.58000183105469 +41.71 83.06999969482422 +41.81 89.8600025177002 +41.85 76.21999740600586 +41.87 47.48999881744385 +41.89 41.88999938964844 +42.24 72.6500015258789 +42.31 52.91000175476074 +42.42 154.69999504089355 +42.48 71.12999927997589 +42.51 61.069997787475586 +42.55 87.71000003814697 +42.56 71.91000175476074 +42.67 71.45999908447266 +42.76 42.7599983215332 +42.85 85.33999824523926 +43.01 109.53999710083008 +43.02 46.84000039100647 +43.13 43.130001068115234 +43.16 58.079999923706055 +43.17 43.16999816894531 +43.19 156.05999875068665 +43.31 64.10000228881836 +43.37 56.84999895095825 +43.57 43.71999970078468 +43.71 108.69000053405762 +43.73 108.50999927520752 +43.92 48.319998145103455 +44.1 67.1299991607666 +44.22 103.33000373840332 +44.27 74.88000106811523 +44.43 106.65999984741211 +44.57 59.00999927520752 +45.06 45.060001373291016 +45.1 45.099998474121094 +45.19 45.189998626708984 +45.19 117.10000038146973 +45.24 109.34000396728516 +45.34 129.73999881744385 +45.35 109.28999698162079 +45.42 77.61999893188477 +45.45 45.45000076293945 +45.56 137.57999897003174 +45.59 82.54000091552734 +45.68 55.25 +45.92 90.97999954223633 +45.99 47.07000172138214 +46.02 82.27999877929688 +46.09 46.09000015258789 +46.1 98.17999839782715 +46.15 72.7800008058548 +46.18 74.29000091552734 +46.21 52.75999927520752 +46.27 85.61000061035156 +46.43 106.26000118255615 +46.45 110.90000343322754 +46.62 78.0099983215332 +46.8 80.46999740600586 +46.86 62.08000087738037 +46.87 70.80999952554703 +46.88 106.08000183105469 +46.97 88.1100025177002 +47.08 148.28000259399414 +47.27 50.60000038146973 +47.32 118.12999922037125 +47.57 90.32999801635742 +47.59 104.17000007629395 +47.69 99.88999909162521 +47.88 47.880001068115234 +48.01 91.72999802231789 +48.08 79.69000244140625 +48.11 48.11000061035156 +48.15 65.8500018119812 +48.22 105.07000017166138 +48.23 139.52999877929688 +48.25 48.25 +48.28 98.669997215271 +48.37 185.9499979019165 +48.45 48.45000076293945 +48.45 94.54000091552734 +48.52 146.69999885559082 +48.59 89.30999946594238 +49.12 49.119998931884766 +49.28 123.56999969482422 +49.44 110.93999814987183 +49.68 73.63999938964844 +49.77 50.33000046014786 +49.78 66.46999931335449 +50.02 63.380000829696655 +50.08 156.16000366210938 +50.09 106.28000068664551 +50.26 72.05999755859375 +50.28 50.279998779296875 +50.31 117.44000053405762 +50.32 90.13999938964844 +50.4 96.11999988555908 +50.41 98.72999799251556 +50.66 55.37999963760376 +50.7 131.11999821662903 +50.83 98.69000148773193 +50.92 53.70999813079834 +50.96 103.71999835968018 +51.25 67.0 +51.29 87.35000276565552 +51.29 124.93000030517578 +51.72 97.17000198364258 +51.79 139.90000343322754 +51.84 168.94000053405762 +51.85 171.64999961853027 +52.17 206.86999320983887 +52.23 177.1599998474121 +52.44 88.48999881744385 +52.5 105.41000175476074 +52.53 64.98999881744385 +52.72 52.720001220703125 +52.73 74.04999923706055 +52.85 89.63999938964844 +52.87 130.87999725341797 +53.02 100.50999927520752 +53.06 259.9299945831299 +53.18 53.18000030517578 +53.27 53.27000045776367 +53.59 53.59000015258789 +53.78 139.38999938964844 +53.93 57.890000343322754 +53.94 63.529998779296875 +54.1 152.7699956893921 +54.31 77.38000106811523 +54.34 125.46999943256378 +54.43 132.04999923706055 +54.44 103.01999950408936 +54.47 186.52000045776367 +54.73 63.179999351501465 +54.75 112.82999992370605 +54.83 110.82000160217285 +54.99 160.40000343322754 +55.1 161.35999965667725 +55.18 215.58000373840332 +55.2 126.65999984741211 +55.39 137.6699981689453 +55.51 74.55999803543091 +55.63 96.43000030517578 +55.99 187.10999989509583 +56.04 150.5800018310547 +56.07 118.15000057220459 +56.1 135.79000091552734 +56.15 144.64000034332275 +56.33 61.77000188827515 +56.62 78.88999938964844 +56.68 154.13999938964844 +56.81 169.64000129699707 +57.08 69.98000144958496 +57.11 168.0100040435791 +57.12 100.28999710083008 +57.23 65.9399995803833 +57.25 133.46999740600586 +57.29 112.54000091552734 +57.35 110.89999866485596 +57.37 115.109998524189 +57.46 147.78999710083008 +57.64 112.19000101089478 +57.67 57.66999816894531 +57.89 111.15999984741211 +57.93 68.02000045776367 +58.0 123.9399995803833 +58.08 58.08000183105469 +58.09 206.37000274658203 +58.13 105.84999942779541 +58.43 165.0900001525879 +58.52 167.0299997329712 +58.66 136.04000091552734 +58.67 205.36999702453613 +58.75 90.41000083088875 +58.86 165.14000129699707 +59.07 87.86999893188477 +59.16 224.25 +59.21 90.35999870300293 +59.34 127.44999893009663 +59.43 106.50000202655792 +59.45 67.90000057220459 +59.45 197.11999893188477 +59.5 149.63999938964844 +59.55 61.459999203681946 +59.61 85.97000026702881 +59.62 113.3299970626831 +59.68 73.89000034332275 +59.68 94.40999984741211 +59.7 193.1699981689453 +59.71 60.22999906539917 +59.83 145.17000007629395 +59.87 228.80999946594238 +59.99 134.04000282287598 +60.02 66.76000022888184 +60.06 60.060001373291016 +60.12 113.34999942779541 +60.13 214.27000045776367 +60.22 108.10000228881836 +60.26 105.94999847561121 +60.26 165.32999849319458 +60.53 66.37999868392944 +60.6 82.8499984741211 +60.71 72.04999899864197 +60.85 132.36999607086182 +61.21 160.86999797821045 +61.7 127.55000257492065 +61.86 248.9700005054474 +61.88 112.15999984741211 +61.92 125.29999899864197 +61.94 119.6099967956543 +62.14 110.59000015258789 +62.2 149.91000080108643 +62.23 111.3499984741211 +62.3 158.41999912261963 +62.39 110.95999908447266 +62.52 123.97999966144562 +62.72 123.78999900817871 +62.74 153.10000038146973 +62.85 167.01999855041504 +62.9 256.0699996948242 +62.92 129.3899974822998 +63.12 93.47999954223633 +63.33 135.38999938964844 +63.35 116.93999862670898 +63.42 172.76000213623047 +63.51 123.51999855041504 +63.9 135.70000076293945 +64.0 191.55000257492065 +64.22 86.97000122070312 +64.25 131.25 +64.3 122.3800048828125 +64.36 85.59000015258789 +64.46 134.44000053405762 +64.65 143.54000091552734 +64.67 121.15999794006348 +64.77 214.40999603271484 +64.87 194.61000156402588 +64.95 324.87999153137207 +65.02 175.6099967956543 +65.02 259.6299982070923 +65.38 168.7100009918213 +65.43 112.27000069618225 +65.43 289.6800003051758 +65.44 192.89000137150288 +65.55 66.16000306606293 +65.62 139.67000198364258 +65.7 65.69999694824219 +65.72 77.54000091552734 +66.17 177.10999631881714 +66.17 200.60999870300293 +66.36 131.73000144958496 +66.51 83.50000190734863 +66.61 66.61000061035156 +66.61 78.93000030517578 +66.67 129.84999752044678 +66.89 99.96000003814697 +67.12 67.12000274658203 +67.18 234.21000003814697 +67.26 77.9900016784668 +67.38 178.72999572753906 +67.45 197.29999446868896 +67.48 268.0900020599365 +67.59 272.95999336242676 +67.94 125.89000129699707 +67.98 123.36000299453735 +68.01 124.84000396728516 +68.04 166.76000022888184 +68.22 181.57000064849854 +68.25 113.52000045776367 +68.25 163.2900013923645 +68.32 247.04999542236328 +68.41 157.72000312805176 +68.5 156.36999893188477 +68.81 93.02999782562256 +68.85 160.57999649643898 +68.89 89.69999885559082 +68.95 140.99999594688416 +68.96 192.4799976348877 +69.32 156.29000091552734 +69.53 239.17000007629395 +69.74 246.84999418258667 +69.8 101.63000345230103 +69.88 220.45999908447266 +69.96 83.84999942779541 +69.97 169.86000031232834 +70.0 236.76000022888184 +70.04 196.70000076293945 +70.06 92.17999839782715 +70.24 213.77999877929688 +70.35 247.5099983215332 +70.38 179.95999908447266 +70.39 164.79999923706055 +70.52 181.33999824523926 +70.53 70.52999877929688 +70.56 110.53999710083008 +70.85 223.94999885559082 +70.89 149.81999969482422 +70.93 160.79000282287598 +71.01 92.20000267028809 +71.07 112.36000061035156 +71.13 135.149995803833 +71.19 136.1800012588501 +71.26 318.7700004577637 +71.31 231.88999405503273 +71.32 145.21000003814697 +71.35 145.909996509552 +71.5 217.409996509552 +71.54 71.54000091552734 +71.55 90.41000270843506 +71.68 72.9500002861023 +71.68 227.97000122070312 +71.78 189.71999979019165 +71.8 103.66000366210938 +71.89 180.57999992370605 +72.04 156.63000106811523 +72.18 181.46999728679657 +72.51 264.0600047111511 +72.53 265.42000015079975 +72.56 132.61999893188477 +72.62 205.2400016784668 +72.79 72.79000091552734 +72.98 286.76000213623047 +73.18 123.78000068664551 +73.32 175.96999979019165 +73.48 144.8300018310547 +73.63 320.47999143600464 +73.65 114.11000156402588 +73.68 161.0300030708313 +73.88 291.28999376296997 +73.93 156.77999877929688 +74.0 179.8499994277954 +74.02 89.27999687194824 +74.15 74.1500015258789 +74.19 74.47000244259834 +74.19 122.44000244140625 +74.3 159.72000193595886 +74.42 219.62999820709229 +74.45 203.839994430542 +74.52 271.6399955749512 +74.53 342.6200008392334 +74.59 157.12999725341797 +74.62 163.89999961853027 +74.72 139.05000114440918 +74.78 230.83999752998352 +75.03 174.98999881744385 +75.1 214.62999725341797 +75.19 149.6600048840046 +75.29 93.4900016784668 +75.35 256.91999912261963 +75.42 153.46999764442444 +75.66 225.4800033569336 +75.73 236.60000133514404 +75.83 199.4000015258789 +75.88 243.89000129699707 +76.05 179.770001411438 +76.1 257.43999671936035 +76.28 168.48000144958496 +76.28 177.9100022315979 +76.33 363.09000396728516 +76.52 207.39999389648438 +76.69 212.87000370025635 +76.7 307.5399944782257 +76.71 132.17999839782715 +76.72 166.42000007629395 +76.72 258.1899985074997 +76.74 102.81999778747559 +76.92 401.7999897003174 +76.93 384.4699947834015 +77.02 296.6499948501587 +77.1 187.63999557495117 +77.36 281.19999504089355 +77.42 217.32000160217285 +77.57 189.66000270843506 +77.66 267.380003452301 +77.81 178.31999683380127 +77.84 350.7999897003174 +77.89 249.5399990081787 +77.97 178.2599983215332 +78.21 189.1099977493286 +78.26 284.6300048828125 +78.28 306.25 +78.3 168.71000388264656 +78.31 161.80999946594238 +78.62 275.3200035095215 +78.64 94.55999946594238 +78.73 310.6199974119663 +78.89 175.31999969482422 +78.98 293.25000381469727 +79.12 276.419997215271 +79.19 194.30000096559525 +79.21 237.62999820709229 +79.38 151.22999572753906 +79.42 124.6099967956543 +79.48 116.55000305175781 +79.48 200.64000129699707 +79.49 294.1199951171875 +79.54 145.23999786376953 +79.55 254.54000186920166 +79.75 215.13999938964844 +79.83 294.9700012207031 +79.96 79.95999908447266 +79.97 266.4900016784668 +79.99 219.6599998474121 +80.23 248.71000480651855 +80.3 133.4800033569336 +80.3 305.7800064086914 +80.46 249.17000296711922 +80.52 136.21999502182007 +80.58 261.16000175476074 +80.6 281.23999977111816 +80.71 275.0100000500679 +80.74 240.4599997997284 +80.84 142.60999822616577 +80.92 340.5499963760376 +80.96 372.2499928474426 +80.97 93.32000160217285 +80.99 317.74999809265137 +81.17 220.55999755859375 +81.32 158.86000061035156 +81.32 206.16000366210938 +81.47 198.91000175476074 +81.58 173.76000022888184 +81.64 207.5300006866455 +81.66 261.5100030899048 +82.24 243.59999752044678 +82.3 157.18000411987305 +82.34 214.0699977874756 +82.41 135.83000302314758 +82.52 240.23999977111816 +82.55 139.6900042295456 +82.56 175.8799991607666 +82.72 232.63000202178955 +82.97 101.9000015258789 +83.08 181.57000160217285 +83.27 83.2699966430664 +83.33 348.75000198185444 +83.4 196.729998588562 +83.54 299.12000465393066 +83.57 139.38999938964844 +83.58 163.54000091552734 +83.87 263.64000415802 +83.92 432.67000015079975 +83.93 358.9400003552437 +84.03 240.19000244140625 +84.23 209.53000235557556 +84.31 157.0999984741211 +84.38 220.1699981689453 +84.4 443.3400018811226 +84.69 249.4900016784668 +84.72 105.00000095367432 +84.83 157.4800033569336 +85.0 221.21999502182007 +85.03 283.9400005340576 +85.1 217.27999687194824 +85.14 266.7100009918213 +85.23 277.7100009918213 +85.49 261.4599976539612 +85.49 325.9499976634979 +85.51 165.59000205993652 +85.74 341.80999755859375 +85.76 160.57000350952148 +85.87 357.5099983215332 +85.9 202.45000457763672 +86.0 322.60000133514404 +86.22 152.8300018310547 +86.23 152.99000358581543 +86.63 381.5999984741211 +86.69 138.42000198364258 +86.92 147.14999723434448 +86.93 97.59000015258789 +86.93 218.18000030517578 +87.14 219.50999546051025 +87.22 211.01000022888184 +87.4 137.73000198602676 +87.48 153.86000204086304 +87.57 307.0799951553345 +87.61 321.82000064849854 +87.67 248.4600009918213 +87.83 130.96000289916992 +87.94 134.9800043106079 +87.99 227.0399990081787 +88.02 406.7899971008301 +88.05 97.53000259399414 +88.07 210.45000457763672 +88.17 245.6500015258789 +88.22 369.4600009918213 +88.36 305.6399974822998 +88.47 213.9400006532669 +88.48 211.84000635147095 +88.55 199.71000289916992 +88.77 187.49999463558197 +88.78 337.23999977111816 +88.8 254.1300015449524 +88.91 276.40999829769135 +89.01 98.69000244140625 +89.03 431.6499996185303 +89.1 113.83999919891357 +89.15 234.32000160217285 +89.28 343.82000064849854 +89.38 496.16999435424805 +89.53 153.05999755859375 +89.55 89.55000305175781 +89.55 96.27000284194946 +89.55 172.62000274658203 +89.8 205.1400022506714 +89.81 225.64000058174133 +89.93 89.93000030517578 +90.05 90.05000305175781 +90.05 247.1500015258789 +90.07 137.82999801635742 +90.12 234.76000308990479 +90.2 208.32999616861343 +90.25 386.8999948501587 +90.28 334.17000007629395 +90.35 158.36999893188477 +90.38 148.65999841690063 +90.51 268.7700004577637 +90.56 298.8899937272072 +90.69 181.1000051498413 +90.69 523.360002592206 +90.73 260.59000366926193 +90.77 203.03999733924866 +91.05 263.67000579833984 +91.16 298.55999755859375 +91.42 196.41999912261963 +91.48 144.56000471115112 +91.53 301.9800033569336 +91.61 215.59000027179718 +91.63 432.17999362945557 +91.78 330.9499988555908 +91.88 277.8299951553345 +91.97 205.4900016784668 +92.05 317.69000363349915 +92.11 307.70000088214874 +92.33 92.33000183105469 +92.37 226.410005569458 +92.4 265.1600036621094 +92.55 295.59000039100647 +92.61 249.38999938964844 +92.82 182.87000274658203 +92.96 185.98999691009521 +92.98 494.779993057251 +93.03 165.80999958515167 +93.09 456.1800003051758 +93.11 427.2800006866455 +93.61 465.8599934577942 +93.64 314.1999969482422 +93.73 475.3300018310547 +94.08 187.56000137329102 +94.15 362.9200019836426 +94.25 308.1900006532669 +94.27 351.70999336242676 +94.31 216.75 +94.33 193.02000427246094 +94.34 320.75000190734863 +94.38 260.18999683856964 +94.43 174.89999771118164 +94.54 340.19000244140625 +94.66 192.19000625610352 +94.68 189.23999977111816 +94.68 344.1700019836426 +94.72 274.6800003051758 +95.07 417.67000102996826 +95.11 393.99999433755875 +95.28 140.37999725341797 +95.33 244.99000671505928 +95.34 192.92999649047852 +95.38 230.36000156402588 +95.48 406.1000007688999 +95.53 248.58999633789062 +95.53 252.71000289916992 +95.81 134.81999588012695 +95.81 551.9899978637695 +95.84 274.1599931716919 +96.09 191.52999687194824 +96.23 398.2100067138672 +96.25 261.8400020599365 +96.29 199.10999870300293 +96.38 323.4199962615967 +96.62 309.4900064468384 +96.68 336.87000274658203 +96.73 272.0500030517578 +96.78 190.27000045776367 +96.91 180.18000030517578 +96.94 216.54999923706055 +97.09 428.03999519348145 +97.24 346.41000083088875 +97.26 373.67999935150146 +97.39 257.9600028991699 +97.46 449.1699924468994 +97.51 132.09000396728516 +97.56 97.55999755859375 +97.57 592.3499927520752 +97.65 196.34000301361084 +97.68 258.7100033760071 +97.71 175.70000076293945 +97.81 297.20999908447266 +97.83 396.38999938964844 +97.87 254.50000381469727 +98.18 105.29000043869019 +98.22 438.4100036621094 +98.23 525.5100040435791 +98.31 335.93999576568604 +98.48 286.0400047302246 +98.51 132.86000061035156 +98.57 404.3500061035156 +98.72 263.8600025177002 +98.96 288.1999988555908 +99.13 243.6900019645691 +99.15 210.11000061035156 +99.21 347.7999954223633 +99.24 537.6500015258789 +99.29 291.48000717163086 +99.36 106.41000080108643 +99.62 417.3700008392334 +99.65 185.62000179290771 +99.67 180.92999649047852 +99.68 230.6400032043457 +99.91 367.2900071144104 +99.92 376.32999646663666 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-7-20fdc99aa046b2c41d9b85ab338c749c b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-7-20fdc99aa046b2c41d9b85ab338c749c new file mode 100644 index 0000000000000000000000000000000000000000..1a4528978b74f17db18fc6aa60a5c7396bdf2831 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-7-20fdc99aa046b2c41d9b85ab338c749c @@ -0,0 +1,1049 @@ + 65560 3.1 + 65718 2.38 + 65740 3.04 +alice allen 65662 1.55 +alice allen 65720 1.79 +alice allen 65758 1.98 +alice brown 65696 0.13 +alice carson 65559 4.2 +alice davidson 65547 1.51 +alice falkner 65669 4.19 +alice garcia 65613 0.72 +alice hernandez 65737 0.92 +alice hernandez 65784 2.09 +alice johnson 65739 2.55 +alice king 65660 3.84 +alice king 65734 2.96 +alice king 65738 2.14 +alice laertes 65669 0.28 +alice laertes 65671 1.16 +alice miller 65590 4.95 +alice nixon 65586 2.98 +alice nixon 65595 2.36 +alice nixon 65604 2.45 +alice ovid 65737 3.2 +alice polk 65548 1.23 +alice quirinius 65636 0.23 +alice quirinius 65728 0.82 +alice robinson 65606 3.99 +alice robinson 65789 4.35 +alice steinbeck 65578 4.72 +alice steinbeck 65673 3.97 +alice steinbeck 65786 3.92 +alice underhill 65750 2.06 +alice van buren 65562 2.43 +alice xylophone 65578 2.22 +alice xylophone 65585 2.11 +alice xylophone 65599 2.92 +alice zipper 65553 3.78 +alice zipper 65662 2.61 +alice zipper 65766 3.12 +bob brown 65584 2.09 +bob brown 65777 1.62 +bob brown 65783 2.4 +bob carson 65713 3.87 +bob davidson 65664 4.25 +bob davidson 65693 3.54 +bob davidson 65768 2.91 +bob ellison 65591 2.23 +bob ellison 65624 1.69 +bob ellison 65721 1.69 +bob ellison 65760 1.69 +bob falkner 65789 0.21 +bob garcia 65585 3.91 +bob garcia 65598 2.64 +bob garcia 65673 3.26 +bob garcia 65754 3.29 +bob garcia 65782 2.86 +bob hernandez 65557 3.72 +bob ichabod 65549 1.39 +bob king 65715 2.76 +bob king 65757 1.71 +bob king 65783 2.24 +bob laertes 65602 4.98 +bob laertes 65663 3.56 +bob miller 65608 4.95 +bob ovid 65564 1.23 +bob ovid 65619 1.53 +bob ovid 65686 1.84 +bob ovid 65726 2.38 +bob polk 65594 0.28 +bob quirinius 65700 3.82 +bob steinbeck 65637 0.22 +bob van buren 65778 2.89 +bob white 65543 4.75 +bob white 65605 2.89 +bob xylophone 65574 1.7 +bob xylophone 65666 2.51 +bob young 65556 0.95 +bob zipper 65559 3.18 +bob zipper 65633 3.2 +bob zipper 65739 3.24 +calvin allen 65669 2.3 +calvin brown 65537 1.1 +calvin brown 65580 2.82 +calvin brown 65677 2.57 +calvin carson 65637 1.33 +calvin davidson 65541 1.98 +calvin davidson 65564 1.6 +calvin ellison 65667 1.85 +calvin falkner 65573 3.52 +calvin falkner 65596 3.31 +calvin falkner 65738 2.36 +calvin falkner 65762 2.26 +calvin falkner 65778 2.7 +calvin falkner 65784 2.98 +calvin garcia 65664 2.9 +calvin hernandez 65578 2.08 +calvin johnson 65731 4.69 +calvin laertes 65570 4.7 +calvin laertes 65684 3.68 +calvin nixon 65654 2.74 +calvin nixon 65724 3.27 +calvin nixon 65749 2.88 +calvin ovid 65554 3.34 +calvin ovid 65643 2.38 +calvin ovid 65663 2.74 +calvin ovid 65715 2.47 +calvin polk 65731 4.36 +calvin quirinius 65741 4.0 +calvin quirinius 65769 2.2 +calvin robinson 65581 3.32 +calvin steinbeck 65680 1.29 +calvin steinbeck 65762 1.3 +calvin steinbeck 65779 1.98 +calvin thompson 65560 4.08 +calvin thompson 65640 3.33 +calvin underhill 65732 2.41 +calvin van buren 65552 1.05 +calvin van buren 65771 1.33 +calvin white 65553 4.7 +calvin white 65561 2.68 +calvin xylophone 65575 4.6 +calvin xylophone 65596 4.77 +calvin xylophone 65713 3.23 +calvin young 65574 0.27 +calvin young 65746 0.9 +calvin zipper 65669 4.4 +calvin zipper 65739 2.29 +david allen 65588 3.86 +david allen 65617 3.18 +david brown 65637 1.17 +david brown 65760 1.01 +david davidson 65559 1.37 +david davidson 65756 1.57 +david davidson 65778 1.89 +david davidson 65779 2.43 +david ellison 65634 3.23 +david ellison 65724 2.95 +david ellison 65724 2.95 +david hernandez 65763 1.15 +david ichabod 65699 1.67 +david ichabod 65715 1.26 +david laertes 65762 1.38 +david nixon 65536 1.27 +david ovid 65623 0.23 +david ovid 65628 1.15 +david quirinius 65697 1.14 +david quirinius 65759 1.65 +david quirinius 65779 1.93 +david robinson 65762 3.51 +david robinson 65775 3.38 +david thompson 65550 3.3 +david underhill 65602 0.12 +david underhill 65662 2.29 +david underhill 65751 2.43 +david van buren 65625 1.55 +david van buren 65634 3.25 +david white 65678 0.17 +david xylophone 65537 1.07 +david xylophone 65670 0.72 +david xylophone 65764 0.94 +david young 65551 4.51 +david young 65694 2.74 +ethan allen 65747 3.61 +ethan brown 65539 2.9 +ethan brown 65617 1.59 +ethan brown 65685 2.17 +ethan brown 65685 2.17 +ethan brown 65722 2.64 +ethan brown 65733 2.75 +ethan carson 65742 2.84 +ethan ellison 65714 4.87 +ethan ellison 65732 3.9 +ethan falkner 65577 3.61 +ethan falkner 65614 1.95 +ethan garcia 65736 4.63 +ethan hernandez 65618 0.46 +ethan johnson 65536 1.76 +ethan king 65614 0.92 +ethan laertes 65562 2.89 +ethan laertes 65597 3.45 +ethan laertes 65628 3.15 +ethan laertes 65643 3.53 +ethan laertes 65680 3.18 +ethan laertes 65745 3.36 +ethan laertes 65760 3.01 +ethan miller 65712 1.97 +ethan nixon 65766 4.1 +ethan ovid 65697 3.81 +ethan polk 65589 0.7 +ethan polk 65615 1.92 +ethan polk 65622 2.0 +ethan polk 65622 2.0 +ethan quirinius 65542 4.64 +ethan quirinius 65591 3.97 +ethan quirinius 65706 2.88 +ethan robinson 65547 2.2 +ethan robinson 65659 2.17 +ethan underhill 65570 2.45 +ethan van buren 65572 1.11 +ethan white 65677 3.42 +ethan white 65788 4.09 +ethan xylophone 65595 4.66 +ethan zipper 65593 2.1 +ethan zipper 65680 2.53 +fred davidson 65552 0.37 +fred davidson 65595 2.31 +fred davidson 65721 2.65 +fred ellison 65548 4.32 +fred ellison 65691 3.44 +fred ellison 65771 3.1 +fred falkner 65637 4.59 +fred falkner 65648 2.79 +fred falkner 65783 2.35 +fred hernandez 65541 3.87 +fred ichabod 65572 1.45 +fred ichabod 65789 1.68 +fred johnson 65758 3.86 +fred king 65694 4.05 +fred king 65745 4.5 +fred laertes 65769 3.89 +fred miller 65536 2.59 +fred nixon 65560 2.52 +fred nixon 65612 2.0 +fred nixon 65703 1.35 +fred nixon 65705 1.16 +fred polk 65603 2.9 +fred polk 65656 1.65 +fred polk 65701 1.75 +fred polk 65706 1.4 +fred quirinius 65697 1.91 +fred quirinius 65782 3.24 +fred robinson 65623 2.6 +fred steinbeck 65544 2.0 +fred steinbeck 65651 3.32 +fred steinbeck 65755 3.51 +fred underhill 65629 2.56 +fred van buren 65537 4.97 +fred van buren 65561 3.28 +fred van buren 65745 3.24 +fred van buren 65789 3.27 +fred white 65589 3.16 +fred young 65594 0.34 +fred young 65773 1.56 +fred zipper 65553 0.37 +gabriella allen 65646 1.68 +gabriella allen 65677 1.6 +gabriella brown 65704 0.02 +gabriella brown 65753 1.86 +gabriella carson 65586 0.37 +gabriella davidson 65565 3.45 +gabriella ellison 65706 1.15 +gabriella ellison 65716 2.06 +gabriella falkner 65623 2.09 +gabriella falkner 65711 2.48 +gabriella falkner 65767 1.82 +gabriella garcia 65571 3.17 +gabriella hernandez 65587 0.74 +gabriella hernandez 65717 0.96 +gabriella ichabod 65559 0.63 +gabriella ichabod 65633 2.42 +gabriella ichabod 65702 3.27 +gabriella ichabod 65712 3.6 +gabriella ichabod 65717 3.56 +gabriella king 65651 3.59 +gabriella king 65657 2.17 +gabriella laertes 65781 2.81 +gabriella miller 65646 3.47 +gabriella ovid 65556 1.23 +gabriella ovid 65583 1.95 +gabriella polk 65701 3.58 +gabriella polk 65790 2.08 +gabriella steinbeck 65582 3.6 +gabriella steinbeck 65653 2.7 +gabriella thompson 65682 1.78 +gabriella thompson 65755 3.21 +gabriella thompson 65766 2.71 +gabriella van buren 65581 3.36 +gabriella van buren 65644 2.6 +gabriella white 65638 4.55 +gabriella young 65699 4.13 +gabriella young 65774 3.58 +gabriella zipper 65540 0.96 +gabriella zipper 65754 2.13 +holly allen 65596 0.05 +holly brown 65599 3.23 +holly brown 65619 3.4 +holly falkner 65720 4.06 +holly hernandez 65602 3.67 +holly hernandez 65686 3.39 +holly hernandez 65750 3.2 +holly hernandez 65788 2.95 +holly ichabod 65711 4.73 +holly ichabod 65749 3.54 +holly ichabod 65752 3.27 +holly johnson 65655 4.19 +holly johnson 65662 3.84 +holly johnson 65755 2.65 +holly king 65549 3.61 +holly king 65648 2.37 +holly laertes 65664 4.14 +holly miller 65653 3.84 +holly nixon 65539 4.09 +holly nixon 65658 3.04 +holly polk 65743 2.1 +holly polk 65751 2.78 +holly robinson 65564 0.24 +holly thompson 65538 2.39 +holly thompson 65578 1.49 +holly thompson 65713 1.54 +holly underhill 65634 4.69 +holly underhill 65654 3.08 +holly underhill 65721 3.14 +holly underhill 65759 2.61 +holly van buren 65727 0.04 +holly white 65536 4.59 +holly white 65602 4.13 +holly xylophone 65544 1.49 +holly young 65606 4.39 +holly young 65765 3.81 +holly zipper 65607 4.12 +holly zipper 65755 3.3 +irene allen 65556 3.45 +irene brown 65633 4.8 +irene brown 65650 3.77 +irene brown 65765 3.53 +irene carson 65590 2.83 +irene ellison 65659 3.15 +irene ellison 65696 2.0 +irene falkner 65620 0.99 +irene falkner 65661 1.41 +irene garcia 65660 1.55 +irene garcia 65711 1.72 +irene garcia 65787 1.57 +irene ichabod 65645 0.95 +irene ichabod 65722 1.49 +irene johnson 65583 4.51 +irene laertes 65664 2.5 +irene laertes 65710 1.5 +irene laertes 65722 2.01 +irene miller 65730 4.33 +irene nixon 65631 2.36 +irene nixon 65643 3.43 +irene nixon 65653 2.43 +irene ovid 65691 3.24 +irene ovid 65734 3.17 +irene ovid 65753 3.18 +irene polk 65551 4.51 +irene polk 65575 2.97 +irene polk 65579 3.58 +irene polk 65595 2.82 +irene polk 65610 2.99 +irene quirinius 65724 3.5 +irene quirinius 65769 3.85 +irene quirinius 65773 4.21 +irene robinson 65554 2.67 +irene steinbeck 65683 1.48 +irene thompson 65688 0.06 +irene underhill 65591 3.61 +irene underhill 65787 4.01 +irene van buren 65579 4.26 +irene van buren 65589 4.37 +irene xylophone 65775 4.81 +jessica brown 65588 2.87 +jessica carson 65553 1.02 +jessica carson 65672 1.82 +jessica carson 65747 1.91 +jessica davidson 65549 4.48 +jessica davidson 65606 2.72 +jessica davidson 65675 2.23 +jessica davidson 65727 2.12 +jessica ellison 65567 3.0 +jessica ellison 65663 3.15 +jessica falkner 65584 2.11 +jessica garcia 65676 2.13 +jessica garcia 65789 3.54 +jessica ichabod 65704 3.48 +jessica johnson 65607 3.55 +jessica johnson 65720 3.0 +jessica miller 65733 2.9 +jessica nixon 65590 2.18 +jessica nixon 65774 3.1 +jessica ovid 65582 3.23 +jessica ovid 65751 3.14 +jessica polk 65637 1.12 +jessica quirinius 65562 3.06 +jessica quirinius 65608 1.75 +jessica quirinius 65712 1.51 +jessica quirinius 65716 1.37 +jessica robinson 65576 1.11 +jessica thompson 65581 4.94 +jessica thompson 65675 3.56 +jessica underhill 65656 3.97 +jessica underhill 65702 3.01 +jessica underhill 65783 3.5 +jessica van buren 65615 2.15 +jessica white 65544 1.89 +jessica white 65570 1.92 +jessica white 65594 2.67 +jessica white 65673 2.1 +jessica white 65779 2.04 +jessica xylophone 65562 0.49 +jessica young 65623 0.5 +jessica young 65711 2.37 +jessica zipper 65600 1.71 +jessica zipper 65657 1.0 +jessica zipper 65778 0.9 +katie allen 65542 1.3 +katie brown 65590 3.06 +katie davidson 65619 1.89 +katie ellison 65675 1.92 +katie ellison 65699 2.55 +katie falkner 65728 2.42 +katie garcia 65625 4.18 +katie garcia 65747 4.33 +katie hernandez 65550 2.1 +katie ichabod 65658 1.84 +katie ichabod 65726 2.41 +katie ichabod 65757 3.2 +katie king 65629 0.86 +katie king 65647 2.09 +katie king 65776 2.74 +katie miller 65541 0.66 +katie miller 65661 1.39 +katie nixon 65669 2.06 +katie ovid 65681 1.61 +katie polk 65746 4.74 +katie polk 65784 2.57 +katie robinson 65697 4.74 +katie van buren 65643 0.61 +katie van buren 65730 2.79 +katie white 65620 0.42 +katie white 65719 1.38 +katie xylophone 65585 3.02 +katie young 65644 2.64 +katie young 65746 2.45 +katie young 65764 2.66 +katie zipper 65568 2.33 +katie zipper 65733 1.97 +luke allen 65547 2.04 +luke allen 65552 1.49 +luke allen 65576 2.14 +luke allen 65681 2.2 +luke allen 65776 1.92 +luke brown 65719 4.3 +luke davidson 65656 3.37 +luke davidson 65791 3.9 +luke ellison 65582 0.23 +luke ellison 65664 0.51 +luke ellison 65779 0.87 +luke falkner 65589 2.22 +luke falkner 65618 1.22 +luke garcia 65687 4.73 +luke garcia 65778 3.56 +luke ichabod 65629 3.07 +luke ichabod 65654 3.58 +luke johnson 65545 4.33 +luke johnson 65716 3.01 +luke johnson 65718 3.17 +luke laertes 65608 3.79 +luke laertes 65657 3.89 +luke laertes 65685 2.82 +luke laertes 65730 2.96 +luke laertes 65756 3.19 +luke miller 65752 4.7 +luke ovid 65569 4.4 +luke ovid 65693 2.58 +luke polk 65645 0.57 +luke polk 65658 2.73 +luke quirinius 65655 4.1 +luke robinson 65634 4.9 +luke robinson 65772 4.19 +luke thompson 65626 0.15 +luke underhill 65553 1.28 +luke underhill 65571 0.84 +luke underhill 65651 1.14 +luke van buren 65678 0.34 +luke white 65693 0.91 +luke xylophone 65597 2.27 +luke zipper 65641 4.63 +mike allen 65706 3.06 +mike brown 65654 3.57 +mike carson 65698 4.46 +mike carson 65700 3.89 +mike carson 65751 3.58 +mike davidson 65658 2.06 +mike davidson 65759 3.34 +mike ellison 65598 3.96 +mike ellison 65606 3.28 +mike ellison 65718 3.38 +mike ellison 65738 2.56 +mike ellison 65760 3.03 +mike falkner 65609 4.85 +mike garcia 65571 1.82 +mike garcia 65600 1.42 +mike garcia 65770 1.92 +mike hernandez 65548 1.42 +mike hernandez 65672 1.75 +mike ichabod 65621 3.73 +mike king 65563 4.34 +mike king 65586 3.75 +mike king 65591 3.09 +mike king 65642 2.69 +mike king 65769 2.36 +mike king 65776 2.55 +mike miller 65549 3.96 +mike nixon 65619 0.09 +mike nixon 65704 2.15 +mike polk 65619 4.13 +mike polk 65658 4.27 +mike polk 65704 3.77 +mike quirinius 65717 2.81 +mike steinbeck 65550 3.18 +mike steinbeck 65564 2.58 +mike steinbeck 65573 2.12 +mike steinbeck 65749 1.72 +mike van buren 65620 0.09 +mike van buren 65770 0.88 +mike white 65648 1.72 +mike white 65685 1.12 +mike white 65769 1.74 +mike white 65778 2.05 +mike young 65545 1.69 +mike young 65581 0.92 +mike young 65736 1.84 +mike zipper 65552 4.8 +mike zipper 65695 4.16 +mike zipper 65779 4.22 +nick allen 65641 3.78 +nick allen 65786 3.74 +nick brown 65724 4.61 +nick davidson 65601 0.88 +nick ellison 65691 4.04 +nick ellison 65745 3.22 +nick falkner 65583 4.44 +nick falkner 65676 3.08 +nick garcia 65695 1.05 +nick garcia 65712 2.18 +nick garcia 65720 1.94 +nick ichabod 65572 2.62 +nick ichabod 65681 2.99 +nick ichabod 65737 3.55 +nick johnson 65585 0.56 +nick johnson 65784 0.42 +nick laertes 65624 0.16 +nick miller 65757 4.23 +nick nixon 65650 0.7 +nick ovid 65719 3.93 +nick polk 65716 3.66 +nick quirinius 65588 2.88 +nick quirinius 65723 2.42 +nick robinson 65547 0.21 +nick robinson 65675 0.57 +nick steinbeck 65689 4.11 +nick thompson 65610 2.32 +nick underhill 65619 2.73 +nick van buren 65603 1.84 +nick xylophone 65644 2.06 +nick young 65654 2.06 +nick young 65660 2.28 +nick zipper 65757 3.8 +nick zipper 65765 2.04 +oscar allen 65644 3.01 +oscar brown 65614 3.95 +oscar carson 65537 4.29 +oscar carson 65548 2.89 +oscar carson 65549 1.95 +oscar carson 65624 2.56 +oscar carson 65697 3.03 +oscar davidson 65556 0.6 +oscar ellison 65630 1.47 +oscar ellison 65630 1.47 +oscar falkner 65692 3.44 +oscar garcia 65751 3.71 +oscar hernandez 65683 3.32 +oscar hernandez 65707 2.25 +oscar ichabod 65536 1.8 +oscar ichabod 65562 1.18 +oscar ichabod 65637 1.91 +oscar ichabod 65763 1.96 +oscar johnson 65645 1.69 +oscar johnson 65778 1.59 +oscar king 65541 3.96 +oscar king 65550 4.31 +oscar king 65787 3.54 +oscar laertes 65625 2.26 +oscar laertes 65690 2.55 +oscar laertes 65756 2.85 +oscar laertes 65790 2.39 +oscar nixon 65596 3.12 +oscar ovid 65536 2.94 +oscar ovid 65615 2.95 +oscar ovid 65659 3.41 +oscar polk 65541 1.12 +oscar polk 65643 1.62 +oscar quirinius 65541 3.5 +oscar quirinius 65560 2.87 +oscar quirinius 65689 3.35 +oscar quirinius 65720 3.03 +oscar robinson 65537 0.29 +oscar robinson 65658 0.29 +oscar robinson 65687 1.5 +oscar robinson 65782 1.78 +oscar steinbeck 65709 4.96 +oscar thompson 65542 0.48 +oscar thompson 65698 2.07 +oscar thompson 65727 2.01 +oscar thompson 65738 1.8 +oscar underhill 65626 3.49 +oscar van buren 65581 2.33 +oscar van buren 65635 2.45 +oscar van buren 65705 2.68 +oscar white 65552 3.05 +oscar white 65564 2.58 +oscar white 65671 2.56 +oscar white 65735 2.47 +oscar xylophone 65773 1.51 +oscar xylophone 65773 1.51 +oscar xylophone 65775 1.82 +oscar zipper 65568 4.77 +oscar zipper 65740 3.81 +oscar zipper 65777 3.26 +priscilla brown 65670 2.91 +priscilla brown 65690 2.83 +priscilla brown 65749 2.07 +priscilla carson 65658 1.43 +priscilla carson 65687 2.97 +priscilla carson 65755 2.87 +priscilla ichabod 65627 4.95 +priscilla ichabod 65759 3.41 +priscilla johnson 65543 3.85 +priscilla johnson 65633 2.98 +priscilla johnson 65668 2.23 +priscilla johnson 65681 1.98 +priscilla johnson 65755 1.94 +priscilla king 65646 1.93 +priscilla nixon 65564 0.31 +priscilla nixon 65600 2.35 +priscilla ovid 65541 3.8 +priscilla ovid 65790 2.37 +priscilla polk 65747 3.1 +priscilla quirinius 65672 0.81 +priscilla thompson 65654 2.04 +priscilla underhill 65715 0.82 +priscilla underhill 65729 1.33 +priscilla van buren 65607 0.23 +priscilla van buren 65685 0.81 +priscilla van buren 65749 1.84 +priscilla white 65652 4.79 +priscilla xylophone 65538 3.56 +priscilla xylophone 65763 2.48 +priscilla xylophone 65774 1.84 +priscilla young 65585 2.92 +priscilla young 65658 3.77 +priscilla zipper 65622 4.62 +priscilla zipper 65726 2.67 +quinn allen 65657 3.02 +quinn allen 65708 3.35 +quinn brown 65691 4.37 +quinn brown 65700 3.28 +quinn brown 65733 3.27 +quinn davidson 65549 0.03 +quinn davidson 65714 1.02 +quinn davidson 65776 2.19 +quinn davidson 65779 2.66 +quinn ellison 65705 0.54 +quinn ellison 65778 2.74 +quinn garcia 65568 1.04 +quinn garcia 65604 0.79 +quinn garcia 65610 1.85 +quinn garcia 65773 1.67 +quinn ichabod 65564 0.65 +quinn king 65558 2.38 +quinn king 65649 1.53 +quinn laertes 65542 2.33 +quinn laertes 65560 2.03 +quinn laertes 65627 2.08 +quinn nixon 65659 0.72 +quinn ovid 65699 2.87 +quinn quirinius 65747 0.86 +quinn robinson 65627 4.14 +quinn steinbeck 65578 4.92 +quinn steinbeck 65763 3.18 +quinn thompson 65643 4.62 +quinn thompson 65774 2.43 +quinn underhill 65549 3.89 +quinn underhill 65694 2.39 +quinn underhill 65767 2.48 +quinn van buren 65725 0.53 +quinn young 65647 3.54 +quinn zipper 65579 1.83 +quinn zipper 65693 2.1 +rachel allen 65661 4.66 +rachel allen 65709 4.53 +rachel brown 65586 0.43 +rachel brown 65587 2.69 +rachel brown 65587 2.69 +rachel brown 65610 2.67 +rachel brown 65693 3.07 +rachel carson 65677 4.58 +rachel carson 65682 4.54 +rachel davidson 65755 2.28 +rachel ellison 65761 0.05 +rachel falkner 65616 1.2 +rachel falkner 65681 2.01 +rachel falkner 65693 2.56 +rachel falkner 65764 2.91 +rachel johnson 65658 3.02 +rachel king 65604 2.32 +rachel king 65643 2.4 +rachel laertes 65562 4.43 +rachel laertes 65624 2.8 +rachel ovid 65721 0.3 +rachel ovid 65736 0.92 +rachel polk 65686 2.56 +rachel quirinius 65787 2.95 +rachel robinson 65544 0.06 +rachel robinson 65717 1.55 +rachel robinson 65724 2.27 +rachel thompson 65648 2.49 +rachel thompson 65662 3.16 +rachel thompson 65733 2.51 +rachel underhill 65667 2.29 +rachel white 65615 1.99 +rachel white 65717 3.08 +rachel young 65727 1.75 +rachel zipper 65757 2.82 +rachel zipper 65785 3.62 +sarah carson 65679 1.04 +sarah carson 65693 0.85 +sarah carson 65694 2.06 +sarah ellison 65611 1.64 +sarah falkner 65606 0.77 +sarah falkner 65680 2.85 +sarah garcia 65563 4.89 +sarah garcia 65638 4.78 +sarah garcia 65661 3.63 +sarah ichabod 65667 3.94 +sarah ichabod 65671 2.33 +sarah johnson 65659 3.51 +sarah johnson 65716 4.21 +sarah johnson 65731 3.81 +sarah johnson 65751 3.37 +sarah king 65650 1.05 +sarah king 65699 0.99 +sarah miller 65557 0.2 +sarah ovid 65550 3.21 +sarah robinson 65677 4.9 +sarah robinson 65763 2.99 +sarah steinbeck 65721 2.82 +sarah white 65622 0.07 +sarah white 65747 2.29 +sarah xylophone 65678 0.15 +sarah young 65595 2.15 +sarah zipper 65550 2.22 +tom brown 65593 1.64 +tom brown 65675 2.83 +tom carson 65539 4.38 +tom carson 65624 4.28 +tom carson 65780 4.03 +tom davidson 65780 2.4 +tom ellison 65578 3.3 +tom ellison 65670 4.04 +tom ellison 65756 3.51 +tom falkner 65574 1.09 +tom falkner 65583 2.05 +tom hernandez 65575 2.35 +tom hernandez 65632 2.64 +tom ichabod 65588 1.48 +tom johnson 65536 4.68 +tom johnson 65789 4.6 +tom king 65576 2.87 +tom laertes 65617 1.51 +tom laertes 65701 1.93 +tom miller 65594 1.14 +tom miller 65603 1.25 +tom miller 65704 1.26 +tom nixon 65672 0.04 +tom ovid 65628 1.95 +tom polk 65652 2.54 +tom polk 65742 2.26 +tom quirinius 65563 4.51 +tom quirinius 65783 4.17 +tom robinson 65626 3.12 +tom robinson 65632 3.61 +tom robinson 65691 3.13 +tom robinson 65758 3.45 +tom steinbeck 65666 1.34 +tom van buren 65621 0.66 +tom van buren 65652 2.71 +tom van buren 65669 3.47 +tom white 65548 2.13 +tom young 65544 3.73 +tom young 65546 2.02 +tom zipper 65789 1.51 +ulysses brown 65735 2.92 +ulysses carson 65602 0.34 +ulysses carson 65643 2.11 +ulysses carson 65703 1.43 +ulysses carson 65716 1.41 +ulysses davidson 65750 3.04 +ulysses ellison 65575 4.39 +ulysses garcia 65666 4.2 +ulysses hernandez 65651 1.75 +ulysses hernandez 65702 2.42 +ulysses hernandez 65786 2.53 +ulysses ichabod 65551 0.33 +ulysses ichabod 65566 2.19 +ulysses johnson 65776 4.79 +ulysses king 65649 4.46 +ulysses laertes 65691 4.55 +ulysses laertes 65711 3.54 +ulysses laertes 65781 3.66 +ulysses miller 65610 0.24 +ulysses miller 65637 1.08 +ulysses nixon 65603 1.85 +ulysses ovid 65656 3.17 +ulysses polk 65563 1.32 +ulysses polk 65580 3.05 +ulysses polk 65612 3.46 +ulysses polk 65777 3.75 +ulysses quirinius 65786 2.13 +ulysses robinson 65744 1.97 +ulysses steinbeck 65611 2.74 +ulysses steinbeck 65680 2.64 +ulysses thompson 65788 1.51 +ulysses underhill 65570 0.38 +ulysses underhill 65616 0.8 +ulysses underhill 65620 2.09 +ulysses underhill 65623 2.69 +ulysses underhill 65641 2.54 +ulysses underhill 65713 2.9 +ulysses underhill 65785 2.97 +ulysses van buren 65684 1.42 +ulysses white 65654 0.14 +ulysses white 65675 1.51 +ulysses xylophone 65623 2.3 +ulysses xylophone 65636 2.69 +ulysses xylophone 65781 3.22 +ulysses young 65675 1.34 +ulysses young 65736 2.01 +ulysses young 65748 2.24 +victor allen 65684 0.83 +victor allen 65707 2.31 +victor brown 65550 4.57 +victor brown 65555 3.54 +victor brown 65622 2.61 +victor brown 65673 2.34 +victor davidson 65579 0.61 +victor davidson 65628 1.52 +victor davidson 65783 2.25 +victor ellison 65641 1.32 +victor ellison 65782 2.59 +victor hernandez 65571 3.62 +victor hernandez 65659 3.68 +victor hernandez 65708 3.35 +victor hernandez 65735 2.88 +victor hernandez 65775 2.62 +victor johnson 65606 3.03 +victor johnson 65607 2.3 +victor johnson 65607 2.3 +victor king 65721 4.09 +victor king 65743 2.45 +victor laertes 65638 1.46 +victor laertes 65644 2.38 +victor miller 65570 0.1 +victor nixon 65709 0.74 +victor nixon 65791 1.73 +victor ovid 65649 4.93 +victor polk 65625 1.04 +victor quirinius 65620 1.32 +victor quirinius 65651 3.15 +victor robinson 65596 0.92 +victor robinson 65673 1.76 +victor steinbeck 65618 2.87 +victor steinbeck 65661 2.19 +victor steinbeck 65686 2.81 +victor thompson 65548 1.59 +victor van buren 65664 4.44 +victor van buren 65774 4.06 +victor white 65548 4.67 +victor white 65601 3.87 +victor xylophone 65549 3.8 +victor xylophone 65618 2.13 +victor xylophone 65644 1.59 +victor xylophone 65677 1.89 +victor xylophone 65755 2.27 +victor young 65628 3.16 +victor zipper 65743 3.98 +wendy allen 65628 3.8 +wendy allen 65711 3.44 +wendy allen 65782 2.4 +wendy brown 65580 4.67 +wendy brown 65657 4.68 +wendy ellison 65545 1.51 +wendy ellison 65603 1.6 +wendy falkner 65595 0.58 +wendy falkner 65604 0.82 +wendy falkner 65635 1.59 +wendy garcia 65659 3.47 +wendy garcia 65746 2.35 +wendy garcia 65747 1.67 +wendy garcia 65777 1.32 +wendy hernandez 65650 2.26 +wendy ichabod 65730 0.44 +wendy king 65586 4.46 +wendy king 65664 4.25 +wendy king 65670 2.94 +wendy laertes 65566 3.13 +wendy laertes 65683 3.99 +wendy laertes 65727 3.57 +wendy miller 65582 1.53 +wendy miller 65626 1.4 +wendy nixon 65611 0.26 +wendy nixon 65746 2.27 +wendy ovid 65589 4.75 +wendy ovid 65643 3.42 +wendy polk 65656 0.62 +wendy polk 65692 1.36 +wendy quirinius 65766 1.35 +wendy quirinius 65767 2.76 +wendy robinson 65622 0.85 +wendy robinson 65715 2.13 +wendy robinson 65774 1.8 +wendy steinbeck 65612 0.07 +wendy thompson 65650 2.27 +wendy thompson 65737 3.2 +wendy underhill 65662 4.55 +wendy underhill 65758 2.84 +wendy underhill 65775 2.54 +wendy van buren 65680 1.1 +wendy van buren 65699 1.0 +wendy white 65705 0.5 +wendy xylophone 65687 0.46 +wendy xylophone 65773 1.39 +wendy young 65674 0.48 +wendy young 65685 1.31 +xavier allen 65611 1.53 +xavier allen 65618 2.07 +xavier allen 65771 2.33 +xavier brown 65600 0.89 +xavier brown 65704 0.58 +xavier brown 65723 1.55 +xavier carson 65731 4.42 +xavier carson 65758 3.91 +xavier davidson 65644 1.84 +xavier davidson 65664 3.4 +xavier davidson 65755 2.67 +xavier ellison 65541 1.47 +xavier ellison 65654 2.49 +xavier garcia 65672 2.76 +xavier hernandez 65541 0.96 +xavier hernandez 65544 1.47 +xavier hernandez 65766 1.28 +xavier ichabod 65597 4.76 +xavier ichabod 65663 2.99 +xavier johnson 65654 1.34 +xavier johnson 65744 3.06 +xavier king 65590 2.68 +xavier king 65601 1.4 +xavier laertes 65743 0.75 +xavier ovid 65788 2.06 +xavier polk 65587 0.99 +xavier polk 65653 1.15 +xavier polk 65675 1.9 +xavier polk 65696 1.93 +xavier quirinius 65599 1.66 +xavier quirinius 65650 1.94 +xavier quirinius 65656 2.46 +xavier quirinius 65737 1.92 +xavier thompson 65608 1.65 +xavier underhill 65710 0.13 +xavier white 65703 0.98 +xavier white 65732 2.22 +xavier xylophone 65572 1.0 +xavier zipper 65561 0.94 +yuri allen 65565 2.03 +yuri allen 65682 1.14 +yuri brown 65538 2.73 +yuri brown 65688 2.02 +yuri carson 65670 3.06 +yuri carson 65769 3.43 +yuri ellison 65570 1.05 +yuri ellison 65581 1.68 +yuri falkner 65658 2.85 +yuri falkner 65681 2.14 +yuri garcia 65639 3.41 +yuri hernandez 65706 1.64 +yuri johnson 65587 1.27 +yuri johnson 65697 1.44 +yuri johnson 65712 2.29 +yuri king 65721 0.33 +yuri laertes 65637 4.3 +yuri laertes 65773 2.15 +yuri nixon 65635 4.02 +yuri nixon 65740 4.18 +yuri polk 65607 0.08 +yuri polk 65713 0.37 +yuri polk 65742 1.25 +yuri quirinius 65544 2.58 +yuri quirinius 65617 2.1 +yuri quirinius 65695 1.91 +yuri steinbeck 65592 4.89 +yuri steinbeck 65679 3.24 +yuri thompson 65676 2.67 +yuri underhill 65718 2.86 +yuri underhill 65750 2.51 +yuri white 65659 4.59 +yuri xylophone 65714 2.53 +zach allen 65667 0.88 +zach brown 65559 4.88 +zach brown 65588 4.53 +zach brown 65691 3.49 +zach brown 65759 3.4 +zach brown 65762 3.55 +zach carson 65572 2.03 +zach ellison 65748 1.76 +zach falkner 65620 0.34 +zach falkner 65627 0.25 +zach garcia 65544 0.99 +zach garcia 65623 2.84 +zach garcia 65629 3.01 +zach garcia 65786 2.55 +zach ichabod 65599 3.36 +zach ichabod 65612 1.92 +zach king 65556 2.36 +zach king 65702 1.52 +zach king 65773 2.58 +zach miller 65584 1.6 +zach miller 65665 0.99 +zach miller 65719 1.55 +zach ovid 65578 1.51 +zach ovid 65703 1.92 +zach ovid 65750 2.63 +zach ovid 65784 2.72 +zach quirinius 65691 2.95 +zach robinson 65599 2.87 +zach steinbeck 65602 2.45 +zach steinbeck 65695 1.86 +zach thompson 65636 0.25 +zach thompson 65696 0.51 +zach underhill 65573 3.97 +zach white 65733 2.31 +zach xylophone 65542 1.69 +zach xylophone 65780 0.88 +zach young 65576 1.82 +zach zipper 65579 4.5 +zach zipper 65649 4.02 +zach zipper 65676 3.12 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-8-45a1d7c2aba45d761e19ff4dfdf5463e b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-8-45a1d7c2aba45d761e19ff4dfdf5463e new file mode 100644 index 0000000000000000000000000000000000000000..84b934fad85b17b209410489fb7c58f7368a2fae --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-8-45a1d7c2aba45d761e19ff4dfdf5463e @@ -0,0 +1,1049 @@ + 65560 20.0 + 65718 20.0 + 65740 20.0 +alice allen 65662 20.0 +alice allen 65720 20.0 +alice allen 65758 20.0 +alice brown 65696 20.0 +alice carson 65559 20.0 +alice davidson 65547 20.0 +alice falkner 65669 20.0 +alice garcia 65613 20.0 +alice hernandez 65737 20.0 +alice hernandez 65784 20.0 +alice johnson 65739 20.0 +alice king 65660 20.0 +alice king 65734 20.0 +alice king 65738 20.0 +alice laertes 65669 20.0 +alice laertes 65671 20.0 +alice miller 65590 20.0 +alice nixon 65586 20.0 +alice nixon 65595 20.0 +alice nixon 65604 20.0 +alice ovid 65737 20.0 +alice polk 65548 20.0 +alice quirinius 65636 20.0 +alice quirinius 65728 20.0 +alice robinson 65606 20.0 +alice robinson 65789 20.0 +alice steinbeck 65578 20.0 +alice steinbeck 65673 20.0 +alice steinbeck 65786 20.0 +alice underhill 65750 20.0 +alice van buren 65562 20.0 +alice xylophone 65578 20.0 +alice xylophone 65585 20.0 +alice xylophone 65599 20.0 +alice zipper 65553 20.0 +alice zipper 65662 20.0 +alice zipper 65766 20.0 +bob brown 65584 20.0 +bob brown 65777 20.0 +bob brown 65783 20.0 +bob carson 65713 20.0 +bob davidson 65664 20.0 +bob davidson 65693 20.0 +bob davidson 65768 20.0 +bob ellison 65591 20.0 +bob ellison 65624 20.0 +bob ellison 65721 20.0 +bob ellison 65760 20.0 +bob falkner 65789 20.0 +bob garcia 65585 20.0 +bob garcia 65598 20.0 +bob garcia 65673 20.0 +bob garcia 65754 20.0 +bob garcia 65782 20.0 +bob hernandez 65557 20.0 +bob ichabod 65549 20.0 +bob king 65715 20.0 +bob king 65757 20.0 +bob king 65783 20.0 +bob laertes 65602 20.0 +bob laertes 65663 20.0 +bob miller 65608 20.0 +bob ovid 65564 20.0 +bob ovid 65619 20.0 +bob ovid 65686 20.0 +bob ovid 65726 20.0 +bob polk 65594 20.0 +bob quirinius 65700 20.0 +bob steinbeck 65637 20.0 +bob van buren 65778 20.0 +bob white 65543 20.0 +bob white 65605 20.0 +bob xylophone 65574 20.0 +bob xylophone 65666 20.0 +bob young 65556 20.0 +bob zipper 65559 20.0 +bob zipper 65633 20.0 +bob zipper 65739 20.0 +calvin allen 65669 20.0 +calvin brown 65537 20.0 +calvin brown 65580 20.0 +calvin brown 65677 20.0 +calvin carson 65637 20.0 +calvin davidson 65541 20.0 +calvin davidson 65564 20.0 +calvin ellison 65667 20.0 +calvin falkner 65573 20.0 +calvin falkner 65596 20.0 +calvin falkner 65738 20.0 +calvin falkner 65762 20.0 +calvin falkner 65778 20.0 +calvin falkner 65784 20.0 +calvin garcia 65664 20.0 +calvin hernandez 65578 20.0 +calvin johnson 65731 20.0 +calvin laertes 65570 20.0 +calvin laertes 65684 20.0 +calvin nixon 65654 20.0 +calvin nixon 65724 20.0 +calvin nixon 65749 20.0 +calvin ovid 65554 20.0 +calvin ovid 65643 20.0 +calvin ovid 65663 20.0 +calvin ovid 65715 20.0 +calvin polk 65731 20.0 +calvin quirinius 65741 20.0 +calvin quirinius 65769 20.0 +calvin robinson 65581 20.0 +calvin steinbeck 65680 20.0 +calvin steinbeck 65762 20.0 +calvin steinbeck 65779 20.0 +calvin thompson 65560 20.0 +calvin thompson 65640 20.0 +calvin underhill 65732 20.0 +calvin van buren 65552 20.0 +calvin van buren 65771 20.0 +calvin white 65553 20.0 +calvin white 65561 20.0 +calvin xylophone 65575 20.0 +calvin xylophone 65596 20.0 +calvin xylophone 65713 20.0 +calvin young 65574 20.0 +calvin young 65746 20.0 +calvin zipper 65669 20.0 +calvin zipper 65739 20.0 +david allen 65588 20.0 +david allen 65617 20.0 +david brown 65637 20.0 +david brown 65760 20.0 +david davidson 65559 20.0 +david davidson 65756 20.0 +david davidson 65778 20.0 +david davidson 65779 20.0 +david ellison 65634 20.0 +david ellison 65724 20.0 +david ellison 65724 20.0 +david hernandez 65763 20.0 +david ichabod 65699 20.0 +david ichabod 65715 20.0 +david laertes 65762 20.0 +david nixon 65536 20.0 +david ovid 65623 20.0 +david ovid 65628 20.0 +david quirinius 65697 20.0 +david quirinius 65759 20.0 +david quirinius 65779 20.0 +david robinson 65762 20.0 +david robinson 65775 20.0 +david thompson 65550 20.0 +david underhill 65602 20.0 +david underhill 65662 20.0 +david underhill 65751 20.0 +david van buren 65625 20.0 +david van buren 65634 20.0 +david white 65678 20.0 +david xylophone 65537 20.0 +david xylophone 65670 20.0 +david xylophone 65764 20.0 +david young 65551 20.0 +david young 65694 20.0 +ethan allen 65747 20.0 +ethan brown 65539 20.0 +ethan brown 65617 20.0 +ethan brown 65685 20.0 +ethan brown 65685 20.0 +ethan brown 65722 20.0 +ethan brown 65733 20.0 +ethan carson 65742 20.0 +ethan ellison 65714 20.0 +ethan ellison 65732 20.0 +ethan falkner 65577 20.0 +ethan falkner 65614 20.0 +ethan garcia 65736 20.0 +ethan hernandez 65618 20.0 +ethan johnson 65536 20.0 +ethan king 65614 20.0 +ethan laertes 65562 20.0 +ethan laertes 65597 20.0 +ethan laertes 65628 20.0 +ethan laertes 65643 20.0 +ethan laertes 65680 20.0 +ethan laertes 65745 20.0 +ethan laertes 65760 20.0 +ethan miller 65712 20.0 +ethan nixon 65766 20.0 +ethan ovid 65697 20.0 +ethan polk 65589 20.0 +ethan polk 65615 20.0 +ethan polk 65622 20.0 +ethan polk 65622 20.0 +ethan quirinius 65542 20.0 +ethan quirinius 65591 20.0 +ethan quirinius 65706 20.0 +ethan robinson 65547 20.0 +ethan robinson 65659 20.0 +ethan underhill 65570 20.0 +ethan van buren 65572 20.0 +ethan white 65677 20.0 +ethan white 65788 20.0 +ethan xylophone 65595 20.0 +ethan zipper 65593 20.0 +ethan zipper 65680 20.0 +fred davidson 65552 20.0 +fred davidson 65595 20.0 +fred davidson 65721 20.0 +fred ellison 65548 20.0 +fred ellison 65691 20.0 +fred ellison 65771 20.0 +fred falkner 65637 20.0 +fred falkner 65648 20.0 +fred falkner 65783 20.0 +fred hernandez 65541 20.0 +fred ichabod 65572 20.0 +fred ichabod 65789 20.0 +fred johnson 65758 20.0 +fred king 65694 20.0 +fred king 65745 20.0 +fred laertes 65769 20.0 +fred miller 65536 20.0 +fred nixon 65560 20.0 +fred nixon 65612 20.0 +fred nixon 65703 20.0 +fred nixon 65705 20.0 +fred polk 65603 20.0 +fred polk 65656 20.0 +fred polk 65701 20.0 +fred polk 65706 20.0 +fred quirinius 65697 20.0 +fred quirinius 65782 20.0 +fred robinson 65623 20.0 +fred steinbeck 65544 20.0 +fred steinbeck 65651 20.0 +fred steinbeck 65755 20.0 +fred underhill 65629 20.0 +fred van buren 65537 20.0 +fred van buren 65561 20.0 +fred van buren 65745 20.0 +fred van buren 65789 20.0 +fred white 65589 20.0 +fred young 65594 20.0 +fred young 65773 20.0 +fred zipper 65553 20.0 +gabriella allen 65646 20.0 +gabriella allen 65677 20.0 +gabriella brown 65704 20.0 +gabriella brown 65753 20.0 +gabriella carson 65586 20.0 +gabriella davidson 65565 20.0 +gabriella ellison 65706 20.0 +gabriella ellison 65716 20.0 +gabriella falkner 65623 20.0 +gabriella falkner 65711 20.0 +gabriella falkner 65767 20.0 +gabriella garcia 65571 20.0 +gabriella hernandez 65587 20.0 +gabriella hernandez 65717 20.0 +gabriella ichabod 65559 20.0 +gabriella ichabod 65633 20.0 +gabriella ichabod 65702 20.0 +gabriella ichabod 65712 20.0 +gabriella ichabod 65717 20.0 +gabriella king 65651 20.0 +gabriella king 65657 20.0 +gabriella laertes 65781 20.0 +gabriella miller 65646 20.0 +gabriella ovid 65556 20.0 +gabriella ovid 65583 20.0 +gabriella polk 65701 20.0 +gabriella polk 65790 20.0 +gabriella steinbeck 65582 20.0 +gabriella steinbeck 65653 20.0 +gabriella thompson 65682 20.0 +gabriella thompson 65755 20.0 +gabriella thompson 65766 20.0 +gabriella van buren 65581 20.0 +gabriella van buren 65644 20.0 +gabriella white 65638 20.0 +gabriella young 65699 20.0 +gabriella young 65774 20.0 +gabriella zipper 65540 20.0 +gabriella zipper 65754 20.0 +holly allen 65596 20.0 +holly brown 65599 20.0 +holly brown 65619 20.0 +holly falkner 65720 20.0 +holly hernandez 65602 20.0 +holly hernandez 65686 20.0 +holly hernandez 65750 20.0 +holly hernandez 65788 20.0 +holly ichabod 65711 20.0 +holly ichabod 65749 20.0 +holly ichabod 65752 20.0 +holly johnson 65655 20.0 +holly johnson 65662 20.0 +holly johnson 65755 20.0 +holly king 65549 20.0 +holly king 65648 20.0 +holly laertes 65664 20.0 +holly miller 65653 20.0 +holly nixon 65539 20.0 +holly nixon 65658 20.0 +holly polk 65743 20.0 +holly polk 65751 20.0 +holly robinson 65564 20.0 +holly thompson 65538 20.0 +holly thompson 65578 20.0 +holly thompson 65713 20.0 +holly underhill 65634 20.0 +holly underhill 65654 20.0 +holly underhill 65721 20.0 +holly underhill 65759 20.0 +holly van buren 65727 20.0 +holly white 65536 20.0 +holly white 65602 20.0 +holly xylophone 65544 20.0 +holly young 65606 20.0 +holly young 65765 20.0 +holly zipper 65607 20.0 +holly zipper 65755 20.0 +irene allen 65556 20.0 +irene brown 65633 20.0 +irene brown 65650 20.0 +irene brown 65765 20.0 +irene carson 65590 20.0 +irene ellison 65659 20.0 +irene ellison 65696 20.0 +irene falkner 65620 20.0 +irene falkner 65661 20.0 +irene garcia 65660 20.0 +irene garcia 65711 20.0 +irene garcia 65787 20.0 +irene ichabod 65645 20.0 +irene ichabod 65722 20.0 +irene johnson 65583 20.0 +irene laertes 65664 20.0 +irene laertes 65710 20.0 +irene laertes 65722 20.0 +irene miller 65730 20.0 +irene nixon 65631 20.0 +irene nixon 65643 20.0 +irene nixon 65653 20.0 +irene ovid 65691 20.0 +irene ovid 65734 20.0 +irene ovid 65753 20.0 +irene polk 65551 20.0 +irene polk 65575 20.0 +irene polk 65579 20.0 +irene polk 65595 20.0 +irene polk 65610 20.0 +irene quirinius 65724 20.0 +irene quirinius 65769 20.0 +irene quirinius 65773 20.0 +irene robinson 65554 20.0 +irene steinbeck 65683 20.0 +irene thompson 65688 20.0 +irene underhill 65591 20.0 +irene underhill 65787 20.0 +irene van buren 65579 20.0 +irene van buren 65589 20.0 +irene xylophone 65775 20.0 +jessica brown 65588 20.0 +jessica carson 65553 20.0 +jessica carson 65672 20.0 +jessica carson 65747 20.0 +jessica davidson 65549 20.0 +jessica davidson 65606 20.0 +jessica davidson 65675 20.0 +jessica davidson 65727 20.0 +jessica ellison 65567 20.0 +jessica ellison 65663 20.0 +jessica falkner 65584 20.0 +jessica garcia 65676 20.0 +jessica garcia 65789 20.0 +jessica ichabod 65704 20.0 +jessica johnson 65607 20.0 +jessica johnson 65720 20.0 +jessica miller 65733 20.0 +jessica nixon 65590 20.0 +jessica nixon 65774 20.0 +jessica ovid 65582 20.0 +jessica ovid 65751 20.0 +jessica polk 65637 20.0 +jessica quirinius 65562 20.0 +jessica quirinius 65608 20.0 +jessica quirinius 65712 20.0 +jessica quirinius 65716 20.0 +jessica robinson 65576 20.0 +jessica thompson 65581 20.0 +jessica thompson 65675 20.0 +jessica underhill 65656 20.0 +jessica underhill 65702 20.0 +jessica underhill 65783 20.0 +jessica van buren 65615 20.0 +jessica white 65544 20.0 +jessica white 65570 20.0 +jessica white 65594 20.0 +jessica white 65673 20.0 +jessica white 65779 20.0 +jessica xylophone 65562 20.0 +jessica young 65623 20.0 +jessica young 65711 20.0 +jessica zipper 65600 20.0 +jessica zipper 65657 20.0 +jessica zipper 65778 20.0 +katie allen 65542 20.0 +katie brown 65590 20.0 +katie davidson 65619 20.0 +katie ellison 65675 20.0 +katie ellison 65699 20.0 +katie falkner 65728 20.0 +katie garcia 65625 20.0 +katie garcia 65747 20.0 +katie hernandez 65550 20.0 +katie ichabod 65658 20.0 +katie ichabod 65726 20.0 +katie ichabod 65757 20.0 +katie king 65629 20.0 +katie king 65647 20.0 +katie king 65776 20.0 +katie miller 65541 20.0 +katie miller 65661 20.0 +katie nixon 65669 20.0 +katie ovid 65681 20.0 +katie polk 65746 20.0 +katie polk 65784 20.0 +katie robinson 65697 20.0 +katie van buren 65643 20.0 +katie van buren 65730 20.0 +katie white 65620 20.0 +katie white 65719 20.0 +katie xylophone 65585 20.0 +katie young 65644 20.0 +katie young 65746 20.0 +katie young 65764 20.0 +katie zipper 65568 20.0 +katie zipper 65733 20.0 +luke allen 65547 20.0 +luke allen 65552 20.0 +luke allen 65576 20.0 +luke allen 65681 20.0 +luke allen 65776 20.0 +luke brown 65719 20.0 +luke davidson 65656 20.0 +luke davidson 65791 20.0 +luke ellison 65582 20.0 +luke ellison 65664 20.0 +luke ellison 65779 20.0 +luke falkner 65589 20.0 +luke falkner 65618 20.0 +luke garcia 65687 20.0 +luke garcia 65778 20.0 +luke ichabod 65629 20.0 +luke ichabod 65654 20.0 +luke johnson 65545 20.0 +luke johnson 65716 20.0 +luke johnson 65718 20.0 +luke laertes 65608 20.0 +luke laertes 65657 20.0 +luke laertes 65685 20.0 +luke laertes 65730 20.0 +luke laertes 65756 20.0 +luke miller 65752 20.0 +luke ovid 65569 20.0 +luke ovid 65693 20.0 +luke polk 65645 20.0 +luke polk 65658 20.0 +luke quirinius 65655 20.0 +luke robinson 65634 20.0 +luke robinson 65772 20.0 +luke thompson 65626 20.0 +luke underhill 65553 20.0 +luke underhill 65571 20.0 +luke underhill 65651 20.0 +luke van buren 65678 20.0 +luke white 65693 20.0 +luke xylophone 65597 20.0 +luke zipper 65641 20.0 +mike allen 65706 20.0 +mike brown 65654 20.0 +mike carson 65698 20.0 +mike carson 65700 20.0 +mike carson 65751 20.0 +mike davidson 65658 20.0 +mike davidson 65759 20.0 +mike ellison 65598 20.0 +mike ellison 65606 20.0 +mike ellison 65718 20.0 +mike ellison 65738 20.0 +mike ellison 65760 20.0 +mike falkner 65609 20.0 +mike garcia 65571 20.0 +mike garcia 65600 20.0 +mike garcia 65770 20.0 +mike hernandez 65548 20.0 +mike hernandez 65672 20.0 +mike ichabod 65621 20.0 +mike king 65563 20.0 +mike king 65586 20.0 +mike king 65591 20.0 +mike king 65642 20.0 +mike king 65769 20.0 +mike king 65776 20.0 +mike miller 65549 20.0 +mike nixon 65619 20.0 +mike nixon 65704 20.0 +mike polk 65619 20.0 +mike polk 65658 20.0 +mike polk 65704 20.0 +mike quirinius 65717 20.0 +mike steinbeck 65550 20.0 +mike steinbeck 65564 20.0 +mike steinbeck 65573 20.0 +mike steinbeck 65749 20.0 +mike van buren 65620 20.0 +mike van buren 65770 20.0 +mike white 65648 20.0 +mike white 65685 20.0 +mike white 65769 20.0 +mike white 65778 20.0 +mike young 65545 20.0 +mike young 65581 20.0 +mike young 65736 20.0 +mike zipper 65552 20.0 +mike zipper 65695 20.0 +mike zipper 65779 20.0 +nick allen 65641 20.0 +nick allen 65786 20.0 +nick brown 65724 20.0 +nick davidson 65601 20.0 +nick ellison 65691 20.0 +nick ellison 65745 20.0 +nick falkner 65583 20.0 +nick falkner 65676 20.0 +nick garcia 65695 20.0 +nick garcia 65712 20.0 +nick garcia 65720 20.0 +nick ichabod 65572 20.0 +nick ichabod 65681 20.0 +nick ichabod 65737 20.0 +nick johnson 65585 20.0 +nick johnson 65784 20.0 +nick laertes 65624 20.0 +nick miller 65757 20.0 +nick nixon 65650 20.0 +nick ovid 65719 20.0 +nick polk 65716 20.0 +nick quirinius 65588 20.0 +nick quirinius 65723 20.0 +nick robinson 65547 20.0 +nick robinson 65675 20.0 +nick steinbeck 65689 20.0 +nick thompson 65610 20.0 +nick underhill 65619 20.0 +nick van buren 65603 20.0 +nick xylophone 65644 20.0 +nick young 65654 20.0 +nick young 65660 20.0 +nick zipper 65757 20.0 +nick zipper 65765 20.0 +oscar allen 65644 20.0 +oscar brown 65614 20.0 +oscar carson 65537 20.0 +oscar carson 65548 20.0 +oscar carson 65549 20.0 +oscar carson 65624 20.0 +oscar carson 65697 20.0 +oscar davidson 65556 20.0 +oscar ellison 65630 20.0 +oscar ellison 65630 20.0 +oscar falkner 65692 20.0 +oscar garcia 65751 20.0 +oscar hernandez 65683 20.0 +oscar hernandez 65707 20.0 +oscar ichabod 65536 20.0 +oscar ichabod 65562 20.0 +oscar ichabod 65637 20.0 +oscar ichabod 65763 20.0 +oscar johnson 65645 20.0 +oscar johnson 65778 20.0 +oscar king 65541 20.0 +oscar king 65550 20.0 +oscar king 65787 20.0 +oscar laertes 65625 20.0 +oscar laertes 65690 20.0 +oscar laertes 65756 20.0 +oscar laertes 65790 20.0 +oscar nixon 65596 20.0 +oscar ovid 65536 20.0 +oscar ovid 65615 20.0 +oscar ovid 65659 20.0 +oscar polk 65541 20.0 +oscar polk 65643 20.0 +oscar quirinius 65541 20.0 +oscar quirinius 65560 20.0 +oscar quirinius 65689 20.0 +oscar quirinius 65720 20.0 +oscar robinson 65537 20.0 +oscar robinson 65658 20.0 +oscar robinson 65687 20.0 +oscar robinson 65782 20.0 +oscar steinbeck 65709 20.0 +oscar thompson 65542 20.0 +oscar thompson 65698 20.0 +oscar thompson 65727 20.0 +oscar thompson 65738 20.0 +oscar underhill 65626 20.0 +oscar van buren 65581 20.0 +oscar van buren 65635 20.0 +oscar van buren 65705 20.0 +oscar white 65552 20.0 +oscar white 65564 20.0 +oscar white 65671 20.0 +oscar white 65735 20.0 +oscar xylophone 65773 20.0 +oscar xylophone 65773 20.0 +oscar xylophone 65775 20.0 +oscar zipper 65568 20.0 +oscar zipper 65740 20.0 +oscar zipper 65777 20.0 +priscilla brown 65670 20.0 +priscilla brown 65690 20.0 +priscilla brown 65749 20.0 +priscilla carson 65658 20.0 +priscilla carson 65687 20.0 +priscilla carson 65755 20.0 +priscilla ichabod 65627 20.0 +priscilla ichabod 65759 20.0 +priscilla johnson 65543 20.0 +priscilla johnson 65633 20.0 +priscilla johnson 65668 20.0 +priscilla johnson 65681 20.0 +priscilla johnson 65755 20.0 +priscilla king 65646 20.0 +priscilla nixon 65564 20.0 +priscilla nixon 65600 20.0 +priscilla ovid 65541 20.0 +priscilla ovid 65790 20.0 +priscilla polk 65747 20.0 +priscilla quirinius 65672 20.0 +priscilla thompson 65654 20.0 +priscilla underhill 65715 20.0 +priscilla underhill 65729 20.0 +priscilla van buren 65607 20.0 +priscilla van buren 65685 20.0 +priscilla van buren 65749 20.0 +priscilla white 65652 20.0 +priscilla xylophone 65538 20.0 +priscilla xylophone 65763 20.0 +priscilla xylophone 65774 20.0 +priscilla young 65585 20.0 +priscilla young 65658 20.0 +priscilla zipper 65622 20.0 +priscilla zipper 65726 20.0 +quinn allen 65657 20.0 +quinn allen 65708 20.0 +quinn brown 65691 20.0 +quinn brown 65700 20.0 +quinn brown 65733 20.0 +quinn davidson 65549 20.0 +quinn davidson 65714 20.0 +quinn davidson 65776 20.0 +quinn davidson 65779 20.0 +quinn ellison 65705 20.0 +quinn ellison 65778 20.0 +quinn garcia 65568 20.0 +quinn garcia 65604 20.0 +quinn garcia 65610 20.0 +quinn garcia 65773 20.0 +quinn ichabod 65564 20.0 +quinn king 65558 20.0 +quinn king 65649 20.0 +quinn laertes 65542 20.0 +quinn laertes 65560 20.0 +quinn laertes 65627 20.0 +quinn nixon 65659 20.0 +quinn ovid 65699 20.0 +quinn quirinius 65747 20.0 +quinn robinson 65627 20.0 +quinn steinbeck 65578 20.0 +quinn steinbeck 65763 20.0 +quinn thompson 65643 20.0 +quinn thompson 65774 20.0 +quinn underhill 65549 20.0 +quinn underhill 65694 20.0 +quinn underhill 65767 20.0 +quinn van buren 65725 20.0 +quinn young 65647 20.0 +quinn zipper 65579 20.0 +quinn zipper 65693 20.0 +rachel allen 65661 20.0 +rachel allen 65709 20.0 +rachel brown 65586 20.0 +rachel brown 65587 20.0 +rachel brown 65587 20.0 +rachel brown 65610 20.0 +rachel brown 65693 20.0 +rachel carson 65677 20.0 +rachel carson 65682 20.0 +rachel davidson 65755 20.0 +rachel ellison 65761 20.0 +rachel falkner 65616 20.0 +rachel falkner 65681 20.0 +rachel falkner 65693 20.0 +rachel falkner 65764 20.0 +rachel johnson 65658 20.0 +rachel king 65604 20.0 +rachel king 65643 20.0 +rachel laertes 65562 20.0 +rachel laertes 65624 20.0 +rachel ovid 65721 20.0 +rachel ovid 65736 20.0 +rachel polk 65686 20.0 +rachel quirinius 65787 20.0 +rachel robinson 65544 20.0 +rachel robinson 65717 20.0 +rachel robinson 65724 20.0 +rachel thompson 65648 20.0 +rachel thompson 65662 20.0 +rachel thompson 65733 20.0 +rachel underhill 65667 20.0 +rachel white 65615 20.0 +rachel white 65717 20.0 +rachel young 65727 20.0 +rachel zipper 65757 20.0 +rachel zipper 65785 20.0 +sarah carson 65679 20.0 +sarah carson 65693 20.0 +sarah carson 65694 20.0 +sarah ellison 65611 20.0 +sarah falkner 65606 20.0 +sarah falkner 65680 20.0 +sarah garcia 65563 20.0 +sarah garcia 65638 20.0 +sarah garcia 65661 20.0 +sarah ichabod 65667 20.0 +sarah ichabod 65671 20.0 +sarah johnson 65659 20.0 +sarah johnson 65716 20.0 +sarah johnson 65731 20.0 +sarah johnson 65751 20.0 +sarah king 65650 20.0 +sarah king 65699 20.0 +sarah miller 65557 20.0 +sarah ovid 65550 20.0 +sarah robinson 65677 20.0 +sarah robinson 65763 20.0 +sarah steinbeck 65721 20.0 +sarah white 65622 20.0 +sarah white 65747 20.0 +sarah xylophone 65678 20.0 +sarah young 65595 20.0 +sarah zipper 65550 20.0 +tom brown 65593 20.0 +tom brown 65675 20.0 +tom carson 65539 20.0 +tom carson 65624 20.0 +tom carson 65780 20.0 +tom davidson 65780 20.0 +tom ellison 65578 20.0 +tom ellison 65670 20.0 +tom ellison 65756 20.0 +tom falkner 65574 20.0 +tom falkner 65583 20.0 +tom hernandez 65575 20.0 +tom hernandez 65632 20.0 +tom ichabod 65588 20.0 +tom johnson 65536 20.0 +tom johnson 65789 20.0 +tom king 65576 20.0 +tom laertes 65617 20.0 +tom laertes 65701 20.0 +tom miller 65594 20.0 +tom miller 65603 20.0 +tom miller 65704 20.0 +tom nixon 65672 20.0 +tom ovid 65628 20.0 +tom polk 65652 20.0 +tom polk 65742 20.0 +tom quirinius 65563 20.0 +tom quirinius 65783 20.0 +tom robinson 65626 20.0 +tom robinson 65632 20.0 +tom robinson 65691 20.0 +tom robinson 65758 20.0 +tom steinbeck 65666 20.0 +tom van buren 65621 20.0 +tom van buren 65652 20.0 +tom van buren 65669 20.0 +tom white 65548 20.0 +tom young 65544 20.0 +tom young 65546 20.0 +tom zipper 65789 20.0 +ulysses brown 65735 20.0 +ulysses carson 65602 20.0 +ulysses carson 65643 20.0 +ulysses carson 65703 20.0 +ulysses carson 65716 20.0 +ulysses davidson 65750 20.0 +ulysses ellison 65575 20.0 +ulysses garcia 65666 20.0 +ulysses hernandez 65651 20.0 +ulysses hernandez 65702 20.0 +ulysses hernandez 65786 20.0 +ulysses ichabod 65551 20.0 +ulysses ichabod 65566 20.0 +ulysses johnson 65776 20.0 +ulysses king 65649 20.0 +ulysses laertes 65691 20.0 +ulysses laertes 65711 20.0 +ulysses laertes 65781 20.0 +ulysses miller 65610 20.0 +ulysses miller 65637 20.0 +ulysses nixon 65603 20.0 +ulysses ovid 65656 20.0 +ulysses polk 65563 20.0 +ulysses polk 65580 20.0 +ulysses polk 65612 20.0 +ulysses polk 65777 20.0 +ulysses quirinius 65786 20.0 +ulysses robinson 65744 20.0 +ulysses steinbeck 65611 20.0 +ulysses steinbeck 65680 20.0 +ulysses thompson 65788 20.0 +ulysses underhill 65570 20.0 +ulysses underhill 65616 20.0 +ulysses underhill 65620 20.0 +ulysses underhill 65623 20.0 +ulysses underhill 65641 20.0 +ulysses underhill 65713 20.0 +ulysses underhill 65785 20.0 +ulysses van buren 65684 20.0 +ulysses white 65654 20.0 +ulysses white 65675 20.0 +ulysses xylophone 65623 20.0 +ulysses xylophone 65636 20.0 +ulysses xylophone 65781 20.0 +ulysses young 65675 20.0 +ulysses young 65736 20.0 +ulysses young 65748 20.0 +victor allen 65684 20.0 +victor allen 65707 20.0 +victor brown 65550 20.0 +victor brown 65555 20.0 +victor brown 65622 20.0 +victor brown 65673 20.0 +victor davidson 65579 20.0 +victor davidson 65628 20.0 +victor davidson 65783 20.0 +victor ellison 65641 20.0 +victor ellison 65782 20.0 +victor hernandez 65571 20.0 +victor hernandez 65659 20.0 +victor hernandez 65708 20.0 +victor hernandez 65735 20.0 +victor hernandez 65775 20.0 +victor johnson 65606 20.0 +victor johnson 65607 20.0 +victor johnson 65607 20.0 +victor king 65721 20.0 +victor king 65743 20.0 +victor laertes 65638 20.0 +victor laertes 65644 20.0 +victor miller 65570 20.0 +victor nixon 65709 20.0 +victor nixon 65791 20.0 +victor ovid 65649 20.0 +victor polk 65625 20.0 +victor quirinius 65620 20.0 +victor quirinius 65651 20.0 +victor robinson 65596 20.0 +victor robinson 65673 20.0 +victor steinbeck 65618 20.0 +victor steinbeck 65661 20.0 +victor steinbeck 65686 20.0 +victor thompson 65548 20.0 +victor van buren 65664 20.0 +victor van buren 65774 20.0 +victor white 65548 20.0 +victor white 65601 20.0 +victor xylophone 65549 20.0 +victor xylophone 65618 20.0 +victor xylophone 65644 20.0 +victor xylophone 65677 20.0 +victor xylophone 65755 20.0 +victor young 65628 20.0 +victor zipper 65743 20.0 +wendy allen 65628 20.0 +wendy allen 65711 20.0 +wendy allen 65782 20.0 +wendy brown 65580 20.0 +wendy brown 65657 20.0 +wendy ellison 65545 20.0 +wendy ellison 65603 20.0 +wendy falkner 65595 20.0 +wendy falkner 65604 20.0 +wendy falkner 65635 20.0 +wendy garcia 65659 20.0 +wendy garcia 65746 20.0 +wendy garcia 65747 20.0 +wendy garcia 65777 20.0 +wendy hernandez 65650 20.0 +wendy ichabod 65730 20.0 +wendy king 65586 20.0 +wendy king 65664 20.0 +wendy king 65670 20.0 +wendy laertes 65566 20.0 +wendy laertes 65683 20.0 +wendy laertes 65727 20.0 +wendy miller 65582 20.0 +wendy miller 65626 20.0 +wendy nixon 65611 20.0 +wendy nixon 65746 20.0 +wendy ovid 65589 20.0 +wendy ovid 65643 20.0 +wendy polk 65656 20.0 +wendy polk 65692 20.0 +wendy quirinius 65766 20.0 +wendy quirinius 65767 20.0 +wendy robinson 65622 20.0 +wendy robinson 65715 20.0 +wendy robinson 65774 20.0 +wendy steinbeck 65612 20.0 +wendy thompson 65650 20.0 +wendy thompson 65737 20.0 +wendy underhill 65662 20.0 +wendy underhill 65758 20.0 +wendy underhill 65775 20.0 +wendy van buren 65680 20.0 +wendy van buren 65699 20.0 +wendy white 65705 20.0 +wendy xylophone 65687 20.0 +wendy xylophone 65773 20.0 +wendy young 65674 20.0 +wendy young 65685 20.0 +xavier allen 65611 20.0 +xavier allen 65618 20.0 +xavier allen 65771 20.0 +xavier brown 65600 20.0 +xavier brown 65704 20.0 +xavier brown 65723 20.0 +xavier carson 65731 20.0 +xavier carson 65758 20.0 +xavier davidson 65644 20.0 +xavier davidson 65664 20.0 +xavier davidson 65755 20.0 +xavier ellison 65541 20.0 +xavier ellison 65654 20.0 +xavier garcia 65672 20.0 +xavier hernandez 65541 20.0 +xavier hernandez 65544 20.0 +xavier hernandez 65766 20.0 +xavier ichabod 65597 20.0 +xavier ichabod 65663 20.0 +xavier johnson 65654 20.0 +xavier johnson 65744 20.0 +xavier king 65590 20.0 +xavier king 65601 20.0 +xavier laertes 65743 20.0 +xavier ovid 65788 20.0 +xavier polk 65587 20.0 +xavier polk 65653 20.0 +xavier polk 65675 20.0 +xavier polk 65696 20.0 +xavier quirinius 65599 20.0 +xavier quirinius 65650 20.0 +xavier quirinius 65656 20.0 +xavier quirinius 65737 20.0 +xavier thompson 65608 20.0 +xavier underhill 65710 20.0 +xavier white 65703 20.0 +xavier white 65732 20.0 +xavier xylophone 65572 20.0 +xavier zipper 65561 20.0 +yuri allen 65565 20.0 +yuri allen 65682 20.0 +yuri brown 65538 20.0 +yuri brown 65688 20.0 +yuri carson 65670 20.0 +yuri carson 65769 20.0 +yuri ellison 65570 20.0 +yuri ellison 65581 20.0 +yuri falkner 65658 20.0 +yuri falkner 65681 20.0 +yuri garcia 65639 20.0 +yuri hernandez 65706 20.0 +yuri johnson 65587 20.0 +yuri johnson 65697 20.0 +yuri johnson 65712 20.0 +yuri king 65721 20.0 +yuri laertes 65637 20.0 +yuri laertes 65773 20.0 +yuri nixon 65635 20.0 +yuri nixon 65740 20.0 +yuri polk 65607 20.0 +yuri polk 65713 20.0 +yuri polk 65742 20.0 +yuri quirinius 65544 20.0 +yuri quirinius 65617 20.0 +yuri quirinius 65695 20.0 +yuri steinbeck 65592 20.0 +yuri steinbeck 65679 20.0 +yuri thompson 65676 20.0 +yuri underhill 65718 20.0 +yuri underhill 65750 20.0 +yuri white 65659 20.0 +yuri xylophone 65714 20.0 +zach allen 65667 20.0 +zach brown 65559 20.0 +zach brown 65588 20.0 +zach brown 65691 20.0 +zach brown 65759 20.0 +zach brown 65762 20.0 +zach carson 65572 20.0 +zach ellison 65748 20.0 +zach falkner 65620 20.0 +zach falkner 65627 20.0 +zach garcia 65544 20.0 +zach garcia 65623 20.0 +zach garcia 65629 20.0 +zach garcia 65786 20.0 +zach ichabod 65599 20.0 +zach ichabod 65612 20.0 +zach king 65556 20.0 +zach king 65702 20.0 +zach king 65773 20.0 +zach miller 65584 20.0 +zach miller 65665 20.0 +zach miller 65719 20.0 +zach ovid 65578 20.0 +zach ovid 65703 20.0 +zach ovid 65750 20.0 +zach ovid 65784 20.0 +zach quirinius 65691 20.0 +zach robinson 65599 20.0 +zach steinbeck 65602 20.0 +zach steinbeck 65695 20.0 +zach thompson 65636 20.0 +zach thompson 65696 20.0 +zach underhill 65573 20.0 +zach white 65733 20.0 +zach xylophone 65542 20.0 +zach xylophone 65780 20.0 +zach young 65576 20.0 +zach zipper 65579 20.0 +zach zipper 65649 20.0 +zach zipper 65676 20.0 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 027056d4b865f07afbc7fe2d1313dbab11f2a491..5ead5f3c19908e947b411ea346066b1e3cae4afd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -300,7 +300,7 @@ abstract class HiveComparisonTest val hiveQueries = queryList.map(new TestHive.HiveQLQueryExecution(_)) // Make sure we can at least parse everything before attempting hive execution. - hiveQueries.foreach(_.analyzed) + hiveQueries.foreach(_.logical) val computedResults = (queryList.zipWithIndex, hiveQueries, hiveCacheFiles).zipped.map { case ((queryString, i), hiveQuery, cachedAnswerFile)=> try { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala new file mode 100644 index 0000000000000000000000000000000000000000..934452fe579a1db085603c29ce5e6c4c8efaad40 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -0,0 +1,845 @@ +/* + * 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 java.io.File +import java.util.{Locale, TimeZone} + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.util.Utils + +/** + * The test suite for window functions. To actually compare results with Hive, + * every test should be created by `createQueryTest`. Because we are reusing tables + * for different tests and there are a few properties needed to let Hive generate golden + * files, every `createQueryTest` calls should explicitly set `reset` to `false`. + */ +abstract class HiveWindowFunctionQueryBaseSuite extends HiveComparisonTest with BeforeAndAfter { + private val originalTimeZone = TimeZone.getDefault + private val originalLocale = Locale.getDefault + private val testTempDir = Utils.createTempDir() + + override def beforeAll() { + TestHive.cacheTables = true + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + + // Create the table used in windowing.q + sql("DROP TABLE IF EXISTS part") + 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) + val testData1 = TestHive.getHiveFile("data/files/part_tiny.txt").getCanonicalPath + sql( + s""" + |LOAD DATA LOCAL INPATH '$testData1' overwrite into table part + """.stripMargin) + + sql("DROP TABLE IF EXISTS over1k") + sql( + """ + |create table over1k( + | t tinyint, + | si smallint, + | i int, + | b bigint, + | f float, + | d double, + | bo boolean, + | s string, + | ts timestamp, + | dec decimal(4,2), + | bin binary) + |row format delimited + |fields terminated by '|' + """.stripMargin) + val testData2 = TestHive.getHiveFile("data/files/over1k").getCanonicalPath + sql( + s""" + |LOAD DATA LOCAL INPATH '$testData2' overwrite into table over1k + """.stripMargin) + + // The following settings are used for generating golden files with Hive. + // We have to use kryo to correctly let Hive serialize plans with window functions. + // This is used to generate golden files. + sql("set hive.plan.serialization.format=kryo") + // Explicitly set fs to local fs. + sql(s"set fs.default.name=file://$testTempDir/") + // Ask Hive to run jobs in-process as a single map and reduce task. + sql("set mapred.job.tracker=local") + } + + override def afterAll() { + TestHive.cacheTables = false + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + TestHive.reset() + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_multipartitioning.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_multipartitioning.q (deterministic) 1", + s""" + |select s, + |rank() over (partition by s order by si) r, + |sum(b) over (partition by s order by si) sum + |from over1k + |order by s, r, sum; + """.stripMargin, reset = false) + + /* timestamp comparison issue with Hive? + createQueryTest("windowing_multipartitioning.q (deterministic) 2", + s""" + |select s, + |rank() over (partition by s order by dec desc) r, + |sum(b) over (partition by s order by ts desc) as sum + |from over1k + |where s = 'tom allen' or s = 'bob steinbeck' + |order by s, r, sum; + """.stripMargin, reset = false) + */ + + createQueryTest("windowing_multipartitioning.q (deterministic) 3", + s""" + |select s, sum(i) over (partition by s), sum(f) over (partition by si) + |from over1k where s = 'tom allen' or s = 'bob steinbeck'; + """.stripMargin, reset = false) + + createQueryTest("windowing_multipartitioning.q (deterministic) 4", + s""" + |select s, rank() over (partition by s order by bo), + |rank() over (partition by si order by bin desc) from over1k + |where s = 'tom allen' or s = 'bob steinbeck'; + """.stripMargin, reset = false) + + createQueryTest("windowing_multipartitioning.q (deterministic) 5", + s""" + |select s, sum(f) over (partition by i), row_number() over (order by f) + |from over1k where s = 'tom allen' or s = 'bob steinbeck'; + """.stripMargin, reset = false) + + createQueryTest("windowing_multipartitioning.q (deterministic) 6", + s""" + |select s, rank() over w1, + |rank() over w2 + |from over1k + |where s = 'tom allen' or s = 'bob steinbeck' + |window + |w1 as (partition by s order by dec), + |w2 as (partition by si order by f) ; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_navfn.q + // Results of the original query file are not deterministic. + // Also, the original query of + // select i, lead(s) over (partition by bin order by d,i desc) from over1k ; + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_navfn.q (deterministic)", + s""" + |select s, row_number() over (partition by d order by dec) rn from over1k + |order by s, rn desc; + |select i, lead(s) over (partition by cast(bin as string) order by d,i desc) as l + |from over1k + |order by i desc, l; + |select i, lag(dec) over (partition by i order by s,i,dec) l from over1k + |order by i, l; + |select s, last_value(t) over (partition by d order by f) l from over1k + |order by s, l; + |select s, first_value(s) over (partition by bo order by s) f from over1k + |order by s, f; + |select t, s, i, last_value(i) over (partition by t order by s) + |from over1k where (s = 'oscar allen' or s = 'oscar carson') and t = 10; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_ntile.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_ntile.q (deterministic)", + s""" + |select i, ntile(10) over (partition by s order by i) n from over1k + |order by i, n; + |select s, ntile(100) over (partition by i order by s) n from over1k + |order by s, n; + |select f, ntile(4) over (partition by d order by f) n from over1k + |order by f, n; + |select d, ntile(1000) over (partition by dec order by d) n from over1k + |order by d, n; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_udaf.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_udaf.q (deterministic)", + s""" + |select s, min(i) over (partition by s) m from over1k + |order by s, m; + |select s, avg(f) over (partition by si order by s) a from over1k + |order by s, a; + |select s, avg(i) over (partition by t, b order by s) a from over1k + |order by s, a; + |select max(i) over w m from over1k + |order by m window w as (partition by f) ; + |select s, avg(d) over (partition by t order by f) a from over1k + |order by s, a; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_windowspec.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_windowspec.q (deterministic)", + s""" + |select s, sum(b) over (partition by i order by s,b rows unbounded preceding) as sum + |from over1k order by s, sum; + |select s, sum(f) over (partition by d order by s,f rows unbounded preceding) as sum + |from over1k order by s, sum; + |select s, sum(f) over + |(partition by ts order by f range between current row and unbounded following) as sum + |from over1k order by s, sum; + |select s, avg(f) + |over (partition by ts order by s,f rows between current row and 5 following) avg + |from over1k order by s, avg; + |select s, avg(d) over + |(partition by t order by s,d desc rows between 5 preceding and 5 following) avg + |from over1k order by s, avg; + |select s, sum(i) over(partition by ts order by s) sum from over1k + |order by s, sum; + |select f, sum(f) over + |(partition by ts order by f range between unbounded preceding and current row) sum + |from over1k order by f, sum; + |select s, i, round(avg(d) over (partition by s order by i) / 10.0 , 2) avg + |from over1k order by s, i, avg; + |select s, i, round((avg(d) over w1 + 10.0) - (avg(d) over w1 - 10.0),2) avg + |from over1k + |order by s, i, avg window w1 as (partition by s order by i); + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_rank.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_rank.q (deterministic) 1", + s""" + |select s, rank() over (partition by f order by t) r from over1k order by s, r; + |select s, dense_rank() over (partition by ts order by i,s desc) as r from over1k + |order by s desc, r desc; + |select s, cume_dist() over (partition by bo order by b,s) cd from over1k + |order by s, cd; + |select s, percent_rank() over (partition by dec order by f) r from over1k + |order by s desc, r desc; + """.stripMargin, reset = false) + + createQueryTest("windowing_rank.q (deterministic) 2", + s""" + |select ts, dec, rnk + |from + | (select ts, dec, + | rank() over (partition by ts order by dec) as rnk + | from + | (select other.ts, other.dec + | from over1k other + | join over1k on (other.b = over1k.b) + | ) joined + | ) ranked + |where rnk = 1 + |order by ts, dec, rnk; + """.stripMargin, reset = false) + + createQueryTest("windowing_rank.q (deterministic) 3", + s""" + |select ts, dec, rnk + |from + | (select ts, dec, + | rank() over (partition by ts order by dec) as rnk + | from + | (select other.ts, other.dec + | from over1k other + | join over1k on (other.b = over1k.b) + | ) joined + | ) ranked + |where dec = 89.5 + |order by ts, dec, rnk; + """.stripMargin, reset = false) + + createQueryTest("windowing_rank.q (deterministic) 4", + s""" + |select ts, dec, rnk + |from + | (select ts, dec, + | rank() over (partition by ts order by dec) as rnk + | from + | (select other.ts, other.dec + | from over1k other + | join over1k on (other.b = over1k.b) + | where other.t < 10 + | ) joined + | ) ranked + |where rnk = 1 + |order by ts, dec, rnk; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests from windowing.q + // We port tests in windowing.q to here because this query file contains too + // many tests and the syntax of test "-- 7. testJoinWithWindowingAndPTF" + // is not supported right now. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing.q -- 1. testWindowing", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |sum(p_retailprice) over + |(distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 2. testGroupByWithPartitioning", + s""" + |select p_mfgr, p_name, p_size, + |min(p_retailprice), + |rank() over(distribute by p_mfgr sort by p_name)as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |group by p_mfgr, p_name, p_size + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 3. testGroupByHavingWithSWQ", + s""" + |select p_mfgr, p_name, p_size, min(p_retailprice), + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |group by p_mfgr, p_name, p_size + |having p_size > 0 + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 4. testCount", + s""" + |select p_mfgr, p_name, + |count(p_size) over(distribute by p_mfgr sort by p_name) as cd + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 5. testCountWithWindowingUDAF", + s""" + |select p_mfgr, p_name, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |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, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 6. testCountInSubQ", + s""" + |select sub1.r, sub1.dr, sub1.cd, sub1.s1, sub1.deltaSz + |from (select p_mfgr, p_name, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |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, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |) sub1 + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 8. testMixedCaseAlias", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name, p_size desc) as R + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 9. testHavingWithWindowingNoGBY", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_name + | rows between unbounded preceding and current row) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 10. testHavingWithWindowingCondRankNoGBY", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_name + | rows between unbounded preceding and current row) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 11. testFirstLast", + s""" + |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, 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) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 12. testFirstLastWithWhere", + s""" + |select p_mfgr,p_name, p_size, + |rank() 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, 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) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 13. testSumWindow", + s""" + |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, reset = false) + + createQueryTest("windowing.q -- 14. testNoSortClause", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 15. testExpressions", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, + |percent_rank() over(distribute by p_mfgr sort by p_name) as pr, + |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, reset = false) + + createQueryTest("windowing.q -- 16. testMultipleWindows", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, + |sum(p_size) over (distribute by p_mfgr sort by p_name + |range between unbounded preceding and current row) as s1, + |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, reset = false) + + + createQueryTest("windowing.q -- 17. testCountStar", + s""" + |select p_mfgr,p_name, p_size, + |count(*) over(distribute by p_mfgr sort by p_name ) as c, + |count(p_size) over(distribute by p_mfgr sort by p_name) as ca, + |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, reset = false) + + createQueryTest("windowing.q -- 18. testUDAFs", + s""" + |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, reset = false) + + createQueryTest("windowing.q -- 19. testUDAFsWithGBY", + """ + |select p_mfgr,p_name, p_size, p_retailprice, + |sum(p_retailprice) over w1 as s, + |min(p_retailprice) as mi , + |max(p_retailprice) as ma , + |avg(p_retailprice) over w1 as ag + |from part + |group by p_mfgr,p_name, p_size, p_retailprice + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following); + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 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, reset = false) + + createQueryTest("windowing.q -- 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, reset = false) + + createQueryTest("windowing.q -- 24. testLateralViews", + """ + |select p_mfgr, p_name, + |lv_col, p_size, sum(p_size) over w1 as s + |from (select p_mfgr, p_name, p_size, array(1,2,3) arr from part) p + |lateral view explode(arr) part_lv as lv_col + |window w1 as (distribute by p_mfgr sort by p_size, lv_col + | rows between 2 preceding and current row) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 26. testGroupByHavingWithSWQAndAlias", + """ + |select p_mfgr, p_name, p_size, min(p_retailprice) as mi, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |group by p_mfgr, p_name, p_size + |having p_size > 0 + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 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, reset = false) + + createQueryTest("windowing.q -- 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, reset = false) + + createQueryTest("windowing.q -- 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, reset = false) + + createQueryTest("windowing.q -- 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, reset = false) + + /* p_name is not a numeric column. What is Hive's semantic? + createQueryTest("windowing.q -- 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_name + | range between 2 preceding and 2 following), + | w2 as w1 + """.stripMargin, reset = false) + */ + /* + createQueryTest("windowing.q -- 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_name + | range between 2 preceding and 2 following), + | w2 as (w1 rows between unbounded preceding and current row) + """.stripMargin, reset = false) + */ + + /* p_name is not a numeric column. What is Hive's semantic? + createQueryTest("windowing.q -- 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_name + | range between 2 preceding and 2 following), + | w2 as w3, + | w3 as (distribute by p_mfgr sort by p_name + | range between unbounded preceding and current row) + """.stripMargin, reset = false) + */ + /* + createQueryTest("windowing.q -- 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_name + | range between 2 preceding and 2 following), + | w2 as w3, + | w3 as (distribute by p_mfgr sort by p_name + | range between unbounded preceding and current row) + """.stripMargin, reset = false) + */ + + /* Seems Hive evaluate SELECT DISTINCT before window functions? + createQueryTest("windowing.q -- 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, reset = false) + */ + + createQueryTest("windowing.q -- 36. testRankWithPartitioning", + """ + |select p_mfgr, p_name, p_size, + |rank() over (partition by p_mfgr order by p_name ) as r + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 37. testPartitioningVariousForms", + """ + |select p_mfgr, + |round(sum(p_retailprice) over (partition by p_mfgr order by p_mfgr),2) as s1, + |min(p_retailprice) over (partition by p_mfgr) as s2, + |max(p_retailprice) over (distribute by p_mfgr sort by p_mfgr) as s3, + |round(avg(p_retailprice) over (distribute by p_mfgr),2) as s4, + |count(p_retailprice) over (cluster by p_mfgr ) as s5 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 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, reset = false) + + createQueryTest("windowing.q -- 39. testUDFOnOrderCols", + """ + |select p_mfgr, p_type, substr(p_type, 2) as short_ptype, + |rank() over (partition by p_mfgr order by substr(p_type, 2)) as r + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 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, reset = false) + + createQueryTest("windowing.q -- 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, reset = false) + + createQueryTest("windowing.q -- 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, reset = false) + + createQueryTest("windowing.q -- 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, reset = false) + + createQueryTest("windowing.q -- 44. testOverNoPartitionSingleAggregate", + """ + |select p_name, p_retailprice, + |round(avg(p_retailprice) over(),2) + |from part + |order by p_name + """.stripMargin, reset = false) +} + +class HiveWindowFunctionQueryWithoutCodeGenSuite extends HiveWindowFunctionQueryBaseSuite { + var originalCodegenEnabled: Boolean = _ + override def beforeAll(): Unit = { + super.beforeAll() + originalCodegenEnabled = conf.codegenEnabled + sql("set spark.sql.codegen=false") + } + + override def afterAll(): Unit = { + sql(s"set spark.sql.codegen=$originalCodegenEnabled") + super.afterAll() + } +} + +abstract class HiveWindowFunctionQueryFileBaseSuite + extends HiveCompatibilitySuite with BeforeAndAfter { + private val originalTimeZone = TimeZone.getDefault + private val originalLocale = Locale.getDefault + private val testTempDir = Utils.createTempDir() + + override def beforeAll() { + TestHive.cacheTables = true + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + + // The following settings are used for generating golden files with Hive. + // We have to use kryo to correctly let Hive serialize plans with window functions. + // This is used to generate golden files. + sql("set hive.plan.serialization.format=kryo") + // Explicitly set fs to local fs. + sql(s"set fs.default.name=file://$testTempDir/") + // Ask Hive to run jobs in-process as a single map and reduce task. + sql("set mapred.job.tracker=local") + } + + override def afterAll() { + TestHive.cacheTables = false + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + TestHive.reset() + } + + override def blackList: Seq[String] = Seq( + // Partitioned table functions are not supported. + "ptf*", + // tests of windowing.q are in HiveWindowFunctionQueryBaseSuite + "windowing.q", + + // This one failed on the expression of + // sum(lag(p_retailprice,1,0.0)) over w1 + // lag(p_retailprice,1,0.0) is a GenericUDF and the argument inspector of + // p_retailprice created by HiveInspectors is + // PrimitiveObjectInspectorFactory.javaDoubleObjectInspector. + // However, seems Hive assumes it is + // PrimitiveObjectInspectorFactory.writableDoubleObjectInspector, which introduces an error. + "windowing_expressions", + + // Hive's results are not deterministic + "windowing_multipartitioning", + "windowing_navfn", + "windowing_ntile", + "windowing_udaf", + "windowing_windowspec", + "windowing_rank" + ) + + override def whiteList: Seq[String] = Seq( + "windowing_udaf2", + "windowing_columnPruning", + "windowing_adjust_rowcontainer_sz" + ) + + override def testCases: Seq[(String, File)] = super.testCases.filter { + case (name, _) => realWhiteList.contains(name) + } +} + +class HiveWindowFunctionQueryFileWithoutCodeGenSuite extends HiveWindowFunctionQueryFileBaseSuite { + var originalCodegenEnabled: Boolean = _ + override def beforeAll(): Unit = { + super.beforeAll() + originalCodegenEnabled = conf.codegenEnabled + sql("set spark.sql.codegen=false") + } + + override def afterAll(): Unit = { + sql(s"set spark.sql.codegen=$originalCodegenEnabled") + super.afterAll() + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 630dec8fa05a09571a4b6d03d782748b0dd7c98a..616352d223c5bf32e757b2c52fe074dd634e4297 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -48,6 +48,10 @@ case class Order( state: String, month: Int) +case class WindowData( + month: Int, + area: String, + product: Int) /** A SQL Dialect for testing purpose, and it can not be nested type */ class MyDialect extends DefaultDialect @@ -604,4 +608,147 @@ class SQLQuerySuite extends QueryTest { sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat' AS (a,b,c) FROM script_trans") .queryExecution.toRdd.count()) } + + test("window function: udaf with aggregate expressin") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().registerTempTable("windowData") + + checkAnswer( + sql( + """ + |select area, sum(product), sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 11), + ("a", 6, 11), + ("b", 7, 15), + ("b", 8, 15), + ("c", 9, 19), + ("c", 10, 19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product) - 1, sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 4, 11), + ("a", 5, 11), + ("b", 6, 15), + ("b", 7, 15), + ("c", 8, 19), + ("c", 9, 19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product), sum(product) / sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 5d/11), + ("a", 6, 6d/11), + ("b", 7, 7d/15), + ("b", 8, 8d/15), + ("c", 10, 10d/19), + ("c", 9, 9d/19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product), sum(product) / sum(sum(product) - 1) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 5d/9), + ("a", 6, 6d/9), + ("b", 7, 7d/13), + ("b", 8, 8d/13), + ("c", 10, 10d/17), + ("c", 9, 9d/17) + ).map(i => Row(i._1, i._2, i._3))) + } + + test("window function: partition and order expressions") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().registerTempTable("windowData") + + checkAnswer( + sql( + """ + |select month, area, product, sum(product + 1) over (partition by 1 order by 2) + |from windowData + """.stripMargin), + Seq( + (1, "a", 5, 51), + (2, "a", 6, 51), + (3, "b", 7, 51), + (4, "b", 8, 51), + (5, "c", 9, 51), + (6, "c", 10, 51) + ).map(i => Row(i._1, i._2, i._3, i._4))) + + checkAnswer( + sql( + """ + |select month, area, product, sum(product) + |over (partition by month % 2 order by 10 - product) + |from windowData + """.stripMargin), + Seq( + (1, "a", 5, 21), + (2, "a", 6, 24), + (3, "b", 7, 16), + (4, "b", 8, 18), + (5, "c", 9, 9), + (6, "c", 10, 10) + ).map(i => Row(i._1, i._2, i._3, i._4))) + } + + test("window function: expressions in arguments of a window functions") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().registerTempTable("windowData") + + checkAnswer( + sql( + """ + |select month, area, month % 2, + |lag(product, 1 + 1, product) over (partition by month % 2 order by area) + |from windowData + """.stripMargin), + Seq( + (1, "a", 1, 5), + (2, "a", 0, 6), + (3, "b", 1, 7), + (4, "b", 0, 8), + (5, "c", 1, 5), + (6, "c", 0, 6) + ).map(i => Row(i._1, i._2, i._3, i._4))) + } }