You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2015/09/09 04:18:05 UTC

[01/11] hive git commit: HIVE-11657 : HIVE-2573 introduces some issues during metastore init (and CLI init) (Sergey Shelukhin, reviewed by Sushanth Sowmyan)

Repository: hive
Updated Branches:
  refs/heads/llap 6413e8679 -> b20330db7


HIVE-11657 : HIVE-2573 introduces some issues during metastore init (and CLI init) (Sergey Shelukhin, reviewed by Sushanth Sowmyan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/730a4040
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/730a4040
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/730a4040

Branch: refs/heads/llap
Commit: 730a40401c81e18fe773ac41012125184f776a04
Parents: bb4f5e7
Author: Sergey Shelukhin <se...@apache.org>
Authored: Thu Sep 3 19:57:40 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Thu Sep 3 19:57:40 2015 -0700

----------------------------------------------------------------------
 .../hive/metastore/RetryingMetaStoreClient.java | 32 +++++++----
 .../hadoop/hive/ql/exec/FunctionTask.java       |  2 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    | 56 ++++++++++++++++----
 3 files changed, 70 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/730a4040/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java
index e282981..5087098 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java
@@ -161,16 +161,27 @@ public class RetryingMetaStoreClient implements InvocationHandler {
       } catch (UndeclaredThrowableException e) {
         throw e.getCause();
       } catch (InvocationTargetException e) {
-        if ((e.getCause() instanceof TApplicationException) ||
-            (e.getCause() instanceof TProtocolException) ||
-            (e.getCause() instanceof TTransportException)) {
-          caughtException = (TException) e.getCause();
-        } else if ((e.getCause() instanceof MetaException) &&
-            e.getCause().getMessage().matches
-            ("(?s).*(JDO[a-zA-Z]*|TApplication|TProtocol|TTransport)Exception.*")) {
-          caughtException = (MetaException) e.getCause();
+        Throwable t = e.getCause();
+        if (t instanceof TApplicationException) {
+          TApplicationException tae = (TApplicationException)t;
+          switch (tae.getType()) {
+          case TApplicationException.UNSUPPORTED_CLIENT_TYPE:
+          case TApplicationException.UNKNOWN_METHOD:
+          case TApplicationException.WRONG_METHOD_NAME:
+          case TApplicationException.INVALID_PROTOCOL:
+            throw t;
+          default:
+            // TODO: most other options are probably unrecoverable... throw?
+            caughtException = tae;
+          }
+        } else if ((t instanceof TProtocolException) || (t instanceof TTransportException)) {
+          // TODO: most protocol exceptions are probably unrecoverable... throw?
+          caughtException = (TException)t;
+        } else if ((t instanceof MetaException) && t.getMessage().matches(
+            "(?s).*(JDO[a-zA-Z]*|TProtocol|TTransport)Exception.*")) {
+          caughtException = (MetaException)t;
         } else {
-          throw e.getCause();
+          throw t;
         }
       } catch (MetaException e) {
         if (e.getMessage().matches("(?s).*(IO|TTransport)Exception.*")) {
@@ -180,7 +191,8 @@ public class RetryingMetaStoreClient implements InvocationHandler {
         }
       }
 
-      if (retriesMade >=  retryLimit) {
+
+      if (retriesMade >= retryLimit) {
         throw caughtException;
       }
       retriesMade++;

http://git-wip-us.apache.org/repos/asf/hive/blob/730a4040/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
index 6390740..7671d29 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
@@ -98,7 +98,7 @@ public class FunctionTask extends Task<FunctionWork> {
 
     if (work.getReloadFunctionDesc() != null) {
       try {
-        Hive.reloadFunctions();
+        Hive.get().reloadFunctions();
       } catch (Exception e) {
         setException(e);
         LOG.error(stringifyException(e));

http://git-wip-us.apache.org/repos/asf/hive/blob/730a4040/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index 396c070..c449aee 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -113,6 +113,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE;
 import static org.apache.hadoop.hive.serde.serdeConstants.COLLECTION_DELIM;
@@ -160,24 +161,61 @@ public class Hive {
     }
   };
 
+  // Note that while this is an improvement over static initialization, it is still not,
+  // technically, valid, cause nothing prevents us from connecting to several metastores in
+  // the same process. This will still only get the functions from the first metastore.
+  private final static AtomicInteger didRegisterAllFuncs = new AtomicInteger(0);
+  private final static int REG_FUNCS_NO = 0, REG_FUNCS_DONE = 2, REG_FUNCS_PENDING = 1;
+
   // register all permanent functions. need improvement
-  static {
+  private void registerAllFunctionsOnce() {
+    boolean breakLoop = false;
+    while (!breakLoop) {
+      int val = didRegisterAllFuncs.get();
+      switch (val) {
+      case REG_FUNCS_NO: {
+        if (didRegisterAllFuncs.compareAndSet(val, REG_FUNCS_PENDING)) {
+          breakLoop = true;
+          break;
+        }
+        continue;
+      }
+      case REG_FUNCS_PENDING: {
+        synchronized (didRegisterAllFuncs) {
+          try {
+            didRegisterAllFuncs.wait(100);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            return;
+          }
+        }
+        continue;
+      }
+      case REG_FUNCS_DONE: return;
+      default: throw new AssertionError(val);
+      }
+    }
     try {
       reloadFunctions();
     } catch (Exception e) {
-      LOG.warn("Failed to access metastore. This class should not accessed in runtime.",e);
+      LOG.warn("Failed to register all functions.", e);
+    } finally {
+      boolean result = didRegisterAllFuncs.compareAndSet(REG_FUNCS_PENDING, REG_FUNCS_DONE);
+      assert result;
+      synchronized (didRegisterAllFuncs) {
+        didRegisterAllFuncs.notifyAll();
+      }
     }
   }
 
-  public static void reloadFunctions() throws HiveException {
-    Hive db = Hive.get();
-    for (Function function : db.getAllFunctions()) {
+  public void reloadFunctions() throws HiveException {
+    for (Function function : getAllFunctions()) {
       String functionName = function.getFunctionName();
       try {
         LOG.info("Registering function " + functionName + " " + function.getClassName());
-        FunctionRegistry.registerPermanentFunction(
-                FunctionUtils.qualifyFunctionName(functionName, function.getDbName()), function.getClassName(),
-                false, FunctionTask.toFunctionResource(function.getResourceUris()));
+        FunctionRegistry.registerPermanentFunction(FunctionUtils.qualifyFunctionName(
+                    functionName, function.getDbName()), function.getClassName(), false,
+                    FunctionTask.toFunctionResource(function.getResourceUris()));
       } catch (Exception e) {
         LOG.warn("Failed to register persistent function " +
                 functionName + ":" + function.getClassName() + ". Ignore and continue.");
@@ -268,6 +306,7 @@ public class Hive {
    */
   private Hive(HiveConf c) throws HiveException {
     conf = c;
+    registerAllFunctionsOnce();
   }
 
 
@@ -2675,7 +2714,6 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
 
     //Check if different encryption zones
-    HadoopShims.HdfsFileStatus destStatus = null;
     HadoopShims.HdfsEncryptionShim hdfsEncryptionShim = SessionState.get().getHdfsEncryptionShim();
     return hdfsEncryptionShim != null && (hdfsEncryptionShim.isPathEncrypted(srcf) || hdfsEncryptionShim.isPathEncrypted(destf))
       && !hdfsEncryptionShim.arePathsOnSameEncryptionZone(srcf, destf);


[02/11] hive git commit: HIVE-11600 : Hive Parser to Support multi col in clause (x, y..) in ((..), ..., ()) (Pengcheng Xiong, reviewed by Laljo John Pullokkaran)

Posted by se...@apache.org.
HIVE-11600 : Hive Parser to Support multi col in clause (x,y..) in ((..),..., ()) (Pengcheng Xiong, reviewed by Laljo John Pullokkaran)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/8bed378e
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8bed378e
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8bed378e

Branch: refs/heads/llap
Commit: 8bed378eac538a1bf1f4599b50929abedc735891
Parents: 730a404
Author: Pengcheng Xiong <px...@apache.org>
Authored: Fri Sep 4 10:13:49 2015 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Fri Sep 4 10:13:49 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |  46 ++-
 .../TestSQL11ReservedKeyWordsNegative.java      |  32 +-
 .../TestSQL11ReservedKeyWordsPositive.java      |  23 +-
 ql/src/test/queries/clientpositive/char_udf1.q  |   9 +-
 ql/src/test/queries/clientpositive/keyword_2.q  |  14 +
 .../queries/clientpositive/multi_column_in.q    |  71 ++++
 .../test/queries/clientpositive/varchar_udf1.q  |   6 +-
 .../clientpositive/char_udf1.q.java1.7.out      |  22 +-
 .../test/results/clientpositive/keyword_2.q.out |  51 +++
 .../clientpositive/multi_column_in.q.out        | 410 +++++++++++++++++++
 .../clientpositive/varchar_udf1.q.java1.7.out   |  12 +-
 11 files changed, 663 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8bed378e/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
index 64af7d1..bac0d22 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
@@ -430,24 +430,31 @@ precedenceEqualOperator
 subQueryExpression 
     : 
     LPAREN! selectStatement[true] RPAREN!     
- ;
- 
+    ;
+
 precedenceEqualExpression
     :
+    (LPAREN precedenceBitwiseOrExpression COMMA) => precedenceEqualExpressionMutiple
+    |
+    precedenceEqualExpressionSingle
+    ;
+
+precedenceEqualExpressionSingle
+    :
     (left=precedenceBitwiseOrExpression -> $left)
     (
        (KW_NOT precedenceEqualNegatableOperator notExpr=precedenceBitwiseOrExpression)
-       -> ^(KW_NOT ^(precedenceEqualNegatableOperator $precedenceEqualExpression $notExpr))
+       -> ^(KW_NOT ^(precedenceEqualNegatableOperator $precedenceEqualExpressionSingle $notExpr))
     | (precedenceEqualOperator equalExpr=precedenceBitwiseOrExpression)
-       -> ^(precedenceEqualOperator $precedenceEqualExpression $equalExpr)
+       -> ^(precedenceEqualOperator $precedenceEqualExpressionSingle $equalExpr)
     | (KW_NOT KW_IN LPAREN KW_SELECT)=>  (KW_NOT KW_IN subQueryExpression) 
-       -> ^(KW_NOT ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_IN) subQueryExpression $precedenceEqualExpression))
+       -> ^(KW_NOT ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_IN) subQueryExpression $precedenceEqualExpressionSingle))
     | (KW_NOT KW_IN expressions)
-       -> ^(KW_NOT ^(TOK_FUNCTION KW_IN $precedenceEqualExpression expressions))
+       -> ^(KW_NOT ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionSingle expressions))
     | (KW_IN LPAREN KW_SELECT)=>  (KW_IN subQueryExpression) 
-       -> ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_IN) subQueryExpression $precedenceEqualExpression)
+       -> ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_IN) subQueryExpression $precedenceEqualExpressionSingle)
     | (KW_IN expressions)
-       -> ^(TOK_FUNCTION KW_IN $precedenceEqualExpression expressions)
+       -> ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionSingle expressions)
     | ( KW_NOT KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) )
        -> ^(TOK_FUNCTION Identifier["between"] KW_TRUE $left $min $max)
     | ( KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) )
@@ -458,7 +465,22 @@ precedenceEqualExpression
 
 expressions
     :
-    LPAREN expression (COMMA expression)* RPAREN -> expression*
+    LPAREN expression (COMMA expression)* RPAREN -> expression+
+    ;
+
+//we transform the (col0, col1) in ((v00,v01),(v10,v11)) into struct(col0, col1) in (struct(v00,v01),struct(v10,v11))
+precedenceEqualExpressionMutiple
+    :
+    (LPAREN precedenceBitwiseOrExpression (COMMA precedenceBitwiseOrExpression)+ RPAREN -> ^(TOK_FUNCTION Identifier["struct"] precedenceBitwiseOrExpression+))
+    ( (KW_IN LPAREN expressionsToStruct (COMMA expressionsToStruct)+ RPAREN)
+       -> ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionMutiple expressionsToStruct+)
+    | (KW_NOT KW_IN LPAREN expressionsToStruct (COMMA expressionsToStruct)+ RPAREN)
+       -> ^(KW_NOT ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionMutiple expressionsToStruct+)))
+    ;
+
+expressionsToStruct
+    :
+    LPAREN expression (COMMA expression)* RPAREN -> ^(TOK_FUNCTION Identifier["struct"] expression+)
     ;
 
 precedenceNotOperator
@@ -635,7 +657,7 @@ nonReserved
     | KW_MAPJOIN | KW_MATERIALIZED | KW_METADATA | KW_MINUS | KW_MINUTE | KW_MONTH | KW_MSCK | KW_NOSCAN | KW_NO_DROP | KW_OFFLINE
     | KW_OPTION | KW_OUTPUTDRIVER | KW_OUTPUTFORMAT | KW_OVERWRITE | KW_OWNER | KW_PARTITIONED | KW_PARTITIONS | KW_PLUS | KW_PRETTY
     | KW_PRINCIPALS | KW_PROTECTION | KW_PURGE | KW_READ | KW_READONLY | KW_REBUILD | KW_RECORDREADER | KW_RECORDWRITER
-    | KW_REGEXP | KW_RELOAD | KW_RENAME | KW_REPAIR | KW_REPLACE | KW_REPLICATION | KW_RESTRICT | KW_REWRITE | KW_RLIKE
+    | KW_RELOAD | KW_RENAME | KW_REPAIR | KW_REPLACE | KW_REPLICATION | KW_RESTRICT | KW_REWRITE
     | KW_ROLE | KW_ROLES | KW_SCHEMA | KW_SCHEMAS | KW_SECOND | KW_SEMI | KW_SERDE | KW_SERDEPROPERTIES | KW_SERVER | KW_SETS | KW_SHARED
     | KW_SHOW | KW_SHOW_DATABASE | KW_SKEWED | KW_SORT | KW_SORTED | KW_SSL | KW_STATISTICS | KW_STORED
     | KW_STREAMTABLE | KW_STRING | KW_STRUCT | KW_TABLES | KW_TBLPROPERTIES | KW_TEMPORARY | KW_TERMINATED
@@ -668,5 +690,7 @@ sql11ReservedKeywordsUsedAsIdentifier
     | KW_LEFT | KW_LIKE | KW_LOCAL | KW_NONE | KW_NULL | KW_OF | KW_ORDER | KW_OUT | KW_OUTER | KW_PARTITION 
     | KW_PERCENT | KW_PROCEDURE | KW_RANGE | KW_READS | KW_REVOKE | KW_RIGHT 
     | KW_ROLLUP | KW_ROW | KW_ROWS | KW_SET | KW_SMALLINT | KW_TABLE | KW_TIMESTAMP | KW_TO | KW_TRIGGER | KW_TRUE 
-    | KW_TRUNCATE | KW_UNION | KW_UPDATE | KW_USER | KW_USING | KW_VALUES | KW_WITH
+    | KW_TRUNCATE | KW_UNION | KW_UPDATE | KW_USER | KW_USING | KW_VALUES | KW_WITH 
+//The following two keywords come from MySQL. Although they are not keywords in SQL2011, they are reserved keywords in MySQL.    
+    | KW_REGEXP | KW_RLIKE
     ;

http://git-wip-us.apache.org/repos/asf/hive/blob/8bed378e/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsNegative.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsNegative.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsNegative.java
index 61b5892..97ae0d9 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsNegative.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsNegative.java
@@ -30,7 +30,7 @@ import org.junit.Test;
 
 /**
  * Parser tests for SQL11 Reserved KeyWords. Please find more information in
- * HIVE-6617. Total number : 74
+ * HIVE-6617. Total number : 74 + 2 (MySQL)
  */
 public class TestSQL11ReservedKeyWordsNegative {
   private static HiveConf conf;
@@ -1070,4 +1070,34 @@ public class TestSQL11ReservedKeyWordsNegative {
               ex.getMessage());
     }
   }
