You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2013/04/02 16:16:37 UTC

svn commit: r1463556 [10/15] - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ data/files/ ql/if/ ql/src/gen/thrift/gen-cpp/ ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/ ql/src/gen/thrift/gen-php/ ql/src/gen/thrift/ge...

Added: hive/trunk/ql/src/test/queries/clientpositive/windowing_rank.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/windowing_rank.q?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/windowing_rank.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/windowing_rank.q Tue Apr  2 14:16:34 2013
@@ -0,0 +1,66 @@
+drop table over10k;
+
+create table over10k(
+           t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+	   ts timestamp, 
+           dec decimal,  
+           bin binary)
+       row format delimited
+       fields terminated by '|';
+
+load data local inpath '../data/files/over10k' into table over10k;
+
+select s, rank() over (partition by f order by t) from over10k limit 100;
+
+select s, dense_rank() over (partition by ts order by i desc) from over10k limit 100;
+
+select s, cume_dist() over (partition by bo order by b) from over10k limit 100;
+
+select s, percent_rank() over (partition by dec order by f) from over10k limit 100;
+
+-- If following tests fail, look for the comments in class PTFPPD::process()
+
+select ts, dec, rnk
+from
+  (select ts, dec,
+          rank() over (partition by ts)  as rnk
+          from
+            (select other.ts, other.dec
+             from over10k other
+             join over10k on (other.b = over10k.b)
+            ) joined
+  ) ranked
+where rnk =  1 limit 10;
+
+select ts, dec, rnk
+from
+  (select ts, dec,
+          rank() over (partition by ts)  as rnk
+          from
+            (select other.ts, other.dec
+             from over10k other
+             join over10k on (other.b = over10k.b)
+            ) joined
+  ) ranked
+where dec = 89.5 limit 10;
+
+select ts, dec, rnk
+from
+  (select ts, dec,
+          rank() over (partition by ts)  as rnk
+          from
+            (select other.ts, other.dec
+             from over10k other
+             join over10k on (other.b = over10k.b)
+             where other.t < 10
+            ) joined
+  ) ranked
+where rnk = 1 limit 10;
+

Added: hive/trunk/ql/src/test/queries/clientpositive/windowing_udaf.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/windowing_udaf.q?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/windowing_udaf.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/windowing_udaf.q Tue Apr  2 14:16:34 2013
@@ -0,0 +1,28 @@
+drop table over10k;
+
+create table over10k(
+           t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+	   ts timestamp, 
+           dec decimal,  
+           bin binary)
+       row format delimited
+       fields terminated by '|';
+
+load data local inpath '../data/files/over10k' into table over10k;
+
+select s, min(i) over (partition by s) from over10k limit 100;
+
+select s, avg(f) over (partition by si order by t) from over10k limit 100;
+
+select s, avg(i) over (partition by t, b) from over10k limit 100;
+
+select max(i) over w from over10k window w as (partition by f) limit 100;
+
+select s, avg(d) over (partition by t order by f) from over10k limit 100;

Added: hive/trunk/ql/src/test/queries/clientpositive/windowing_windowspec.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/windowing_windowspec.q?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/windowing_windowspec.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/windowing_windowspec.q Tue Apr  2 14:16:34 2013
@@ -0,0 +1,34 @@
+drop table over10k;
+
+create table over10k(
+           t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+	   ts timestamp, 
+           dec decimal,  
+           bin binary)
+       row format delimited
+       fields terminated by '|';
+
+load data local inpath '../data/files/over10k' into table over10k;
+
+select s, sum(b) over (partition by i order by si rows unbounded preceding) from over10k limit 100;
+
+select s, sum(f) over (partition by d order by i rows unbounded preceding) from over10k limit 100;
+
+select s, sum(f) over (partition by ts order by b range between current row and unbounded following) from over10k limit 100;
+
+select s, avg(f) over (partition by bin order by s rows between current row and 5 following) from over10k limit 100;
+
+select s, avg(d) over (partition by t order by ts desc rows between 5 preceding and 5 following) from over10k limit 100;
+
+select s, sum(i) over() from over10k limit 100;
+
+select f, sum(f) over (order by f range between unbounded preceding and current row) from over10k limit 100;
+
+

Added: hive/trunk/ql/src/test/results/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q.out?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q.out Tue Apr  2 14:16:34 2013
@@ -0,0 +1,36 @@
+PREHOOK: query: DROP TABLE part
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE part
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part
+PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part
+PREHOOK: type: LOAD
+PREHOOK: Output: default@part
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@part
+FAILED: SemanticException [Error 10025]: Line 4:7 Expression not in GROUP BY key 'p_mfgr'

Added: hive/trunk/ql/src/test/results/clientnegative/ptf_negative_AmbiguousWindowDefn.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/ptf_negative_AmbiguousWindowDefn.q.out?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/ptf_negative_AmbiguousWindowDefn.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/ptf_negative_AmbiguousWindowDefn.q.out Tue Apr  2 14:16:34 2013
@@ -0,0 +1,36 @@
+PREHOOK: query: DROP TABLE part
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE part
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part
+PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part
+PREHOOK: type: LOAD
+PREHOOK: Output: default@part
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@part
+FAILED: SemanticException Cycle in Window references [w3, w3]

Added: hive/trunk/ql/src/test/results/clientnegative/ptf_negative_DistributeByOrderBy.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/ptf_negative_DistributeByOrderBy.q.out?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/ptf_negative_DistributeByOrderBy.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/ptf_negative_DistributeByOrderBy.q.out Tue Apr  2 14:16:34 2013
@@ -0,0 +1,32 @@
+PREHOOK: query: DROP TABLE part
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE part
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part
+FAILED: ParseException line 5:46 missing ) at 'order' near 'by'
+line 5:55 missing EOF at 'p_mfgr' near 'by'
+

Added: hive/trunk/ql/src/test/results/clientnegative/ptf_negative_DuplicateWindowAlias.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/ptf_negative_DuplicateWindowAlias.q.out?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/ptf_negative_DuplicateWindowAlias.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/ptf_negative_DuplicateWindowAlias.q.out Tue Apr  2 14:16:34 2013
@@ -0,0 +1,30 @@
+PREHOOK: query: DROP TABLE part
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE part
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part
+FAILED: SemanticException 10:7 Duplicate definition of window w2 is not allowed. Error encountered near token 'w2'

