You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2015/05/06 19:43:27 UTC

[12/13] spark git commit: [SPARK-1442] [SQL] Window Function Support for Spark SQL

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
----------------------------------------------------------------------
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 4b6f0ad..fd0b6f0 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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 1. testWindowing-0-327a8cd39fe30255ff492ee86f660522
----------------------------------------------------------------------
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 0000000..850c41c
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 10. testHavingWithWindowingCondRankNoGBY-0-fef4bf638d52a9a601845347010602fd
----------------------------------------------------------------------
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 0000000..850c41c
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 11. testFirstLast-0-86bb9c97d92fdcd941bcb5143513e2e6
----------------------------------------------------------------------
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 0000000..921679c
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 12. testFirstLastWithWhere-0-84345a9f685ba63b87caa4bb16b122b5
----------------------------------------------------------------------
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 0000000..09e30c7
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 13. testSumWindow-0-6cfc8840d3a4469b0fe11d63182cb59f
----------------------------------------------------------------------
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 0000000..01ee88f
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 14. testNoSortClause-0-88d96a526d3cae6ed8168c5b228974d1
----------------------------------------------------------------------
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 0000000..c78eb64
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 15. testExpressions-0-11f6c13cf2710ce7054654cca136e73e
----------------------------------------------------------------------
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 0000000..050138c
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 16. testMultipleWindows-0-efd1476255eeb1b1961149144f574b7a
----------------------------------------------------------------------
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 0000000..c108888
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 17. testCountStar-0-1b1fc185c8fddf68e58e92f29052ab2d
----------------------------------------------------------------------
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 0000000..b1309a4
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 18. testUDAFs-0-6974e5959e41a661e09db18547fef58a
----------------------------------------------------------------------
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 0000000..52d2ee8
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 19. testUDAFsWithGBY-0-67d15ee5915ac64a738fd4b60d75eb35
----------------------------------------------------------------------
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 0000000..6461642
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 2. testGroupByWithPartitioning-0-cb5618b1e626f3a9d4a030b508b5d251
----------------------------------------------------------------------
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 0000000..2c30e65
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838
----------------------------------------------------------------------
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 0000000..1f7e8a5
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad
----------------------------------------------------------------------
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 0000000..e7c39f4
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 24. testLateralViews-0-dea06072f0a64fe4537fae854944ed5a
----------------------------------------------------------------------
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 0000000..dc83c9f
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 26. testGroupByHavingWithSWQAndAlias-0-b996a664b06e5741c08079d5c38241bc
----------------------------------------------------------------------
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 0000000..2c30e65
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 27. testMultipleRangeWindows-0-227e080e337d734dd88ff814b3b412e4
----------------------------------------------------------------------
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 0000000..b2a91ba
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 28. testPartOrderInUDAFInvoke-0-25912ae7d18c91cc09e17e57968fb5db
----------------------------------------------------------------------
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 0000000..5bcb0fa
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 29. testPartOrderInWdwDef-0-88945892370ccbc1125a927a3d55342a
----------------------------------------------------------------------
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 0000000..5bcb0fa
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 3. testGroupByHavingWithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100
----------------------------------------------------------------------
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 0000000..2c30e65
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 30. testDefaultPartitioningSpecRules-0-fa80b09c99e3c1487de48ea71a88dada
----------------------------------------------------------------------
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 0000000..698a443
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 36. testRankWithPartitioning-0-45ccbaf0ee083858f7661c66b11d4768
----------------------------------------------------------------------
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 0000000..e35257d
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 37. testPartitioningVariousForms-0-3436e50214f9afdec84334e10faa931a
----------------------------------------------------------------------
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 0000000..9c0ca6c
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 38. testPartitioningVariousForms2-0-cba9d84a6b1bb5e36595338d4602377e
----------------------------------------------------------------------
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 0000000..fc27df2
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 39. testUDFOnOrderCols-0-7647562850dd367ef1e6c63117805423
----------------------------------------------------------------------
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 0000000..e5a541f
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 4. testCount-0-e6e97e884327df86f16b870527ec026c
----------------------------------------------------------------------
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 0000000..bf8e620
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 40. testNoBetweenForRows-0-99007f45b6406869e048b0e4eb9213f1
----------------------------------------------------------------------
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 0000000..1e29df6
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 41. testNoBetweenForRange-0-d81a591e90950de291d2f133793e9283
----------------------------------------------------------------------
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 0000000..a620479
--- /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

http://git-wip-us.apache.org/repos/asf/spark/blob/f2c47082/sql/hive/src/test/resources/golden/windowing.q -- 42. testUnboundedFollowingForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0
----------------------------------------------------------------------
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 0000000..74147d2
--- /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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org