+
+  // MySQL reserved keywords.
+  @Test
+  public void testSQL11ReservedKeyWords_RLIKE() {
+    try {
+      parse("CREATE TABLE RLIKE (col STRING)");
+      Assert.assertFalse("Expected ParseException", true);
+    } catch (ParseException ex) {
+      Assert
+          .assertEquals(
+              "Failure didn't match.",
+              "line 1:13 Failed to recognize predicate 'RLIKE'. Failed rule: 'identifier' in table name",
+              ex.getMessage());
+    }
+  }
+
+  @Test
+  public void testSQL11ReservedKeyWords_REGEXP() {
+    try {
+      parse("CREATE TABLE REGEXP (col STRING)");
+      Assert.assertFalse("Expected ParseException", true);
+    } catch (ParseException ex) {
+      Assert
+          .assertEquals(
+              "Failure didn't match.",
+              "line 1:13 Failed to recognize predicate 'REGEXP'. Failed rule: 'identifier' in table name",
+              ex.getMessage());
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/8bed378e/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsPositive.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsPositive.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsPositive.java
index 4c84e91..2a68899 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsPositive.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsPositive.java
@@ -30,7 +30,7 @@ import org.junit.Test;
 
 /**
  * Parser tests for SQL11 Reserved KeyWords. Please find more information in
- * HIVE-6617. Total number : 74
+ * HIVE-6617. Total number : 74 + 2 (MySQL)
  */
 public class TestSQL11ReservedKeyWordsPositive {
   private static HiveConf conf;
@@ -798,4 +798,25 @@ public class TestSQL11ReservedKeyWordsPositive {
             "(TOK_CREATETABLE (TOK_TABNAME WITH) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL col TOK_STRING)))",
             ast.toStringTree());
   }
+
+  // MySQL reserved keywords.
+  @Test
+  public void testSQL11ReservedKeyWords_RLIKE() throws ParseException {
+    ASTNode ast = parse("CREATE TABLE RLIKE (col STRING)");
+    Assert
+        .assertEquals(
+            "AST doesn't match",
+            "(TOK_CREATETABLE (TOK_TABNAME RLIKE) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL col TOK_STRING)))",
+            ast.toStringTree());
+  }
+
+  @Test
+  public void testSQL11ReservedKeyWords_REGEXP() throws ParseException {
+    ASTNode ast = parse("CREATE TABLE REGEXP (col STRING)");
+    Assert
+        .assertEquals(
+            "AST doesn't match",
+            "(TOK_CREATETABLE (TOK_TABNAME REGEXP) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL col TOK_STRING)))",
+            ast.toStringTree());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/8bed378e/ql/src/test/queries/clientpositive/char_udf1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/char_udf1.q b/ql/src/test/queries/clientpositive/char_udf1.q
index 8848609..09012b4 100644
--- a/ql/src/test/queries/clientpositive/char_udf1.q
+++ b/ql/src/test/queries/clientpositive/char_udf1.q
@@ -74,10 +74,13 @@ select
   ltrim(c2) = ltrim(c4)
 from char_udf_1 limit 1;
 
+-- In hive wiki page https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF
+-- we only allow A regexp B, not regexp (A,B).
+
 select
-  regexp(c2, 'val'),
-  regexp(c4, 'val'),
-  regexp(c2, 'val') = regexp(c4, 'val')
+  c2 regexp 'val',
+  c4 regexp 'val',
+  (c2 regexp 'val') = (c4 regexp 'val')
 from char_udf_1 limit 1;
 
 select

http://git-wip-us.apache.org/repos/asf/hive/blob/8bed378e/ql/src/test/queries/clientpositive/keyword_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/keyword_2.q b/ql/src/test/queries/clientpositive/keyword_2.q
new file mode 100644
index 0000000..054e26a
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/keyword_2.q
@@ -0,0 +1,14 @@
+set hive.support.sql11.reserved.keywords=false;
+drop table varchar_udf_1;
+
+create table varchar_udf_1 (c1 string, c2 string, c3 varchar(10), c4 varchar(20));
+insert overwrite table varchar_udf_1
+  select key, value, key, value from src where key = '238' limit 1;
+
+select
+  regexp(c2, 'val'),
+  regexp(c4, 'val'),
+  regexp(c2, 'val') = regexp(c4, 'val')
+from varchar_udf_1 limit 1;
+
+drop table varchar_udf_1;

http://git-wip-us.apache.org/repos/asf/hive/blob/8bed378e/ql/src/test/queries/clientpositive/multi_column_in.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/multi_column_in.q b/ql/src/test/queries/clientpositive/multi_column_in.q
new file mode 100644
index 0000000..18a56cc
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/multi_column_in.q
@@ -0,0 +1,71 @@
+drop table emps;
+
+create table emps (empno int, deptno int, empname string);
+
+insert into table emps values (1,2,"11"),(1,2,"11"),(3,4,"33"),(1,3,"11"),(2,5,"22"),(2,5,"22");
+
+select * from emps;
+
+select * from emps where (int(empno+deptno/2), int(deptno/3)) in ((2,0),(3,2));
+
+select * from emps where (int(empno+deptno/2), int(deptno/3)) not in ((2,0),(3,2));
+
+select * from emps where (empno,deptno) in ((1,2),(3,2));
+
+select * from emps where (empno,deptno) not in ((1,2),(3,2));
+
+select * from emps where (empno,deptno) in ((1,2),(1,3));
+
+select * from emps where (empno,deptno) not in ((1,2),(1,3));
+
+explain
+select * from emps where (empno+1,deptno) in ((1,2),(3,2));
+
+explain 
+select * from emps where (empno+1,deptno) not in ((1,2),(3,2));
+
+select * from emps where empno in (1,2);
+
+select * from emps where empno in (1,2) and deptno > 2;
+
+select * from emps where (empno) in (1,2) and deptno > 2;
+
+select * from emps where ((empno) in (1,2) and deptno > 2);
+
+explain select * from emps where ((empno*2)|1,deptno) in ((empno+1,2),(empno+2,2));
+
+select * from emps where ((empno*2)|1,deptno) in ((empno+1,2),(empno+2,2));
+
+select (empno*2)|1,substr(empname,1,1) from emps;
+
+select * from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+2,'2'));
+
+select * from emps where ((empno*2)|1,substr(empname,1,1)) not in ((empno+1,'2'),(empno+2,'2'));
+
+select * from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2'));
+
+select * from emps where ((empno*2)|1,substr(empname,1,1)) not in ((empno+1,'2'),(empno+3,'2'));
+
+
+select sum(empno), empname from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2'))
+group by empname;
+
+select * from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2'))
+union
+select * from emps where (empno,deptno) in ((1,2),(3,2));
+
+drop view v;
+
+create view v as 
+select * from(
+select * from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2'))
+union
+select * from emps where (empno,deptno) in ((1,2),(3,2)))subq order by empno desc;
+
+select * from v;
+
+select subq.e1 from 
+(select (empno*2)|1 as e1, substr(empname,1,1) as n1 from emps)subq
+join
+(select empno as e2 from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2')))subq2
+on e1=e2+1;

http://git-wip-us.apache.org/repos/asf/hive/blob/8bed378e/ql/src/test/queries/clientpositive/varchar_udf1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/varchar_udf1.q b/ql/src/test/queries/clientpositive/varchar_udf1.q
index 395fb12..ff40b31 100644
--- a/ql/src/test/queries/clientpositive/varchar_udf1.q
+++ b/ql/src/test/queries/clientpositive/varchar_udf1.q
@@ -75,9 +75,9 @@ select
 from varchar_udf_1 limit 1;
 
 select
-  regexp(c2, 'val'),
-  regexp(c4, 'val'),
-  regexp(c2, 'val') = regexp(c4, 'val')
+  c2 regexp 'val',
+  c4 regexp 'val',
+  (c2 regexp 'val') = (c4 regexp 'val')
 from varchar_udf_1 limit 1;
 
 select

http://git-wip-us.apache.org/repos/asf/hive/blob/8bed378e/ql/src/test/results/clientpositive/char_udf1.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/char_udf1.q.java1.7.out b/ql/src/test/results/clientpositive/char_udf1.q.java1.7.out
index ced0132..bfed116 100644
--- a/ql/src/test/results/clientpositive/char_udf1.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/char_udf1.q.java1.7.out
@@ -219,18 +219,24 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@char_udf_1
 #### A masked pattern was here ####
 val_238	val_238	true
-PREHOOK: query: select
-  regexp(c2, 'val'),
-  regexp(c4, 'val'),
-  regexp(c2, 'val') = regexp(c4, 'val')
+PREHOOK: query: -- In hive wiki page https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF
+-- we only allow A regexp B, not regexp (A,B).
+
+select
+  c2 regexp 'val',
+  c4 regexp 'val',
+  (c2 regexp 'val') = (c4 regexp 'val')
 from char_udf_1 limit 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@char_udf_1
 #### A masked pattern was here ####
-POSTHOOK: query: select
-  regexp(c2, 'val'),
-  regexp(c4, 'val'),
-  regexp(c2, 'val') = regexp(c4, 'val')
+POSTHOOK: query: -- In hive wiki page https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF
+-- we only allow A regexp B, not regexp (A,B).
+
+select
+  c2 regexp 'val',
+  c4 regexp 'val',
+  (c2 regexp 'val') = (c4 regexp 'val')
 from char_udf_1 limit 1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@char_udf_1

http://git-wip-us.apache.org/repos/asf/hive/blob/8bed378e/ql/src/test/results/clientpositive/keyword_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/keyword_2.q.out b/ql/src/test/results/clientpositive/keyword_2.q.out
new file mode 100644
index 0000000..3d248fb
--- /dev/null
+++ b/ql/src/test/results/clientpositive/keyword_2.q.out
@@ -0,0 +1,51 @@
+PREHOOK: query: drop table varchar_udf_1
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table varchar_udf_1
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table varchar_udf_1 (c1 string, c2 string, c3 varchar(10), c4 varchar(20))
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@varchar_udf_1
+POSTHOOK: query: create table varchar_udf_1 (c1 string, c2 string, c3 varchar(10), c4 varchar(20))
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@varchar_udf_1
+PREHOOK: query: insert overwrite table varchar_udf_1
+  select key, value, key, value from src where key = '238' limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@varchar_udf_1
+POSTHOOK: query: insert overwrite table varchar_udf_1
+  select key, value, key, value from src where key = '238' limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@varchar_udf_1
+POSTHOOK: Lineage: varchar_udf_1.c1 SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: varchar_udf_1.c2 SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: varchar_udf_1.c3 EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: varchar_udf_1.c4 EXPRESSION [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: select
+  regexp(c2, 'val'),
+  regexp(c4, 'val'),
+  regexp(c2, 'val') = regexp(c4, 'val')
+from varchar_udf_1 limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@varchar_udf_1
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  regexp(c2, 'val'),
+  regexp(c4, 'val'),
+  regexp(c2, 'val') = regexp(c4, 'val')
+from varchar_udf_1 limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@varchar_udf_1
+#### A masked pattern was here ####
+true	true	true
+PREHOOK: query: drop table varchar_udf_1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@varchar_udf_1
+PREHOOK: Output: default@varchar_udf_1
+POSTHOOK: query: drop table varchar_udf_1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@varchar_udf_1
+POSTHOOK: Output: default@varchar_udf_1

http://git-wip-us.apache.org/repos/asf/hive/blob/8bed378e/ql/src/test/results/clientpositive/multi_column_in.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/multi_column_in.q.out b/ql/src/test/results/clientpositive/multi_column_in.q.out
new file mode 100644
index 0000000..e0ec848
--- /dev/null
+++ b/ql/src/test/results/clientpositive/multi_column_in.q.out
@@ -0,0 +1,410 @@
+PREHOOK: query: drop table emps
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table emps
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table emps (empno int, deptno int, empname string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@emps
+POSTHOOK: query: create table emps (empno int, deptno int, empname string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@emps
+PREHOOK: query: insert into table emps values (1,2,"11"),(1,2,"11"),(3,4,"33"),(1,3,"11"),(2,5,"22"),(2,5,"22")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@emps
+POSTHOOK: query: insert into table emps values (1,2,"11"),(1,2,"11"),(3,4,"33"),(1,3,"11"),(2,5,"22"),(2,5,"22")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@emps
+POSTHOOK: Lineage: emps.deptno EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: emps.empname SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: emps.empno EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: select * from emps
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+1	2	11
+1	2	11
+3	4	33
+1	3	11
+2	5	22
+2	5	22
+PREHOOK: query: select * from emps where (int(empno+deptno/2), int(deptno/3)) in ((2,0),(3,2))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where (int(empno+deptno/2), int(deptno/3)) in ((2,0),(3,2))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+1	2	11
+1	2	11
+PREHOOK: query: select * from emps where (int(empno+deptno/2), int(deptno/3)) not in ((2,0),(3,2))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where (int(empno+deptno/2), int(deptno/3)) not in ((2,0),(3,2))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+3	4	33
+1	3	11
+2	5	22
+2	5	22
+PREHOOK: query: select * from emps where (empno,deptno) in ((1,2),(3,2))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where (empno,deptno) in ((1,2),(3,2))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+1	2	11
+1	2	11
+PREHOOK: query: select * from emps where (empno,deptno) not in ((1,2),(3,2))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where (empno,deptno) not in ((1,2),(3,2))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+3	4	33
+1	3	11
+2	5	22
+2	5	22
+PREHOOK: query: select * from emps where (empno,deptno) in ((1,2),(1,3))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where (empno,deptno) in ((1,2),(1,3))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+1	2	11
+1	2	11
+1	3	11
+PREHOOK: query: select * from emps where (empno,deptno) not in ((1,2),(1,3))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where (empno,deptno) not in ((1,2),(1,3))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+3	4	33
+2	5	22
+2	5	22
+PREHOOK: query: explain
+select * from emps where (empno+1,deptno) in ((1,2),(3,2))
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select * from emps where (empno+1,deptno) in ((1,2),(3,2))
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: emps
+            Statistics: Num rows: 6 Data size: 36 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (struct((empno + 1),deptno)) IN (const struct(1,2), const struct(3,2)) (type: boolean)
+              Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: empno (type: int), deptno (type: int), empname (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain 
+select * from emps where (empno+1,deptno) not in ((1,2),(3,2))
+PREHOOK: type: QUERY
+POSTHOOK: query: explain 
+select * from emps where (empno+1,deptno) not in ((1,2),(3,2))
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: emps
+            Statistics: Num rows: 6 Data size: 36 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (not (struct((empno + 1),deptno)) IN (const struct(1,2), const struct(3,2))) (type: boolean)
+              Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: empno (type: int), deptno (type: int), empname (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select * from emps where empno in (1,2)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where empno in (1,2)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+1	2	11
+1	2	11
+1	3	11
+2	5	22
+2	5	22
+PREHOOK: query: select * from emps where empno in (1,2) and deptno > 2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where empno in (1,2) and deptno > 2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+1	3	11
+2	5	22
+2	5	22
+PREHOOK: query: select * from emps where (empno) in (1,2) and deptno > 2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where (empno) in (1,2) and deptno > 2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+1	3	11
+2	5	22
+2	5	22
+PREHOOK: query: select * from emps where ((empno) in (1,2) and deptno > 2)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where ((empno) in (1,2) and deptno > 2)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+1	3	11
+2	5	22
+2	5	22
+PREHOOK: query: explain select * from emps where ((empno*2)|1,deptno) in ((empno+1,2),(empno+2,2))
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from emps where ((empno*2)|1,deptno) in ((empno+1,2),(empno+2,2))
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: emps
+            Statistics: Num rows: 6 Data size: 36 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (struct(((empno * 2) | 1),deptno)) IN (struct((empno + 1),2), struct((empno + 2),2)) (type: boolean)
+              Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: empno (type: int), deptno (type: int), empname (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select * from emps where ((empno*2)|1,deptno) in ((empno+1,2),(empno+2,2))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where ((empno*2)|1,deptno) in ((empno+1,2),(empno+2,2))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+1	2	11
+1	2	11
+PREHOOK: query: select (empno*2)|1,substr(empname,1,1) from emps
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select (empno*2)|1,substr(empname,1,1) from emps
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+3	1
+3	1
+7	3
+3	1
+5	2
+5	2
+PREHOOK: query: select * from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+2,'2'))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+2,'2'))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+PREHOOK: query: select * from emps where ((empno*2)|1,substr(empname,1,1)) not in ((empno+1,'2'),(empno+2,'2'))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where ((empno*2)|1,substr(empname,1,1)) not in ((empno+1,'2'),(empno+2,'2'))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+1	2	11
+1	2	11
+3	4	33
+1	3	11
+2	5	22
+2	5	22
+PREHOOK: query: select * from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2'))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2'))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+2	5	22
+2	5	22
+PREHOOK: query: select * from emps where ((empno*2)|1,substr(empname,1,1)) not in ((empno+1,'2'),(empno+3,'2'))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where ((empno*2)|1,substr(empname,1,1)) not in ((empno+1,'2'),(empno+3,'2'))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+1	2	11
+1	2	11
+3	4	33
+1	3	11
+PREHOOK: query: select sum(empno), empname from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2'))
+group by empname
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select sum(empno), empname from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2'))
+group by empname
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+4	22
+PREHOOK: query: select * from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2'))
+union
+select * from emps where (empno,deptno) in ((1,2),(3,2))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select * from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2'))
+union
+select * from emps where (empno,deptno) in ((1,2),(3,2))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+1	2	11
+2	5	22
+PREHOOK: query: drop view v
+PREHOOK: type: DROPVIEW
+POSTHOOK: query: drop view v
+POSTHOOK: type: DROPVIEW
+PREHOOK: query: create view v as 
+select * from(
+select * from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2'))
+union
+select * from emps where (empno,deptno) in ((1,2),(3,2)))subq order by empno desc
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: default@emps
+PREHOOK: Output: database:default
+PREHOOK: Output: default@v
+POSTHOOK: query: create view v as 
+select * from(
+select * from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2'))
+union
+select * from emps where (empno,deptno) in ((1,2),(3,2)))subq order by empno desc
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: default@emps
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@v
+PREHOOK: query: select * from v
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+PREHOOK: Input: default@v
+#### A masked pattern was here ####
+POSTHOOK: query: select * from v
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+POSTHOOK: Input: default@v
+#### A masked pattern was here ####
+2	5	22
+1	2	11
+PREHOOK: query: select subq.e1 from 
+(select (empno*2)|1 as e1, substr(empname,1,1) as n1 from emps)subq
+join
+(select empno as e2 from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2')))subq2
+on e1=e2+1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@emps
+#### A masked pattern was here ####
+POSTHOOK: query: select subq.e1 from 
+(select (empno*2)|1 as e1, substr(empname,1,1) as n1 from emps)subq
+join
+(select empno as e2 from emps where ((empno*2)|1,substr(empname,1,1)) in ((empno+1,'2'),(empno+3,'2')))subq2
+on e1=e2+1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@emps
+#### A masked pattern was here ####
+3
+3
+3
+3
+3
+3

http://git-wip-us.apache.org/repos/asf/hive/blob/8bed378e/ql/src/test/results/clientpositive/varchar_udf1.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/varchar_udf1.q.java1.7.out b/ql/src/test/results/clientpositive/varchar_udf1.q.java1.7.out
index 96ba06e..853bc4a 100644
--- a/ql/src/test/results/clientpositive/varchar_udf1.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/varchar_udf1.q.java1.7.out
@@ -220,17 +220,17 @@ POSTHOOK: Input: default@varchar_udf_1
 #### A masked pattern was here ####
 val_238	val_238	true
 PREHOOK: query: select
-  regexp(c2, 'val'),
-  regexp(c4, 'val'),
-  regexp(c2, 'val') = regexp(c4, 'val')
+  c2 regexp 'val',
+  c4 regexp 'val',
+  (c2 regexp 'val') = (c4 regexp 'val')
 from varchar_udf_1 limit 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@varchar_udf_1
 #### A masked pattern was here ####
 POSTHOOK: query: select
-  regexp(c2, 'val'),
-  regexp(c4, 'val'),
-  regexp(c2, 'val') = regexp(c4, 'val')
+  c2 regexp 'val',
+  c4 regexp 'val',
+  (c2 regexp 'val') = (c4 regexp 'val')
 from varchar_udf_1 limit 1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@varchar_udf_1


[10/11] hive git commit: HIVE-11329: Column prefix in key of hbase column prefix map (Wojciech Indyk, reviewed by Sergio Pena)

Posted by se...@apache.org.
HIVE-11329: Column prefix in key of hbase column prefix map (Wojciech Indyk, reviewed by Sergio Pena)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d51c62a4
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d51c62a4
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d51c62a4

Branch: refs/heads/llap
Commit: d51c62a455eb08ee49f10ea2e117ca90de0bf47b
Parents: 7281a46
Author: Sergio Pena <se...@cloudera.com>
Authored: Tue Sep 8 08:39:10 2015 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Tue Sep 8 08:39:10 2015 -0500

----------------------------------------------------------------------
 .../hadoop/hive/hbase/ColumnMappings.java       |  5 ++
 .../apache/hadoop/hive/hbase/HBaseSerDe.java    | 19 +++++-
 .../hadoop/hive/hbase/HBaseSerDeParameters.java |  8 ++-
 .../hadoop/hive/hbase/HBaseStorageHandler.java  | 13 ++--
 .../hadoop/hive/hbase/LazyHBaseCellMap.java     | 19 +++++-
 .../apache/hadoop/hive/hbase/LazyHBaseRow.java  |  5 +-
 .../hadoop/hive/hbase/LazyHBaseCellMapTest.java | 72 ++++++++++++++++++++
 .../positive/hbase_binary_map_queries_prefix.q  | 15 +++-
 .../hbase_binary_map_queries_prefix.q.out       | 40 +++++++++++
 9 files changed, 177 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java
index 5cb3752..d7d4461 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java
@@ -354,6 +354,7 @@ public class ColumnMappings implements Iterable<ColumnMappings.ColumnMapping> {
     String mappingSpec;
     String qualifierPrefix;
     byte[] qualifierPrefixBytes;
+    boolean doPrefixCut;
 
     public String getColumnName() {
       return columnName;
@@ -399,6 +400,10 @@ public class ColumnMappings implements Iterable<ColumnMappings.ColumnMapping> {
       return qualifierPrefixBytes;
     }
 
+    public boolean isDoPrefixCut(){
+      return doPrefixCut;
+    }
+
     public boolean isCategory(ObjectInspector.Category category) {
       return columnType.getCategory() == category;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java
index e843bd8..ed2df5f 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java
@@ -93,6 +93,13 @@ public class HBaseSerDe extends AbstractSerDe {
    **/
   public static final String SERIALIZATION_TYPE = "serialization.type";
 
+  /**
+   * Defines if the prefix column from hbase should be hidden.
+   * It works only when @HBASE_COLUMNS_REGEX_MATCHING is true.
+   * Default value of this parameter is false
+   */
+  public static final String HBASE_COLUMNS_PREFIX_HIDE = "hbase.columns.mapping.prefix.hide";
+
   private ObjectInspector cachedObjectInspector;
   private LazyHBaseRow cachedHBaseRow;
 
@@ -136,6 +143,11 @@ public class HBaseSerDe extends AbstractSerDe {
       throws SerDeException {
     return parseColumnsMapping(columnsMappingSpec, true);
   }
+
+  public static ColumnMappings parseColumnsMapping(
+          String columnsMappingSpec, boolean doColumnRegexMatching) throws SerDeException {
+	return parseColumnsMapping(columnsMappingSpec, doColumnRegexMatching, false);
+  }
   /**
    * Parses the HBase columns mapping specifier to identify the column families, qualifiers
    * and also caches the byte arrays corresponding to them. One of the Hive table
@@ -143,11 +155,12 @@ public class HBaseSerDe extends AbstractSerDe {
    *
    * @param columnsMappingSpec string hbase.columns.mapping specified when creating table
    * @param doColumnRegexMatching whether to do a regex matching on the columns or not
+   * @param hideColumnPrefix whether to hide a prefix of column mapping in key name in a map (works only if @doColumnRegexMatching is true)
    * @return List<ColumnMapping> which contains the column mapping information by position
    * @throws org.apache.hadoop.hive.serde2.SerDeException
    */
   public static ColumnMappings parseColumnsMapping(
-      String columnsMappingSpec, boolean doColumnRegexMatching) throws SerDeException {
+      String columnsMappingSpec, boolean doColumnRegexMatching, boolean hideColumnPrefix) throws SerDeException {
 
     if (columnsMappingSpec == null) {
       throw new SerDeException("Error: hbase.columns.mapping missing for this HBase table.");
@@ -206,6 +219,8 @@ public class HBaseSerDe extends AbstractSerDe {
             // we have a prefix with a wildcard
             columnMapping.qualifierPrefix = parts[1].substring(0, parts[1].length() - 2);
             columnMapping.qualifierPrefixBytes = Bytes.toBytes(columnMapping.qualifierPrefix);
+            //pass a flag to hide prefixes
+            columnMapping.doPrefixCut=hideColumnPrefix;
             // we weren't provided any actual qualifier name. Set these to
             // null.
             columnMapping.qualifierName = null;
@@ -214,6 +229,8 @@ public class HBaseSerDe extends AbstractSerDe {
             // set the regular provided qualifier names
             columnMapping.qualifierName = parts[1];
             columnMapping.qualifierNameBytes = Bytes.toBytes(parts[1]);
+            //if there is no prefix then we don't cut anything
+            columnMapping.doPrefixCut=false;
           }
         } else {
           columnMapping.qualifierName = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDeParameters.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDeParameters.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDeParameters.java
index 71f5da5..43c1f0c 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDeParameters.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDeParameters.java
@@ -22,6 +22,8 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
+import javax.annotation.Nullable;
+
 import org.apache.avro.Schema;
 import org.apache.avro.reflect.ReflectData;
 import org.apache.hadoop.conf.Configuration;
@@ -38,8 +40,6 @@ import org.apache.hadoop.hive.serde2.lazy.LazySerDeParameters;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.util.ReflectionUtils;
 
-import javax.annotation.Nullable;
-
 /**
  * HBaseSerDeParameters encapsulates SerDeParameters and additional configurations that are specific for
  * HBaseSerDe.
@@ -57,6 +57,7 @@ public class HBaseSerDeParameters {
   private final String columnMappingString;
   private final ColumnMappings columnMappings;
   private final boolean doColumnRegexMatching;
+  private final boolean doColumnPrefixCut;
 
   private final long putTimestamp;
   private final HBaseKeyFactory keyFactory;
@@ -69,8 +70,9 @@ public class HBaseSerDeParameters {
     columnMappingString = tbl.getProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING);
     doColumnRegexMatching =
         Boolean.valueOf(tbl.getProperty(HBaseSerDe.HBASE_COLUMNS_REGEX_MATCHING, "true"));
+    doColumnPrefixCut = Boolean.valueOf(tbl.getProperty(HBaseSerDe.HBASE_COLUMNS_PREFIX_HIDE, "false"));
     // Parse and initialize the HBase columns mapping
-    columnMappings = HBaseSerDe.parseColumnsMapping(columnMappingString, doColumnRegexMatching);
+    columnMappings = HBaseSerDe.parseColumnsMapping(columnMappingString, doColumnRegexMatching, doColumnPrefixCut);
 
     // Build the type property string if not supplied
     String columnTypeProperty = tbl.getProperty(serdeConstants.LIST_COLUMN_TYPES);

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
index c0eff23..cedb6e0 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
@@ -44,13 +44,10 @@ import org.apache.hadoop.hbase.mapred.TableOutputFormat;
 import org.apache.hadoop.hbase.mapreduce.TableInputFormatBase;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
-import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector;
+import org.apache.hadoop.hbase.security.token.TokenUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping;
-import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.hive.metastore.HiveMetaHook;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -66,15 +63,11 @@ import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.serde2.Deserializer;
 import org.apache.hadoop.hive.serde2.SerDe;
 import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.hbase.security.token.TokenUtil;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.OutputFormat;
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.zookeeper.KeeperException;
 
 import com.yammer.metrics.core.MetricsRegistry;
 
@@ -348,7 +341,9 @@ public class HBaseStorageHandler extends DefaultStorageHandler
       HBaseSerDe.HBASE_COLUMNS_MAPPING,
       tableProperties.getProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING));
     jobProperties.put(HBaseSerDe.HBASE_COLUMNS_REGEX_MATCHING,
-        tableProperties.getProperty(HBaseSerDe.HBASE_COLUMNS_REGEX_MATCHING, "true"));
+            tableProperties.getProperty(HBaseSerDe.HBASE_COLUMNS_REGEX_MATCHING, "true"));
+    jobProperties.put(HBaseSerDe.HBASE_COLUMNS_PREFIX_HIDE,
+            tableProperties.getProperty(HBaseSerDe.HBASE_COLUMNS_PREFIX_HIDE, "false"));
     jobProperties.put(HBaseSerDe.HBASE_TABLE_DEFAULT_STORAGE_TYPE,
       tableProperties.getProperty(HBaseSerDe.HBASE_TABLE_DEFAULT_STORAGE_TYPE,"string"));
     String scanCache = tableProperties.getProperty(HBaseSerDe.HBASE_SCAN_CACHE);

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseCellMap.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseCellMap.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseCellMap.java
index 09cbf52..459d934 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseCellMap.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseCellMap.java
@@ -45,8 +45,9 @@ public class LazyHBaseCellMap extends LazyMap {
   private byte [] columnFamilyBytes;
   private byte[] qualPrefix;
   private List<Boolean> binaryStorage;
+  private boolean hideQualPrefix;
 
-  /**
+	/**
    * Construct a LazyCellMap object with the ObjectInspector.
    * @param oi
    */
@@ -62,15 +63,23 @@ public class LazyHBaseCellMap extends LazyMap {
     init(r, columnFamilyBytes, binaryStorage, null);
   }
 
+	public void init(
+			Result r,
+			byte [] columnFamilyBytes,
+			List<Boolean> binaryStorage, byte[] qualPrefix) {
+		init(r, columnFamilyBytes, binaryStorage, qualPrefix, false);
+	}
+
   public void init(
       Result r,
       byte [] columnFamilyBytes,
-      List<Boolean> binaryStorage, byte[] qualPrefix) {
+      List<Boolean> binaryStorage, byte[] qualPrefix, boolean hideQualPrefix) {
     this.isNull = false;
     this.result = r;
     this.columnFamilyBytes = columnFamilyBytes;
     this.binaryStorage = binaryStorage;
     this.qualPrefix = qualPrefix;
+    this.hideQualPrefix = hideQualPrefix;
     setParsed(false);
   }
 
@@ -106,7 +115,11 @@ public class LazyHBaseCellMap extends LazyMap {
               binaryStorage.get(0));
 
         ByteArrayRef keyRef = new ByteArrayRef();
-        keyRef.setData(e.getKey());
+		  if (qualPrefix!=null && hideQualPrefix){
+			  keyRef.setData(Bytes.tail(e.getKey(), e.getKey().length-qualPrefix.length)); //cut prefix from hive's map key
+		  }else{
+			  keyRef.setData(e.getKey()); //for non-prefix maps
+		  }
         key.init(keyRef, 0, keyRef.getData().length);
 
         // Value

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java
index 868d81f..003dd1c 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping;
 import org.apache.hadoop.hive.hbase.struct.HBaseValueFactory;
@@ -36,6 +35,8 @@ import org.apache.hadoop.hive.serde2.lazy.LazyTimestamp;
 import org.apache.hadoop.hive.serde2.lazy.objectinspector.LazySimpleStructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * LazyObject for storing an HBase row.  The field of an HBase row can be
  * primitive or non-primitive.
@@ -148,7 +149,7 @@ public class LazyHBaseRow extends LazyStruct {
         // qualifier prefix to cherry pick the qualifiers that match the prefix instead of picking
         // up everything
         ((LazyHBaseCellMap) fields[fieldID]).init(
-            result, colMap.familyNameBytes, colMap.binaryStorage, colMap.qualifierPrefixBytes);
+            result, colMap.familyNameBytes, colMap.binaryStorage, colMap.qualifierPrefixBytes, colMap.isDoPrefixCut());
         return fields[fieldID].getObject();
       }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/test/org/apache/hadoop/hive/hbase/LazyHBaseCellMapTest.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/LazyHBaseCellMapTest.java b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/LazyHBaseCellMapTest.java
new file mode 100644
index 0000000..eb13e14
--- /dev/null
+++ b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/LazyHBaseCellMapTest.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.hbase;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hive.serde2.lazy.LazyFactory;
+import org.apache.hadoop.hive.serde2.lazy.objectinspector.LazyMapObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.Text;
+
+import junit.framework.TestCase;
+
+public class LazyHBaseCellMapTest extends TestCase {
+	public static final byte[] TEST_ROW = Bytes.toBytes("test-row");
+	public static final byte[] COLUMN_FAMILY = Bytes.toBytes("a");
+	public static final String QUAL_PREFIX = "col_";
+
+
+	public void testInitColumnPrefix() throws Exception {
+		Text nullSequence = new Text("\\N");
+		ObjectInspector oi = LazyFactory.createLazyObjectInspector(
+				TypeInfoUtils.getTypeInfosFromTypeString("map<string,string>").get(0),
+				new byte[] { (byte) 1, (byte) 2 }, 0, nullSequence, false, (byte) 0);
+
+		LazyHBaseCellMap b = new LazyHBaseCellMap((LazyMapObjectInspector) oi);
+
+		// Initialize a result
+		Cell[] cells = new KeyValue[2];
+
+		final String col1="1";
+		final String col2="2";
+		cells[0] = new KeyValue(TEST_ROW, COLUMN_FAMILY,
+				Bytes.toBytes(QUAL_PREFIX+col1), Bytes.toBytes("cfacol1"));
+		cells[1]=new KeyValue(TEST_ROW, COLUMN_FAMILY,
+				Bytes.toBytes(QUAL_PREFIX+col2), Bytes.toBytes("cfacol2"));
+
+		Result r = Result.create(cells);
+
+		List<Boolean> mapBinaryStorage = new ArrayList<Boolean>();
+		mapBinaryStorage.add(false);
+		mapBinaryStorage.add(false);
+
+		b.init(r, COLUMN_FAMILY, mapBinaryStorage, Bytes.toBytes(QUAL_PREFIX), true);
+
+		assertNotNull(b.getMapValueElement(new Text(col1)));
+		assertNotNull(b.getMapValueElement(new Text(col2)));
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/test/queries/positive/hbase_binary_map_queries_prefix.q
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/queries/positive/hbase_binary_map_queries_prefix.q b/hbase-handler/src/test/queries/positive/hbase_binary_map_queries_prefix.q
index 3d3f5ea..9ff4366 100644
--- a/hbase-handler/src/test/queries/positive/hbase_binary_map_queries_prefix.q
+++ b/hbase-handler/src/test/queries/positive/hbase_binary_map_queries_prefix.q
@@ -49,4 +49,17 @@ TBLPROPERTIES ("hbase.table.name"="t_hive_maps");
 
 SELECT * FROM t_ext_hbase_maps ORDER BY key;
 
-DROP TABLE t_ext_hbase_maps;
\ No newline at end of file
+DROP TABLE t_ext_hbase_maps;
+
+DROP TABLE t_ext_hbase_maps_cut_prefix;
+
+CREATE EXTERNAL TABLE t_ext_hbase_maps_cut_prefix(key STRING,
+                                       string_map_cols MAP<STRING, STRING>, simple_string_col STRING)
+STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
+WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key,cf-string:string_.*,cf-string:simple_string_col"
+    ,"hbase.columns.mapping.prefix.hide"="true")
+TBLPROPERTIES ("hbase.table.name"="t_hive_maps");
+
+SELECT * FROM t_ext_hbase_maps_cut_prefix ORDER BY key;
+
+DROP TABLE t_ext_hbase_maps_cut_prefix;

http://git-wip-us.apache.org/repos/asf/hive/blob/d51c62a4/hbase-handler/src/test/results/positive/hbase_binary_map_queries_prefix.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_binary_map_queries_prefix.q.out b/hbase-handler/src/test/results/positive/hbase_binary_map_queries_prefix.q.out
index 5d21134..f6432b3 100644
--- a/hbase-handler/src/test/results/positive/hbase_binary_map_queries_prefix.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_binary_map_queries_prefix.q.out
@@ -152,3 +152,43 @@ POSTHOOK: query: DROP TABLE t_ext_hbase_maps
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@t_ext_hbase_maps
 POSTHOOK: Output: default@t_ext_hbase_maps
+PREHOOK: query: DROP TABLE t_ext_hbase_maps_cut_prefix
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE t_ext_hbase_maps_cut_prefix
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE EXTERNAL TABLE t_ext_hbase_maps_cut_prefix(key STRING,
+                                       string_map_cols MAP<STRING, STRING>, simple_string_col STRING)
+STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
+WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key,cf-string:string_.*,cf-string:simple_string_col"
+    ,"hbase.columns.mapping.prefix.hide"="true")
+TBLPROPERTIES ("hbase.table.name"="t_hive_maps")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t_ext_hbase_maps_cut_prefix
+POSTHOOK: query: CREATE EXTERNAL TABLE t_ext_hbase_maps_cut_prefix(key STRING,
+                                       string_map_cols MAP<STRING, STRING>, simple_string_col STRING)
+STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
+WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key,cf-string:string_.*,cf-string:simple_string_col"
+    ,"hbase.columns.mapping.prefix.hide"="true")
+TBLPROPERTIES ("hbase.table.name"="t_hive_maps")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t_ext_hbase_maps_cut_prefix
+PREHOOK: query: SELECT * FROM t_ext_hbase_maps_cut_prefix ORDER BY key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t_ext_hbase_maps_cut_prefix
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM t_ext_hbase_maps_cut_prefix ORDER BY key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t_ext_hbase_maps_cut_prefix
+#### A masked pattern was here ####
+125	{"col":"val_125"}	val_125
+126	{"col":"val_126"}	val_126
+PREHOOK: query: DROP TABLE t_ext_hbase_maps_cut_prefix
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@t_ext_hbase_maps_cut_prefix
+PREHOOK: Output: default@t_ext_hbase_maps_cut_prefix
+POSTHOOK: query: DROP TABLE t_ext_hbase_maps_cut_prefix
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@t_ext_hbase_maps_cut_prefix
+POSTHOOK: Output: default@t_ext_hbase_maps_cut_prefix


[11/11] hive git commit: HIVE-11767 : LLAP: merge master into branch (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-11767 : LLAP: merge master into branch (Sergey Shelukhin)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/b20330db
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b20330db
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b20330db

Branch: refs/heads/llap
Commit: b20330db780247801df7886f9c58414cfaaf8fdc
Parents: 6413e86 d51c62a
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Sep 8 19:17:49 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Sep 8 19:17:49 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/hbase/ColumnMappings.java       |    5 +
 .../apache/hadoop/hive/hbase/HBaseSerDe.java    |   19 +-
 .../hadoop/hive/hbase/HBaseSerDeParameters.java |    8 +-
 .../hadoop/hive/hbase/HBaseStorageHandler.java  |   13 +-
 .../hadoop/hive/hbase/LazyHBaseCellMap.java     |   19 +-
 .../apache/hadoop/hive/hbase/LazyHBaseRow.java  |    5 +-
 .../hadoop/hive/hbase/LazyHBaseCellMapTest.java |   72 +
 .../positive/hbase_binary_map_queries_prefix.q  |   15 +-
 .../hbase_binary_map_queries_prefix.q.out       |   40 +
 .../hive/metastore/RetryingMetaStoreClient.java |   32 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |    2 +
 .../hadoop/hive/ql/exec/FunctionTask.java       |    2 +-
 .../hadoop/hive/ql/exec/mr/ExecDriver.java      |    1 -
 .../hadoop/hive/ql/lib/DefaultGraphWalker.java  |    4 +-
 .../hadoop/hive/ql/lib/ForwardWalker.java       |    2 +-
 .../hadoop/hive/ql/lib/LevelOrderWalker.java    |  153 ++
 .../hadoop/hive/ql/lib/PreOrderWalker.java      |    2 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   56 +-
 .../hadoop/hive/ql/optimizer/ColumnPruner.java  |    2 +-
 .../hive/ql/optimizer/ConstantPropagate.java    |    2 +-
 .../calcite/translator/ExprNodeConverter.java   |   25 +-
 .../calcite/translator/HiveOpConverter.java     |    5 +-
 .../hive/ql/optimizer/lineage/Generator.java    |    4 +-
 .../annotation/AnnotateWithOpTraits.java        |    6 +-
 .../annotation/AnnotateWithStatistics.java      |    6 +-
 .../ql/optimizer/unionproc/UnionProcessor.java  |   10 +-
 .../hadoop/hive/ql/parse/GenMapRedWalker.java   |    2 +-
 .../hadoop/hive/ql/parse/GenTezWorkWalker.java  |    2 +-
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |   46 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |    1 +
 .../apache/hadoop/hive/ql/parse/TezWalker.java  |    2 +-
 .../hive/ql/parse/spark/GenSparkWorkWalker.java |    2 +-
 .../ql/ppd/PredicateTransitivePropagate.java    |    4 +-
 .../hive/ql/udf/generic/GenericUDFAesBase.java  |  205 ++
 .../ql/udf/generic/GenericUDFAesDecrypt.java    |   50 +
 .../ql/udf/generic/GenericUDFAesEncrypt.java    |   50 +
 .../ql/udf/generic/GenericUDFParamUtils.java    |    8 +-
 .../TestSQL11ReservedKeyWordsNegative.java      |   32 +-
 .../TestSQL11ReservedKeyWordsPositive.java      |   23 +-
 .../udf/generic/TestGenericUDFAesDecrypt.java   |  233 ++
 .../udf/generic/TestGenericUDFAesEncrypt.java   |  228 ++
 .../queries/clientpositive/cbo_rp_windowing_2.q |  439 ++++
 ql/src/test/queries/clientpositive/char_udf1.q  |    9 +-
 ql/src/test/queries/clientpositive/groupby13.q  |   16 +
 ql/src/test/queries/clientpositive/keyword_2.q  |   14 +
 .../queries/clientpositive/multi_column_in.q    |   71 +
 .../queries/clientpositive/udf_aes_decrypt.q    |   21 +
 .../queries/clientpositive/udf_aes_encrypt.q    |   21 +
 .../test/queries/clientpositive/varchar_udf1.q  |    6 +-
 .../clientpositive/cbo_rp_windowing_2.q.out     | 2338 ++++++++++++++++++
 .../clientpositive/char_udf1.q.java1.7.out      |   22 +-
 .../clientpositive/correlationoptimizer5.q.out  |    6 +-
 .../test/results/clientpositive/groupby13.q.out |   86 +
 ql/src/test/results/clientpositive/join32.q.out |    2 +-
 .../clientpositive/join32_lessSize.q.out        |    6 +-
 ql/src/test/results/clientpositive/join33.q.out |    2 +-
 .../test/results/clientpositive/keyword_2.q.out |   51 +
 .../test/results/clientpositive/lineage2.q.out  |   10 +-
 .../test/results/clientpositive/lineage3.q.out  |   18 +-
 .../clientpositive/multi_column_in.q.out        |  410 +++
 .../results/clientpositive/show_functions.q.out |    2 +
 .../results/clientpositive/spark/join32.q.out   |    2 +-
 .../clientpositive/spark/join32_lessSize.q.out  |    6 +-
 .../results/clientpositive/spark/join33.q.out   |    2 +-
 .../spark/subquery_multiinsert.q.java1.7.out    |   16 +-
 .../subquery_multiinsert.q.java1.7.out          |   16 +-
 .../clientpositive/tez/explainuser_2.q.out      |   12 +-
 .../clientpositive/udf_aes_decrypt.q.out        |   79 +
 .../clientpositive/udf_aes_encrypt.q.out        |   79 +
 .../clientpositive/varchar_udf1.q.java1.7.out   |   12 +-
 70 files changed, 5016 insertions(+), 156 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b20330db/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------


[09/11] hive git commit: HIVE-11617: Explain plan for multiple lateral views is very slow (Aihua Xu, reviewed by Jesus Camacho Rodriguez)

Posted by se...@apache.org.
HIVE-11617: Explain plan for multiple lateral views is very slow (Aihua Xu, reviewed by Jesus Camacho Rodriguez)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7281a460
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7281a460
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7281a460

Branch: refs/heads/llap
Commit: 7281a46062bd4a6dea0c4ef80930246fad16bdea
Parents: 2d3316b
Author: Aihua Xu <ai...@gmail.com>
Authored: Tue Sep 8 11:37:01 2015 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Tue Sep 8 11:37:01 2015 +0100

----------------------------------------------------------------------
 .../hadoop/hive/ql/lib/DefaultGraphWalker.java  |   4 +-
 .../hadoop/hive/ql/lib/ForwardWalker.java       |   2 +-
 .../hadoop/hive/ql/lib/LevelOrderWalker.java    | 153 +++++++++++++++++++
 .../hadoop/hive/ql/lib/PreOrderWalker.java      |   2 +-
 .../hadoop/hive/ql/optimizer/ColumnPruner.java  |   2 +-
 .../hive/ql/optimizer/ConstantPropagate.java    |   2 +-
 .../hive/ql/optimizer/lineage/Generator.java    |   4 +-
 .../annotation/AnnotateWithOpTraits.java        |   6 +-
 .../annotation/AnnotateWithStatistics.java      |   6 +-
 .../ql/optimizer/unionproc/UnionProcessor.java  |  10 +-
 .../hadoop/hive/ql/parse/GenMapRedWalker.java   |   2 +-
 .../hadoop/hive/ql/parse/GenTezWorkWalker.java  |   2 +-
 .../apache/hadoop/hive/ql/parse/TezWalker.java  |   2 +-
 .../hive/ql/parse/spark/GenSparkWorkWalker.java |   2 +-
 .../ql/ppd/PredicateTransitivePropagate.java    |   4 +-
 .../clientpositive/correlationoptimizer5.q.out  |   6 +-
 ql/src/test/results/clientpositive/join32.q.out |   2 +-
 .../clientpositive/join32_lessSize.q.out        |   6 +-
 ql/src/test/results/clientpositive/join33.q.out |   2 +-
 .../test/results/clientpositive/lineage2.q.out  |  10 +-
 .../test/results/clientpositive/lineage3.q.out  |  18 +--
 .../results/clientpositive/spark/join32.q.out   |   2 +-
 .../clientpositive/spark/join32_lessSize.q.out  |   6 +-
 .../results/clientpositive/spark/join33.q.out   |   2 +-
 .../spark/subquery_multiinsert.q.java1.7.out    |  16 +-
 .../subquery_multiinsert.q.java1.7.out          |  16 +-
 .../clientpositive/tez/explainuser_2.q.out      |  12 +-
 27 files changed, 228 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/lib/DefaultGraphWalker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/DefaultGraphWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/DefaultGraphWalker.java
index 07d2734..d452f50 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/DefaultGraphWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/DefaultGraphWalker.java
@@ -72,7 +72,7 @@ public class DefaultGraphWalker implements GraphWalker {
   /**
    * @return the doneList
    */
-  public Set<Node> getDispatchedList() {
+  protected Set<Node> getDispatchedList() {
     return retMap.keySet();
   }
 
@@ -143,7 +143,7 @@ public class DefaultGraphWalker implements GraphWalker {
    *          current operator in the graph
    * @throws SemanticException
    */
-  public void walk(Node nd) throws SemanticException {    
+  protected void walk(Node nd) throws SemanticException {
     // Push the node in the stack
     opStack.push(nd);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/lib/ForwardWalker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/ForwardWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/ForwardWalker.java
index 67b4700..a10dc52 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/ForwardWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/ForwardWalker.java
@@ -63,7 +63,7 @@ public class ForwardWalker extends DefaultGraphWalker {
    * @throws SemanticException
    */
   @Override
-  public void walk(Node nd) throws SemanticException {
+  protected void walk(Node nd) throws SemanticException {
     if (opStack.empty() || nd != opStack.peek()) {
       opStack.push(nd);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/lib/LevelOrderWalker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/LevelOrderWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/LevelOrderWalker.java
new file mode 100644
index 0000000..cf05d5f
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/LevelOrderWalker.java
@@ -0,0 +1,153 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.lib;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Stack;
+
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+
+/**
+ * This is a level-wise walker implementation which dispatches the node in the order
+ * that the node will only get dispatched after all the parents are dispatched.
+ *
+ * Each node will be accessed once while it could be dispatched multiple times.
+ * e.g., for a lineage generator with operator tree, 2 levels of current node's
+ * ancestors need to keep in the operator stack.
+ *                  FIL(2) FIL(4)
+ *                      |    |
+ *                    RS(3) RS(5)
+ *                       \  /
+ *                      JOIN(7)
+ * The join lineage needs to be called twice for JOIN(7) node with different operator
+ * ancestors.
+ */
+public class LevelOrderWalker extends DefaultGraphWalker {
+  // Only specified nodes of these types will be walked.
+  // Empty set means all the nodes will be walked.
+  private HashSet<Class<? extends Node>> nodeTypes = new HashSet<Class<? extends Node>>();
+
+  // How many levels of ancestors to keep in the stack during dispatching
+  private final int numLevels;
+
+  /**
+   * Constructor with keeping all the ancestors in the operator stack during dispatching.
+   * @param disp Dispatcher to call for each op encountered
+   */
+  public LevelOrderWalker(Dispatcher disp) {
+    super(disp);
+    this.numLevels = Integer.MAX_VALUE;
+  }
+
+  /**
+   * Constructor with specified number of ancestor levels to keep in the operator
+   * stack during dispatching.
+   * @param disp      Dispatcher to call for each op encountered
+   * @param numLevels Number of ancestor levels
+   */
+  public LevelOrderWalker(Dispatcher disp, int numLevels) {
+    super(disp);
+    this.numLevels = numLevels;
+  }
+
+  @SuppressWarnings("unchecked")
+  public void setNodeTypes(Class<? extends Node> ...nodeTypes) {
+    this.nodeTypes.addAll(Arrays.asList(nodeTypes));
+  }
+
+  /**
+   * starting point for walking.
+   *
+   * @throws SemanticException
+   */
+  @SuppressWarnings("unchecked")
+  @Override
+  public void startWalking(Collection<Node> startNodes,
+      HashMap<Node, Object> nodeOutput) throws SemanticException {
+    toWalk.addAll(startNodes);
+
+    // Starting from the startNodes, add the children whose parents have been
+    // included in the list.
+    HashSet<Node> addedNodes = new HashSet<Node>();
+    for (Node node : startNodes) {
+      addedNodes.add(node);
+    }
+    int index = 0;
+    while(index < toWalk.size()) {
+      if (toWalk.get(index).getChildren() != null) {
+        for(Node child : toWalk.get(index).getChildren()) {
+          Operator<? extends OperatorDesc> childOP =
+              (Operator<? extends OperatorDesc>) child;
+
+          if (!addedNodes.contains(child) &&
+              (childOP.getParentOperators() == null ||
+              addedNodes.containsAll(childOP.getParentOperators()))) {
+            toWalk.add(child);
+            addedNodes.add(child);
+          }
+        }
+      }
+      ++index;
+    }
+
+    for(Node nd : toWalk) {
+      if (!nodeTypes.isEmpty() && !nodeTypes.contains(nd.getClass())) {
+        continue;
+      }
+
+      opStack.clear();
+      opStack.push(nd);
+      walk(nd, 0, opStack);
+      if (nodeOutput != null && getDispatchedList().contains(nd)) {
+        nodeOutput.put(nd, retMap.get(nd));
+      }
+    }
+  }
+
+  /**
+   * Enumerate numLevels of ancestors by putting them in the stack and dispatch
+   * the current node.
+   * @param nd current operator in the ancestor tree
+   * @param level how many level of ancestors included in the stack
+   * @param stack operator stack
+   * @throws SemanticException
+   */
+  @SuppressWarnings("unchecked")
+  private void walk(Node nd, int level, Stack<Node> stack) throws SemanticException {
+    List<Operator<? extends OperatorDesc>> parents =
+        ((Operator<? extends OperatorDesc>)nd).getParentOperators();
+
+    if (level >= numLevels || parents == null || parents.isEmpty()) {
+      dispatch(stack.peek(), stack);
+      return;
+    }
+
+    for(Node parent : parents) {
+      stack.add(0, parent);
+      walk(parent, level+1, stack);
+      stack.remove(0);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderWalker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderWalker.java
index f22694b..8d8dab8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderWalker.java
@@ -51,7 +51,7 @@ public class PreOrderWalker extends DefaultGraphWalker {
    * @throws SemanticException
    */
   @Override
-  public void walk(Node nd) throws SemanticException {
+  protected void walk(Node nd) throws SemanticException {
     opStack.push(nd);
     dispatch(nd, opStack);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java
index 735b448..561b8fc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java
@@ -150,7 +150,7 @@ public class ColumnPruner implements Transform {
      * Walk the given operator.
      */
     @Override
-    public void walk(Node nd) throws SemanticException {
+    protected void walk(Node nd) throws SemanticException {
       boolean walkChildren = true;
       opStack.push(nd);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagate.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagate.java
index b6f1f27..aacded6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagate.java
@@ -140,7 +140,7 @@ public class ConstantPropagate implements Transform {
     }
 
     @Override
-    public void walk(Node nd) throws SemanticException {
+    protected void walk(Node nd) throws SemanticException {
 
       List<Node> parents = ((Operator) nd).getParentOperators();
       if ((parents == null)

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java
index 9a5cf55..82e26d9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java
@@ -37,9 +37,9 @@ import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
 import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.LevelOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.PreOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Rule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
@@ -94,7 +94,7 @@ public class Generator implements Transform {
 
     // The dispatcher fires the processor corresponding to the closest matching rule and passes the context along
     Dispatcher disp = new DefaultRuleDispatcher(OpProcFactory.getDefaultProc(), opRules, lCtx);
-    GraphWalker ogw = new PreOrderWalker(disp);
+    GraphWalker ogw = new LevelOrderWalker(disp, 2);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/AnnotateWithOpTraits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/AnnotateWithOpTraits.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/AnnotateWithOpTraits.java
index c304e97..0398115 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/AnnotateWithOpTraits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/AnnotateWithOpTraits.java
@@ -35,9 +35,9 @@ import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
 import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.LevelOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.PreOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Rule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
@@ -55,7 +55,7 @@ public class AnnotateWithOpTraits implements Transform {
   public ParseContext transform(ParseContext pctx) throws SemanticException {
     AnnotateOpTraitsProcCtx annotateCtx = new AnnotateOpTraitsProcCtx(pctx);
 
-    // create a walker which walks the tree in a DFS manner while maintaining the
+    // create a walker which walks the tree in a BFS manner while maintaining the
     // operator stack. The dispatcher generates the plan from the operator tree
     Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
     opRules.put(new RuleRegExp("TS", TableScanOperator.getOperatorName() + "%"),
@@ -83,7 +83,7 @@ public class AnnotateWithOpTraits implements Transform {
     // rule and passes the context along
     Dispatcher disp = new DefaultRuleDispatcher(OpTraitsRulesProcFactory.getDefaultRule(), opRules,
         annotateCtx);
-    GraphWalker ogw = new PreOrderWalker(disp);
+    GraphWalker ogw = new LevelOrderWalker(disp, 0);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/AnnotateWithStatistics.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/AnnotateWithStatistics.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/AnnotateWithStatistics.java
index 4aeeff2..c8b3545 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/AnnotateWithStatistics.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/AnnotateWithStatistics.java
@@ -33,9 +33,9 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
 import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.LevelOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.PreOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Rule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
@@ -48,7 +48,7 @@ public class AnnotateWithStatistics implements Transform {
   public ParseContext transform(ParseContext pctx) throws SemanticException {
     AnnotateStatsProcCtx aspCtx = new AnnotateStatsProcCtx(pctx);
 
-    // create a walker which walks the tree in a DFS manner while maintaining the
+    // create a walker which walks the tree in a BFS manner while maintaining the
     // operator stack. The dispatcher generates the plan from the operator tree
     Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
     opRules.put(new RuleRegExp("TS", TableScanOperator.getOperatorName() + "%"),
@@ -70,7 +70,7 @@ public class AnnotateWithStatistics implements Transform {
     // rule and passes the context along
     Dispatcher disp = new DefaultRuleDispatcher(StatsRulesProcFactory.getDefaultRule(), opRules,
         aspCtx);
-    GraphWalker ogw = new PreOrderWalker(disp);
+    GraphWalker ogw = new LevelOrderWalker(disp, 0);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcessor.java
index 9937343..b1286e0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcessor.java
@@ -31,9 +31,9 @@ import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
 import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.LevelOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.PreOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Rule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
@@ -66,7 +66,7 @@ public class UnionProcessor implements Transform {
    *          the current parse context
    */
   public ParseContext transform(ParseContext pCtx) throws SemanticException {
-    // create a walker which walks the tree in a DFS manner while maintaining
+    // create a walker which walks the tree in a BFS manner while maintaining
     // the operator stack.
     Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
     opRules.put(new RuleRegExp("R1",
@@ -85,7 +85,8 @@ public class UnionProcessor implements Transform {
     uCtx.setParseContext(pCtx);
     Dispatcher disp = new DefaultRuleDispatcher(UnionProcFactory.getNoUnion(),
         opRules, uCtx);
-    GraphWalker ogw = new PreOrderWalker(disp);
+    LevelOrderWalker ogw = new LevelOrderWalker(disp);
+    ogw.setNodeTypes(UnionOperator.class);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
@@ -109,7 +110,8 @@ public class UnionProcessor implements Transform {
         UnionProcFactory.getUnionNoProcessFile());
 
       disp = new DefaultRuleDispatcher(UnionProcFactory.getNoUnion(), opRules, uCtx);
-      ogw = new PreOrderWalker(disp);
+      ogw = new LevelOrderWalker(disp);
+      ogw.setNodeTypes(FileSinkOperator.class);
 
       // Create a list of topop nodes
       topNodes.clear();

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/parse/GenMapRedWalker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenMapRedWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenMapRedWalker.java
index 9583a1b..c1056ae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenMapRedWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenMapRedWalker.java
@@ -46,7 +46,7 @@ public class GenMapRedWalker extends DefaultGraphWalker {
    *          operator being walked
    */
   @Override
-  public void walk(Node nd) throws SemanticException {
+  protected void walk(Node nd) throws SemanticException {
     List<? extends Node> children = nd.getChildren();
 
     // maintain the stack of operators encountered

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWorkWalker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWorkWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWorkWalker.java
index 2d8c8b2..8927579 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWorkWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWorkWalker.java
@@ -82,7 +82,7 @@ public class GenTezWorkWalker extends DefaultGraphWalker {
    * @param nd operator being walked
    */
   @Override
-  public void walk(Node nd) throws SemanticException {
+  protected void walk(Node nd) throws SemanticException {
     List<? extends Node> children = nd.getChildren();
 
     // maintain the stack of operators encountered

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/parse/TezWalker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezWalker.java
index 2f63c1a..3187497 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezWalker.java
@@ -46,7 +46,7 @@ public class TezWalker extends DefaultGraphWalker {
    *          operator being walked
    */
   @Override
-  public void walk(Node nd) throws SemanticException {
+  protected void walk(Node nd) throws SemanticException {
     List<? extends Node> children = nd.getChildren();
 
     // maintain the stack of operators encountered

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java
index e31c025..4450079 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java
@@ -82,7 +82,7 @@ public class GenSparkWorkWalker extends DefaultGraphWalker {
    * @param nd operator being walked
    */
   @Override
-  public void walk(Node nd) throws SemanticException {
+  protected void walk(Node nd) throws SemanticException {
     List<? extends Node> children = nd.getChildren();
 
     // maintain the stack of operators encountered

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
index ea1f713..fb76d5d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
@@ -37,10 +37,10 @@ import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
 import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.LevelOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.PreOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Rule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
@@ -74,7 +74,7 @@ public class PredicateTransitivePropagate implements Transform {
     // rule and passes the context along
     TransitiveContext context = new TransitiveContext();
     Dispatcher disp = new DefaultRuleDispatcher(null, opRules, context);
-    GraphWalker ogw = new PreOrderWalker(disp);
+    GraphWalker ogw = new LevelOrderWalker(disp, 2);
 
     // Create a list of topop nodes
     List<Node> topNodes = new ArrayList<Node>();

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/test/results/clientpositive/correlationoptimizer5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/correlationoptimizer5.q.out b/ql/src/test/results/clientpositive/correlationoptimizer5.q.out
index 63741fc..7f2e19f 100644
--- a/ql/src/test/results/clientpositive/correlationoptimizer5.q.out
+++ b/ql/src/test/results/clientpositive/correlationoptimizer5.q.out
@@ -280,7 +280,7 @@ POSTHOOK: Input: default@t2
 POSTHOOK: Input: default@t3
 POSTHOOK: Input: default@t4
 POSTHOOK: Output: default@dest_co1
-POSTHOOK: Lineage: dest_co1.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: dest_co1.key SIMPLE [(t1)x.FieldSchema(name:key, type:int, comment:null), ]
 POSTHOOK: Lineage: dest_co1.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ]
 PREHOOK: query: EXPLAIN
 INSERT OVERWRITE TABLE dest_co2
@@ -461,7 +461,7 @@ POSTHOOK: Input: default@t2
 POSTHOOK: Input: default@t3
 POSTHOOK: Input: default@t4
 POSTHOOK: Output: default@dest_co2
-POSTHOOK: Lineage: dest_co2.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: dest_co2.key SIMPLE [(t1)x.FieldSchema(name:key, type:int, comment:null), ]
 POSTHOOK: Lineage: dest_co2.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ]
 PREHOOK: query: -- Enable hive.auto.convert.join.
 EXPLAIN
@@ -754,7 +754,7 @@ POSTHOOK: Input: default@t2
 POSTHOOK: Input: default@t3
 POSTHOOK: Input: default@t4
 POSTHOOK: Output: default@dest_co3
-POSTHOOK: Lineage: dest_co3.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: dest_co3.key SIMPLE [(t1)x.FieldSchema(name:key, type:int, comment:null), ]
 POSTHOOK: Lineage: dest_co3.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ]
 PREHOOK: query: -- dest_co1, dest_co2 and dest_co3 should be same
 -- SELECT * FROM dest_co1 x ORDER BY x.key, x.val;

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/test/results/clientpositive/join32.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join32.q.out b/ql/src/test/results/clientpositive/join32.q.out
index 5795669..afb373d 100644
--- a/ql/src/test/results/clientpositive/join32.q.out
+++ b/ql/src/test/results/clientpositive/join32.q.out
@@ -406,7 +406,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
 POSTHOOK: Output: default@dest_j1
 POSTHOOK: Lineage: dest_j1.key SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: dest_j1.val2 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: dest_j1.value EXPRESSION [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: dest_j1.value SIMPLE [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: select * from dest_j1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest_j1

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/test/results/clientpositive/join32_lessSize.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join32_lessSize.q.out b/ql/src/test/results/clientpositive/join32_lessSize.q.out
index c027dba..3c5f9e2 100644
--- a/ql/src/test/results/clientpositive/join32_lessSize.q.out
+++ b/ql/src/test/results/clientpositive/join32_lessSize.q.out
@@ -471,7 +471,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
 POSTHOOK: Output: default@dest_j1
 POSTHOOK: Lineage: dest_j1.key SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: dest_j1.val2 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: dest_j1.value EXPRESSION [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: dest_j1.value SIMPLE [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: select * from dest_j1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest_j1
@@ -1107,9 +1107,9 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Input: default@src1
 POSTHOOK: Output: default@dest_j1
-POSTHOOK: Lineage: dest_j1.key EXPRESSION [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: dest_j1.key SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: dest_j1.val2 SIMPLE [(src)w.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: dest_j1.value EXPRESSION [(src1)x.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: dest_j1.value SIMPLE [(src1)x.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: select * from dest_j1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest_j1

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/test/results/clientpositive/join33.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join33.q.out b/ql/src/test/results/clientpositive/join33.q.out
index 5795669..afb373d 100644
--- a/ql/src/test/results/clientpositive/join33.q.out
+++ b/ql/src/test/results/clientpositive/join33.q.out
@@ -406,7 +406,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
 POSTHOOK: Output: default@dest_j1
 POSTHOOK: Lineage: dest_j1.key SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: dest_j1.val2 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: dest_j1.value EXPRESSION [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: dest_j1.value SIMPLE [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: select * from dest_j1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest_j1

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/test/results/clientpositive/lineage2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/lineage2.q.out b/ql/src/test/results/clientpositive/lineage2.q.out
index 9b227c6..549b5f6 100644
--- a/ql/src/test/results/clientpositive/lineage2.q.out
+++ b/ql/src/test/results/clientpositive/lineage2.q.out
@@ -523,14 +523,14 @@ PREHOOK: Input: default@src1
 PREHOOK: Input: default@src2
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest3
-{"version":"1.0","engine":"mr","hash":"a2c4e9a3ec678039814f5d84b1e38ce4","queryText":"create table dest3 as\n  select * from src1 JOIN src2 ON src1.key = src2.key2 WHERE length(key) > 1","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[4],"targets":[0,1,2,3],"expression":"(length(src1.key) > 1)","edgeType":"PREDICATE"},{"sources":[4,6],"targets":[0,1,2,3],"expression":"(src1.key = src2.key2)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest3.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest3.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest3.key2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest3.value2"},{"id":4,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":5,"vertexType":"COLUMN","vertexId":"default.src1.value"
 },{"id":6,"vertexType":"COLUMN","vertexId":"default.src2.key2"},{"id":7,"vertexType":"COLUMN","vertexId":"default.src2.value2"}]}
+{"version":"1.0","engine":"mr","hash":"a2c4e9a3ec678039814f5d84b1e38ce4","queryText":"create table dest3 as\n  select * from src1 JOIN src2 ON src1.key = src2.key2 WHERE length(key) > 1","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[4,6],"targets":[0,1,2,3],"expression":"(src1.key = src2.key2)","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1,2,3],"expression":"(length(src1.key) > 1)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest3.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest3.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest3.key2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest3.value2"},{"id":4,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":5,"vertexType":"COLUMN","vertexId":"default.src1.value"
 },{"id":6,"vertexType":"COLUMN","vertexId":"default.src2.key2"},{"id":7,"vertexType":"COLUMN","vertexId":"default.src2.value2"}]}
 PREHOOK: query: insert overwrite table dest2
   select * from src1 JOIN src2 ON src1.key = src2.key2 WHERE length(key) > 3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 PREHOOK: Input: default@src2
 PREHOOK: Output: default@dest2
-{"version":"1.0","engine":"mr","hash":"76d84512204ddc576ad4d93f252e4358","queryText":"insert overwrite table dest2\n  select * from src1 JOIN src2 ON src1.key = src2.key2 WHERE length(key) > 3","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[4],"targets":[0,1,2,3],"expression":"(length(src1.key) > 3)","edgeType":"PREDICATE"},{"sources":[4,6],"targets":[0,1,2,3],"expression":"(src1.key = src2.key2)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest2.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest2.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest2.key2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest2.value2"},{"id":4,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":5,"vertexType":"COLUMN","vertexId":"default.src1
 .value"},{"id":6,"vertexType":"COLUMN","vertexId":"default.src2.key2"},{"id":7,"vertexType":"COLUMN","vertexId":"default.src2.value2"}]}
+{"version":"1.0","engine":"mr","hash":"76d84512204ddc576ad4d93f252e4358","queryText":"insert overwrite table dest2\n  select * from src1 JOIN src2 ON src1.key = src2.key2 WHERE length(key) > 3","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[4,6],"targets":[0,1,2,3],"expression":"(src1.key = src2.key2)","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1,2,3],"expression":"(length(src1.key) > 3)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest2.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest2.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest2.key2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest2.value2"},{"id":4,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":5,"vertexType":"COLUMN","vertexId":"default.src1
 .value"},{"id":6,"vertexType":"COLUMN","vertexId":"default.src2.key2"},{"id":7,"vertexType":"COLUMN","vertexId":"default.src2.value2"}]}
 PREHOOK: query: drop table if exists dest_l1
 PREHOOK: type: DROPTABLE
 PREHOOK: query: CREATE TABLE dest_l1(key INT, value STRING) STORED AS TEXTFILE
@@ -593,7 +593,7 @@ PREHOOK: Input: default@dept
 PREHOOK: Input: default@emp
 PREHOOK: Input: default@project
 PREHOOK: Output: default@tgt
-{"version":"1.0","engine":"mr","hash":"f59797e0422d2e51515063374dfac361","queryText":"INSERT INTO TABLE tgt\nSELECT emd.dept_name, emd.name, emd.emp_id, emd.mgr_id, p.project_id, p.project_name\nFROM (\n  SELECT d.dept_name, em.name, em.emp_id, em.mgr_id, em.dept_id\n  FROM (\n    SELECT e.name, e.dept_id, e.emp_id emp_id, m.emp_id mgr_id\n    FROM emp e JOIN emp m ON e.emp_id = m.emp_id\n    ) em\n  JOIN dept d ON d.dept_id = em.dept_id\n  ) emd JOIN project p ON emd.dept_id = p.project_id","edges":[{"sources":[6],"targets":[0],"edgeType":"PROJECTION"},{"sources":[7],"targets":[1],"expression":"emd.name","edgeType":"PROJECTION"},{"sources":[8],"targets":[2],"expression":"emd.emp_id","edgeType":"PROJECTION"},{"sources":[8],"targets":[3],"expression":"emd.mgr_id","edgeType":"PROJECTION"},{"sources":[9],"targets":[4],"edgeType":"PROJECTION"},{"sources":[10],"targets":[5],"edgeType":"PROJECTION"},{"sources":[8],"targets":[0,1,2,3,4,5],"expression":"(e.emp_id = m.emp_id)","edgeType":"PR
 EDICATE"},{"sources":[11,12],"targets":[0,1,2,3,4,5],"expression":"(em._col1 = d.dept_id)","edgeType":"PREDICATE"},{"sources":[11,9],"targets":[0,1,2,3,4,5],"expression":"(emd._col4 = p.project_id)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.tgt.dept_name"},{"id":1,"vertexType":"COLUMN","vertexId":"default.tgt.name"},{"id":2,"vertexType":"COLUMN","vertexId":"default.tgt.emp_id"},{"id":3,"vertexType":"COLUMN","vertexId":"default.tgt.mgr_id"},{"id":4,"vertexType":"COLUMN","vertexId":"default.tgt.proj_id"},{"id":5,"vertexType":"COLUMN","vertexId":"default.tgt.proj_name"},{"id":6,"vertexType":"COLUMN","vertexId":"default.dept.dept_name"},{"id":7,"vertexType":"COLUMN","vertexId":"default.emp.name"},{"id":8,"vertexType":"COLUMN","vertexId":"default.emp.emp_id"},{"id":9,"vertexType":"COLUMN","vertexId":"default.project.project_id"},{"id":10,"vertexType":"COLUMN","vertexId":"default.project.project_name"},{"id":11,"vertexType":"COLUMN","vertexId":
 "default.emp.dept_id"},{"id":12,"vertexType":"COLUMN","vertexId":"default.dept.dept_id"}]}
+{"version":"1.0","engine":"mr","hash":"f59797e0422d2e51515063374dfac361","queryText":"INSERT INTO TABLE tgt\nSELECT emd.dept_name, emd.name, emd.emp_id, emd.mgr_id, p.project_id, p.project_name\nFROM (\n  SELECT d.dept_name, em.name, em.emp_id, em.mgr_id, em.dept_id\n  FROM (\n    SELECT e.name, e.dept_id, e.emp_id emp_id, m.emp_id mgr_id\n    FROM emp e JOIN emp m ON e.emp_id = m.emp_id\n    ) em\n  JOIN dept d ON d.dept_id = em.dept_id\n  ) emd JOIN project p ON emd.dept_id = p.project_id","edges":[{"sources":[6],"targets":[0],"edgeType":"PROJECTION"},{"sources":[7],"targets":[1],"edgeType":"PROJECTION"},{"sources":[8],"targets":[2,3],"edgeType":"PROJECTION"},{"sources":[9],"targets":[4],"edgeType":"PROJECTION"},{"sources":[10],"targets":[5],"edgeType":"PROJECTION"},{"sources":[8],"targets":[0,1,2,3,4,5],"expression":"(e.emp_id = m.emp_id)","edgeType":"PREDICATE"},{"sources":[11,12],"targets":[0,1,2,3,4,5],"expression":"(em._col1 = d.dept_id)","edgeType":"PREDICATE"},{"sources":[1
 1,9],"targets":[0,1,2,3,4,5],"expression":"(emd._col4 = p.project_id)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.tgt.dept_name"},{"id":1,"vertexType":"COLUMN","vertexId":"default.tgt.name"},{"id":2,"vertexType":"COLUMN","vertexId":"default.tgt.emp_id"},{"id":3,"vertexType":"COLUMN","vertexId":"default.tgt.mgr_id"},{"id":4,"vertexType":"COLUMN","vertexId":"default.tgt.proj_id"},{"id":5,"vertexType":"COLUMN","vertexId":"default.tgt.proj_name"},{"id":6,"vertexType":"COLUMN","vertexId":"default.dept.dept_name"},{"id":7,"vertexType":"COLUMN","vertexId":"default.emp.name"},{"id":8,"vertexType":"COLUMN","vertexId":"default.emp.emp_id"},{"id":9,"vertexType":"COLUMN","vertexId":"default.project.project_id"},{"id":10,"vertexType":"COLUMN","vertexId":"default.project.project_name"},{"id":11,"vertexType":"COLUMN","vertexId":"default.emp.dept_id"},{"id":12,"vertexType":"COLUMN","vertexId":"default.dept.dept_id"}]}
 PREHOOK: query: drop table if exists dest_l2
 PREHOOK: type: DROPTABLE
 PREHOOK: query: create table dest_l2 (id int, c1 tinyint, c2 int, c3 bigint) stored as textfile
@@ -646,7 +646,7 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@dest_l2
 PREHOOK: Input: default@dest_l3
 #### A masked pattern was here ####
-{"version":"1.0","engine":"mr","hash":"01879c619517509d9f5b6ead998bb4bb","queryText":"select sum(a.c1), count(b.c1), b.c2, b.c3\nfrom dest_l2 a join dest_l3 b on (a.id = b.id)\nwhere a.c2 != 10 and b.c3 > 0\ngroup by a.c1, a.c2, a.id, b.c1, b.c2, b.c3\nhaving count(a.c2) > 0\norder by b.c3 limit 5","edges":[{"sources":[4],"targets":[0],"expression":"sum(default.dest_l2.c1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"expression":"count(default.dest_l3.c1)","edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[8,7],"targets":[0,1,2,3],"expression":"((a.c2 <> 10) and (b.c3 > 0))","edgeType":"PREDICATE"},{"sources":[8],"targets":[0,1,2,3],"expression":"(count(default.dest_l2.c2) > 0)","edgeType":"PREDICATE"},{"sources":[9,10],"targets":[0,1,2,3],"expression":"(a.id = b.id)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"_c0"},{"id":1,"vertexType":"COLUMN
 ","vertexId":"_c1"},{"id":2,"vertexType":"COLUMN","vertexId":"b.c2"},{"id":3,"vertexType":"COLUMN","vertexId":"b.c3"},{"id":4,"vertexType":"COLUMN","vertexId":"default.dest_l2.c1"},{"id":5,"vertexType":"COLUMN","vertexId":"default.dest_l3.c1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.dest_l3.c2"},{"id":7,"vertexType":"COLUMN","vertexId":"default.dest_l3.c3"},{"id":8,"vertexType":"COLUMN","vertexId":"default.dest_l2.c2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.dest_l2.id"},{"id":10,"vertexType":"COLUMN","vertexId":"default.dest_l3.id"}]}
+{"version":"1.0","engine":"mr","hash":"01879c619517509d9f5b6ead998bb4bb","queryText":"select sum(a.c1), count(b.c1), b.c2, b.c3\nfrom dest_l2 a join dest_l3 b on (a.id = b.id)\nwhere a.c2 != 10 and b.c3 > 0\ngroup by a.c1, a.c2, a.id, b.c1, b.c2, b.c3\nhaving count(a.c2) > 0\norder by b.c3 limit 5","edges":[{"sources":[4],"targets":[0],"expression":"sum(default.dest_l2.c1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"expression":"count(default.dest_l3.c1)","edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[8,9],"targets":[0,1,2,3],"expression":"(a.id = b.id)","edgeType":"PREDICATE"},{"sources":[10,7],"targets":[0,1,2,3],"expression":"((a.c2 <> 10) and (b.c3 > 0))","edgeType":"PREDICATE"},{"sources":[10],"targets":[0,1,2,3],"expression":"(count(default.dest_l2.c2) > 0)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"_c0"},{"id":1,"vertexType":"COLUM
 N","vertexId":"_c1"},{"id":2,"vertexType":"COLUMN","vertexId":"b.c2"},{"id":3,"vertexType":"COLUMN","vertexId":"b.c3"},{"id":4,"vertexType":"COLUMN","vertexId":"default.dest_l2.c1"},{"id":5,"vertexType":"COLUMN","vertexId":"default.dest_l3.c1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.dest_l3.c2"},{"id":7,"vertexType":"COLUMN","vertexId":"default.dest_l3.c3"},{"id":8,"vertexType":"COLUMN","vertexId":"default.dest_l2.id"},{"id":9,"vertexType":"COLUMN","vertexId":"default.dest_l3.id"},{"id":10,"vertexType":"COLUMN","vertexId":"default.dest_l2.c2"}]}
 1	1	s2	15
 PREHOOK: query: drop table if exists t
 PREHOOK: type: DROPTABLE
@@ -659,7 +659,7 @@ PREHOOK: Input: default@dest_l2
 PREHOOK: Input: default@dest_l3
 PREHOOK: Output: database:default
 PREHOOK: Output: default@t
-{"version":"1.0","engine":"mr","hash":"0d2f15b494111ffe236d5be42a76fa28","queryText":"create table t as\nselect distinct a.c2, a.c3 from dest_l2 a\ninner join dest_l3 b on (a.id = b.id)\nwhere a.id > 0 and b.c3 = 15","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[4,5],"targets":[0,1],"expression":"((a.id > 0) and (b.c3 = 15))","edgeType":"PREDICATE"},{"sources":[4,6],"targets":[0,1],"expression":"(a.id = b.id)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.t.c2"},{"id":1,"vertexType":"COLUMN","vertexId":"default.t.c3"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_l2.c2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest_l2.c3"},{"id":4,"vertexType":"COLUMN","vertexId":"default.dest_l2.id"},{"id":5,"vertexType":"COLUMN","vertexId":"default.dest_l3.c3"},{"id":6,"vertexType":"COLUMN","vertexId":"default.dest_l3.id"}]}
+{"version":"1.0","engine":"mr","hash":"0d2f15b494111ffe236d5be42a76fa28","queryText":"create table t as\nselect distinct a.c2, a.c3 from dest_l2 a\ninner join dest_l3 b on (a.id = b.id)\nwhere a.id > 0 and b.c3 = 15","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[4,5],"targets":[0,1],"expression":"(a.id = b.id)","edgeType":"PREDICATE"},{"sources":[4,6],"targets":[0,1],"expression":"((a.id > 0) and (b.c3 = 15))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.t.c2"},{"id":1,"vertexType":"COLUMN","vertexId":"default.t.c3"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_l2.c2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest_l2.c3"},{"id":4,"vertexType":"COLUMN","vertexId":"default.dest_l2.id"},{"id":5,"vertexType":"COLUMN","vertexId":"default.dest_l3.id"},{"id":6,"vertexType":"COLUMN","vertexId":"default.dest_l3.c3"}]}
 PREHOOK: query: SELECT substr(src1.key,1,1), count(DISTINCT substr(src1.value,5)),
 concat(substr(src1.key,1,1),sum(substr(src1.value,5)))
 from src1

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/test/results/clientpositive/lineage3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/lineage3.q.out b/ql/src/test/results/clientpositive/lineage3.q.out
index b6b4e0b..6fd2aa4 100644
--- a/ql/src/test/results/clientpositive/lineage3.q.out
+++ b/ql/src/test/results/clientpositive/lineage3.q.out
@@ -25,7 +25,7 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: default@d1
 PREHOOK: Output: default@d2
-{"version":"1.0","engine":"mr","hash":"8703e4091ebd4c96afd3cac83e3a2957","queryText":"from (select a.ctinyint x, b.cstring1 y\nfrom alltypesorc a join alltypesorc b on a.cint = b.cbigint) t\ninsert into table d1 select x where y is null\ninsert into table d2 select y where x > 0","edges":[{"sources":[2],"targets":[0],"expression":"UDFToInteger(x)","edgeType":"PROJECTION"},{"sources":[3],"targets":[0,1],"expression":"t.y is null","edgeType":"PREDICATE"},{"sources":[4,5],"targets":[0,1],"expression":"(UDFToLong(a.cint) = b.cbigint)","edgeType":"PREDICATE"},{"sources":[3],"targets":[1],"expression":"CAST( y AS varchar(128))","edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(t.x > 0)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.d1.a"},{"id":1,"vertexType":"COLUMN","vertexId":"default.d2.b"},{"id":2,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.
 cstring1"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"}]}
+{"version":"1.0","engine":"mr","hash":"8703e4091ebd4c96afd3cac83e3a2957","queryText":"from (select a.ctinyint x, b.cstring1 y\nfrom alltypesorc a join alltypesorc b on a.cint = b.cbigint) t\ninsert into table d1 select x where y is null\ninsert into table d2 select y where x > 0","edges":[{"sources":[2],"targets":[0],"expression":"UDFToInteger(x)","edgeType":"PROJECTION"},{"sources":[3,4],"targets":[0,1],"expression":"(UDFToLong(a.cint) = b.cbigint)","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1],"expression":"t.y is null","edgeType":"PREDICATE"},{"sources":[5],"targets":[1],"expression":"CAST( y AS varchar(128))","edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(t.x > 0)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.d1.a"},{"id":1,"vertexType":"COLUMN","vertexId":"default.d2.b"},{"id":2,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.
 cint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"}]}
 PREHOOK: query: drop table if exists t
 PREHOOK: type: DROPTABLE
 PREHOOK: query: create table t as
@@ -116,7 +116,7 @@ order by a.cbigint, a.ctinyint, b.cint, b.ctinyint limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 #### A masked pattern was here ####
-{"version":"1.0","engine":"mr","hash":"afd760470fc5aa6d3e8348dee03af97f","queryText":"select a.cbigint, a.ctinyint, b.cint, b.ctinyint\nfrom\n  (select ctinyint, cbigint from alltypesorc\n   union all\n   select ctinyint, cbigint from alltypesorc) a\n  inner join\n  alltypesorc b\n  on (a.ctinyint = b.ctinyint)\nwhere b.ctinyint < 100 and a.cbigint is not null and b.cint is not null\norder by a.cbigint, a.ctinyint, b.cint, b.ctinyint limit 5","edges":[{"sources":[4],"targets":[0],"expression":"a.ctinyint","edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"expression":"a.cbigint","edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[5],"targets":[3],"edgeType":"PROJECTION"},{"sources":[4],"targets":[0,1,2,3],"expression":"alltypesorc.cbigint is not null","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1,2,3],"expression":"(alltypesorc.ctinyint < 100)","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1,2,3],"expression":"(ctinyint < 100)
 ","edgeType":"PREDICATE"},{"sources":[5,6],"targets":[0,1,2,3],"expression":"((alltypesorc.ctinyint < 100) and alltypesorc.cint is not null)","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1,2,3],"expression":"(a.cbigint = alltypesorc.ctinyint)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.cbigint"},{"id":1,"vertexType":"COLUMN","vertexId":"a.ctinyint"},{"id":2,"vertexType":"COLUMN","vertexId":"b.cint"},{"id":3,"vertexType":"COLUMN","vertexId":"b.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"}]}
+{"version":"1.0","engine":"mr","hash":"afd760470fc5aa6d3e8348dee03af97f","queryText":"select a.cbigint, a.ctinyint, b.cint, b.ctinyint\nfrom\n  (select ctinyint, cbigint from alltypesorc\n   union all\n   select ctinyint, cbigint from alltypesorc) a\n  inner join\n  alltypesorc b\n  on (a.ctinyint = b.ctinyint)\nwhere b.ctinyint < 100 and a.cbigint is not null and b.cint is not null\norder by a.cbigint, a.ctinyint, b.cint, b.ctinyint limit 5","edges":[{"sources":[4],"targets":[0],"expression":"cbigint","edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"expression":"ctinyint","edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[5],"targets":[3],"edgeType":"PROJECTION"},{"sources":[4],"targets":[0,1,2,3],"expression":"alltypesorc.cbigint is not null","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1,2,3],"expression":"(ctinyint < 100)","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1,2,3],"expression":"(ctinyint = alltypesorc.ctinyint
 )","edgeType":"PREDICATE"},{"sources":[5,6],"targets":[0,1,2,3],"expression":"((alltypesorc.ctinyint < 100) and alltypesorc.cint is not null)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.cbigint"},{"id":1,"vertexType":"COLUMN","vertexId":"a.ctinyint"},{"id":2,"vertexType":"COLUMN","vertexId":"b.cint"},{"id":3,"vertexType":"COLUMN","vertexId":"b.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"}]}
 -2147311592	-51	-1071480828	-51
 -2147311592	-51	-1071480828	-51
 -2147311592	-51	-1067683781	-51
@@ -135,7 +135,7 @@ and x.ctinyint + length(c.cstring2) < 1000
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 #### A masked pattern was here ####
-{"version":"1.0","engine":"mr","hash":"3a12ad24b2622a8958df12d0bdc60f8a","queryText":"select x.ctinyint, x.cint, c.cbigint-100, c.cstring1\nfrom alltypesorc c\njoin (\n   select a.ctinyint ctinyint, b.cint cint\n   from (select * from alltypesorc a where cboolean1=false) a\n   join alltypesorc b on (a.cint = b.cbigint - 224870380)\n ) x on (x.cint = c.cint)\nwhere x.ctinyint > 10\nand x.cint < 4.5\nand x.ctinyint + length(c.cstring2) < 1000","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"expression":"cint","edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"expression":"(c.cbigint - UDFToLong(100))","edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[5],"targets":[0,1,2,3],"expression":"(UDFToDouble(c.cint) < 4.5)","edgeType":"PREDICATE"},{"sources":[8],"targets":[0,1,2,3],"expression":"((UDFToInteger(ctinyint) + length(c.cstring2)) < 1000)","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1,2
 ,3],"expression":"(c.cint = c.cint)","edgeType":"PREDICATE"},{"sources":[9],"targets":[0,1,2,3],"expression":"(c.cboolean1 = false)","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1,2,3],"expression":"(c.ctinyint > 10)","edgeType":"PREDICATE"},{"sources":[6,5],"targets":[0,1,2,3],"expression":"((c.cbigint - UDFToLong(224870380)) = UDFToLong(c.cint))","edgeType":"PREDICATE"},{"sources":[4,8],"targets":[0,1,2,3],"expression":"((UDFToInteger(c.ctinyint) + length(c.cstring2)) < 1000)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"x.ctinyint"},{"id":1,"vertexType":"COLUMN","vertexId":"x.cint"},{"id":2,"vertexType":"COLUMN","vertexId":"c2"},{"id":3,"vertexType":"COLUMN","vertexId":"c.cstring1"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":7,"vertexType":"COLUMN","vertexId":"defau
 lt.alltypesorc.cstring1"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"}]}
+{"version":"1.0","engine":"mr","hash":"3a12ad24b2622a8958df12d0bdc60f8a","queryText":"select x.ctinyint, x.cint, c.cbigint-100, c.cstring1\nfrom alltypesorc c\njoin (\n   select a.ctinyint ctinyint, b.cint cint\n   from (select * from alltypesorc a where cboolean1=false) a\n   join alltypesorc b on (a.cint = b.cbigint - 224870380)\n ) x on (x.cint = c.cint)\nwhere x.ctinyint > 10\nand x.cint < 4.5\nand x.ctinyint + length(c.cstring2) < 1000","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"expression":"(c.cbigint - UDFToLong(100))","edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[5],"targets":[0,1,2,3],"expression":"(UDFToDouble(c.cint) < 4.5)","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1,2,3],"expression":"(c.cint = c.cint)","edgeType":"PREDICATE"},{"sources":[6,5],"targets":[0,1,2,3],"expression":"((c.cbigint - UDFToLong(224870380)) =
  UDFToLong(c.cint))","edgeType":"PREDICATE"},{"sources":[8],"targets":[0,1,2,3],"expression":"(c.cboolean1 = false)","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1,2,3],"expression":"(c.ctinyint > 10)","edgeType":"PREDICATE"},{"sources":[4,9],"targets":[0,1,2,3],"expression":"((UDFToInteger(c.ctinyint) + length(c.cstring2)) < 1000)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"x.ctinyint"},{"id":1,"vertexType":"COLUMN","vertexId":"x.cint"},{"id":2,"vertexType":"COLUMN","vertexId":"c2"},{"id":3,"vertexType":"COLUMN","vertexId":"c.cstring1"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":9,"vertexType":"COLUMN","vertexId":"default
 .alltypesorc.cstring2"}]}
 11	-654374827	857266369	OEfPnHnIYueoup
 PREHOOK: query: select c1, x2, x3
 from (
@@ -178,7 +178,7 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-{"version":"1.0","engine":"mr","hash":"8bf193b0658183be94e2428a79d91d10","queryText":"select * from src1 a\nwhere exists\n  (select cint from alltypesorc b\n   where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > UDFToDouble(300))","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"(1 = 1)","edgeType":"PREDICATE"},{"sources":[2,4],"targets":[0,1],"expression":"(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
+{"version":"1.0","engine":"mr","hash":"8bf193b0658183be94e2428a79d91d10","queryText":"select * from src1 a\nwhere exists\n  (select cint from alltypesorc b\n   where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > UDFToDouble(300))","edgeType":"PREDICATE"},{"sources":[2,4],"targets":[0,1],"expression":"(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"(1 = 1)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
 311	val_311
 Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: select key, value from src1
@@ -186,7 +186,7 @@ where key not in (select key+18 from src1) order by key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-{"version":"1.0","engine":"mr","hash":"9b488fe1d7cf018aad3825173808cd36","queryText":"select key, value from src1\nwhere key not in (select key+18 from src1) order by key","edges":[{"sources":[2],"targets":[0],"expression":"key","edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"value","edgeType":"PROJECTION"},{"sources":[],"targets":[0,1],"expression":"(1 = 1)","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"_o__c0 is null","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) + UDFToDouble(18)) is null","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"(count(*) = 0)","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"true","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(key) = (UDFToDouble(src1.key) + UDFToDouble(18)))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"key"},{"id":1,"vertexType":"COLUMN","ver
 texId":"value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"TABLE","vertexId":"default.src1"}]}
+{"version":"1.0","engine":"mr","hash":"9b488fe1d7cf018aad3825173808cd36","queryText":"select key, value from src1\nwhere key not in (select key+18 from src1) order by key","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[],"targets":[0,1],"expression":"(1 = 1)","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) + UDFToDouble(18)) is null","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"(count(*) = 0)","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"true","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) = (UDFToDouble(src1.key) + UDFToDouble(18)))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"key"},{"id":1,"vertexType":"COLUMN","vertexId":"value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"
 default.src1.value"},{"id":4,"vertexType":"TABLE","vertexId":"default.src1"}]}
 PREHOOK: query: select * from src1 a
 where not exists
   (select cint from alltypesorc b
@@ -196,7 +196,7 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-{"version":"1.0","engine":"mr","hash":"53191056e05af9080a30de853e8cea9c","queryText":"select * from src1 a\nwhere not exists\n  (select cint from alltypesorc b\n   where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > UDFToDouble(300))","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"sq_corr_0 is null","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"(1 = 1)","edgeType":"PREDICATE"},{"sources":[2,4],"targets":[0,1],"expression":"(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"(UDFToInteger(b.ctinyint) + 300) is null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUM
 N","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
+{"version":"1.0","engine":"mr","hash":"53191056e05af9080a30de853e8cea9c","queryText":"select * from src1 a\nwhere not exists\n  (select cint from alltypesorc b\n   where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > UDFToDouble(300))","edgeType":"PREDICATE"},{"sources":[2,4],"targets":[0,1],"expression":"(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"(1 = 1)","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"(UDFToInteger(b.ctinyint) + 300) is null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src
 1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
 369	
 401	val_401
 406	val_406
@@ -297,7 +297,7 @@ PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest_v3
-{"version":"1.0","engine":"mr","hash":"a0c2481ce1c24895a43a950f93a10da7","queryText":"create view dest_v3 (a1, a2, a3, a4, a5, a6, a7) as\n  select x.csmallint, x.cbigint bint1, x.ctinyint, c.cbigint bint2, x.cint, x.cfloat, c.cstring1\n  from alltypesorc c\n  join (\n     select a.csmallint csmallint, a.ctinyint ctinyint, a.cstring2 cstring2,\n           a.cint cint, a.cstring1 ctring1, b.cfloat cfloat, b.cbigint cbigint\n     from ( select * from alltypesorc a where cboolean1=true ) a\n     join alltypesorc b on (a.csmallint = b.cint)\n   ) x on (x.ctinyint = c.cbigint)\n  where x.csmallint=11\n  and x.cint > 899\n  and x.cfloat > 4.5\n  and c.cstring1 < '7'\n  and x.cint + x.cfloat + length(c.cstring1) < 1000","edges":[{"sources":[7],"targets":[0],"expression":"x._col15","edgeType":"PROJECTION"},{"sources":[8],"targets":[1,2],"edgeType":"PROJECTION"},{"sources":[9],"targets":[3],"expression":"x._col16","edgeType":"PROJECTION"},{"sources":[10],"targets":[4],"expression":"x._col18"
 ,"edgeType":"PROJECTION"},{"sources":[11],"targets":[5],"edgeType":"PROJECTION"},{"sources":[12],"targets":[6],"edgeType":"PROJECTION"},{"sources":[13],"targets":[0,1,3,2,4,5,6],"expression":"(a.cboolean1 = true)","edgeType":"PREDICATE"},{"sources":[7,10,12,11],"targets":[0,1,3,2,4,5,6],"expression":"((x.csmallint = 11) and (x.cint > 899) and (x.cfloat > 4.5) and (c.cstring1 < '7') and (((x.cint + x.cfloat) + length(c.cstring1)) < 1000))","edgeType":"PREDICATE"},{"sources":[7,10],"targets":[0,1,3,2,4,5,6],"expression":"(UDFToInteger(a._col1) = b.cint)","edgeType":"PREDICATE"},{"sources":[8,9],"targets":[0,1,3,2,4,5,6],"expression":"(c.cbigint = UDFToLong(x._col1))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_v3.csmallint"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_v3.bint1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_v3.bint2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest_v3.ctinyint"},{"id":4,"vertex
 Type":"COLUMN","vertexId":"default.dest_v3.cint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.dest_v3.cfloat"},{"id":6,"vertexType":"COLUMN","vertexId":"default.dest_v3.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":10,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":11,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"},{"id":12,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":13,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"}]}
+{"version":"1.0","engine":"mr","hash":"a0c2481ce1c24895a43a950f93a10da7","queryText":"create view dest_v3 (a1, a2, a3, a4, a5, a6, a7) as\n  select x.csmallint, x.cbigint bint1, x.ctinyint, c.cbigint bint2, x.cint, x.cfloat, c.cstring1\n  from alltypesorc c\n  join (\n     select a.csmallint csmallint, a.ctinyint ctinyint, a.cstring2 cstring2,\n           a.cint cint, a.cstring1 ctring1, b.cfloat cfloat, b.cbigint cbigint\n     from ( select * from alltypesorc a where cboolean1=true ) a\n     join alltypesorc b on (a.csmallint = b.cint)\n   ) x on (x.ctinyint = c.cbigint)\n  where x.csmallint=11\n  and x.cint > 899\n  and x.cfloat > 4.5\n  and c.cstring1 < '7'\n  and x.cint + x.cfloat + length(c.cstring1) < 1000","edges":[{"sources":[7],"targets":[0],"edgeType":"PROJECTION"},{"sources":[8],"targets":[1,2],"edgeType":"PROJECTION"},{"sources":[9],"targets":[3],"edgeType":"PROJECTION"},{"sources":[10],"targets":[4],"edgeType":"PROJECTION"},{"sources":[11],"targets":[5],"edgeType":"PROJ
 ECTION"},{"sources":[12],"targets":[6],"edgeType":"PROJECTION"},{"sources":[8,9],"targets":[0,1,3,2,4,5,6],"expression":"(c.cbigint = UDFToLong(x._col1))","edgeType":"PREDICATE"},{"sources":[13],"targets":[0,1,3,2,4,5,6],"expression":"(a.cboolean1 = true)","edgeType":"PREDICATE"},{"sources":[7,10],"targets":[0,1,3,2,4,5,6],"expression":"(UDFToInteger(a._col1) = b.cint)","edgeType":"PREDICATE"},{"sources":[7,10,11,12],"targets":[0,1,3,2,4,5,6],"expression":"((x.csmallint = 11) and (x.cint > 899) and (x.cfloat > 4.5) and (c.cstring1 < '7') and (((x.cint + x.cfloat) + length(c.cstring1)) < 1000))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_v3.csmallint"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_v3.bint1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_v3.bint2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest_v3.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.dest_v3.cint"},{"id":5,"vertexType":"
 COLUMN","vertexId":"default.dest_v3.cfloat"},{"id":6,"vertexType":"COLUMN","vertexId":"default.dest_v3.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":10,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":11,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"},{"id":12,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":13,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"}]}
 PREHOOK: query: alter view dest_v3 as
   select * from (
     select sum(a.ctinyint) over (partition by a.csmallint order by a.csmallint) a,
@@ -311,12 +311,12 @@ PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest_v3
-{"version":"1.0","engine":"mr","hash":"949093880975cc807ad1a8003e8a8c7c","queryText":"alter view dest_v3 as\n  select * from (\n    select sum(a.ctinyint) over (partition by a.csmallint order by a.csmallint) a,\n      count(b.cstring1) x, b.cboolean1\n    from alltypesorc a join alltypesorc b on (a.cint = b.cint)\n    where a.cboolean2 = true and b.cfloat > 0\n    group by a.ctinyint, a.csmallint, b.cboolean1\n    having count(a.cint) > 10\n    order by a, x, b.cboolean1 limit 10) t","edges":[{"sources":[3,4,5,6,7],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col a) ctinyint) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col a) csmallint)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col a) csmallint))))))","edgeType":"PROJECTION"},{"sources":[6],"targets":[1],"expression":"count(default.alltypesorc.cstring1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[2],"edgeType":"PROJECTION"},{"sources":[8,9],"targets":[0,1,2],"express
 ion":"((a.cboolean2 = true) and (b.cfloat > 0.0))","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(count(default.alltypesorc.cint) > 10)","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(a.cint = b.cint)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_v3.a"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_v3.x"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_v3.cboolean1"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"}]}
+{"version":"1.0","engine":"mr","hash":"949093880975cc807ad1a8003e8a8c7c","queryText":"alter view dest_v3 as\n  select * from (\n    select sum(a.ctinyint) over (partition by a.csmallint order by a.csmallint) a,\n      count(b.cstring1) x, b.cboolean1\n    from alltypesorc a join alltypesorc b on (a.cint = b.cint)\n    where a.cboolean2 = true and b.cfloat > 0\n    group by a.ctinyint, a.csmallint, b.cboolean1\n    having count(a.cint) > 10\n    order by a, x, b.cboolean1 limit 10) t","edges":[{"sources":[3,4,5,6,7],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col a) ctinyint) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col a) csmallint)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col a) csmallint))))))","edgeType":"PROJECTION"},{"sources":[6],"targets":[1],"expression":"count(default.alltypesorc.cstring1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"targets":[0,1,2],"expressio
 n":"(a.cint = b.cint)","edgeType":"PREDICATE"},{"sources":[8,9],"targets":[0,1,2],"expression":"((a.cboolean2 = true) and (b.cfloat > 0.0))","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(count(default.alltypesorc.cint) > 10)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_v3.a"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_v3.x"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_v3.cboolean1"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"}]}
 PREHOOK: query: select * from dest_v3 limit 2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Input: default@dest_v3
 #### A masked pattern was here ####
-{"version":"1.0","engine":"mr","hash":"40bccc0722002f798d0548b59e369e83","queryText":"select * from dest_v3 limit 2","edges":[{"sources":[3,4,5,6,7],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col $hdt$_0) $f0) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) $f1)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col $hdt$_0) $f1)))) (tok_windowvalues (preceding 2147483647) current)))","edgeType":"PROJECTION"},{"sources":[6],"targets":[1],"expression":"count(default.alltypesorc.cstring1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[2],"edgeType":"PROJECTION"},{"sources":[8],"targets":[0,1,2],"expression":"(a.cboolean2 = true)","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(count(default.alltypesorc.cint) > 10)","edgeType":"PREDICATE"},{"sources":[9],"targets":[0,1,2],"expression":"(a.cfloat > 0.0)","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(a.cint = a.cint)","edge
 Type":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"dest_v3.a"},{"id":1,"vertexType":"COLUMN","vertexId":"dest_v3.x"},{"id":2,"vertexType":"COLUMN","vertexId":"dest_v3.cboolean1"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"}]}
+{"version":"1.0","engine":"mr","hash":"40bccc0722002f798d0548b59e369e83","queryText":"select * from dest_v3 limit 2","edges":[{"sources":[3,4,5,6,7],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col $hdt$_0) $f0) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) $f1)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col $hdt$_0) $f1)))) (tok_windowvalues (preceding 2147483647) current)))","edgeType":"PROJECTION"},{"sources":[6],"targets":[1],"expression":"count(default.alltypesorc.cstring1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[2],"edgeType":"PROJECTION"},{"sources":[8],"targets":[0,1,2],"expression":"(a.cboolean2 = true)","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(a.cint = a.cint)","edgeType":"PREDICATE"},{"sources":[9],"targets":[0,1,2],"expression":"(a.cfloat > 0.0)","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(count(default.alltypesorc.cint) > 10)","edge
 Type":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"dest_v3.a"},{"id":1,"vertexType":"COLUMN","vertexId":"dest_v3.x"},{"id":2,"vertexType":"COLUMN","vertexId":"dest_v3.cboolean1"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"}]}
 38	216	false
 38	229	true

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/test/results/clientpositive/spark/join32.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join32.q.out b/ql/src/test/results/clientpositive/spark/join32.q.out
index 4ae9dc6..1c1c103 100644
--- a/ql/src/test/results/clientpositive/spark/join32.q.out
+++ b/ql/src/test/results/clientpositive/spark/join32.q.out
@@ -423,7 +423,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
 POSTHOOK: Output: default@dest_j1
 POSTHOOK: Lineage: dest_j1.key SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: dest_j1.val2 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: dest_j1.value EXPRESSION [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: dest_j1.value SIMPLE [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: select * from dest_j1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest_j1

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out b/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out
index 78bb655..937e8fc 100644
--- a/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out
+++ b/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out
@@ -431,7 +431,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
 POSTHOOK: Output: default@dest_j1
 POSTHOOK: Lineage: dest_j1.key SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: dest_j1.val2 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: dest_j1.value EXPRESSION [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: dest_j1.value SIMPLE [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: select * from dest_j1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest_j1
@@ -1027,9 +1027,9 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Input: default@src1
 POSTHOOK: Output: default@dest_j1
-POSTHOOK: Lineage: dest_j1.key EXPRESSION [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: dest_j1.key SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: dest_j1.val2 SIMPLE [(src)w.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: dest_j1.value EXPRESSION [(src1)x.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: dest_j1.value SIMPLE [(src1)x.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: select * from dest_j1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest_j1

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/test/results/clientpositive/spark/join33.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join33.q.out b/ql/src/test/results/clientpositive/spark/join33.q.out
index 4ae9dc6..1c1c103 100644
--- a/ql/src/test/results/clientpositive/spark/join33.q.out
+++ b/ql/src/test/results/clientpositive/spark/join33.q.out
@@ -423,7 +423,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
 POSTHOOK: Output: default@dest_j1
 POSTHOOK: Lineage: dest_j1.key SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: dest_j1.val2 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: dest_j1.value EXPRESSION [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: dest_j1.value SIMPLE [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: select * from dest_j1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest_j1

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.java1.7.out b/ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.java1.7.out
index 1bfdba2..3aac389 100644
--- a/ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.java1.7.out
@@ -310,10 +310,10 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@src_4
 POSTHOOK: Output: default@src_5
-POSTHOOK: Lineage: src_4.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: src_4.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: src_5.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: src_5.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: src_4.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src_4.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: src_5.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src_5.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ]
 RUN: Stage-2:MAPRED
 RUN: Stage-1:MOVE
 RUN: Stage-0:MOVE
@@ -732,10 +732,10 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@src_4
 POSTHOOK: Output: default@src_5
-POSTHOOK: Lineage: src_4.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: src_4.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: src_5.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: src_5.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: src_4.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src_4.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: src_5.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src_5.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ]
 RUN: Stage-5:MAPRED
 RUN: Stage-2:MAPRED
 RUN: Stage-1:MOVE

http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.7.out b/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.7.out
index 3a2473f..b668694 100644
--- a/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.7.out
@@ -333,10 +333,10 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@src_4
 POSTHOOK: Output: default@src_5
-POSTHOOK: Lineage: src_4.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: src_4.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: src_5.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: src_5.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: src_4.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src_4.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: src_5.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src_5.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ]
 RUN: Stage-10:MAPRED
 RUN: Stage-2:MAPRED
 RUN: Stage-3:MAPRED
@@ -839,10 +839,10 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@src_4
 POSTHOOK: Output: default@src_5
-POSTHOOK: Lineage: src_4.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: src_4.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: src_5.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: src_5.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: src_4.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src_4.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: src_5.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src_5.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ]
 RUN: Stage-10:MAPRED
 RUN: Stage-14:CONDITIONAL
 RUN: Stage-17:MAPREDLOCAL


[06/11] hive git commit: HIVE-11646: CBO: Calcite Operator To Hive Operator (Calcite Return Path): fix multiple window spec for PTF operator (Pengcheng Xiong, reviewed by Jesus Camacho Rodriguez)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/2d3316b9/ql/src/test/results/clientpositive/cbo_rp_windowing_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_windowing_2.q.out b/ql/src/test/results/clientpositive/cbo_rp_windowing_2.q.out
new file mode 100644
index 0000000..aa34d3d
--- /dev/null
+++ b/ql/src/test/results/clientpositive/cbo_rp_windowing_2.q.out
@@ -0,0 +1,2338 @@
+PREHOOK: query: -- SORT_QUERY_RESULTS
+
+-- 1. testWindowing
+select p_mfgr, p_name, p_size,
+rank() over(distribute by p_mfgr sort by p_name) as r,
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr,
+sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1
+from part
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- SORT_QUERY_RESULTS
+
+-- 1. testWindowing
+select p_mfgr, p_name, p_size,
+rank() over(distribute by p_mfgr sort by p_name) as r,
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr,
+sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1
+from part
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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
+PREHOOK: query: -- 2. testGroupByWithPartitioning
+select p_mfgr, p_name, p_size, 
+min(p_retailprice),
+rank() over(distribute by p_mfgr sort by p_name)as r,
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr,
+p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz
+from part
+group by p_mfgr, p_name, p_size
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 2. testGroupByWithPartitioning
+select p_mfgr, p_name, p_size, 
+min(p_retailprice),
+rank() over(distribute by p_mfgr sort by p_name)as r,
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr,
+p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz
+from part
+group by p_mfgr, p_name, p_size
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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
+PREHOOK: query: -- 3. testGroupByHavingWithSWQ
+select p_mfgr, p_name, p_size, min(p_retailprice),
+rank() over(distribute by p_mfgr sort by p_name) as r,
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr,
+p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz
+from part
+group by p_mfgr, p_name, p_size
+having p_size > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 3. testGroupByHavingWithSWQ
+select p_mfgr, p_name, p_size, min(p_retailprice),
+rank() over(distribute by p_mfgr sort by p_name) as r,
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr,
+p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz
+from part
+group by p_mfgr, p_name, p_size
+having p_size > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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
+PREHOOK: query: -- 4. testCount
+select p_mfgr, p_name, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as cd 
+from part
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 4. testCount
+select p_mfgr, p_name, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as cd 
+from part
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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
+PREHOOK: query: -- 5. testCountWithWindowingUDAF
+select p_mfgr, p_name, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as cd, 
+p_retailprice, sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1, 
+p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz 
+from part
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 5. testCountWithWindowingUDAF
+select p_mfgr, p_name, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as cd, 
+p_retailprice, sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1, 
+p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz 
+from part
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	1	1	2	1173.15	1173.15	2	0
+Manufacturer#1	almond antique burnished rose metallic	1	1	2	1173.15	2346.3	2	0
+Manufacturer#1	almond antique chartreuse lavender yellow	3	2	3	1753.76	4100.06	34	32
+Manufacturer#1	almond antique salmon chartreuse burlywood	4	3	4	1602.59	5702.650000000001	6	-28
+Manufacturer#1	almond aquamarine burnished black steel	5	4	5	1414.42	7117.070000000001	28	22
+Manufacturer#1	almond aquamarine pink moccasin thistle	6	5	6	1632.66	8749.730000000001	42	14
+Manufacturer#2	almond antique violet chocolate turquoise	1	1	1	1690.68	1690.68	14	0
+Manufacturer#2	almond antique violet turquoise frosted	2	2	2	1800.7	3491.38	40	26
+Manufacturer#2	almond aquamarine midnight light salmon	3	3	3	2031.98	5523.360000000001	2	-38
+Manufacturer#2	almond aquamarine rose maroon antique	4	4	4	1698.66	7222.02	25	23
+Manufacturer#2	almond aquamarine sandy cyan gainsboro	5	5	5	1701.6	8923.62	18	-7
+Manufacturer#3	almond antique chartreuse khaki white	1	1	1	1671.68	1671.68	17	0
+Manufacturer#3	almond antique forest lavender goldenrod	2	2	2	1190.27	2861.95	14	-3
+Manufacturer#3	almond antique metallic orange dim	3	3	3	1410.39	4272.34	19	5
+Manufacturer#3	almond antique misty red olive	4	4	4	1922.98	6195.32	1	-18
+Manufacturer#3	almond antique olive coral navajo	5	5	5	1337.29	7532.61	45	44
+Manufacturer#4	almond antique gainsboro frosted violet	1	1	1	1620.67	1620.67	10	0
+Manufacturer#4	almond antique violet mint lemon	2	2	2	1375.42	2996.09	39	29
+Manufacturer#4	almond aquamarine floral ivory bisque	3	3	3	1206.26	4202.35	27	-12
+Manufacturer#4	almond aquamarine yellow dodger mint	4	4	4	1844.92	6047.27	7	-20
+Manufacturer#4	almond azure aquamarine papaya violet	5	5	5	1290.35	7337.620000000001	12	5
+Manufacturer#5	almond antique blue firebrick mint	1	1	1	1789.69	1789.69	31	0
+Manufacturer#5	almond antique medium spring khaki	2	2	2	1611.66	3401.3500000000004	6	-25
+Manufacturer#5	almond antique sky peru orange	3	3	3	1788.73	5190.08	2	-4
+Manufacturer#5	almond aquamarine dodger light gainsboro	4	4	4	1018.1	6208.18	46	44
+Manufacturer#5	almond azure blanched chiffon midnight	5	5	5	1464.48	7672.66	23	-23
+PREHOOK: query: -- 6. testCountInSubQ
+select sub1.r, sub1.dr, sub1.cd, sub1.s1, sub1.deltaSz 
+from (select p_mfgr, p_name, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as cd, 
+p_retailprice, sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1, 
+p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz 
+from part 
+) sub1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 6. testCountInSubQ
+select sub1.r, sub1.dr, sub1.cd, sub1.s1, sub1.deltaSz 
+from (select p_mfgr, p_name, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as cd, 
+p_retailprice, sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1, 
+p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz 
+from part 
+) sub1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+1	1	1	1620.67	0
+1	1	1	1671.68	0
+1	1	1	1690.68	0
+1	1	1	1789.69	0
+1	1	2	1173.15	0
+1	1	2	2346.3	0
+2	2	2	2861.95	-3
+2	2	2	2996.09	29
+2	2	2	3401.3500000000004	-25
+2	2	2	3491.38	26
+3	2	3	4100.06	32
+3	3	3	4202.35	-12
+3	3	3	4272.34	5
+3	3	3	5190.08	-4
+3	3	3	5523.360000000001	-38
+4	3	4	5702.650000000001	-28
+4	4	4	6047.27	-20
+4	4	4	6195.32	-18
+4	4	4	6208.18	44
+4	4	4	7222.02	23
+5	4	5	7117.070000000001	22
+5	5	5	7337.620000000001	5
+5	5	5	7532.61	44
+5	5	5	7672.66	-23
+5	5	5	8923.62	-7
+6	5	6	8749.730000000001	14
+PREHOOK: query: -- 7. testJoinWithWindowingAndPTF
+select abc.p_mfgr, abc.p_name, 
+rank() over(distribute by abc.p_mfgr sort by abc.p_name) as r, 
+dense_rank() over(distribute by abc.p_mfgr sort by abc.p_name) as dr, 
+abc.p_retailprice, sum(abc.p_retailprice) over (distribute by abc.p_mfgr sort by abc.p_name rows between unbounded preceding and current row) as s1, 
+abc.p_size, abc.p_size - lag(abc.p_size,1,abc.p_size) over(distribute by abc.p_mfgr sort by abc.p_name) as deltaSz 
+from noop(on part 
+partition by p_mfgr 
+order by p_name 
+) abc join part p1 on abc.p_partkey = p1.p_partkey
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 7. testJoinWithWindowingAndPTF
+select abc.p_mfgr, abc.p_name, 
+rank() over(distribute by abc.p_mfgr sort by abc.p_name) as r, 
+dense_rank() over(distribute by abc.p_mfgr sort by abc.p_name) as dr, 
+abc.p_retailprice, sum(abc.p_retailprice) over (distribute by abc.p_mfgr sort by abc.p_name rows between unbounded preceding and current row) as s1, 
+abc.p_size, abc.p_size - lag(abc.p_size,1,abc.p_size) over(distribute by abc.p_mfgr sort by abc.p_name) as deltaSz 
+from noop(on part 
+partition by p_mfgr 
+order by p_name 
+) abc join part p1 on abc.p_partkey = p1.p_partkey
+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 burnished rose metallic	1	1	1173.15	3519.4500000000003	2	0
+Manufacturer#1	almond antique burnished rose metallic	1	1	1173.15	4692.6	2	0
+Manufacturer#1	almond antique chartreuse lavender yellow	5	2	1753.76	6446.360000000001	34	32
+Manufacturer#1	almond antique salmon chartreuse burlywood	6	3	1602.59	8048.950000000001	6	-28
+Manufacturer#1	almond aquamarine burnished black steel	7	4	1414.42	9463.37	28	22
+Manufacturer#1	almond aquamarine pink moccasin thistle	8	5	1632.66	11096.03	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: -- 8. testMixedCaseAlias
+select p_mfgr, p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name, p_size desc) as R
+from part
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 8. testMixedCaseAlias
+select p_mfgr, p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name, p_size desc) as R
+from part
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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
+PREHOOK: query: -- 9. testHavingWithWindowingNoGBY
+select p_mfgr, p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row)  as s1
+from part
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 9. testHavingWithWindowingNoGBY
+select p_mfgr, p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row)  as s1
+from part
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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
+PREHOOK: query: -- 10. testHavingWithWindowingCondRankNoGBY
+select p_mfgr, p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1 
+from part
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 10. testHavingWithWindowingCondRankNoGBY
+select p_mfgr, p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1 
+from part
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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
+PREHOOK: query: -- 11. testFirstLast   
+select  p_mfgr,p_name, p_size, 
+sum(p_size) over (distribute by p_mfgr sort by p_name rows between current row and current row) as s2, 
+first_value(p_size) over w1  as f, 
+last_value(p_size, false) over w1  as l 
+from part 
+window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 11. testFirstLast   
+select  p_mfgr,p_name, p_size, 
+sum(p_size) over (distribute by p_mfgr sort by p_name rows between current row and current row) as s2, 
+first_value(p_size) over w1  as f, 
+last_value(p_size, false) over w1  as l 
+from part 
+window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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
+PREHOOK: query: -- 12. testFirstLastWithWhere
+select  p_mfgr,p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+sum(p_size) over (distribute by p_mfgr sort by p_name rows between current row and current row) as s2, 
+first_value(p_size) over w1 as f,  
+last_value(p_size, false) over w1 as l 
+from part 
+where p_mfgr = 'Manufacturer#3'  
+window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 12. testFirstLastWithWhere
+select  p_mfgr,p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+sum(p_size) over (distribute by p_mfgr sort by p_name rows between current row and current row) as s2, 
+first_value(p_size) over w1 as f,  
+last_value(p_size, false) over w1 as l 
+from part 
+where p_mfgr = 'Manufacturer#3'  
+window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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
+PREHOOK: query: -- 13. testSumWindow
+select  p_mfgr,p_name, p_size,  
+sum(p_size) over w1 as s1, 
+sum(p_size) over (distribute by p_mfgr  sort by p_name rows between current row and current row)  as s2 
+from part 
+window w1 as (distribute by p_mfgr  sort by p_name rows between 2 preceding and 2 following)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 13. testSumWindow
+select  p_mfgr,p_name, p_size,  
+sum(p_size) over w1 as s1, 
+sum(p_size) over (distribute by p_mfgr  sort by p_name rows between current row and current row)  as s2 
+from part 
+window w1 as (distribute by p_mfgr  sort by p_name rows between 2 preceding and 2 following)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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
+PREHOOK: query: -- 14. testNoSortClause
+select  p_mfgr,p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name) as r, dense_rank() over(distribute by p_mfgr sort by p_name) as dr 
+from part  
+window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 14. testNoSortClause
+select  p_mfgr,p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name) as r, dense_rank() over(distribute by p_mfgr sort by p_name) as dr 
+from part  
+window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following)
+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	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
+PREHOOK: query: -- 15. testExpressions
+select  p_mfgr,p_name, p_size,  
+rank() over(distribute by p_mfgr sort by p_name) as r,  
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+cume_dist() over(distribute by p_mfgr sort by p_name) as cud, 
+percent_rank() over(distribute by p_mfgr sort by p_name) as pr, 
+ntile(3) over(distribute by p_mfgr sort by p_name) as nt, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as ca, 
+avg(p_size) over(distribute by p_mfgr sort by p_name) as avg, 
+stddev(p_size) over(distribute by p_mfgr sort by p_name) as st, 
+first_value(p_size % 5) over(distribute by p_mfgr sort by p_name) as fv, 
+last_value(p_size) over(distribute by p_mfgr sort by p_name) as lv, 
+first_value(p_size) over w1  as fvW1
+from part 
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 15. testExpressions
+select  p_mfgr,p_name, p_size,  
+rank() over(distribute by p_mfgr sort by p_name) as r,  
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+cume_dist() over(distribute by p_mfgr sort by p_name) as cud, 
+percent_rank() over(distribute by p_mfgr sort by p_name) as pr, 
+ntile(3) over(distribute by p_mfgr sort by p_name) as nt, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as ca, 
+avg(p_size) over(distribute by p_mfgr sort by p_name) as avg, 
+stddev(p_size) over(distribute by p_mfgr sort by p_name) as st, 
+first_value(p_size % 5) over(distribute by p_mfgr sort by p_name) as fv, 
+last_value(p_size) over(distribute by p_mfgr sort by p_name) as lv, 
+first_value(p_size) over w1  as fvW1
+from part 
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	2	1	1	0	0.0	1	2	2.0	0.0	2	2	2
+Manufacturer#1	almond antique burnished rose metallic	2	1	1	0	0.0	1	2	2.0	0.0	2	2	2
+Manufacturer#1	almond antique chartreuse lavender yellow	34	3	2	0	0.4	2	3	12.666666666666666	15.084944665313014	2	34	2
+Manufacturer#1	almond antique salmon chartreuse burlywood	6	4	3	0	0.6	2	4	11.0	13.379088160259652	2	6	2
+Manufacturer#1	almond aquamarine burnished black steel	28	5	4	0	0.8	3	5	14.4	13.763720427268202	2	28	34
+Manufacturer#1	almond aquamarine pink moccasin thistle	42	6	5	1	1.0	3	6	19.0	16.237815945091466	2	42	6
+Manufacturer#2	almond antique violet chocolate turquoise	14	1	1	0	0.0	1	1	14.0	0.0	4	14	14
+Manufacturer#2	almond antique violet turquoise frosted	40	2	2	0	0.25	1	2	27.0	13.0	4	40	14
+Manufacturer#2	almond aquamarine midnight light salmon	2	3	3	0	0.5	2	3	18.666666666666668	15.86050300449376	4	2	14
+Manufacturer#2	almond aquamarine rose maroon antique	25	4	4	0	0.75	2	4	20.25	14.00669482783144	4	25	40
+Manufacturer#2	almond aquamarine sandy cyan gainsboro	18	5	5	1	1.0	3	5	19.8	12.560254774486067	4	18	2
+Manufacturer#3	almond antique chartreuse khaki white	17	1	1	0	0.0	1	1	17.0	0.0	2	17	17
+Manufacturer#3	almond antique forest lavender goldenrod	14	2	2	0	0.25	1	2	15.5	1.5	2	14	17
+Manufacturer#3	almond antique metallic orange dim	19	3	3	0	0.5	2	3	16.666666666666668	2.0548046676563256	2	19	17
+Manufacturer#3	almond antique misty red olive	1	4	4	0	0.75	2	4	12.75	7.013380069552769	2	1	14
+Manufacturer#3	almond antique olive coral navajo	45	5	5	1	1.0	3	5	19.2	14.344336861632886	2	45	19
+Manufacturer#4	almond antique gainsboro frosted violet	10	1	1	0	0.0	1	1	10.0	0.0	0	10	10
+Manufacturer#4	almond antique violet mint lemon	39	2	2	0	0.25	1	2	24.5	14.5	0	39	10
+Manufacturer#4	almond aquamarine floral ivory bisque	27	3	3	0	0.5	2	3	25.333333333333332	11.897712198383164	0	27	10
+Manufacturer#4	almond aquamarine yellow dodger mint	7	4	4	0	0.75	2	4	20.75	13.007209539328564	0	7	39
+Manufacturer#4	almond azure aquamarine papaya violet	12	5	5	1	1.0	3	5	19.0	12.149074038789951	0	12	27
+Manufacturer#5	almond antique blue firebrick mint	31	1	1	0	0.0	1	1	31.0	0.0	1	31	31
+Manufacturer#5	almond antique medium spring khaki	6	2	2	0	0.25	1	2	18.5	12.5	1	6	31
+Manufacturer#5	almond antique sky peru orange	2	3	3	0	0.5	2	3	13.0	12.832251036613439	1	2	31
+Manufacturer#5	almond aquamarine dodger light gainsboro	46	4	4	0	0.75	2	4	21.25	18.102140757380052	1	46	6
+Manufacturer#5	almond azure blanched chiffon midnight	23	5	5	1	1.0	3	5	21.6	16.206171663906314	1	23	2
+PREHOOK: query: -- 16. testMultipleWindows
+select  p_mfgr,p_name, p_size,  
+  rank() over(distribute by p_mfgr sort by p_name) as r, 
+  dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+cume_dist() over(distribute by p_mfgr sort by p_name) as cud, 
+sum(p_size) over (distribute by p_mfgr sort by p_name range between unbounded preceding and current row) as s1, 
+sum(p_size) over (distribute by p_mfgr sort by p_size range between 5 preceding and current row) as s2, 
+first_value(p_size) over w1  as fv1
+from part 
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 16. testMultipleWindows
+select  p_mfgr,p_name, p_size,  
+  rank() over(distribute by p_mfgr sort by p_name) as r, 
+  dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+cume_dist() over(distribute by p_mfgr sort by p_name) as cud, 
+sum(p_size) over (distribute by p_mfgr sort by p_name range between unbounded preceding and current row) as s1, 
+sum(p_size) over (distribute by p_mfgr sort by p_size range between 5 preceding and current row) as s2, 
+first_value(p_size) over w1  as fv1
+from part 
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	2	1	1	0	4	4	2
+Manufacturer#1	almond antique burnished rose metallic	2	1	1	0	4	4	2
+Manufacturer#1	almond antique chartreuse lavender yellow	34	3	2	0	38	34	2
+Manufacturer#1	almond antique salmon chartreuse burlywood	6	4	3	0	44	10	2
+Manufacturer#1	almond aquamarine burnished black steel	28	5	4	0	72	28	34
+Manufacturer#1	almond aquamarine pink moccasin thistle	42	6	5	1	114	42	6
+Manufacturer#2	almond antique violet chocolate turquoise	14	1	1	0	14	14	14
+Manufacturer#2	almond antique violet turquoise frosted	40	2	2	0	54	40	14
+Manufacturer#2	almond aquamarine midnight light salmon	2	3	3	0	56	2	14
+Manufacturer#2	almond aquamarine rose maroon antique	25	4	4	0	81	25	40
+Manufacturer#2	almond aquamarine sandy cyan gainsboro	18	5	5	1	99	32	2
+Manufacturer#3	almond antique chartreuse khaki white	17	1	1	0	17	31	17
+Manufacturer#3	almond antique forest lavender goldenrod	14	2	2	0	31	14	17
+Manufacturer#3	almond antique metallic orange dim	19	3	3	0	50	50	17
+Manufacturer#3	almond antique misty red olive	1	4	4	0	51	1	14
+Manufacturer#3	almond antique olive coral navajo	45	5	5	1	96	45	19
+Manufacturer#4	almond antique gainsboro frosted violet	10	1	1	0	10	17	10
+Manufacturer#4	almond antique violet mint lemon	39	2	2	0	49	39	10
+Manufacturer#4	almond aquamarine floral ivory bisque	27	3	3	0	76	27	10
+Manufacturer#4	almond aquamarine yellow dodger mint	7	4	4	0	83	7	39
+Manufacturer#4	almond azure aquamarine papaya violet	12	5	5	1	95	29	27
+Manufacturer#5	almond antique blue firebrick mint	31	1	1	0	31	31	31
+Manufacturer#5	almond antique medium spring khaki	6	2	2	0	37	8	31
+Manufacturer#5	almond antique sky peru orange	2	3	3	0	39	2	31
+Manufacturer#5	almond aquamarine dodger light gainsboro	46	4	4	0	85	46	6
+Manufacturer#5	almond azure blanched chiffon midnight	23	5	5	1	108	23	2
+PREHOOK: query: -- 17. testCountStar
+select  p_mfgr,p_name, p_size,
+count(*) over(distribute by p_mfgr sort by p_name ) as c, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as ca, 
+first_value(p_size) over w1  as fvW1
+from part 
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 17. testCountStar
+select  p_mfgr,p_name, p_size,
+count(*) over(distribute by p_mfgr sort by p_name ) as c, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as ca, 
+first_value(p_size) over w1  as fvW1
+from part 
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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
+PREHOOK: query: -- 18. testUDAFs
+select  p_mfgr,p_name, p_size, 
+sum(p_retailprice) over w1 as s, 
+min(p_retailprice) over w1 as mi,
+max(p_retailprice) over w1 as ma,
+avg(p_retailprice) over w1 as ag
+from part
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 18. testUDAFs
+select  p_mfgr,p_name, p_size, 
+sum(p_retailprice) over w1 as s, 
+min(p_retailprice) over w1 as mi,
+max(p_retailprice) over w1 as ma,
+avg(p_retailprice) over w1 as ag
+from part
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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.580000000002	1173.15	1753.76	1515.3160000000003
+Manufacturer#1	almond aquamarine burnished black steel	28	6403.430000000001	1414.42	1753.76	1600.8575000000003
+Manufacturer#1	almond aquamarine pink moccasin thistle	42	4649.670000000001	1414.42	1632.66	1549.8900000000003
+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.240000000001	1698.66	2031.98	1810.746666666667
+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.969999999999	1018.1	1788.73	1470.7424999999998
+Manufacturer#5	almond azure blanched chiffon midnight	23	4271.3099999999995	1018.1	1788.73	1423.7699999999998
+PREHOOK: query: -- 19. testUDAFsWithGBY
+select  p_mfgr,p_name, p_size, p_retailprice, 
+sum(p_retailprice) over w1 as s, 
+min(p_retailprice) as mi ,
+max(p_retailprice) as ma ,
+avg(p_retailprice) over w1 as ag
+from part
+group by p_mfgr,p_name, p_size, p_retailprice
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 19. testUDAFsWithGBY
+select  p_mfgr,p_name, p_size, p_retailprice, 
+sum(p_retailprice) over w1 as s, 
+min(p_retailprice) as mi ,
+max(p_retailprice) as ma ,
+avg(p_retailprice) over w1 as ag
+from part
+group by p_mfgr,p_name, p_size, p_retailprice
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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.240000000001	1701.6	1701.6	1810.746666666667
+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.969999999999	1018.1	1018.1	1470.7424999999998
+Manufacturer#5	almond azure blanched chiffon midnight	23	1464.48	4271.3099999999995	1464.48	1464.48	1423.7699999999998
+PREHOOK: query: -- 20. testSTATs
+select  p_mfgr,p_name, p_size, 
+stddev(p_retailprice) over w1 as sdev, 
+stddev_pop(p_retailprice) over w1 as sdev_pop, 
+collect_set(p_size) over w1 as uniq_size, 
+variance(p_retailprice) over w1 as var,
+corr(p_size, p_retailprice) over w1 as cor,
+covar_pop(p_size, p_retailprice) over w1 as covarp
+from part
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 20. testSTATs
+select  p_mfgr,p_name, p_size, 
+stddev(p_retailprice) over w1 as sdev, 
+stddev_pop(p_retailprice) over w1 as sdev_pop, 
+collect_set(p_size) over w1 as uniq_size, 
+variance(p_retailprice) over w1 as var,
+corr(p_size, p_retailprice) over w1 as cor,
+covar_pop(p_size, p_retailprice) over w1 as covarp
+from part
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	2	258.10677784349235	258.10677784349235	[2,34,6]	66619.10876874991	0.811328754177887	2801.7074999999995
+Manufacturer#1	almond antique burnished rose metallic	2	273.70217881648074	273.70217881648074	[2,34]	74912.8826888888	1.0	4128.782222222221
+Manufacturer#1	almond antique chartreuse lavender yellow	34	230.90151585470358	230.90151585470358	[2,34,6,28]	53315.51002399992	0.695639377397664	2210.7864
+Manufacturer#1	almond antique salmon chartreuse burlywood	6	202.73109328368946	202.73109328368946	[2,34,6,28,42]	41099.896184	0.630785977101214	2009.9536000000007
+Manufacturer#1	almond aquamarine burnished black steel	28	121.6064517973862	121.6064517973862	[34,6,28,42]	14788.129118750014	0.2036684720435979	331.1337500000004
+Manufacturer#1	almond aquamarine pink moccasin thistle	42	96.5751586416853	96.5751586416853	[6,28,42]	9326.761266666683	-1.4442181184933883E-4	-0.20666666666708502
+Manufacturer#2	almond antique violet chocolate turquoise	14	142.2363169751898	142.2363169751898	[14,40,2]	20231.169866666663	-0.49369526554523185	-1113.7466666666658
+Manufacturer#2	almond antique violet turquoise frosted	40	137.76306498840682	137.76306498840682	[14,40,2,25]	18978.662075	-0.5205630897335946	-1004.4812499999995
+Manufacturer#2	almond aquamarine midnight light salmon	2	130.03972279269132	130.03972279269132	[14,40,2,25,18]	16910.329504000005	-0.46908967495720255	-766.1791999999995
+Manufacturer#2	almond aquamarine rose maroon antique	25	135.55100986344584	135.55100986344584	[40,2,25,18]	18374.07627499999	-0.6091405874714462	-1128.1787499999987
+Manufacturer#2	almond aquamarine sandy cyan gainsboro	18	156.44019460768044	156.44019460768044	[2,25,18]	24473.534488888927	-0.9571686373491608	-1441.4466666666676
+Manufacturer#3	almond antique chartreuse khaki white	17	196.7742266885805	196.7742266885805	[17,14,19]	38720.09628888887	0.5557168646224995	224.6944444444446
+Manufacturer#3	almond antique forest lavender goldenrod	14	275.14144189852607	275.14144189852607	[17,14,19,1]	75702.81305	-0.6720833036576083	-1296.9000000000003
+Manufacturer#3	almond antique metallic orange dim	19	260.23473614412046	260.23473614412046	[17,14,19,1,45]	67722.117896	-0.5703526513979519	-2129.0664
+Manufacturer#3	almond antique misty red olive	1	275.9139962356932	275.9139962356932	[14,19,1,45]	76128.53331875012	-0.577476899644802	-2547.7868749999993
+Manufacturer#3	almond antique olive coral navajo	45	260.5815918713796	260.5815918713796	[19,1,45]	67902.76602222225	-0.8710736366736884	-4099.731111111111
+Manufacturer#4	almond antique gainsboro frosted violet	10	170.13011889596618	170.13011889596618	[10,39,27]	28944.25735555559	-0.6656975320098423	-1347.4777777777779
+Manufacturer#4	almond antique violet mint lemon	39	242.26834609323197	242.26834609323197	[10,39,27,7]	58693.95151875002	-0.8051852719193339	-2537.328125
+Manufacturer#4	almond aquamarine floral ivory bisque	27	234.10001662537326	234.10001662537326	[10,39,27,7,12]	54802.817784000035	-0.6046935574240581	-1719.8079999999995
+Manufacturer#4	almond aquamarine yellow dodger mint	7	247.3342714197732	247.3342714197732	[39,27,7,12]	61174.24181875003	-0.5508665654707869	-1719.0368749999975
+Manufacturer#4	almond azure aquamarine papaya violet	12	283.3344330566893	283.3344330566893	[27,7,12]	80278.40095555557	-0.7755740084632333	-1867.4888888888881
+Manufacturer#5	almond antique blue firebrick mint	31	83.69879024746363	83.69879024746363	[31,6,2]	7005.487488888913	0.39004303087285047	418.9233333333353
+Manufacturer#5	almond antique medium spring khaki	6	316.68049612345885	316.68049612345885	[31,6,2,46]	100286.53662500004	-0.713612911776183	-4090.853749999999
+Manufacturer#5	almond antique sky peru orange	2	285.40506298242155	285.40506298242155	[31,6,2,46,23]	81456.04997600002	-0.712858514567818	-3297.2011999999986
+Manufacturer#5	almond aquamarine dodger light gainsboro	46	285.43749038756283	285.43749038756283	[6,2,46,23]	81474.56091875004	-0.984128787153391	-4871.028125000002
+Manufacturer#5	almond azure blanched chiffon midnight	23	315.9225931564038	315.9225931564038	[2,46,23]	99807.08486666664	-0.9978877469246936	-5664.856666666666
+PREHOOK: query: -- 21. testDISTs
+select  p_mfgr,p_name, p_size, 
+histogram_numeric(p_retailprice, 5) over w1 as hist, 
+percentile(p_partkey, 0.5) over w1 as per,
+row_number() over(distribute by p_mfgr sort by p_mfgr, p_name) as rn
+from part
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 21. testDISTs
+select  p_mfgr,p_name, p_size, 
+histogram_numeric(p_retailprice, 5) over w1 as hist, 
+percentile(p_partkey, 0.5) over w1 as per,
+row_number() over(distribute by p_mfgr sort by p_mfgr, p_name) as rn
+from part
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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 burnished rose metallic	2	[{"x":1173.15,"y":2.0},{"x":1753.76,"y":1.0}]	121152.0	1
+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
+PREHOOK: query: -- 22. testViewAsTableInputWithWindowing
+create view IF NOT EXISTS mfgr_price_view as 
+select p_mfgr, p_brand, 
+round(sum(p_retailprice),2) as s 
+from part 
+group by p_mfgr, p_brand
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: default@part
+PREHOOK: Output: database:default
+PREHOOK: Output: default@mfgr_price_view
+POSTHOOK: query: -- 22. testViewAsTableInputWithWindowing
+create view IF NOT EXISTS mfgr_price_view as 
+select p_mfgr, p_brand, 
+round(sum(p_retailprice),2) as s 
+from part 
+group by p_mfgr, p_brand
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: default@part
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@mfgr_price_view
+PREHOOK: query: select * 
+from (
+select p_mfgr, p_brand, s, 
+round(sum(s) over w1 , 2)  as s1
+from mfgr_price_view 
+window w1 as (distribute by p_mfgr sort by p_mfgr )
+) sq
+order by p_mfgr, p_brand
+PREHOOK: type: QUERY
+PREHOOK: Input: default@mfgr_price_view
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: select * 
+from (
+select p_mfgr, p_brand, s, 
+round(sum(s) over w1 , 2)  as s1
+from mfgr_price_view 
+window w1 as (distribute by p_mfgr sort by p_mfgr )
+) sq
+order by p_mfgr, p_brand
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@mfgr_price_view
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	Brand#12	4800.84	8749.73
+Manufacturer#1	Brand#14	2346.3	8749.73
+Manufacturer#1	Brand#15	1602.59	8749.73
+Manufacturer#2	Brand#22	3491.38	8923.62
+Manufacturer#2	Brand#23	2031.98	8923.62
+Manufacturer#2	Brand#24	1698.66	8923.62
+Manufacturer#2	Brand#25	1701.6	8923.62
+Manufacturer#3	Brand#31	1671.68	7532.61
+Manufacturer#3	Brand#32	3333.37	7532.61
+Manufacturer#3	Brand#34	1337.29	7532.61
+Manufacturer#3	Brand#35	1190.27	7532.61
+Manufacturer#4	Brand#41	4755.94	7337.62
+Manufacturer#4	Brand#42	2581.68	7337.62
+Manufacturer#5	Brand#51	1611.66	7672.66
+Manufacturer#5	Brand#52	3254.17	7672.66
+Manufacturer#5	Brand#53	2806.83	7672.66
+PREHOOK: query: select p_mfgr, p_brand, s, 
+round(sum(s) over w1 ,2)  as s1
+from mfgr_price_view 
+window w1 as (distribute by p_mfgr sort by p_brand rows between 2 preceding and current row)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@mfgr_price_view
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: select p_mfgr, p_brand, s, 
+round(sum(s) over w1 ,2)  as s1
+from mfgr_price_view 
+window w1 as (distribute by p_mfgr sort by p_brand rows between 2 preceding and current row)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@mfgr_price_view
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	Brand#12	4800.84	4800.84
+Manufacturer#1	Brand#14	2346.3	7147.14
+Manufacturer#1	Brand#15	1602.59	8749.73
+Manufacturer#2	Brand#22	3491.38	3491.38
+Manufacturer#2	Brand#23	2031.98	5523.36
+Manufacturer#2	Brand#24	1698.66	7222.02
+Manufacturer#2	Brand#25	1701.6	5432.24
+Manufacturer#3	Brand#31	1671.68	1671.68
+Manufacturer#3	Brand#32	3333.37	5005.05
+Manufacturer#3	Brand#34	1337.29	6342.34
+Manufacturer#3	Brand#35	1190.27	5860.93
+Manufacturer#4	Brand#41	4755.94	4755.94
+Manufacturer#4	Brand#42	2581.68	7337.62
+Manufacturer#5	Brand#51	1611.66	1611.66
+Manufacturer#5	Brand#52	3254.17	4865.83
+Manufacturer#5	Brand#53	2806.83	7672.66
+PREHOOK: query: -- 23. testCreateViewWithWindowingQuery
+create view IF NOT EXISTS mfgr_brand_price_view as 
+select p_mfgr, p_brand, 
+sum(p_retailprice) over w1  as s
+from part 
+window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and current row)
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: default@part
+PREHOOK: Output: database:default
+PREHOOK: Output: default@mfgr_brand_price_view
+POSTHOOK: query: -- 23. testCreateViewWithWindowingQuery
+create view IF NOT EXISTS mfgr_brand_price_view as 
+select p_mfgr, p_brand, 
+sum(p_retailprice) over w1  as s
+from part 
+window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and current row)
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: default@part
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@mfgr_brand_price_view
+PREHOOK: query: select * from mfgr_brand_price_view
+PREHOOK: type: QUERY
+PREHOOK: Input: default@mfgr_brand_price_view
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: select * from mfgr_brand_price_view
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@mfgr_brand_price_view
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	Brand#12	4100.06
+Manufacturer#1	Brand#12	4649.670000000001
+Manufacturer#1	Brand#12	4770.77
+Manufacturer#1	Brand#14	1173.15
+Manufacturer#1	Brand#14	2346.3
+Manufacturer#1	Brand#15	4529.5
+Manufacturer#2	Brand#22	1690.68
+Manufacturer#2	Brand#22	3491.38
+Manufacturer#2	Brand#23	5523.360000000001
+Manufacturer#2	Brand#24	5531.34
+Manufacturer#2	Brand#25	5432.240000000001
+Manufacturer#3	Brand#31	1671.68
+Manufacturer#3	Brand#32	4272.34
+Manufacturer#3	Brand#32	4523.639999999999
+Manufacturer#3	Brand#34	4670.66
+Manufacturer#3	Brand#35	2861.95
+Manufacturer#4	Brand#41	1620.67
+Manufacturer#4	Brand#41	4341.530000000001
+Manufacturer#4	Brand#41	4426.6
+Manufacturer#4	Brand#42	2996.09
+Manufacturer#4	Brand#42	4202.35
+Manufacturer#5	Brand#51	3401.3500000000004
+Manufacturer#5	Brand#52	1789.69
+Manufacturer#5	Brand#52	4271.3099999999995
+Manufacturer#5	Brand#53	4418.49
+Manufacturer#5	Brand#53	5190.08
+PREHOOK: query: -- 24. testLateralViews
+select p_mfgr, p_name, 
+lv_col, p_size, sum(p_size) over w1   as s
+from (select p_mfgr, p_name, p_size, array(1,2,3) arr from part) p 
+lateral view explode(arr) part_lv as lv_col
+window w1 as (distribute by p_mfgr sort by p_size, lv_col rows between 2 preceding and current row)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: -- 24. testLateralViews
+select p_mfgr, p_name, 
+lv_col, p_size, sum(p_size) over w1   as s
+from (select p_mfgr, p_name, p_size, array(1,2,3) arr from part) p 
+lateral view explode(arr) part_lv as lv_col
+window w1 as (distribute by p_mfgr sort by p_size, lv_col rows between 2 preceding and current row)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+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 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 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 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 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 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#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 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 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#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 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 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 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 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 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 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#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 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 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#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 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 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 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
+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
+PREHOOK: query: -- 25. testMultipleInserts3SWQs
+CREATE TABLE part_1( 
+p_mfgr STRING, 
+p_name STRING, 
+p_size INT, 
+r INT, 
+dr INT, 
+s DOUBLE)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@part_1
+POSTHOOK: query: -- 25. testMultipleInserts3SWQs
+CREATE TABLE part_1( 
+p_mfgr STRING, 
+p_name STRING, 
+p_size INT, 
+r INT, 
+dr INT, 
+s DOUBLE)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@part_1
+PREHOOK: query: CREATE TABLE part_2( 
+p_mfgr STRING, 
+p_name STRING, 
+p_size INT, 
+r INT, 
+dr INT, 
+cud INT,  
+s2 DOUBLE, 
+fv1 INT)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@part_2
+POSTHOOK: query: CREATE TABLE part_2( 
+p_mfgr STRING, 
+p_name STRING, 
+p_size INT, 
+r INT, 
+dr INT, 
+cud INT,  
+s2 DOUBLE, 
+fv1 INT)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@part_2
+PREHOOK: query: CREATE TABLE part_3( 
+p_mfgr STRING, 
+p_name STRING, 
+p_size INT, 
+c INT, 
+ca INT, 
+fv INT)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@part_3
+POSTHOOK: query: CREATE TABLE part_3( 
+p_mfgr STRING, 
+p_name STRING, 
+p_size INT, 
+c INT, 
+ca INT, 
+fv INT)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@part_3
+PREHOOK: query: from part 
+INSERT OVERWRITE TABLE part_1 
+select p_mfgr, p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name ) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name ) as dr, 
+sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row)  as s
+INSERT OVERWRITE TABLE part_2 
+select  p_mfgr,p_name, p_size,  
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+cume_dist() over(distribute by p_mfgr sort by p_name) as cud, 
+round(sum(p_size) over (distribute by p_mfgr sort by p_size range between 5 preceding and current row),1) as s2, 
+first_value(p_size) over w1  as fv1
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following) 
+INSERT OVERWRITE TABLE part_3 
+select  p_mfgr,p_name, p_size,  
+count(*) over(distribute by p_mfgr sort by p_name) as c, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as ca, 
+first_value(p_size) over w1  as fv
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+PREHOOK: Output: default@part_1
+PREHOOK: Output: default@part_2
+PREHOOK: Output: default@part_3
+POSTHOOK: query: from part 
+INSERT OVERWRITE TABLE part_1 
+select p_mfgr, p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name ) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name ) as dr, 
+sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row)  as s
+INSERT OVERWRITE TABLE part_2 
+select  p_mfgr,p_name, p_size,  
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+cume_dist() over(distribute by p_mfgr sort by p_name) as cud, 
+round(sum(p_size) over (distribute by p_mfgr sort by p_size range between 5 preceding and current row),1) as s2, 
+first_value(p_size) over w1  as fv1
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following) 
+INSERT OVERWRITE TABLE part_3 
+select  p_mfgr,p_name, p_size,  
+count(*) over(distribute by p_mfgr sort by p_name) as c, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as ca, 
+first_value(p_size) over w1  as fv
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+POSTHOOK: Output: default@part_1
+POSTHOOK: Output: default@part_2
+POSTHOOK: Output: default@part_3
+POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ]
+POSTHOOK: Lineage: part_1.p_mfgr SIMPLE [(part)part.FieldSchema(name:p_mfgr, type:string, comment:null), ]
+POSTHOOK: Lineage: part_1.p_name SIMPLE [(part)part.FieldSchema(name:p_name, type:string, comment:null), ]
+POSTHOOK: Lineage: part_1.p_size SIMPLE [(part)part.FieldSchema(name:p_size, type:int, comment:null), ]
+POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ]
+POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ]
+POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ]
+POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ]
+POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ]
+POSTHOOK: Lineage: part_2.p_mfgr SIMPLE [(part)part.FieldSchema(name:p_mfgr, type:string, comment:null), ]
+POSTHOOK: Lineage: part_2.p_name SIMPLE [(part)part.FieldSchema(name:p_name, type:string, comment:null), ]
+POSTHOOK: Lineage: part_2.p_size SIMPLE [(part)part.FieldSchema(name:p_size, type:int, comment:null), ]
+POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ]
+POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ]
+POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ]
+POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ]
+POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ]
+POSTHOOK: Lineage: part_3.p_mfgr SIMPLE [(part)part.FieldSchema(name:p_mfgr, type:string, comment:null), ]
+POSTHOOK: Lineage: part_3.p_name SIMPLE [(part)part.FieldSchema(name:p_name, type:string, comment:null), ]
+POSTHOOK: Lineage: part_3.p_size SIMPLE [(part)part.FieldSchema(name:p_size, type:int, comment:null), ]
+PREHOOK: query: select * from part_1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_1
+#### A masked pattern was here ####
+POSTHOOK: query: select * from part_1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_1
+#### A masked pattern was here ####
+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
+PREHOOK: query: select * from part_2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_2
+#### A masked pattern was here ####
+POSTHOOK: query: select * from part_2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_2
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	2	1	1	0	4.0	2
+Manufacturer#1	almond antique burnished rose metallic	2	1	1	0	4.0	2
+Manufacturer#1	almond antique chartreuse lavender yellow	34	3	2	0	34.0	2
+Manufacturer#1	almond antique salmon chartreuse burlywood	6	4	3	0	10.0	2
+Manufacturer#1	almond aquamarine burnished black steel	28	5	4	0	28.0	34
+Manufacturer#1	almond aquamarine pink moccasin thistle	42	6	5	1	42.0	6
+Manufacturer#2	almond antique violet chocolate turquoise	14	1	1	0	14.0	14
+Manufacturer#2	almond antique violet turquoise frosted	40	2	2	0	40.0	14
+Manufacturer#2	almond aquamarine midnight light salmon	2	3	3	0	2.0	14
+Manufacturer#2	almond aquamarine rose maroon antique	25	4	4	0	25.0	40
+Manufacturer#2	almond aquamarine sandy cyan gainsboro	18	5	5	1	32.0	2
+Manufacturer#3	almond antique chartreuse khaki white	17	1	1	0	31.0	17
+Manufacturer#3	almond antique forest 

<TRUNCATED>

[08/11] hive git commit: HIVE-11617: Explain plan for multiple lateral views is very slow (Aihua Xu, reviewed by Jesus Camacho Rodriguez)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/7281a460/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_2.q.out b/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
index 71d8f41..566b451 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
@@ -58,10 +58,10 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
 POSTHOOK: Output: default@ss
 POSTHOOK: Lineage: ss.k1 SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: ss.k2 SIMPLE [(src)y.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: ss.k3 EXPRESSION [(srcpart)z.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: ss.k3 SIMPLE [(srcpart)z.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: ss.v1 SIMPLE [(src1)x.FieldSchema(name:value, type:string, comment:default), ]
 POSTHOOK: Lineage: ss.v2 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: ss.v3 EXPRESSION [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: ss.v3 SIMPLE [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: INSERT OVERWRITE TABLE sr
 SELECT x.key,x.value,y.key,y.value,z.key,z.value
 FROM src1 x 
@@ -86,10 +86,10 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
 POSTHOOK: Output: default@sr
 POSTHOOK: Lineage: sr.k1 SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: sr.k2 SIMPLE [(src)y.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: sr.k3 EXPRESSION [(srcpart)z.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: sr.k3 SIMPLE [(srcpart)z.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: sr.v1 SIMPLE [(src1)x.FieldSchema(name:value, type:string, comment:default), ]
 POSTHOOK: Lineage: sr.v2 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: sr.v3 EXPRESSION [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: sr.v3 SIMPLE [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: INSERT OVERWRITE TABLE cs
 SELECT x.key,x.value,y.key,y.value,z.key,z.value
 FROM src1 x 
@@ -115,10 +115,10 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
 POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
 POSTHOOK: Output: default@cs
 POSTHOOK: Lineage: cs.k1 SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: cs.k2 EXPRESSION [(src)y.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: cs.k2 SIMPLE [(src)y.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: cs.k3 SIMPLE [(srcpart)z.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: cs.v1 SIMPLE [(src1)x.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: cs.v2 EXPRESSION [(src)y.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: cs.v2 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ]
 POSTHOOK: Lineage: cs.v3 SIMPLE [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: ANALYZE TABLE ss COMPUTE STATISTICS
 PREHOOK: type: QUERY


[03/11] hive git commit: HIVE-11593 Add aes_encrypt and aes_decrypt UDFs (Alexander Pivovarov, reviewed by Jason Dere)

Posted by se...@apache.org.
HIVE-11593 Add aes_encrypt and aes_decrypt UDFs (Alexander Pivovarov, reviewed by Jason Dere)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f4acb44f
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f4acb44f
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f4acb44f

Branch: refs/heads/llap
Commit: f4acb44f67b64e85f53de6d8503ef2ae6e0fa524
Parents: 8bed378
Author: Alexander Pivovarov <ap...@gmail.com>
Authored: Sun Aug 16 18:36:32 2015 -0700
Committer: Alexander Pivovarov <ap...@gmail.com>
Committed: Sat Sep 5 19:24:50 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   2 +
 .../hive/ql/udf/generic/GenericUDFAesBase.java  | 205 ++++++++++++++++
 .../ql/udf/generic/GenericUDFAesDecrypt.java    |  50 ++++
 .../ql/udf/generic/GenericUDFAesEncrypt.java    |  50 ++++
 .../ql/udf/generic/GenericUDFParamUtils.java    |   8 +-
 .../udf/generic/TestGenericUDFAesDecrypt.java   | 233 +++++++++++++++++++
 .../udf/generic/TestGenericUDFAesEncrypt.java   | 228 ++++++++++++++++++
 .../queries/clientpositive/udf_aes_decrypt.q    |  21 ++
 .../queries/clientpositive/udf_aes_encrypt.q    |  21 ++
 .../results/clientpositive/show_functions.q.out |   2 +
 .../clientpositive/udf_aes_decrypt.q.out        |  79 +++++++
 .../clientpositive/udf_aes_encrypt.q.out        |  79 +++++++
 12 files changed, 977 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f4acb44f/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index 4c1c53e..f1fe30d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -234,6 +234,8 @@ public final class FunctionRegistry {
     system.registerUDF("md5", UDFMd5.class, false);
     system.registerUDF("sha1", UDFSha1.class, false);
     system.registerUDF("sha", UDFSha1.class, false);
+    system.registerGenericUDF("aes_encrypt", GenericUDFAesEncrypt.class);
+    system.registerGenericUDF("aes_decrypt", GenericUDFAesDecrypt.class);
 
     system.registerGenericUDF("encode", GenericUDFEncode.class);
     system.registerGenericUDF("decode", GenericUDFDecode.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/f4acb44f/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAesBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAesBase.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAesBase.java
new file mode 100644
index 0000000..66a4457
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAesBase.java
@@ -0,0 +1,205 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import static org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveGrouping.BINARY_GROUP;
+import static org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveGrouping.STRING_GROUP;
+
+import java.security.GeneralSecurityException;
+import java.security.NoSuchAlgorithmException;
+
+import javax.crypto.Cipher;
+import javax.crypto.NoSuchPaddingException;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.SecretKeySpec;
+
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * GenericUDFAesBase.
+ *
+ */
+public abstract class GenericUDFAesBase extends GenericUDF {
+  protected transient Converter[] converters = new Converter[2];
+  protected transient PrimitiveCategory[] inputTypes = new PrimitiveCategory[2];
+  protected final BytesWritable output = new BytesWritable();
+  protected transient boolean isStr0;
+  protected transient boolean isStr1;
+  protected transient boolean isKeyConstant;
+  protected transient Cipher cipher;
+  protected transient SecretKey secretKey;
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+    checkArgsSize(arguments, 2, 2);
+
+    checkArgPrimitive(arguments, 0);
+    checkArgPrimitive(arguments, 1);
+
+    // the function should support both string and binary input types
+    if (canParam0BeStr()) {
+      checkArgGroups(arguments, 0, inputTypes, STRING_GROUP, BINARY_GROUP);
+    } else {
+      checkArgGroups(arguments, 0, inputTypes, BINARY_GROUP);
+    }
+    checkArgGroups(arguments, 1, inputTypes, STRING_GROUP, BINARY_GROUP);
+
+    if (isStr0 = PrimitiveObjectInspectorUtils.getPrimitiveGrouping(inputTypes[0]) == STRING_GROUP) {
+      obtainStringConverter(arguments, 0, inputTypes, converters);
+    } else {
+      GenericUDFParamUtils.obtainBinaryConverter(arguments, 0, inputTypes, converters);
+    }
+
+    isKeyConstant = arguments[1] instanceof ConstantObjectInspector;
+    byte[] key = null;
+    int keyLength = 0;
+
+    if (isStr1 = PrimitiveObjectInspectorUtils.getPrimitiveGrouping(inputTypes[1]) == STRING_GROUP) {
+      if (isKeyConstant) {
+        String keyStr = getConstantStringValue(arguments, 1);
+        if (keyStr != null) {
+          key = keyStr.getBytes();
+          keyLength = key.length;
+        }
+      } else {
+        obtainStringConverter(arguments, 1, inputTypes, converters);
+      }
+    } else {
+      if (isKeyConstant) {
+        BytesWritable keyWr = GenericUDFParamUtils.getConstantBytesValue(arguments, 1);
+        if (keyWr != null) {
+          key = keyWr.getBytes();
+          keyLength = keyWr.getLength();
+        }
+      } else {
+        GenericUDFParamUtils.obtainBinaryConverter(arguments, 1, inputTypes, converters);
+      }
+    }
+
+    if (key != null) {
+      secretKey = getSecretKey(key, keyLength);
+    }
+
+    try {
+      cipher = Cipher.getInstance("AES");
+    } catch (NoSuchPaddingException | NoSuchAlgorithmException e) {
+      throw new RuntimeException(e);
+    }
+
+    ObjectInspector outputOI = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    return outputOI;
+  }
+
+  @Override
+  public Object evaluate(DeferredObject[] arguments) throws HiveException {
+    byte[] input;
+    int inputLength;
+
+    if (isStr0) {
+      Text n = GenericUDFParamUtils.getTextValue(arguments, 0, converters);
+      if (n == null) {
+        return null;
+      }
+      input = n.getBytes();
+      inputLength = n.getLength();
+    } else {
+      BytesWritable bWr = GenericUDFParamUtils.getBinaryValue(arguments, 0, converters);
+      if (bWr == null) {
+        return null;
+      }
+      input = bWr.getBytes();
+      inputLength = bWr.getLength();
+    }
+
+    if (input == null) {
+      return null;
+    }
+
+    SecretKey secretKey;
+    if (isKeyConstant) {
+      secretKey = this.secretKey;
+    } else {
+      byte[] key;
+      int keyLength;
+      if (isStr1) {
+        Text n = GenericUDFParamUtils.getTextValue(arguments, 1, converters);
+        if (n == null) {
+          return null;
+        }
+        key = n.getBytes();
+        keyLength = n.getLength();
+      } else {
+        BytesWritable bWr = GenericUDFParamUtils.getBinaryValue(arguments, 1, converters);
+        if (bWr == null) {
+          return null;
+        }
+        key = bWr.getBytes();
+        keyLength = bWr.getLength();
+      }
+      secretKey = getSecretKey(key, keyLength);
+    }
+
+    if (secretKey == null) {
+      return null;
+    }
+
+    byte[] res = aesFunction(input, inputLength, secretKey);
+
+    if (res == null) {
+      return null;
+    }
+
+    output.set(res, 0, res.length);
+    return output;
+  }
+
+  protected SecretKey getSecretKey(byte[] key, int keyLength) {
+    if (keyLength == 16 || keyLength == 32 || keyLength == 24) {
+      return new SecretKeySpec(key, 0, keyLength, "AES");
+    }
+    return null;
+  }
+
+  protected byte[] aesFunction(byte[] input, int inputLength, SecretKey secretKey) {
+    try {
+      cipher.init(getCipherMode(), secretKey);
+      byte[] res = cipher.doFinal(input, 0, inputLength);
+      return res;
+    } catch (GeneralSecurityException e) {
+      return null;
+    }
+  }
+
+  abstract protected int getCipherMode();
+
+  abstract protected boolean canParam0BeStr();
+
+  @Override
+  public String getDisplayString(String[] children) {
+    return getStandardDisplayString(getFuncName(), children);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f4acb44f/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAesDecrypt.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAesDecrypt.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAesDecrypt.java
new file mode 100644
index 0000000..d83fd2d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAesDecrypt.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import javax.crypto.Cipher;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * GenericUDFAesDecrypt.
+ *
+ */
+@Description(name = "aes_decrypt", value = "_FUNC_(input binary, key string/binary) - Decrypt input using AES.",
+    extended = "AES (Advanced Encryption Standard) algorithm. "
+    + "Key lengths of 128, 192 or 256 bits can be used. 192 and 256 bits keys can be used if "
+    + "Java Cryptography Extension (JCE) Unlimited Strength Jurisdiction Policy Files are installed. "
+    + "If either argument is NULL or the key length is not one of the permitted values, the return value is NULL.\n"
+    + "Example: > SELECT _FUNC_(unbase64('y6Ss+zCYObpCbgfWfyNWTw=='), '1234567890123456');\n 'ABC'")
+public class GenericUDFAesDecrypt extends GenericUDFAesBase {
+
+  @Override
+  protected int getCipherMode() {
+    return Cipher.DECRYPT_MODE;
+  }
+
+  @Override
+  protected boolean canParam0BeStr() {
+    return false;
+  }
+
+  @Override
+  protected String getFuncName() {
+    return "aes_decrypt";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f4acb44f/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAesEncrypt.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAesEncrypt.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAesEncrypt.java
new file mode 100644
index 0000000..d916441
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAesEncrypt.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import javax.crypto.Cipher;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * GenericUDFAesEncrypt.
+ *
+ */
+@Description(name = "aes_encrypt", value = "_FUNC_(input string/binary, key string/binary) - Encrypt input using AES.",
+    extended = "AES (Advanced Encryption Standard) algorithm. "
+    + "Key lengths of 128, 192 or 256 bits can be used. 192 and 256 bits keys can be used if "
+    + "Java Cryptography Extension (JCE) Unlimited Strength Jurisdiction Policy Files are installed. "
+    + "If either argument is NULL or the key length is not one of the permitted values, the return value is NULL.\n"
+    + "Example: > SELECT base64(_FUNC_('ABC', '1234567890123456'));\n 'y6Ss+zCYObpCbgfWfyNWTw=='")
+public class GenericUDFAesEncrypt extends GenericUDFAesBase {
+
+  @Override
+  protected int getCipherMode() {
+    return Cipher.ENCRYPT_MODE;
+  }
+
+  @Override
+  protected boolean canParam0BeStr() {
+    return true;
+  }
+
+  @Override
+  protected String getFuncName() {
+    return "aes_encrypt";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f4acb44f/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFParamUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFParamUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFParamUtils.java
index cdbc6ea..0e7eb8e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFParamUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFParamUtils.java
@@ -21,10 +21,11 @@ package org.apache.hadoop.hive.ql.udf.generic;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.BytesWritable;
@@ -68,4 +69,9 @@ public class GenericUDFParamUtils {
     converters[i] = converter;
     inputTypes[i] = inputType;
   }
+
+  public static BytesWritable getConstantBytesValue(ObjectInspector[] arguments, int i) {
+    Object constValue = ((ConstantObjectInspector) arguments[i]).getWritableConstantValue();
+    return (BytesWritable) constValue;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f4acb44f/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAesDecrypt.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAesDecrypt.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAesDecrypt.java
new file mode 100644
index 0000000..729f2fc
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAesDecrypt.java
@@ -0,0 +1,233 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import static org.junit.Assert.assertEquals;
+
+import java.security.NoSuchAlgorithmException;
+
+import javax.crypto.Cipher;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class TestGenericUDFAesDecrypt {
+
+  @Test
+  public void testAesDec128ConstStr() throws HiveException {
+    GenericUDFAesDecrypt udf = new GenericUDFAesDecrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    Text keyWr = new Text("1234567890123456");
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory
+        .getPrimitiveWritableConstantObjectInspector(TypeInfoFactory.stringTypeInfo, keyWr);
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    runAndVerifyStr("y6Ss+zCYObpCbgfWfyNWTw==", keyWr, "ABC", udf);
+    runAndVerifyStr("BQGHoM3lqYcsurCRq3PlUw==", keyWr, "", udf);
+    // null
+    runAndVerifyStr(null, keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesDec256ConstStr() throws HiveException, NoSuchAlgorithmException {
+    int maxKeyLen = Cipher.getMaxAllowedKeyLength("AES");
+    // skip the test if Java Cryptography Extension (JCE) Unlimited Strength
+    // Jurisdiction Policy Files not installed
+    if (maxKeyLen < 256) {
+      return;
+    }
+    GenericUDFAesDecrypt udf = new GenericUDFAesDecrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    Text keyWr = new Text("1234567890123456" + "1234567890123456");
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory
+        .getPrimitiveWritableConstantObjectInspector(TypeInfoFactory.stringTypeInfo, keyWr);
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    runAndVerifyStr("nYfCuJeRd5eD60yXDw7WEA==", keyWr, "ABC", udf);
+    runAndVerifyStr("mVClVqZ6W4VF6b842FOgCA==", keyWr, "", udf);
+    // null
+    runAndVerifyStr(null, keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesDec128Str() throws HiveException {
+    GenericUDFAesDecrypt udf = new GenericUDFAesDecrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    Text keyWr = new Text("1234567890123456");
+    runAndVerifyStr("y6Ss+zCYObpCbgfWfyNWTw==", keyWr, "ABC", udf);
+    runAndVerifyStr("BQGHoM3lqYcsurCRq3PlUw==", keyWr, "", udf);
+    // null
+    runAndVerifyStr(null, keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesDec128ConstBin() throws HiveException {
+    GenericUDFAesDecrypt udf = new GenericUDFAesDecrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    BytesWritable keyWr = new BytesWritable("1234567890123456".getBytes());
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory
+        .getPrimitiveWritableConstantObjectInspector(TypeInfoFactory.binaryTypeInfo, keyWr);
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    runAndVerifyBin("y6Ss+zCYObpCbgfWfyNWTw==", keyWr, "ABC", udf);
+    runAndVerifyBin("BQGHoM3lqYcsurCRq3PlUw==", keyWr, "", udf);
+    // null
+    runAndVerifyBin(null, keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesDec128Bin() throws HiveException {
+    GenericUDFAesDecrypt udf = new GenericUDFAesDecrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    BytesWritable keyWr = new BytesWritable("1234567890123456".getBytes());
+    runAndVerifyBin("y6Ss+zCYObpCbgfWfyNWTw==", keyWr, "ABC", udf);
+    runAndVerifyBin("BQGHoM3lqYcsurCRq3PlUw==", keyWr, "", udf);
+    // null
+    runAndVerifyBin(null, keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesDec192Bin() throws HiveException, NoSuchAlgorithmException {
+    int maxKeyLen = Cipher.getMaxAllowedKeyLength("AES");
+    // skip the test if Java Cryptography Extension (JCE) Unlimited Strength
+    // Jurisdiction Policy Files not installed
+    if (maxKeyLen < 192) {
+      return;
+    }
+    GenericUDFAesDecrypt udf = new GenericUDFAesDecrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    BytesWritable keyWr = new BytesWritable(("1234567890123456" + "12345678").getBytes());
+    runAndVerifyBin("ucvvpP9r2/LfQ6BilQuFtA==", keyWr, "ABC", udf);
+    runAndVerifyBin("KqMT3cF6VwSISMaUVUB4Qw==", keyWr, "", udf);
+    // null
+    runAndVerifyBin(null, keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesDecKeyNullConstStr() throws HiveException {
+    GenericUDFAesDecrypt udf = new GenericUDFAesDecrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    Text keyWr = null;
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory
+        .getPrimitiveWritableConstantObjectInspector(TypeInfoFactory.stringTypeInfo, keyWr);
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    runAndVerifyStr("y6Ss+zCYObpCbgfWfyNWTw==", keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesDecKeyNullStr() throws HiveException {
+    GenericUDFAesDecrypt udf = new GenericUDFAesDecrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    Text keyWr = null;
+    runAndVerifyStr("y6Ss+zCYObpCbgfWfyNWTw==", keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesDecKeyNullConstBin() throws HiveException {
+    GenericUDFAesDecrypt udf = new GenericUDFAesDecrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    BytesWritable keyWr = null;
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory
+        .getPrimitiveWritableConstantObjectInspector(TypeInfoFactory.binaryTypeInfo, keyWr);
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    runAndVerifyBin("y6Ss+zCYObpCbgfWfyNWTw==", keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesDecKeyNullBin() throws HiveException {
+    GenericUDFAesDecrypt udf = new GenericUDFAesDecrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    BytesWritable keyWr = null;
+    runAndVerifyBin("y6Ss+zCYObpCbgfWfyNWTw==", keyWr, null, udf);
+  }
+
+  private void runAndVerifyStr(String strBase64, Text keyWr, String expResult, GenericUDFAesDecrypt udf)
+      throws HiveException {
+    DeferredObject valueObj0 = new DeferredJavaObject(
+        strBase64 != null ? new BytesWritable(Base64.decodeBase64(strBase64)) : null);
+    DeferredObject valueObj1 = new DeferredJavaObject(keyWr);
+    DeferredObject[] args = { valueObj0, valueObj1 };
+    BytesWritable output = (BytesWritable) udf.evaluate(args);
+    String expResultHex = expResult == null ? null : Hex.encodeHexString(expResult.getBytes());
+    assertEquals("aes_decrypt() test ", expResultHex, output != null ? copyBytesAndHex(output) : null);
+  }
+
+  private void runAndVerifyBin(String strBase64, BytesWritable keyWr, String expResult, GenericUDFAesDecrypt udf)
+      throws HiveException {
+    DeferredObject valueObj0 = new DeferredJavaObject(
+        strBase64 != null ? new BytesWritable(Base64.decodeBase64(strBase64)) : null);
+    DeferredObject valueObj1 = new DeferredJavaObject(keyWr);
+    DeferredObject[] args = { valueObj0, valueObj1 };
+    BytesWritable output = (BytesWritable) udf.evaluate(args);
+    String expResultHex = expResult == null ? null : Hex.encodeHexString(expResult.getBytes());
+    assertEquals("aes_decrypt() test ", expResultHex, output != null ? copyBytesAndHex(output) : null);
+  }
+
+  private String copyBytesAndHex(BytesWritable bw) {
+    int size = bw.getLength();
+    byte[] bytes = new byte[size];
+    System.arraycopy(bw.getBytes(), 0, bytes, 0, size);
+    return Hex.encodeHexString(bytes);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f4acb44f/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAesEncrypt.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAesEncrypt.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAesEncrypt.java
new file mode 100644
index 0000000..5e5914d
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAesEncrypt.java
@@ -0,0 +1,228 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import static org.junit.Assert.assertEquals;
+
+import java.security.NoSuchAlgorithmException;
+
+import javax.crypto.Cipher;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class TestGenericUDFAesEncrypt {
+
+  @Test
+  public void testAesEnc128ConstStr() throws HiveException {
+    GenericUDFAesEncrypt udf = new GenericUDFAesEncrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
+    Text keyWr = new Text("1234567890123456");
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory
+        .getPrimitiveWritableConstantObjectInspector(TypeInfoFactory.stringTypeInfo, keyWr);
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    runAndVerifyStr("ABC", keyWr, "y6Ss+zCYObpCbgfWfyNWTw==", udf);
+    runAndVerifyStr("", keyWr, "BQGHoM3lqYcsurCRq3PlUw==", udf);
+    // null
+    runAndVerifyStr(null, keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesEnc256ConstStr() throws HiveException, NoSuchAlgorithmException {
+    int maxKeyLen = Cipher.getMaxAllowedKeyLength("AES");
+    // skip the test if Java Cryptography Extension (JCE) Unlimited Strength
+    // Jurisdiction Policy Files not installed
+    if (maxKeyLen < 256) {
+      return;
+    }
+    GenericUDFAesEncrypt udf = new GenericUDFAesEncrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
+    Text keyWr = new Text("1234567890123456" + "1234567890123456");
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory
+        .getPrimitiveWritableConstantObjectInspector(TypeInfoFactory.stringTypeInfo, keyWr);
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    runAndVerifyStr("ABC", keyWr, "nYfCuJeRd5eD60yXDw7WEA==", udf);
+    runAndVerifyStr("", keyWr, "mVClVqZ6W4VF6b842FOgCA==", udf);
+    // null
+    runAndVerifyStr(null, keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesEnc128Str() throws HiveException {
+    GenericUDFAesEncrypt udf = new GenericUDFAesEncrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    Text keyWr = new Text("1234567890123456");
+    runAndVerifyStr("ABC", keyWr, "y6Ss+zCYObpCbgfWfyNWTw==", udf);
+    runAndVerifyStr("", keyWr, "BQGHoM3lqYcsurCRq3PlUw==", udf);
+    // null
+    runAndVerifyStr(null, keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesEnc128ConstBin() throws HiveException {
+    GenericUDFAesEncrypt udf = new GenericUDFAesEncrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    BytesWritable keyWr = new BytesWritable("1234567890123456".getBytes());
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory
+        .getPrimitiveWritableConstantObjectInspector(TypeInfoFactory.binaryTypeInfo, keyWr);
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    runAndVerifyBin(new byte[] { 65, 66, 67 }, keyWr, "y6Ss+zCYObpCbgfWfyNWTw==", udf);
+    runAndVerifyBin(new byte[0], keyWr, "BQGHoM3lqYcsurCRq3PlUw==", udf);
+    // null
+    runAndVerifyBin(null, keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesEnc128Bin() throws HiveException {
+    GenericUDFAesEncrypt udf = new GenericUDFAesEncrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    BytesWritable keyWr = new BytesWritable("1234567890123456".getBytes());
+    runAndVerifyBin(new byte[] { 65, 66, 67 }, keyWr, "y6Ss+zCYObpCbgfWfyNWTw==", udf);
+    runAndVerifyBin(new byte[0], keyWr, "BQGHoM3lqYcsurCRq3PlUw==", udf);
+    // null
+    runAndVerifyBin(null, keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesEnc192Bin() throws HiveException, NoSuchAlgorithmException {
+    int maxKeyLen = Cipher.getMaxAllowedKeyLength("AES");
+    // skip the test if Java Cryptography Extension (JCE) Unlimited Strength
+    // Jurisdiction Policy Files not installed
+    if (maxKeyLen < 192) {
+      return;
+    }
+    GenericUDFAesEncrypt udf = new GenericUDFAesEncrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    BytesWritable keyWr = new BytesWritable(("1234567890123456" + "12345678").getBytes());
+    runAndVerifyBin(new byte[] { 65, 66, 67 }, keyWr, "ucvvpP9r2/LfQ6BilQuFtA==", udf);
+    runAndVerifyBin(new byte[0], keyWr, "KqMT3cF6VwSISMaUVUB4Qw==", udf);
+    // null
+    runAndVerifyBin(null, keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesEncKeyNullConstStr() throws HiveException {
+    GenericUDFAesEncrypt udf = new GenericUDFAesEncrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
+    Text keyWr = null;
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory
+        .getPrimitiveWritableConstantObjectInspector(TypeInfoFactory.stringTypeInfo, keyWr);
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    runAndVerifyStr("ABC", keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesEncKeyNullStr() throws HiveException {
+    GenericUDFAesEncrypt udf = new GenericUDFAesEncrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    Text keyWr = null;
+    runAndVerifyStr("ABC", keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesEncKeyNullConstBin() throws HiveException {
+    GenericUDFAesEncrypt udf = new GenericUDFAesEncrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    BytesWritable keyWr = null;
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory
+        .getPrimitiveWritableConstantObjectInspector(TypeInfoFactory.binaryTypeInfo, keyWr);
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    runAndVerifyBin(new byte[] { 65, 66, 67 }, keyWr, null, udf);
+  }
+
+  @Test
+  public void testAesEncKeyNullBin() throws HiveException {
+    GenericUDFAesEncrypt udf = new GenericUDFAesEncrypt();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    BytesWritable keyWr = null;
+    runAndVerifyBin(new byte[] { 65, 66, 67 }, keyWr, null, udf);
+  }
+
+  private void runAndVerifyStr(String str, Text keyWr, String expResultBase64, GenericUDFAesEncrypt udf)
+      throws HiveException {
+    DeferredObject valueObj0 = new DeferredJavaObject(str != null ? new Text(str) : null);
+    DeferredObject valueObj1 = new DeferredJavaObject(keyWr);
+    DeferredObject[] args = { valueObj0, valueObj1 };
+    BytesWritable output = (BytesWritable) udf.evaluate(args);
+    assertEquals("aes_encrypt() test ", expResultBase64, output != null ? copyBytesAndBase64(output) : null);
+  }
+
+  private void runAndVerifyBin(byte[] b, BytesWritable keyWr, String expResultBase64, GenericUDFAesEncrypt udf)
+      throws HiveException {
+    DeferredObject valueObj0 = new DeferredJavaObject(b != null ? new BytesWritable(b) : null);
+    DeferredObject valueObj1 = new DeferredJavaObject(keyWr);
+    DeferredObject[] args = { valueObj0, valueObj1 };
+    BytesWritable output = (BytesWritable) udf.evaluate(args);
+    assertEquals("aes_encrypt() test ", expResultBase64, output != null ? copyBytesAndBase64(output) : null);
+  }
+
+  private String copyBytesAndBase64(BytesWritable bw) {
+    int size = bw.getLength();
+    byte[] bytes = new byte[size];
+    System.arraycopy(bw.getBytes(), 0, bytes, 0, size);
+    return new String(Base64.encodeBase64(bytes));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f4acb44f/ql/src/test/queries/clientpositive/udf_aes_decrypt.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udf_aes_decrypt.q b/ql/src/test/queries/clientpositive/udf_aes_decrypt.q
new file mode 100644
index 0000000..36a0cf9
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/udf_aes_decrypt.q
@@ -0,0 +1,21 @@
+DESCRIBE FUNCTION aes_decrypt;
+DESC FUNCTION EXTENDED aes_decrypt;
+
+explain select aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), '1234567890123456');
+
+select
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), '1234567890123456'),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), binary('1234567890123456')),
+aes_decrypt(unbase64("BQGHoM3lqYcsurCRq3PlUw=="), '1234567890123456') = binary(''),
+aes_decrypt(unbase64("BQGHoM3lqYcsurCRq3PlUw=="), binary('1234567890123456')) = binary(''),
+aes_decrypt(cast(null as binary), '1234567890123456'),
+aes_decrypt(cast(null as binary), binary('1234567890123456'));
+
+--bad key
+select
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), '12345678901234567'),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), binary('123456789012345')),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), ''),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), binary('')),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), cast(null as string)),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), cast(null as binary));
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/f4acb44f/ql/src/test/queries/clientpositive/udf_aes_encrypt.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udf_aes_encrypt.q b/ql/src/test/queries/clientpositive/udf_aes_encrypt.q
new file mode 100644
index 0000000..2f03943
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/udf_aes_encrypt.q
@@ -0,0 +1,21 @@
+DESCRIBE FUNCTION aes_encrypt;
+DESC FUNCTION EXTENDED aes_encrypt;
+
+explain select aes_encrypt('ABC', '1234567890123456');
+
+select
+base64(aes_encrypt('ABC', '1234567890123456')),
+base64(aes_encrypt('', '1234567890123456')),
+base64(aes_encrypt(binary('ABC'), binary('1234567890123456'))),
+base64(aes_encrypt(binary(''), binary('1234567890123456'))),
+aes_encrypt(cast(null as string), '1234567890123456'),
+aes_encrypt(cast(null as binary), binary('1234567890123456'));
+
+--bad key
+select
+aes_encrypt('ABC', '12345678901234567'),
+aes_encrypt(binary('ABC'), binary('123456789012345')),
+aes_encrypt('ABC', ''),
+aes_encrypt(binary('ABC'), binary('')),
+aes_encrypt('ABC', cast(null as string)),
+aes_encrypt(binary('ABC'), cast(null as binary));
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/f4acb44f/ql/src/test/results/clientpositive/show_functions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/show_functions.q.out b/ql/src/test/results/clientpositive/show_functions.q.out
index 540079b..fbcd86a 100644
--- a/ql/src/test/results/clientpositive/show_functions.q.out
+++ b/ql/src/test/results/clientpositive/show_functions.q.out
@@ -22,6 +22,8 @@ POSTHOOK: type: SHOWFUNCTIONS
 abs
 acos
 add_months
+aes_decrypt
+aes_encrypt
 and
 array
 array_contains

http://git-wip-us.apache.org/repos/asf/hive/blob/f4acb44f/ql/src/test/results/clientpositive/udf_aes_decrypt.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_aes_decrypt.q.out b/ql/src/test/results/clientpositive/udf_aes_decrypt.q.out
new file mode 100644
index 0000000..83780a9
--- /dev/null
+++ b/ql/src/test/results/clientpositive/udf_aes_decrypt.q.out
@@ -0,0 +1,79 @@
+PREHOOK: query: DESCRIBE FUNCTION aes_decrypt
+PREHOOK: type: DESCFUNCTION
+POSTHOOK: query: DESCRIBE FUNCTION aes_decrypt
+POSTHOOK: type: DESCFUNCTION
+aes_decrypt(input binary, key string/binary) - Decrypt input using AES.
+PREHOOK: query: DESC FUNCTION EXTENDED aes_decrypt
+PREHOOK: type: DESCFUNCTION
+POSTHOOK: query: DESC FUNCTION EXTENDED aes_decrypt
+POSTHOOK: type: DESCFUNCTION
+aes_decrypt(input binary, key string/binary) - Decrypt input using AES.
+AES (Advanced Encryption Standard) algorithm. Key lengths of 128, 192 or 256 bits can be used. 192 and 256 bits keys can be used if Java Cryptography Extension (JCE) Unlimited Strength Jurisdiction Policy Files are installed. If either argument is NULL or the key length is not one of the permitted values, the return value is NULL.
+Example: > SELECT aes_decrypt(unbase64('y6Ss+zCYObpCbgfWfyNWTw=='), '1234567890123456');
+ 'ABC'
+PREHOOK: query: explain select aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), '1234567890123456')
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), '1234567890123456')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: _dummy_table
+          Row Limit Per Split: 1
+          Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE
+          Select Operator
+            expressions: 414243 (type: binary)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+            ListSink
+
+PREHOOK: query: select
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), '1234567890123456'),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), binary('1234567890123456')),
+aes_decrypt(unbase64("BQGHoM3lqYcsurCRq3PlUw=="), '1234567890123456') = binary(''),
+aes_decrypt(unbase64("BQGHoM3lqYcsurCRq3PlUw=="), binary('1234567890123456')) = binary(''),
+aes_decrypt(cast(null as binary), '1234567890123456'),
+aes_decrypt(cast(null as binary), binary('1234567890123456'))
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), '1234567890123456'),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), binary('1234567890123456')),
+aes_decrypt(unbase64("BQGHoM3lqYcsurCRq3PlUw=="), '1234567890123456') = binary(''),
+aes_decrypt(unbase64("BQGHoM3lqYcsurCRq3PlUw=="), binary('1234567890123456')) = binary(''),
+aes_decrypt(cast(null as binary), '1234567890123456'),
+aes_decrypt(cast(null as binary), binary('1234567890123456'))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+ABC	ABC	true	true	NULL	NULL
+PREHOOK: query: --bad key
+select
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), '12345678901234567'),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), binary('123456789012345')),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), ''),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), binary('')),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), cast(null as string)),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), cast(null as binary))
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: --bad key
+select
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), '12345678901234567'),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), binary('123456789012345')),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), ''),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), binary('')),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), cast(null as string)),
+aes_decrypt(unbase64("y6Ss+zCYObpCbgfWfyNWTw=="), cast(null as binary))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+NULL	NULL	NULL	NULL	NULL	NULL

http://git-wip-us.apache.org/repos/asf/hive/blob/f4acb44f/ql/src/test/results/clientpositive/udf_aes_encrypt.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_aes_encrypt.q.out b/ql/src/test/results/clientpositive/udf_aes_encrypt.q.out
new file mode 100644
index 0000000..9e356b2
--- /dev/null
+++ b/ql/src/test/results/clientpositive/udf_aes_encrypt.q.out
@@ -0,0 +1,79 @@
+PREHOOK: query: DESCRIBE FUNCTION aes_encrypt
+PREHOOK: type: DESCFUNCTION
+POSTHOOK: query: DESCRIBE FUNCTION aes_encrypt
+POSTHOOK: type: DESCFUNCTION
+aes_encrypt(input string/binary, key string/binary) - Encrypt input using AES.
+PREHOOK: query: DESC FUNCTION EXTENDED aes_encrypt
+PREHOOK: type: DESCFUNCTION
+POSTHOOK: query: DESC FUNCTION EXTENDED aes_encrypt
+POSTHOOK: type: DESCFUNCTION
+aes_encrypt(input string/binary, key string/binary) - Encrypt input using AES.
+AES (Advanced Encryption Standard) algorithm. Key lengths of 128, 192 or 256 bits can be used. 192 and 256 bits keys can be used if Java Cryptography Extension (JCE) Unlimited Strength Jurisdiction Policy Files are installed. If either argument is NULL or the key length is not one of the permitted values, the return value is NULL.
+Example: > SELECT base64(aes_encrypt('ABC', '1234567890123456'));
+ 'y6Ss+zCYObpCbgfWfyNWTw=='
+PREHOOK: query: explain select aes_encrypt('ABC', '1234567890123456')
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select aes_encrypt('ABC', '1234567890123456')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: _dummy_table
+          Row Limit Per Split: 1
+          Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE
+          Select Operator
+            expressions: CBA4ACFB309839BA426E07D67F23564F (type: binary)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+            ListSink
+
+PREHOOK: query: select
+base64(aes_encrypt('ABC', '1234567890123456')),
+base64(aes_encrypt('', '1234567890123456')),
+base64(aes_encrypt(binary('ABC'), binary('1234567890123456'))),
+base64(aes_encrypt(binary(''), binary('1234567890123456'))),
+aes_encrypt(cast(null as string), '1234567890123456'),
+aes_encrypt(cast(null as binary), binary('1234567890123456'))
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+base64(aes_encrypt('ABC', '1234567890123456')),
+base64(aes_encrypt('', '1234567890123456')),
+base64(aes_encrypt(binary('ABC'), binary('1234567890123456'))),
+base64(aes_encrypt(binary(''), binary('1234567890123456'))),
+aes_encrypt(cast(null as string), '1234567890123456'),
+aes_encrypt(cast(null as binary), binary('1234567890123456'))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+y6Ss+zCYObpCbgfWfyNWTw==	BQGHoM3lqYcsurCRq3PlUw==	y6Ss+zCYObpCbgfWfyNWTw==	BQGHoM3lqYcsurCRq3PlUw==	NULL	NULL
+PREHOOK: query: --bad key
+select
+aes_encrypt('ABC', '12345678901234567'),
+aes_encrypt(binary('ABC'), binary('123456789012345')),
+aes_encrypt('ABC', ''),
+aes_encrypt(binary('ABC'), binary('')),
+aes_encrypt('ABC', cast(null as string)),
+aes_encrypt(binary('ABC'), cast(null as binary))
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: --bad key
+select
+aes_encrypt('ABC', '12345678901234567'),
+aes_encrypt(binary('ABC'), binary('123456789012345')),
+aes_encrypt('ABC', ''),
+aes_encrypt(binary('ABC'), binary('')),
+aes_encrypt('ABC', cast(null as string)),
+aes_encrypt(binary('ABC'), cast(null as binary))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+NULL	NULL	NULL	NULL	NULL	NULL


[04/11] hive git commit: HIVE-11747: Unnecessary error log is shown when executing a "INSERT OVERWRITE LOCAL DIRECTORY" cmd in the embedded mode(Ferdinand Xu, reviewed by Dong Chen)

Posted by se...@apache.org.
HIVE-11747: Unnecessary error log is shown when executing a "INSERT OVERWRITE LOCAL DIRECTORY" cmd in the embedded mode(Ferdinand Xu, reviewed by Dong Chen)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/76fc383e
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/76fc383e
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/76fc383e

Branch: refs/heads/llap
Commit: 76fc383e5af919f2c43b7607bcfe732b6681a45b
Parents: f4acb44
Author: Ferdinand Xu <ch...@intel.com>
Authored: Mon Sep 7 03:06:41 2015 -0400
Committer: Ferdinand Xu <ch...@intel.com>
Committed: Mon Sep 7 03:06:58 2015 -0400

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/76fc383e/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
index 82345ee..d9225a9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
@@ -398,7 +398,6 @@ public class ExecDriver extends Task<MapredWork> implements Serializable, Hadoop
       if (pwd != null) {
         HiveConf.setVar(job, HiveConf.ConfVars.METASTOREPWD, "HIVE");
       }
-      LOG.error(job.get("mapreduce.framework.name"));
       JobClient jc = new JobClient(job);
       // make this client wait if job tracker is not behaving well.
       Throttle.checkJobTracker(job, LOG);


[05/11] hive git commit: HIVE-11737: IndexOutOfBounds compiling query with duplicated groupby keys (Jimmy, reviewed by Szehon)

Posted by se...@apache.org.
HIVE-11737: IndexOutOfBounds compiling query with duplicated groupby keys (Jimmy, reviewed by Szehon)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7aec2726
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7aec2726
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7aec2726

Branch: refs/heads/llap
Commit: 7aec2726dc10c036941bc204020e199885550427
Parents: 76fc383
Author: Jimmy Xiang <jx...@cloudera.com>
Authored: Fri Sep 4 10:34:58 2015 -0700
Committer: Jimmy Xiang <jx...@cloudera.com>
Committed: Mon Sep 7 09:41:15 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  1 +
 ql/src/test/queries/clientpositive/groupby13.q  | 16 ++++
 .../test/results/clientpositive/groupby13.q.out | 86 ++++++++++++++++++++
 3 files changed, 103 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7aec2726/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 778c7b2..f6052e3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -4622,6 +4622,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
       if (ExprNodeDescUtils.indexOf(grpByExprNode, groupByKeys) >= 0) {
         // Skip duplicated grouping keys
+        grpByExprs.remove(i--);
         continue;
       }
       groupByKeys.add(grpByExprNode);

http://git-wip-us.apache.org/repos/asf/hive/blob/7aec2726/ql/src/test/queries/clientpositive/groupby13.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/groupby13.q b/ql/src/test/queries/clientpositive/groupby13.q
new file mode 100644
index 0000000..53feaed
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/groupby13.q
@@ -0,0 +1,16 @@
+CREATE TABLE grpby_test (int_col_5 INT,
+  int_col_7 INT);
+
+SET hive.mapred.mode=strict;
+
+EXPLAIN
+SELECT
+int_col_7,
+MAX(LEAST(COALESCE(int_col_5, -279),
+  COALESCE(int_col_7, 476))) AS int_col
+FROM grpby_test
+GROUP BY
+int_col_7,
+int_col_7,
+LEAST(COALESCE(int_col_5, -279),
+  COALESCE(int_col_7, 476));

http://git-wip-us.apache.org/repos/asf/hive/blob/7aec2726/ql/src/test/results/clientpositive/groupby13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby13.q.out b/ql/src/test/results/clientpositive/groupby13.q.out
new file mode 100644
index 0000000..e698d1a
--- /dev/null
+++ b/ql/src/test/results/clientpositive/groupby13.q.out
@@ -0,0 +1,86 @@
+PREHOOK: query: CREATE TABLE grpby_test (int_col_5 INT,
+  int_col_7 INT)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@grpby_test
+POSTHOOK: query: CREATE TABLE grpby_test (int_col_5 INT,
+  int_col_7 INT)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@grpby_test
+PREHOOK: query: EXPLAIN
+SELECT
+int_col_7,
+MAX(LEAST(COALESCE(int_col_5, -279),
+  COALESCE(int_col_7, 476))) AS int_col
+FROM grpby_test
+GROUP BY
+int_col_7,
+int_col_7,
+LEAST(COALESCE(int_col_5, -279),
+  COALESCE(int_col_7, 476))
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT
+int_col_7,
+MAX(LEAST(COALESCE(int_col_5, -279),
+  COALESCE(int_col_7, 476))) AS int_col
+FROM grpby_test
+GROUP BY
+int_col_7,
+int_col_7,
+LEAST(COALESCE(int_col_5, -279),
+  COALESCE(int_col_7, 476))
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: grpby_test
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Select Operator
+              expressions: int_col_7 (type: int), int_col_5 (type: int)
+              outputColumnNames: int_col_7, int_col_5
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Group By Operator
+                aggregations: max(least(COALESCE(int_col_5,(- 279)),COALESCE(int_col_7,476)))
+                keys: int_col_7 (type: int), least(COALESCE(int_col_5,(- 279)),COALESCE(int_col_7,476)) (type: int)
+                mode: hash
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int), _col1 (type: int)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  value expressions: _col2 (type: int)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: max(VALUE._col0)
+          keys: KEY._col0 (type: int), KEY._col0 (type: int), KEY._col1 (type: int)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: _col1 (type: int), _col3 (type: int)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+


[07/11] hive git commit: HIVE-11646: CBO: Calcite Operator To Hive Operator (Calcite Return Path): fix multiple window spec for PTF operator (Pengcheng Xiong, reviewed by Jesus Camacho Rodriguez)

Posted by se...@apache.org.
HIVE-11646: CBO: Calcite Operator To Hive Operator (Calcite Return Path): fix multiple window spec for PTF operator (Pengcheng Xiong, reviewed by Jesus Camacho Rodriguez)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/2d3316b9
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2d3316b9
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2d3316b9

Branch: refs/heads/llap
Commit: 2d3316b9154cb175b6db15f2ec4551d2d54397d6
Parents: 7aec272
Author: Pengcheng Xiong <px...@hortonworks.com>
Authored: Tue Sep 8 11:33:44 2015 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Tue Sep 8 11:33:44 2015 +0100

----------------------------------------------------------------------
 .../calcite/translator/ExprNodeConverter.java   |   25 +-
 .../calcite/translator/HiveOpConverter.java     |    5 +-
 .../queries/clientpositive/cbo_rp_windowing_2.q |  439 ++++
 .../clientpositive/cbo_rp_windowing_2.q.out     | 2338 ++++++++++++++++++
 4 files changed, 2798 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2d3316b9/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
index ec22f1a..42f1ab6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.translator;
 import java.math.BigDecimal;
 import java.sql.Date;
 import java.sql.Timestamp;
+import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.LinkedList;
 import java.util.List;
@@ -80,11 +81,11 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
   private final String             tabAlias;
   private final String             columnAlias;
   private final RelDataType        inputRowType;
-  private final RelDataType        outputRowType;
   private final ImmutableSet<Integer>       inputVCols;
-  private WindowFunctionSpec wfs;
+  private List<WindowFunctionSpec> windowFunctionSpecs = new ArrayList<>();
   private final RelDataTypeFactory dTFactory;
   protected final Log LOG = LogFactory.getLog(this.getClass().getName());
+  private static long uniqueCounter = 0;
 
   public ExprNodeConverter(String tabAlias, RelDataType inputRowType,
       Set<Integer> vCols, RelDataTypeFactory dTFactory) {
@@ -97,13 +98,12 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
     this.tabAlias = tabAlias;
     this.columnAlias = columnAlias;
     this.inputRowType = inputRowType;
-    this.outputRowType = outputRowType;
     this.inputVCols = ImmutableSet.copyOf(inputVCols);
     this.dTFactory = dTFactory;
   }
 
-  public WindowFunctionSpec getWindowFunctionSpec() {
-    return this.wfs;
+  public List<WindowFunctionSpec> getWindowFunctionSpec() {
+    return this.windowFunctionSpecs;
   }
 
   @Override
@@ -235,7 +235,7 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
     final WindowFrameSpec windowFrameSpec = getWindowRange(window);
     windowSpec.setWindowFrame(windowFrameSpec);
 
-    wfs = new WindowFunctionSpec();
+    WindowFunctionSpec wfs = new WindowFunctionSpec();
     wfs.setWindowSpec(windowSpec);
     final Schema schema = new Schema(tabAlias, inputRowType.getFieldList());
     final ASTNode wUDAFAst = new ASTConverter.RexVisitor(schema).visitOver(over);
@@ -246,10 +246,15 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
       ASTNode child = (ASTNode) wUDAFAst.getChild(i);
       wfs.addArg(child);
     }
+    if (wUDAFAst.getText().equals("TOK_FUNCTIONSTAR")) {
+      wfs.setStar(true);
+    }
+    String columnAlias = getWindowColumnAlias();
     wfs.setAlias(columnAlias);
 
-    RelDataTypeField f = outputRowType.getField(columnAlias, false, false);
-    return new ExprNodeColumnDesc(TypeConverter.convert(f.getType()), columnAlias, tabAlias,
+    this.windowFunctionSpecs.add(wfs);
+
+    return new ExprNodeColumnDesc(TypeConverter.convert(over.getType()), columnAlias, tabAlias,
             false);
   }
 
@@ -343,4 +348,8 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
     return boundarySpec;
   }
 
+  private String getWindowColumnAlias() {
+    return "$win$_col_" + (uniqueCounter++);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/2d3316b9/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
index 9391952..3f66893 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
@@ -81,6 +81,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.parse.UnparseTranslator;
 import org.apache.hadoop.hive.ql.parse.WindowingComponentizer;
 import org.apache.hadoop.hive.ql.parse.WindowingSpec;
+import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowFunctionSpec;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils;
@@ -284,7 +285,9 @@ public class HiveOpConverter {
       exprCols.add(exprCol);
       //TODO: Cols that come through PTF should it retain (VirtualColumness)?
       if (converter.getWindowFunctionSpec() != null) {
-        windowingSpec.addWindowFunction(converter.getWindowFunctionSpec());
+        for (WindowFunctionSpec wfs : converter.getWindowFunctionSpec()) {
+          windowingSpec.addWindowFunction(wfs);
+        }
       }
     }
     if (windowingSpec.getWindowExpressions() != null

http://git-wip-us.apache.org/repos/asf/hive/blob/2d3316b9/ql/src/test/queries/clientpositive/cbo_rp_windowing_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cbo_rp_windowing_2.q b/ql/src/test/queries/clientpositive/cbo_rp_windowing_2.q
new file mode 100644
index 0000000..d02c8be
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/cbo_rp_windowing_2.q
@@ -0,0 +1,439 @@
+set hive.cbo.enable=true;
+set hive.cbo.returnpath.hiveop=true;
+set hive.exec.check.crossproducts=false;
+set mapred.reduce.tasks=4;
+-- SORT_QUERY_RESULTS
+
+-- 1. testWindowing
+select p_mfgr, p_name, p_size,
+rank() over(distribute by p_mfgr sort by p_name) as r,
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr,
+sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1
+from part
+;
+
+-- 2. testGroupByWithPartitioning
+select p_mfgr, p_name, p_size, 
+min(p_retailprice),
+rank() over(distribute by p_mfgr sort by p_name)as r,
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr,
+p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz
+from part
+group by p_mfgr, p_name, p_size
+;
+       
+-- 3. testGroupByHavingWithSWQ
+select p_mfgr, p_name, p_size, min(p_retailprice),
+rank() over(distribute by p_mfgr sort by p_name) as r,
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr,
+p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz
+from part
+group by p_mfgr, p_name, p_size
+having p_size > 0
+;
+
+-- 4. testCount
+select p_mfgr, p_name, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as cd 
+from part 
+;
+
+-- 5. testCountWithWindowingUDAF
+select p_mfgr, p_name, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as cd, 
+p_retailprice, sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1, 
+p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz 
+from part 
+;
+
+-- 6. testCountInSubQ
+select sub1.r, sub1.dr, sub1.cd, sub1.s1, sub1.deltaSz 
+from (select p_mfgr, p_name, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as cd, 
+p_retailprice, sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1, 
+p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz 
+from part 
+) sub1;
+
+-- 7. testJoinWithWindowingAndPTF
+select abc.p_mfgr, abc.p_name, 
+rank() over(distribute by abc.p_mfgr sort by abc.p_name) as r, 
+dense_rank() over(distribute by abc.p_mfgr sort by abc.p_name) as dr, 
+abc.p_retailprice, sum(abc.p_retailprice) over (distribute by abc.p_mfgr sort by abc.p_name rows between unbounded preceding and current row) as s1, 
+abc.p_size, abc.p_size - lag(abc.p_size,1,abc.p_size) over(distribute by abc.p_mfgr sort by abc.p_name) as deltaSz 
+from noop(on part 
+partition by p_mfgr 
+order by p_name 
+) abc join part p1 on abc.p_partkey = p1.p_partkey 
+;
+
+-- 8. testMixedCaseAlias
+select p_mfgr, p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name, p_size desc) as R
+from part 
+;
+
+-- 9. testHavingWithWindowingNoGBY
+select p_mfgr, p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row)  as s1
+from part  
+; 
+
+-- 10. testHavingWithWindowingCondRankNoGBY
+select p_mfgr, p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1 
+from part 
+;
+
+-- 11. testFirstLast   
+select  p_mfgr,p_name, p_size, 
+sum(p_size) over (distribute by p_mfgr sort by p_name rows between current row and current row) as s2, 
+first_value(p_size) over w1  as f, 
+last_value(p_size, false) over w1  as l 
+from part 
+window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following);
+
+-- 12. testFirstLastWithWhere
+select  p_mfgr,p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+sum(p_size) over (distribute by p_mfgr sort by p_name rows between current row and current row) as s2, 
+first_value(p_size) over w1 as f,  
+last_value(p_size, false) over w1 as l 
+from part 
+where p_mfgr = 'Manufacturer#3'  
+window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following);
+
+-- 13. testSumWindow
+select  p_mfgr,p_name, p_size,  
+sum(p_size) over w1 as s1, 
+sum(p_size) over (distribute by p_mfgr  sort by p_name rows between current row and current row)  as s2 
+from part 
+window w1 as (distribute by p_mfgr  sort by p_name rows between 2 preceding and 2 following);
+
+-- 14. testNoSortClause
+select  p_mfgr,p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name) as r, dense_rank() over(distribute by p_mfgr sort by p_name) as dr 
+from part  
+window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following);
+
+-- 15. testExpressions
+select  p_mfgr,p_name, p_size,  
+rank() over(distribute by p_mfgr sort by p_name) as r,  
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+cume_dist() over(distribute by p_mfgr sort by p_name) as cud, 
+percent_rank() over(distribute by p_mfgr sort by p_name) as pr, 
+ntile(3) over(distribute by p_mfgr sort by p_name) as nt, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as ca, 
+avg(p_size) over(distribute by p_mfgr sort by p_name) as avg, 
+stddev(p_size) over(distribute by p_mfgr sort by p_name) as st, 
+first_value(p_size % 5) over(distribute by p_mfgr sort by p_name) as fv, 
+last_value(p_size) over(distribute by p_mfgr sort by p_name) as lv, 
+first_value(p_size) over w1  as fvW1
+from part 
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following);
+
+-- 16. testMultipleWindows
+select  p_mfgr,p_name, p_size,  
+  rank() over(distribute by p_mfgr sort by p_name) as r, 
+  dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+cume_dist() over(distribute by p_mfgr sort by p_name) as cud, 
+sum(p_size) over (distribute by p_mfgr sort by p_name range between unbounded preceding and current row) as s1, 
+sum(p_size) over (distribute by p_mfgr sort by p_size range between 5 preceding and current row) as s2, 
+first_value(p_size) over w1  as fv1
+from part 
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following);
+
+-- 17. testCountStar
+select  p_mfgr,p_name, p_size,
+count(*) over(distribute by p_mfgr sort by p_name ) as c, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as ca, 
+first_value(p_size) over w1  as fvW1
+from part 
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following);
+
+-- 18. testUDAFs
+select  p_mfgr,p_name, p_size, 
+sum(p_retailprice) over w1 as s, 
+min(p_retailprice) over w1 as mi,
+max(p_retailprice) over w1 as ma,
+avg(p_retailprice) over w1 as ag
+from part
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following);
+
+-- 19. testUDAFsWithGBY
+select  p_mfgr,p_name, p_size, p_retailprice, 
+sum(p_retailprice) over w1 as s, 
+min(p_retailprice) as mi ,
+max(p_retailprice) as ma ,
+avg(p_retailprice) over w1 as ag
+from part
+group by p_mfgr,p_name, p_size, p_retailprice
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following);
+
+-- 20. testSTATs
+select  p_mfgr,p_name, p_size, 
+stddev(p_retailprice) over w1 as sdev, 
+stddev_pop(p_retailprice) over w1 as sdev_pop, 
+collect_set(p_size) over w1 as uniq_size, 
+variance(p_retailprice) over w1 as var,
+corr(p_size, p_retailprice) over w1 as cor,
+covar_pop(p_size, p_retailprice) over w1 as covarp
+from part
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following);
+
+-- 21. testDISTs
+select  p_mfgr,p_name, p_size, 
+histogram_numeric(p_retailprice, 5) over w1 as hist, 
+percentile(p_partkey, 0.5) over w1 as per,
+row_number() over(distribute by p_mfgr sort by p_mfgr, p_name) as rn
+from part
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following);
+
+-- 22. testViewAsTableInputWithWindowing
+create view IF NOT EXISTS mfgr_price_view as 
+select p_mfgr, p_brand, 
+round(sum(p_retailprice),2) as s 
+from part 
+group by p_mfgr, p_brand;
+        
+select * 
+from (
+select p_mfgr, p_brand, s, 
+round(sum(s) over w1 , 2)  as s1
+from mfgr_price_view 
+window w1 as (distribute by p_mfgr sort by p_mfgr )
+) sq
+order by p_mfgr, p_brand;
+
+select p_mfgr, p_brand, s, 
+round(sum(s) over w1 ,2)  as s1
+from mfgr_price_view 
+window w1 as (distribute by p_mfgr sort by p_brand rows between 2 preceding and current row);
+
+-- 23. testCreateViewWithWindowingQuery
+create view IF NOT EXISTS mfgr_brand_price_view as 
+select p_mfgr, p_brand, 
+sum(p_retailprice) over w1  as s
+from part 
+window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and current row);
+        
+select * from mfgr_brand_price_view;        
+        
+-- 24. testLateralViews
+select p_mfgr, p_name, 
+lv_col, p_size, sum(p_size) over w1   as s
+from (select p_mfgr, p_name, p_size, array(1,2,3) arr from part) p 
+lateral view explode(arr) part_lv as lv_col
+window w1 as (distribute by p_mfgr sort by p_size, lv_col rows between 2 preceding and current row);        
+
+-- 25. testMultipleInserts3SWQs
+CREATE TABLE part_1( 
+p_mfgr STRING, 
+p_name STRING, 
+p_size INT, 
+r INT, 
+dr INT, 
+s DOUBLE);
+
+CREATE TABLE part_2( 
+p_mfgr STRING, 
+p_name STRING, 
+p_size INT, 
+r INT, 
+dr INT, 
+cud INT,  
+s2 DOUBLE, 
+fv1 INT);
+
+CREATE TABLE part_3( 
+p_mfgr STRING, 
+p_name STRING, 
+p_size INT, 
+c INT, 
+ca INT, 
+fv INT);
+
+from part 
+INSERT OVERWRITE TABLE part_1 
+select p_mfgr, p_name, p_size, 
+rank() over(distribute by p_mfgr sort by p_name ) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name ) as dr, 
+sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row)  as s
+INSERT OVERWRITE TABLE part_2 
+select  p_mfgr,p_name, p_size,  
+rank() over(distribute by p_mfgr sort by p_name) as r, 
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
+cume_dist() over(distribute by p_mfgr sort by p_name) as cud, 
+round(sum(p_size) over (distribute by p_mfgr sort by p_size range between 5 preceding and current row),1) as s2, 
+first_value(p_size) over w1  as fv1
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following) 
+INSERT OVERWRITE TABLE part_3 
+select  p_mfgr,p_name, p_size,  
+count(*) over(distribute by p_mfgr sort by p_name) as c, 
+count(p_size) over(distribute by p_mfgr sort by p_name) as ca, 
+first_value(p_size) over w1  as fv
+window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following);
+
+select * from part_1;
+
+select * from part_2;
+
+select * from part_3;
+
+-- 26. testGroupByHavingWithSWQAndAlias
+select p_mfgr, p_name, p_size, min(p_retailprice) as mi,
+rank() over(distribute by p_mfgr sort by p_name) as r,
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr,
+p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz
+from part
+group by p_mfgr, p_name, p_size
+having p_size > 0
+;
+	 
+-- 27. testMultipleRangeWindows
+select  p_mfgr,p_name, p_size, 
+sum(p_size) over (distribute by p_mfgr sort by p_size range between 10 preceding and current row) as s2, 
+sum(p_size) over (distribute by p_mfgr sort by p_size range between current row and 10 following )  as s1
+from part  
+window w1 as (rows between 2 preceding and 2 following);
+
+-- 28. testPartOrderInUDAFInvoke
+select p_mfgr, p_name, p_size,
+sum(p_size) over (partition by p_mfgr  order by p_name  rows between 2 preceding and 2 following) as s
+from part;
+
+-- 29. testPartOrderInWdwDef
+select p_mfgr, p_name, p_size,
+sum(p_size) over w1 as s
+from part
+window w1 as (partition by p_mfgr  order by p_name  rows between 2 preceding and 2 following);
+
+-- 30. testDefaultPartitioningSpecRules
+select p_mfgr, p_name, p_size,
+sum(p_size) over w1 as s,
+sum(p_size) over w2 as s2
+from part
+window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following),
+       w2 as (partition by p_mfgr order by p_name);
+       
+-- 31. testWindowCrossReference
+select p_mfgr, p_name, p_size, 
+sum(p_size) over w1 as s1, 
+sum(p_size) over w2 as s2
+from part 
+window w1 as (partition by p_mfgr order by p_name range between 2 preceding and 2 following), 
+       w2 as w1;
+       
+               
+-- 32. testWindowInheritance
+select p_mfgr, p_name, p_size, 
+sum(p_size) over w1 as s1, 
+sum(p_size) over w2 as s2 
+from part 
+window w1 as (partition by p_mfgr order by p_name range between 2 preceding and 2 following), 
+       w2 as (w1 rows between unbounded preceding and current row); 
+
+        
+-- 33. testWindowForwardReference
+select p_mfgr, p_name, p_size, 
+sum(p_size) over w1 as s1, 
+sum(p_size) over w2 as s2,
+sum(p_size) over w3 as s3
+from part 
+window w1 as (distribute by p_mfgr sort by p_name range between 2 preceding and 2 following), 
+       w2 as w3,
+       w3 as (distribute by p_mfgr sort by p_name range between unbounded preceding and current row); 
+
+
+-- 34. testWindowDefinitionPropagation
+select p_mfgr, p_name, p_size, 
+sum(p_size) over w1 as s1, 
+sum(p_size) over w2 as s2,
+sum(p_size) over (w3 rows between 2 preceding and 2 following)  as s3
+from part 
+window w1 as (distribute by p_mfgr sort by p_name range between 2 preceding and 2 following), 
+       w2 as w3,
+       w3 as (distribute by p_mfgr sort by p_name range between unbounded preceding and current row); 
+
+-- 35. testDistinctWithWindowing
+select DISTINCT p_mfgr, p_name, p_size,
+sum(p_size) over w1 as s
+from part
+window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following);
+
+-- 36. testRankWithPartitioning
+select p_mfgr, p_name, p_size, 
+rank() over (partition by p_mfgr order by p_name )  as r
+from part;    
+
+-- 37. testPartitioningVariousForms
+select p_mfgr,
+round(sum(p_retailprice) over (partition by p_mfgr order by p_mfgr),2) as s1,
+min(p_retailprice) over (partition by p_mfgr) as s2,
+max(p_retailprice) over (distribute by p_mfgr sort by p_mfgr) as s3,
+round(avg(p_retailprice) over (distribute by p_mfgr),2) as s4,
+count(p_retailprice) over (cluster by p_mfgr ) as s5
+from part;
+
+-- 38. testPartitioningVariousForms2
+select p_mfgr, p_name, p_size,
+sum(p_retailprice) over (partition by p_mfgr, p_name order by p_mfgr, p_name rows between unbounded preceding and current row) as s1,
+min(p_retailprice) over (distribute by p_mfgr, p_name sort by p_mfgr, p_name rows between unbounded preceding and current row) as s2,
+max(p_retailprice) over (partition by p_mfgr, p_name order by p_name) as s3
+from part;
+
+-- 39. testUDFOnOrderCols
+select p_mfgr, p_type, substr(p_type, 2) as short_ptype,
+rank() over (partition by p_mfgr order by substr(p_type, 2))  as r
+from part;
+
+-- 40. testNoBetweenForRows
+select p_mfgr, p_name, p_size,
+    sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows unbounded preceding) as s1
+     from part ;
+
+-- 41. testNoBetweenForRange
+select p_mfgr, p_name, p_size,
+    sum(p_retailprice) over (distribute by p_mfgr sort by p_size range unbounded preceding) as s1
+     from part ;
+
+-- 42. testUnboundedFollowingForRows
+select p_mfgr, p_name, p_size,
+    sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between current row and unbounded following) as s1
+    from part ;
+
+-- 43. testUnboundedFollowingForRange
+select p_mfgr, p_name, p_size,
+    sum(p_retailprice) over (distribute by p_mfgr sort by p_size range between current row and unbounded following) as s1
+    from part ;
+        
+-- 44. testOverNoPartitionSingleAggregate
+select p_name, p_retailprice,
+round(avg(p_retailprice) over(),2)
+from part
+order by p_name;
+
+-- 45. empty partition test
+select p_mfgr, 
+  sum(p_size) over (partition by p_mfgr order by p_size rows between unbounded preceding and current row) 
+from part 
+where p_mfgr = 'Manufacturer#6'
+;
+
+-- 46. window sz is same as partition sz
+select p_retailprice, avg(p_retailprice) over (partition by p_mfgr order by p_name rows between current row and 6 following), 
+sum(p_retailprice) over (partition by p_mfgr order by p_name rows between current row and 6 following) 
+from part 
+where p_mfgr='Manufacturer#1';
+
+-- 47. empty partition
+select sum(p_size) over (partition by p_mfgr )
+from part where p_mfgr = 'm1';