Added: hive/trunk/ql/src/test/results/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q.out?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q.out Tue Apr  2 14:16:34 2013
@@ -0,0 +1,36 @@
+PREHOOK: query: DROP TABLE part
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE part
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part
+PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part
+PREHOOK: type: LOAD
+PREHOOK: Output: default@part
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@part
+FAILED: SemanticException HAVING specified without GROUP BY

Added: hive/trunk/ql/src/test/results/clientnegative/ptf_negative_HavingLeadWithPTF.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/ptf_negative_HavingLeadWithPTF.q.out?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/ptf_negative_HavingLeadWithPTF.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/ptf_negative_HavingLeadWithPTF.q.out Tue Apr  2 14:16:34 2013
@@ -0,0 +1,30 @@
+PREHOOK: query: DROP TABLE part
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE part
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part
+FAILED: SemanticException HAVING specified without GROUP BY

Added: hive/trunk/ql/src/test/results/clientnegative/ptf_negative_InvalidValueBoundary.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/ptf_negative_InvalidValueBoundary.q.out?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/ptf_negative_InvalidValueBoundary.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/ptf_negative_InvalidValueBoundary.q.out Tue Apr  2 14:16:34 2013
@@ -0,0 +1,33 @@
+PREHOOK: query: DROP TABLE part
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE part
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING,
+    p_complex array<int>
+)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING,
+    p_complex array<int>
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part
+FAILED: SemanticException Failed to breakup Windowing invocations into Groups. At least 1 group must only depend on input columns. Also check for circular dependencies.
+Underlying error: Value Boundary expression must be of primitve type. Found: array<int>

Added: hive/trunk/ql/src/test/results/clientnegative/ptf_negative_JoinWithAmbigousAlias.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/ptf_negative_JoinWithAmbigousAlias.q.out?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/ptf_negative_JoinWithAmbigousAlias.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/ptf_negative_JoinWithAmbigousAlias.q.out Tue Apr  2 14:16:34 2013
@@ -0,0 +1,30 @@
+PREHOOK: query: DROP TABLE part
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE part
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part
+FAILED: SemanticException [Error 10008]: Line 8:11 Ambiguous table alias 'part'

Added: hive/trunk/ql/src/test/results/clientnegative/ptf_negative_PartitionBySortBy.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/ptf_negative_PartitionBySortBy.q.out?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/ptf_negative_PartitionBySortBy.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/ptf_negative_PartitionBySortBy.q.out Tue Apr  2 14:16:34 2013
@@ -0,0 +1,32 @@
+PREHOOK: query: DROP TABLE part
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE part
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part
+FAILED: ParseException line 5:45 missing ) at 'sort' near 'by'
+line 5:53 missing EOF at 'p_mfgr' near 'by'
+

Added: hive/trunk/ql/src/test/results/clientnegative/ptf_negative_WhereWithRankCond.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/ptf_negative_WhereWithRankCond.q.out?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/ptf_negative_WhereWithRankCond.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/ptf_negative_WhereWithRankCond.q.out Tue Apr  2 14:16:34 2013
@@ -0,0 +1,36 @@
+PREHOOK: query: DROP TABLE part
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE part
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part
+PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part
+PREHOOK: type: LOAD
+PREHOOK: Output: default@part
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@part
+FAILED: SemanticException [Error 10004]: Line 7:6 Invalid table alias or column reference 'r': (possible column names are: p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment)

Added: hive/trunk/ql/src/test/results/clientnegative/ptf_window_boundaries.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/ptf_window_boundaries.q.out?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/ptf_window_boundaries.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/ptf_window_boundaries.q.out Tue Apr  2 14:16:34 2013
@@ -0,0 +1,29 @@
+PREHOOK: query: -- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part
+FAILED: ParseException line 4:44 mismatched input 'following' expecting KW_PRECEDING near 'unbounded' in windowframestartboundary
+

Added: hive/trunk/ql/src/test/results/clientnegative/ptf_window_boundaries2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/ptf_window_boundaries2.q.out?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/ptf_window_boundaries2.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/ptf_window_boundaries2.q.out Tue Apr  2 14:16:34 2013
@@ -0,0 +1,29 @@
+PREHOOK: query: -- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part
+FAILED: ParseException line 4:45 mismatched input 'following' expecting KW_PRECEDING near 'unbounded' in windowframestartboundary
+

