http://git-wip-us.apache.org/repos/asf/spark/blob/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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/b521a3b0/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-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to