Added: hive/trunk/ql/src/test/results/clientpositive/leadlag.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/leadlag.q.out?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/leadlag.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/leadlag.q.out Tue Apr  2 14:16:34 2013
@@ -0,0 +1,403 @@
+PREHOOK: query: DROP TABLE part
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE part
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: -- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part
+PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part
+PREHOOK: type: LOAD
+PREHOOK: Output: default@part
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@part
+PREHOOK: query: --1. testLagWithPTFWindowing
+select p_mfgr, p_name,
+rank() over (partition by p_mfgr order by p_name) as r,
+dense_rank() over (partition by p_mfgr order by p_name) as dr,
+p_retailprice, sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1,
+p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz
+from noop(on part
+partition by p_mfgr
+order by p_name 
+)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: --1. testLagWithPTFWindowing
+select p_mfgr, p_name,
+rank() over (partition by p_mfgr order by p_name) as r,
+dense_rank() over (partition by p_mfgr order by p_name) as dr,
+p_retailprice, sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1,
+p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz
+from noop(on part
+partition by p_mfgr
+order by p_name 
+)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	1	1	1173.15	1173.15	2	0
+Manufacturer#1	almond antique burnished rose metallic	1	1	1173.15	2346.3	2	0
+Manufacturer#1	almond antique chartreuse lavender yellow	3	2	1753.76	4100.06	34	32
+Manufacturer#1	almond antique salmon chartreuse burlywood	4	3	1602.59	5702.650000000001	6	-28
+Manufacturer#1	almond aquamarine burnished black steel	5	4	1414.42	7117.070000000001	28	22
+Manufacturer#1	almond aquamarine pink moccasin thistle	6	5	1632.66	8749.730000000001	42	14
+Manufacturer#2	almond antique violet chocolate turquoise	1	1	1690.68	1690.68	14	0
+Manufacturer#2	almond antique violet turquoise frosted	2	2	1800.7	3491.38	40	26
+Manufacturer#2	almond aquamarine midnight light salmon	3	3	2031.98	5523.360000000001	2	-38
+Manufacturer#2	almond aquamarine rose maroon antique	4	4	1698.66	7222.02	25	23
+Manufacturer#2	almond aquamarine sandy cyan gainsboro	5	5	1701.6	8923.62	18	-7
+Manufacturer#3	almond antique chartreuse khaki white	1	1	1671.68	1671.68	17	0
+Manufacturer#3	almond antique forest lavender goldenrod	2	2	1190.27	2861.95	14	-3
+Manufacturer#3	almond antique metallic orange dim	3	3	1410.39	4272.34	19	5
+Manufacturer#3	almond antique misty red olive	4	4	1922.98	6195.32	1	-18
+Manufacturer#3	almond antique olive coral navajo	5	5	1337.29	7532.61	45	44
+Manufacturer#4	almond antique gainsboro frosted violet	1	1	1620.67	1620.67	10	0
+Manufacturer#4	almond antique violet mint lemon	2	2	1375.42	2996.09	39	29
+Manufacturer#4	almond aquamarine floral ivory bisque	3	3	1206.26	4202.35	27	-12
+Manufacturer#4	almond aquamarine yellow dodger mint	4	4	1844.92	6047.27	7	-20
+Manufacturer#4	almond azure aquamarine papaya violet	5	5	1290.35	7337.620000000001	12	5
+Manufacturer#5	almond antique blue firebrick mint	1	1	1789.69	1789.69	31	0
+Manufacturer#5	almond antique medium spring khaki	2	2	1611.66	3401.3500000000004	6	-25
+Manufacturer#5	almond antique sky peru orange	3	3	1788.73	5190.08	2	-4
+Manufacturer#5	almond aquamarine dodger light gainsboro	4	4	1018.1	6208.18	46	44
+Manufacturer#5	almond azure blanched chiffon midnight	5	5	1464.48	7672.66	23	-23
+PREHOOK: query: -- 2. testLagWithWindowingNoPTF
+select p_mfgr, p_name,
+rank() over (partition by p_mfgr order by p_name) as r,
+dense_rank() over (partition by p_mfgr order by p_name) as dr,
+p_retailprice, sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1,
+p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz
+from part
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 2. testLagWithWindowingNoPTF
+select p_mfgr, p_name,
+rank() over (partition by p_mfgr order by p_name) as r,
+dense_rank() over (partition by p_mfgr order by p_name) as dr,
+p_retailprice, sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1,
+p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz
+from part
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	1	1	1173.15	1173.15	2	0
+Manufacturer#1	almond antique burnished rose metallic	1	1	1173.15	2346.3	2	0
+Manufacturer#1	almond antique chartreuse lavender yellow	3	2	1753.76	4100.06	34	32
+Manufacturer#1	almond antique salmon chartreuse burlywood	4	3	1602.59	5702.650000000001	6	-28
+Manufacturer#1	almond aquamarine burnished black steel	5	4	1414.42	7117.070000000001	28	22
+Manufacturer#1	almond aquamarine pink moccasin thistle	6	5	1632.66	8749.730000000001	42	14
+Manufacturer#2	almond antique violet chocolate turquoise	1	1	1690.68	1690.68	14	0
+Manufacturer#2	almond antique violet turquoise frosted	2	2	1800.7	3491.38	40	26
+Manufacturer#2	almond aquamarine midnight light salmon	3	3	2031.98	5523.360000000001	2	-38
+Manufacturer#2	almond aquamarine rose maroon antique	4	4	1698.66	7222.02	25	23
+Manufacturer#2	almond aquamarine sandy cyan gainsboro	5	5	1701.6	8923.62	18	-7
+Manufacturer#3	almond antique chartreuse khaki white	1	1	1671.68	1671.68	17	0
+Manufacturer#3	almond antique forest lavender goldenrod	2	2	1190.27	2861.95	14	-3
+Manufacturer#3	almond antique metallic orange dim	3	3	1410.39	4272.34	19	5
+Manufacturer#3	almond antique misty red olive	4	4	1922.98	6195.32	1	-18
+Manufacturer#3	almond antique olive coral navajo	5	5	1337.29	7532.61	45	44
+Manufacturer#4	almond antique gainsboro frosted violet	1	1	1620.67	1620.67	10	0
+Manufacturer#4	almond antique violet mint lemon	2	2	1375.42	2996.09	39	29
+Manufacturer#4	almond aquamarine floral ivory bisque	3	3	1206.26	4202.35	27	-12
+Manufacturer#4	almond aquamarine yellow dodger mint	4	4	1844.92	6047.27	7	-20
+Manufacturer#4	almond azure aquamarine papaya violet	5	5	1290.35	7337.620000000001	12	5
+Manufacturer#5	almond antique blue firebrick mint	1	1	1789.69	1789.69	31	0
+Manufacturer#5	almond antique medium spring khaki	2	2	1611.66	3401.3500000000004	6	-25
+Manufacturer#5	almond antique sky peru orange	3	3	1788.73	5190.08	2	-4
+Manufacturer#5	almond aquamarine dodger light gainsboro	4	4	1018.1	6208.18	46	44
+Manufacturer#5	almond azure blanched chiffon midnight	5	5	1464.48	7672.66	23	-23
+PREHOOK: query: -- 3. testJoinWithLag
+select p1.p_mfgr, p1.p_name,
+p1.p_size, p1.p_size - lag(p1.p_size,1,p1.p_size) over( distribute by p1.p_mfgr sort by p1.p_name) as deltaSz
+from part p1 join part p2 on p1.p_partkey = p2.p_partkey
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 3. testJoinWithLag
+select p1.p_mfgr, p1.p_name,
+p1.p_size, p1.p_size - lag(p1.p_size,1,p1.p_size) over( distribute by p1.p_mfgr sort by p1.p_name) as deltaSz
+from part p1 join part p2 on p1.p_partkey = p2.p_partkey
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	2	0
+Manufacturer#1	almond antique burnished rose metallic	2	0
+Manufacturer#1	almond antique burnished rose metallic	2	0
+Manufacturer#1	almond antique burnished rose metallic	2	0
+Manufacturer#1	almond antique chartreuse lavender yellow	34	32
+Manufacturer#1	almond antique salmon chartreuse burlywood	6	-28
+Manufacturer#1	almond aquamarine burnished black steel	28	22
+Manufacturer#1	almond aquamarine pink moccasin thistle	42	14
+Manufacturer#2	almond antique violet chocolate turquoise	14	0
+Manufacturer#2	almond antique violet turquoise frosted	40	26
+Manufacturer#2	almond aquamarine midnight light salmon	2	-38
+Manufacturer#2	almond aquamarine rose maroon antique	25	23
+Manufacturer#2	almond aquamarine sandy cyan gainsboro	18	-7
+Manufacturer#3	almond antique chartreuse khaki white	17	0
+Manufacturer#3	almond antique forest lavender goldenrod	14	-3
+Manufacturer#3	almond antique metallic orange dim	19	5
+Manufacturer#3	almond antique misty red olive	1	-18
+Manufacturer#3	almond antique olive coral navajo	45	44
+Manufacturer#4	almond antique gainsboro frosted violet	10	0
+Manufacturer#4	almond antique violet mint lemon	39	29
+Manufacturer#4	almond aquamarine floral ivory bisque	27	-12
+Manufacturer#4	almond aquamarine yellow dodger mint	7	-20
+Manufacturer#4	almond azure aquamarine papaya violet	12	5
+Manufacturer#5	almond antique blue firebrick mint	31	0
+Manufacturer#5	almond antique medium spring khaki	6	-25
+Manufacturer#5	almond antique sky peru orange	2	-4
+Manufacturer#5	almond aquamarine dodger light gainsboro	46	44
+Manufacturer#5	almond azure blanched chiffon midnight	23	-23
+PREHOOK: query: -- 4. testLagInSum
+select  p_mfgr,p_name, p_size,   
+sum(p_size - lag(p_size,1)) over(distribute by p_mfgr  sort by p_mfgr ) as deltaSum 
+from part 
+window w1 as (rows between 2 preceding and 2 following)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 4. testLagInSum
+select  p_mfgr,p_name, p_size,   
+sum(p_size - lag(p_size,1)) over(distribute by p_mfgr  sort by p_mfgr ) as deltaSum 
+from part 
+window w1 as (rows between 2 preceding and 2 following)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	2	40
+Manufacturer#1	almond antique chartreuse lavender yellow	34	40
+Manufacturer#1	almond antique burnished rose metallic	2	40
+Manufacturer#1	almond antique salmon chartreuse burlywood	6	40
+Manufacturer#1	almond aquamarine burnished black steel	28	40
+Manufacturer#1	almond aquamarine pink moccasin thistle	42	40
+Manufacturer#2	almond antique violet chocolate turquoise	14	4
+Manufacturer#2	almond antique violet turquoise frosted	40	4
+Manufacturer#2	almond aquamarine midnight light salmon	2	4
+Manufacturer#2	almond aquamarine rose maroon antique	25	4
+Manufacturer#2	almond aquamarine sandy cyan gainsboro	18	4
+Manufacturer#3	almond antique metallic orange dim	19	26
+Manufacturer#3	almond antique chartreuse khaki white	17	26
+Manufacturer#3	almond antique forest lavender goldenrod	14	26
+Manufacturer#3	almond antique misty red olive	1	26
+Manufacturer#3	almond antique olive coral navajo	45	26
+Manufacturer#4	almond antique gainsboro frosted violet	10	2
+Manufacturer#4	almond antique violet mint lemon	39	2
+Manufacturer#4	almond aquamarine floral ivory bisque	27	2
+Manufacturer#4	almond aquamarine yellow dodger mint	7	2
+Manufacturer#4	almond azure aquamarine papaya violet	12	2
+Manufacturer#5	almond antique blue firebrick mint	31	-8
+Manufacturer#5	almond antique medium spring khaki	6	-8
+Manufacturer#5	almond antique sky peru orange	2	-8
+Manufacturer#5	almond aquamarine dodger light gainsboro	46	-8
+Manufacturer#5	almond azure blanched chiffon midnight	23	-8
+PREHOOK: query: -- 5. testLagInSumOverWindow
+select  p_mfgr,p_name, p_size,   
+sum(p_size - lag(p_size,1)) over w1 as deltaSum 
+from part 
+window w1 as (distribute by p_mfgr sort by p_mfgr rows between 2 preceding and 2 following)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 5. testLagInSumOverWindow
+select  p_mfgr,p_name, p_size,   
+sum(p_size - lag(p_size,1)) over w1 as deltaSum 
+from part 
+window w1 as (distribute by p_mfgr sort by p_mfgr rows between 2 preceding and 2 following)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	2	0
+Manufacturer#1	almond antique chartreuse lavender yellow	34	4
+Manufacturer#1	almond antique burnished rose metallic	2	26
+Manufacturer#1	almond antique salmon chartreuse burlywood	6	8
+Manufacturer#1	almond aquamarine burnished black steel	28	40
+Manufacturer#1	almond aquamarine pink moccasin thistle	42	36
+Manufacturer#2	almond antique violet chocolate turquoise	14	-12
+Manufacturer#2	almond antique violet turquoise frosted	40	11
+Manufacturer#2	almond aquamarine midnight light salmon	2	4
+Manufacturer#2	almond aquamarine rose maroon antique	25	-22
+Manufacturer#2	almond aquamarine sandy cyan gainsboro	18	16
+Manufacturer#3	almond antique metallic orange dim	19	-5
+Manufacturer#3	almond antique chartreuse khaki white	17	-18
+Manufacturer#3	almond antique forest lavender goldenrod	14	26
+Manufacturer#3	almond antique misty red olive	1	28
+Manufacturer#3	almond antique olive coral navajo	45	31
+Manufacturer#4	almond antique gainsboro frosted violet	10	17
+Manufacturer#4	almond antique violet mint lemon	39	-3
+Manufacturer#4	almond aquamarine floral ivory bisque	27	2
+Manufacturer#4	almond aquamarine yellow dodger mint	7	-27
+Manufacturer#4	almond azure aquamarine papaya violet	12	-15
+Manufacturer#5	almond antique blue firebrick mint	31	-29
+Manufacturer#5	almond antique medium spring khaki	6	15
+Manufacturer#5	almond antique sky peru orange	2	-8
+Manufacturer#5	almond aquamarine dodger light gainsboro	46	17
+Manufacturer#5	almond azure blanched chiffon midnight	23	21
+PREHOOK: query: -- 6. testRankInLead
+select p_mfgr, p_name, p_size, r1,
+lead(r1,1,r1) over (distribute by p_mfgr sort by p_name) as deltaRank
+from (
+select p_mfgr, p_name, p_size, 
+rank() over(distribute by p_mfgr  sort by p_name) as r1 
+from part 
+) a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 6. testRankInLead
+select p_mfgr, p_name, p_size, r1,
+lead(r1,1,r1) over (distribute by p_mfgr sort by p_name) as deltaRank
+from (
+select p_mfgr, p_name, p_size, 
+rank() over(distribute by p_mfgr  sort by p_name) as r1 
+from part 
+) a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	2	1	1
+Manufacturer#1	almond antique burnished rose metallic	2	1	3
+Manufacturer#1	almond antique chartreuse lavender yellow	34	3	4
+Manufacturer#1	almond antique salmon chartreuse burlywood	6	4	5
+Manufacturer#1	almond aquamarine burnished black steel	28	5	6
+Manufacturer#1	almond aquamarine pink moccasin thistle	42	6	6
+Manufacturer#2	almond antique violet chocolate turquoise	14	1	2
+Manufacturer#2	almond antique violet turquoise frosted	40	2	3
+Manufacturer#2	almond aquamarine midnight light salmon	2	3	4
+Manufacturer#2	almond aquamarine rose maroon antique	25	4	5
+Manufacturer#2	almond aquamarine sandy cyan gainsboro	18	5	5
+Manufacturer#3	almond antique chartreuse khaki white	17	1	2
+Manufacturer#3	almond antique forest lavender goldenrod	14	2	3
+Manufacturer#3	almond antique metallic orange dim	19	3	4
+Manufacturer#3	almond antique misty red olive	1	4	5
+Manufacturer#3	almond antique olive coral navajo	45	5	5
+Manufacturer#4	almond antique gainsboro frosted violet	10	1	2
+Manufacturer#4	almond antique violet mint lemon	39	2	3
+Manufacturer#4	almond aquamarine floral ivory bisque	27	3	4
+Manufacturer#4	almond aquamarine yellow dodger mint	7	4	5
+Manufacturer#4	almond azure aquamarine papaya violet	12	5	5
+Manufacturer#5	almond antique blue firebrick mint	31	1	2
+Manufacturer#5	almond antique medium spring khaki	6	2	3
+Manufacturer#5	almond antique sky peru orange	2	3	4
+Manufacturer#5	almond aquamarine dodger light gainsboro	46	4	5
+Manufacturer#5	almond azure blanched chiffon midnight	23	5	5
+PREHOOK: query: -- 7. testLeadWithPTF
+select p_mfgr, p_name, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+p_size, p_size - lead(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz 
+from noop(on part 
+partition by p_mfgr 
+order by p_name  
+)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 7. testLeadWithPTF
+select p_mfgr, p_name, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+p_size, p_size - lead(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz 
+from noop(on part 
+partition by p_mfgr 
+order by p_name  
+)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	1	1	2	0
+Manufacturer#1	almond antique burnished rose metallic	1	1	2	-32
+Manufacturer#1	almond antique chartreuse lavender yellow	3	2	34	28
+Manufacturer#1	almond antique salmon chartreuse burlywood	4	3	6	-22
+Manufacturer#1	almond aquamarine burnished black steel	5	4	28	-14
+Manufacturer#1	almond aquamarine pink moccasin thistle	6	5	42	0
+Manufacturer#2	almond antique violet chocolate turquoise	1	1	14	-26
+Manufacturer#2	almond antique violet turquoise frosted	2	2	40	38
+Manufacturer#2	almond aquamarine midnight light salmon	3	3	2	-23
+Manufacturer#2	almond aquamarine rose maroon antique	4	4	25	7
+Manufacturer#2	almond aquamarine sandy cyan gainsboro	5	5	18	0
+Manufacturer#3	almond antique chartreuse khaki white	1	1	17	3
+Manufacturer#3	almond antique forest lavender goldenrod	2	2	14	-5
+Manufacturer#3	almond antique metallic orange dim	3	3	19	18
+Manufacturer#3	almond antique misty red olive	4	4	1	-44
+Manufacturer#3	almond antique olive coral navajo	5	5	45	0
+Manufacturer#4	almond antique gainsboro frosted violet	1	1	10	-29
+Manufacturer#4	almond antique violet mint lemon	2	2	39	12
+Manufacturer#4	almond aquamarine floral ivory bisque	3	3	27	20
+Manufacturer#4	almond aquamarine yellow dodger mint	4	4	7	-5
+Manufacturer#4	almond azure aquamarine papaya violet	5	5	12	0
+Manufacturer#5	almond antique blue firebrick mint	1	1	31	25
+Manufacturer#5	almond antique medium spring khaki	2	2	6	4
+Manufacturer#5	almond antique sky peru orange	3	3	2	-44
+Manufacturer#5	almond aquamarine dodger light gainsboro	4	4	46	23
+Manufacturer#5	almond azure blanched chiffon midnight	5	5	23	0
+PREHOOK: query: -- 8. testOverNoPartitionMultipleAggregate
+select p_name, p_retailprice,
+lead(p_retailprice) over() as l1 ,
+lag(p_retailprice)  over() as l2
+from part
+order by p_name
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 8. testOverNoPartitionMultipleAggregate
+select p_name, p_retailprice,
+lead(p_retailprice) over() as l1 ,
+lag(p_retailprice)  over() as l2
+from part
+order by p_name
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+almond antique blue firebrick mint	1789.69	1611.66	1290.35
+almond antique burnished rose metallic	1173.15	1753.76	1173.15
+almond antique burnished rose metallic	1173.15	1173.15	NULL
+almond antique chartreuse khaki white	1671.68	1190.27	1701.6
+almond antique chartreuse lavender yellow	1753.76	1602.59	1173.15
+almond antique forest lavender goldenrod	1190.27	1410.39	1671.68
+almond antique gainsboro frosted violet	1620.67	1375.42	1337.29
+almond antique medium spring khaki	1611.66	1788.73	1789.69
+almond antique metallic orange dim	1410.39	1922.98	1190.27
+almond antique misty red olive	1922.98	1337.29	1410.39
+almond antique olive coral navajo	1337.29	1620.67	1922.98
+almond antique salmon chartreuse burlywood	1602.59	1414.42	1753.76
+almond antique sky peru orange	1788.73	1018.1	1611.66
+almond antique violet chocolate turquoise	1690.68	1800.7	1632.66
+almond antique violet mint lemon	1375.42	1206.26	1620.67
+almond antique violet turquoise frosted	1800.7	2031.98	1690.68
+almond aquamarine burnished black steel	1414.42	1632.66	1602.59
+almond aquamarine dodger light gainsboro	1018.1	1464.48	1788.73
+almond aquamarine floral ivory bisque	1206.26	1844.92	1375.42
+almond aquamarine midnight light salmon	2031.98	1698.66	1800.7
+almond aquamarine pink moccasin thistle	1632.66	1690.68	1414.42
+almond aquamarine rose maroon antique	1698.66	1701.6	2031.98
+almond aquamarine sandy cyan gainsboro	1701.6	1671.68	1698.66
+almond aquamarine yellow dodger mint	1844.92	1290.35	1206.26
+almond azure aquamarine papaya violet	1290.35	1789.69	1844.92
+almond azure blanched chiffon midnight	1464.48	NULL	1018.1

Added: hive/trunk/ql/src/test/results/clientpositive/leadlag_queries.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/leadlag_queries.q.out?rev=1463556&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/leadlag_queries.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/leadlag_queries.q.out Tue Apr  2 14:16:34 2013
@@ -0,0 +1,261 @@
+PREHOOK: query: -- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part
+PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part
+PREHOOK: type: LOAD
+PREHOOK: Output: default@part
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@part
+PREHOOK: query: -- 1. testLeadUDAF
+select p_mfgr, p_retailprice,
+lead(p_retailprice) over (partition by p_mfgr order by p_name) as l1,
+lead(p_retailprice,1) over (partition by p_mfgr order by p_name) as l2,
+lead(p_retailprice,1,10) over (partition by p_mfgr order by p_name) as l3,
+lead(p_retailprice,1, p_retailprice) over (partition by p_mfgr order by p_name) as l4,
+p_retailprice - lead(p_retailprice,1,p_retailprice) over (partition by p_mfgr order by p_name)
+from part
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 1. testLeadUDAF
+select p_mfgr, p_retailprice,
+lead(p_retailprice) over (partition by p_mfgr order by p_name) as l1,
+lead(p_retailprice,1) over (partition by p_mfgr order by p_name) as l2,
+lead(p_retailprice,1,10) over (partition by p_mfgr order by p_name) as l3,
+lead(p_retailprice,1, p_retailprice) over (partition by p_mfgr order by p_name) as l4,
+p_retailprice - lead(p_retailprice,1,p_retailprice) over (partition by p_mfgr order by p_name)
+from part
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	1173.15	1173.15	1173.15	1173.15	1173.15	0.0
+Manufacturer#1	1173.15	1753.76	1753.76	1753.76	1753.76	-580.6099999999999
+Manufacturer#1	1753.76	1602.59	1602.59	1602.59	1602.59	151.17000000000007
+Manufacturer#1	1602.59	1414.42	1414.42	1414.42	1414.42	188.16999999999985
+Manufacturer#1	1414.42	1632.66	1632.66	1632.66	1632.66	-218.24
+Manufacturer#1	1632.66	NULL	NULL	10.0	1632.66	0.0
+Manufacturer#2	1690.68	1800.7	1800.7	1800.7	1800.7	-110.01999999999998
+Manufacturer#2	1800.7	2031.98	2031.98	2031.98	2031.98	-231.27999999999997
+Manufacturer#2	2031.98	1698.66	1698.66	1698.66	1698.66	333.31999999999994
+Manufacturer#2	1698.66	1701.6	1701.6	1701.6	1701.6	-2.939999999999827
+Manufacturer#2	1701.6	NULL	NULL	10.0	1701.6	0.0
+Manufacturer#3	1671.68	1190.27	1190.27	1190.27	1190.27	481.4100000000001
+Manufacturer#3	1190.27	1410.39	1410.39	1410.39	1410.39	-220.12000000000012
+Manufacturer#3	1410.39	1922.98	1922.98	1922.98	1922.98	-512.5899999999999
+Manufacturer#3	1922.98	1337.29	1337.29	1337.29	1337.29	585.69
+Manufacturer#3	1337.29	NULL	NULL	10.0	1337.29	0.0
+Manufacturer#4	1620.67	1375.42	1375.42	1375.42	1375.42	245.25
+Manufacturer#4	1375.42	1206.26	1206.26	1206.26	1206.26	169.16000000000008
+Manufacturer#4	1206.26	1844.92	1844.92	1844.92	1844.92	-638.6600000000001
+Manufacturer#4	1844.92	1290.35	1290.35	1290.35	1290.35	554.5700000000002
+Manufacturer#4	1290.35	NULL	NULL	10.0	1290.35	0.0
+Manufacturer#5	1789.69	1611.66	1611.66	1611.66	1611.66	178.02999999999997
+Manufacturer#5	1611.66	1788.73	1788.73	1788.73	1788.73	-177.06999999999994
+Manufacturer#5	1788.73	1018.1	1018.1	1018.1	1018.1	770.63
+Manufacturer#5	1018.1	1464.48	1464.48	1464.48	1464.48	-446.38
+Manufacturer#5	1464.48	NULL	NULL	10.0	1464.48	0.0
+PREHOOK: query: -- 2.testLeadUDAFPartSz1
+select p_mfgr, p_name, p_retailprice,
+lead(p_retailprice,1) over (partition by p_mfgr, p_name ),
+p_retailprice - lead(p_retailprice,1,p_retailprice) over (partition by p_mfgr, p_name)
+from part
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 2.testLeadUDAFPartSz1
+select p_mfgr, p_name, p_retailprice,
+lead(p_retailprice,1) over (partition by p_mfgr, p_name ),
+p_retailprice - lead(p_retailprice,1,p_retailprice) over (partition by p_mfgr, p_name)
+from part
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	1173.15	1173.15	0.0
+Manufacturer#1	almond antique burnished rose metallic	1173.15	NULL	0.0
+Manufacturer#1	almond antique chartreuse lavender yellow	1753.76	NULL	0.0
+Manufacturer#1	almond antique salmon chartreuse burlywood	1602.59	NULL	0.0
+Manufacturer#1	almond aquamarine burnished black steel	1414.42	NULL	0.0
+Manufacturer#1	almond aquamarine pink moccasin thistle	1632.66	NULL	0.0
+Manufacturer#2	almond antique violet chocolate turquoise	1690.68	NULL	0.0
+Manufacturer#2	almond antique violet turquoise frosted	1800.7	NULL	0.0
+Manufacturer#2	almond aquamarine midnight light salmon	2031.98	NULL	0.0
+Manufacturer#2	almond aquamarine rose maroon antique	1698.66	NULL	0.0
+Manufacturer#2	almond aquamarine sandy cyan gainsboro	1701.6	NULL	0.0
+Manufacturer#3	almond antique chartreuse khaki white	1671.68	NULL	0.0
+Manufacturer#3	almond antique forest lavender goldenrod	1190.27	NULL	0.0
+Manufacturer#3	almond antique metallic orange dim	1410.39	NULL	0.0
+Manufacturer#3	almond antique misty red olive	1922.98	NULL	0.0
+Manufacturer#3	almond antique olive coral navajo	1337.29	NULL	0.0
+Manufacturer#4	almond antique gainsboro frosted violet	1620.67	NULL	0.0
+Manufacturer#4	almond antique violet mint lemon	1375.42	NULL	0.0
+Manufacturer#4	almond aquamarine floral ivory bisque	1206.26	NULL	0.0
+Manufacturer#4	almond aquamarine yellow dodger mint	1844.92	NULL	0.0
+Manufacturer#4	almond azure aquamarine papaya violet	1290.35	NULL	0.0
+Manufacturer#5	almond antique blue firebrick mint	1789.69	NULL	0.0
+Manufacturer#5	almond antique medium spring khaki	1611.66	NULL	0.0
+Manufacturer#5	almond antique sky peru orange	1788.73	NULL	0.0
+Manufacturer#5	almond aquamarine dodger light gainsboro	1018.1	NULL	0.0
+Manufacturer#5	almond azure blanched chiffon midnight	1464.48	NULL	0.0
+PREHOOK: query: -- 3.testLagUDAF
+select p_mfgr, p_retailprice,
+lag(p_retailprice,1) over (partition by p_mfgr order by p_name) as l1,
+lag(p_retailprice) over (partition by p_mfgr order by p_name) as l2,
+lag(p_retailprice,1, p_retailprice) over (partition by p_mfgr order by p_name) as l3,
+lag(p_retailprice,1,10) over (partition by p_mfgr order by p_name) as l4,
+p_retailprice - lag(p_retailprice,1,p_retailprice) over (partition by p_mfgr order by p_name)
+from part
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 3.testLagUDAF
+select p_mfgr, p_retailprice,
+lag(p_retailprice,1) over (partition by p_mfgr order by p_name) as l1,
+lag(p_retailprice) over (partition by p_mfgr order by p_name) as l2,
+lag(p_retailprice,1, p_retailprice) over (partition by p_mfgr order by p_name) as l3,
+lag(p_retailprice,1,10) over (partition by p_mfgr order by p_name) as l4,
+p_retailprice - lag(p_retailprice,1,p_retailprice) over (partition by p_mfgr order by p_name)
+from part
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	1173.15	NULL	NULL	1173.15	10.0	0.0
+Manufacturer#1	1173.15	1173.15	1173.15	1173.15	1173.15	0.0
+Manufacturer#1	1753.76	1173.15	1173.15	1173.15	1173.15	580.6099999999999
+Manufacturer#1	1602.59	1753.76	1753.76	1753.76	1753.76	-151.17000000000007
+Manufacturer#1	1414.42	1602.59	1602.59	1602.59	1602.59	-188.16999999999985
+Manufacturer#1	1632.66	1414.42	1414.42	1414.42	1414.42	218.24
+Manufacturer#2	1690.68	NULL	NULL	1690.68	10.0	0.0
+Manufacturer#2	1800.7	1690.68	1690.68	1690.68	1690.68	110.01999999999998
+Manufacturer#2	2031.98	1800.7	1800.7	1800.7	1800.7	231.27999999999997
+Manufacturer#2	1698.66	2031.98	2031.98	2031.98	2031.98	-333.31999999999994
+Manufacturer#2	1701.6	1698.66	1698.66	1698.66	1698.66	2.939999999999827
+Manufacturer#3	1671.68	NULL	NULL	1671.68	10.0	0.0
+Manufacturer#3	1190.27	1671.68	1671.68	1671.68	1671.68	-481.4100000000001
+Manufacturer#3	1410.39	1190.27	1190.27	1190.27	1190.27	220.12000000000012
+Manufacturer#3	1922.98	1410.39	1410.39	1410.39	1410.39	512.5899999999999
+Manufacturer#3	1337.29	1922.98	1922.98	1922.98	1922.98	-585.69
+Manufacturer#4	1620.67	NULL	NULL	1620.67	10.0	0.0
+Manufacturer#4	1375.42	1620.67	1620.67	1620.67	1620.67	-245.25
+Manufacturer#4	1206.26	1375.42	1375.42	1375.42	1375.42	-169.16000000000008
+Manufacturer#4	1844.92	1206.26	1206.26	1206.26	1206.26	638.6600000000001
+Manufacturer#4	1290.35	1844.92	1844.92	1844.92	1844.92	-554.5700000000002
+Manufacturer#5	1789.69	NULL	NULL	1789.69	10.0	0.0
+Manufacturer#5	1611.66	1789.69	1789.69	1789.69	1789.69	-178.02999999999997
+Manufacturer#5	1788.73	1611.66	1611.66	1611.66	1611.66	177.06999999999994
+Manufacturer#5	1018.1	1788.73	1788.73	1788.73	1788.73	-770.63
+Manufacturer#5	1464.48	1018.1	1018.1	1018.1	1018.1	446.38
+PREHOOK: query: -- 4.testLagUDAFPartSz1
+select p_mfgr, p_name, p_retailprice,
+lag(p_retailprice,1) over (partition by p_mfgr, p_name ),
+p_retailprice - lag(p_retailprice,1,p_retailprice) over (partition by p_mfgr, p_name)
+from part
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 4.testLagUDAFPartSz1
+select p_mfgr, p_name, p_retailprice,
+lag(p_retailprice,1) over (partition by p_mfgr, p_name ),
+p_retailprice - lag(p_retailprice,1,p_retailprice) over (partition by p_mfgr, p_name)
+from part
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	1173.15	NULL	0.0
+Manufacturer#1	almond antique burnished rose metallic	1173.15	1173.15	0.0
+Manufacturer#1	almond antique chartreuse lavender yellow	1753.76	NULL	0.0
+Manufacturer#1	almond antique salmon chartreuse burlywood	1602.59	NULL	0.0
+Manufacturer#1	almond aquamarine burnished black steel	1414.42	NULL	0.0
+Manufacturer#1	almond aquamarine pink moccasin thistle	1632.66	NULL	0.0
+Manufacturer#2	almond antique violet chocolate turquoise	1690.68	NULL	0.0
+Manufacturer#2	almond antique violet turquoise frosted	1800.7	NULL	0.0
+Manufacturer#2	almond aquamarine midnight light salmon	2031.98	NULL	0.0
+Manufacturer#2	almond aquamarine rose maroon antique	1698.66	NULL	0.0
+Manufacturer#2	almond aquamarine sandy cyan gainsboro	1701.6	NULL	0.0
+Manufacturer#3	almond antique chartreuse khaki white	1671.68	NULL	0.0
+Manufacturer#3	almond antique forest lavender goldenrod	1190.27	NULL	0.0
+Manufacturer#3	almond antique metallic orange dim	1410.39	NULL	0.0
+Manufacturer#3	almond antique misty red olive	1922.98	NULL	0.0
+Manufacturer#3	almond antique olive coral navajo	1337.29	NULL	0.0
+Manufacturer#4	almond antique gainsboro frosted violet	1620.67	NULL	0.0
+Manufacturer#4	almond antique violet mint lemon	1375.42	NULL	0.0
+Manufacturer#4	almond aquamarine floral ivory bisque	1206.26	NULL	0.0
+Manufacturer#4	almond aquamarine yellow dodger mint	1844.92	NULL	0.0
+Manufacturer#4	almond azure aquamarine papaya violet	1290.35	NULL	0.0
+Manufacturer#5	almond antique blue firebrick mint	1789.69	NULL	0.0
+Manufacturer#5	almond antique medium spring khaki	1611.66	NULL	0.0
+Manufacturer#5	almond antique sky peru orange	1788.73	NULL	0.0
+Manufacturer#5	almond aquamarine dodger light gainsboro	1018.1	NULL	0.0
+Manufacturer#5	almond azure blanched chiffon midnight	1464.48	NULL	0.0
+PREHOOK: query: -- 5.testLeadLagUDAF
+select p_mfgr, p_retailprice,
+lead(p_retailprice,1) over (partition by p_mfgr order by p_name) as l1,
+lead(p_retailprice,1, p_retailprice) over (partition by p_mfgr order by p_name) as l2,
+p_retailprice - lead(p_retailprice,1,p_retailprice) over (partition by p_mfgr order by p_name),
+lag(p_retailprice,1) over (partition by p_mfgr order by p_name) as l3,
+lag(p_retailprice,1, p_retailprice) over (partition by p_mfgr order by p_name)  as l4
+from part
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 5.testLeadLagUDAF
+select p_mfgr, p_retailprice,
+lead(p_retailprice,1) over (partition by p_mfgr order by p_name) as l1,
+lead(p_retailprice,1, p_retailprice) over (partition by p_mfgr order by p_name) as l2,
+p_retailprice - lead(p_retailprice,1,p_retailprice) over (partition by p_mfgr order by p_name),
+lag(p_retailprice,1) over (partition by p_mfgr order by p_name) as l3,
+lag(p_retailprice,1, p_retailprice) over (partition by p_mfgr order by p_name)  as l4
+from part
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	1173.15	1173.15	1173.15	0.0	NULL	1173.15
+Manufacturer#1	1173.15	1753.76	1753.76	-580.6099999999999	1173.15	1173.15
+Manufacturer#1	1753.76	1602.59	1602.59	151.17000000000007	1173.15	1173.15
+Manufacturer#1	1602.59	1414.42	1414.42	188.16999999999985	1753.76	1753.76
+Manufacturer#1	1414.42	1632.66	1632.66	-218.24	1602.59	1602.59
+Manufacturer#1	1632.66	NULL	1632.66	0.0	1414.42	1414.42
+Manufacturer#2	1690.68	1800.7	1800.7	-110.01999999999998	NULL	1690.68
+Manufacturer#2	1800.7	2031.98	2031.98	-231.27999999999997	1690.68	1690.68
+Manufacturer#2	2031.98	1698.66	1698.66	333.31999999999994	1800.7	1800.7
+Manufacturer#2	1698.66	1701.6	1701.6	-2.939999999999827	2031.98	2031.98
+Manufacturer#2	1701.6	NULL	1701.6	0.0	1698.66	1698.66
+Manufacturer#3	1671.68	1190.27	1190.27	481.4100000000001	NULL	1671.68
+Manufacturer#3	1190.27	1410.39	1410.39	-220.12000000000012	1671.68	1671.68
+Manufacturer#3	1410.39	1922.98	1922.98	-512.5899999999999	1190.27	1190.27
+Manufacturer#3	1922.98	1337.29	1337.29	585.69	1410.39	1410.39
+Manufacturer#3	1337.29	NULL	1337.29	0.0	1922.98	1922.98
+Manufacturer#4	1620.67	1375.42	1375.42	245.25	NULL	1620.67
+Manufacturer#4	1375.42	1206.26	1206.26	169.16000000000008	1620.67	1620.67
+Manufacturer#4	1206.26	1844.92	1844.92	-638.6600000000001	1375.42	1375.42
+Manufacturer#4	1844.92	1290.35	1290.35	554.5700000000002	1206.26	1206.26
+Manufacturer#4	1290.35	NULL	1290.35	0.0	1844.92	1844.92
+Manufacturer#5	1789.69	1611.66	1611.66	178.02999999999997	NULL	1789.69
+Manufacturer#5	1611.66	1788.73	1788.73	-177.06999999999994	1789.69	1789.69
+Manufacturer#5	1788.73	1018.1	1018.1	770.63	1611.66	1611.66
+Manufacturer#5	1018.1	1464.48	1464.48	-446.38	1788.73	1788.73
+Manufacturer#5	1464.48	NULL	1464.48	0.0	1018.1	1018.1