You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ga...@apache.org on 2018/01/18 17:55:42 UTC

[22/70] [abbrv] hive git commit: HIVE-18416: Initial support for TABLE function (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_table.q.out
----------------------------------------------------------------------
diff --git a/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_table.q.out b/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_table.q.out
index bab88eb..2c23a7e 100644
--- a/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_table.q.out
+++ b/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_table.q.out
@@ -14,11 +14,13 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table1
 PREHOOK: query: INSERT OVERWRITE TABLE table1 VALUES (1)
 PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@table1
 POSTHOOK: query: INSERT OVERWRITE TABLE table1 VALUES (1)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@table1
-POSTHOOK: Lineage: table1.id EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: table1.id SCRIPT []
 PREHOOK: query: SELECT * FROM table1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@table1
@@ -30,11 +32,13 @@ POSTHOOK: Input: default@table1
 1
 PREHOOK: query: INSERT OVERWRITE TABLE table1 VALUES (2)
 PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@table1
 POSTHOOK: query: INSERT OVERWRITE TABLE table1 VALUES (2)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@table1
-POSTHOOK: Lineage: table1.id EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: table1.id SCRIPT []
 PREHOOK: query: SELECT * FROM table1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@table1
@@ -63,99 +67,107 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: values__tmp__table__3
-            Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+            alias: _dummy_table
+            Row Limit Per Split: 1
+            Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Select Operator
-              expressions: UDFToInteger(tmp_values_col1) (type: int)
+              expressions: array(const struct(1)) (type: array<struct<col1:int>>)
               outputColumnNames: _col0
-              Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
-              File Output Operator
-                compressed: false
-                GlobalTableId: 1
-                directory: ### BLOBSTORE_STAGING_PATH ###
-                NumFilesPerFileSink: 1
-                Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
-                Stats Publishing Key Prefix: ### BLOBSTORE_STAGING_PATH ###
-                table:
-                    input format: org.apache.hadoop.mapred.TextInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                    properties:
-                      COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"id":"true"}}
-                      bucket_count -1
-                      column.name.delimiter ,
-                      columns id
-                      columns.comments 
-                      columns.types int
+              Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+              UDTF Operator
+                Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                function name: inline
+                Select Operator
+                  expressions: col1 (type: int)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  File Output Operator
+                    compressed: false
+                    GlobalTableId: 1
+                    directory: ### BLOBSTORE_STAGING_PATH ###
+                    NumFilesPerFileSink: 1
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    Stats Publishing Key Prefix: ### BLOBSTORE_STAGING_PATH ###
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        properties:
+                          COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"id":"true"}}
+                          bucket_count -1
+                          column.name.delimiter ,
+                          columns id
+                          columns.comments 
+                          columns.types int
 #### A masked pattern was here ####
-                      location ### test.blobstore.path ###/table1
-                      name default.table1
-                      numFiles 1
-                      numRows 1
-                      rawDataSize 1
-                      serialization.ddl struct table1 { i32 id}
-                      serialization.format 1
-                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      totalSize 2
+                          location ### test.blobstore.path ###/table1
+                          name default.table1
+                          numFiles 1
+                          numRows 1
+                          rawDataSize 1
+                          serialization.ddl struct table1 { i32 id}
+                          serialization.format 1
+                          serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                          totalSize 2
 #### A masked pattern was here ####
-                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    name: default.table1
-                TotalFiles: 1
-                GatherStats: true
-                MultiFileSpray: false
-              Select Operator
-                expressions: _col0 (type: int)
-                outputColumnNames: id
-                Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
-                Group By Operator
-                  aggregations: compute_stats(id, 'hll')
-                  mode: hash
-                  outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 424 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    null sort order: 
-                    sort order: 
-                    Statistics: Num rows: 1 Data size: 424 Basic stats: COMPLETE Column stats: NONE
-                    tag: -1
-                    value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>)
-                    auto parallelism: false
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        name: default.table1
+                    TotalFiles: 1
+                    GatherStats: true
+                    MultiFileSpray: false
+                  Select Operator
+                    expressions: _col0 (type: int)
+                    outputColumnNames: id
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: compute_stats(id, 'hll')
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        null sort order: 
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
+                        tag: -1
+                        value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>)
+                        auto parallelism: false
       Path -> Alias:
 #### A masked pattern was here ####
       Path -> Partition:
 #### A masked pattern was here ####
           Partition
-            base file name: Values__Tmp__Table__3
-            input format: org.apache.hadoop.mapred.TextInputFormat
+            base file name: dummy_path
+            input format: org.apache.hadoop.hive.ql.io.NullRowsInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
               bucket_count -1
               column.name.delimiter ,
-              columns tmp_values_col1
+              columns 
               columns.comments 
-              columns.types string
+              columns.types 
 #### A masked pattern was here ####
-              name default.values__tmp__table__3
-              serialization.ddl struct values__tmp__table__3 { string tmp_values_col1}
+              name _dummy_database._dummy_table
+              serialization.ddl struct _dummy_table { }
               serialization.format 1
-              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              serialization.lib org.apache.hadoop.hive.serde2.NullStructSerDe
+            serde: org.apache.hadoop.hive.serde2.NullStructSerDe
           
-              input format: org.apache.hadoop.mapred.TextInputFormat
+              input format: org.apache.hadoop.hive.ql.io.NullRowsInputFormat
               output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
               properties:
                 bucket_count -1
                 column.name.delimiter ,
-                columns tmp_values_col1
+                columns 
                 columns.comments 
-                columns.types string
+                columns.types 
 #### A masked pattern was here ####
-                name default.values__tmp__table__3
-                serialization.ddl struct values__tmp__table__3 { string tmp_values_col1}
+                name _dummy_database._dummy_table
+                serialization.ddl struct _dummy_table { }
                 serialization.format 1
-                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              name: default.values__tmp__table__3
-            name: default.values__tmp__table__3
+                serialization.lib org.apache.hadoop.hive.serde2.NullStructSerDe
+              serde: org.apache.hadoop.hive.serde2.NullStructSerDe
+              name: _dummy_database._dummy_table
+            name: _dummy_database._dummy_table
       Truncated Path -> Alias:
 #### A masked pattern was here ####
       Needs Tagging: false
@@ -164,13 +176,13 @@ STAGE PLANS:
           aggregations: compute_stats(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
             NumFilesPerFileSink: 1
-            Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: COMPLETE
 #### A masked pattern was here ####
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index d51e5cd..9063f2e 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -35,7 +35,9 @@ disabled.query.files=ql_rewrite_gbtoidx.q,\
   ql_rewrite_gbtoidx_cbo_2.q,\
   rcfile_merge1.q,\
   stats_filemetadata.q,\
-  mm_exim.q
+  mm_exim.q,\
+  cbo_rp_insert.q,\
+  cbo_rp_lineage2.q
 
 # NOTE: Add tests to minitez only if it is very
 # specific to tez and cannot be added to minillap.

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/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 7ca950d..68e6ae3 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
@@ -860,6 +860,15 @@ public final class FunctionRegistry {
       return pcA;
     }
 
+    if (pcA == PrimitiveCategory.VOID) {
+      // Handle NULL, we return the type of pcB
+      return pcB;
+    }
+    if (pcB == PrimitiveCategory.VOID) {
+      // Handle NULL, we return the type of pcA
+      return pcA;
+    }
+
     PrimitiveGrouping pgA = PrimitiveObjectInspectorUtils.getPrimitiveGrouping(pcA);
     PrimitiveGrouping pgB = PrimitiveObjectInspectorUtils.getPrimitiveGrouping(pcB);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java
index 5cc6322..d3dc67d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.Pair;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g
index 3496d9e..b038e81 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g
@@ -87,8 +87,6 @@ fromSource
 @init { gParent.pushMsg("join source", state); }
 @after { gParent.popMsg(state); }
     :
-    virtualTableSource
-    | 
     uniqueJoinToken^ uniqueJoinSource (COMMA! uniqueJoinSource)+
     |
     joinSource
@@ -101,6 +99,8 @@ atomjoinSource
     :
     tableSource (lateralView^)*
     |
+    virtualTableSource (lateralView^)*
+    |
     (subQuerySource) => subQuerySource (lateralView^)*
     |
     partitionedTableFunction (lateralView^)*
@@ -117,7 +117,7 @@ joinSourcePart
 @init { gParent.pushMsg("joinSourcePart", state); }
 @after { gParent.popMsg(state); }
     :
-    (tableSource | subQuerySource | partitionedTableFunction) (lateralView^)*
+    (tableSource | virtualTableSource | subQuerySource | partitionedTableFunction) (lateralView^)*
     ;
 
 uniqueJoinSource
@@ -155,11 +155,14 @@ lateralView
 @init {gParent.pushMsg("lateral view", state); }
 @after {gParent.popMsg(state); }
 	:
-	(KW_LATERAL KW_VIEW KW_OUTER) => KW_LATERAL KW_VIEW KW_OUTER function tableAlias (KW_AS identifier ((COMMA)=> COMMA identifier)*)?
+	(COMMA? KW_LATERAL KW_VIEW KW_OUTER) => KW_LATERAL KW_VIEW KW_OUTER function tableAlias (KW_AS identifier ((COMMA)=> COMMA identifier)*)?
 	-> ^(TOK_LATERAL_VIEW_OUTER ^(TOK_SELECT ^(TOK_SELEXPR function identifier* tableAlias)))
 	|
-	KW_LATERAL KW_VIEW function tableAlias (KW_AS identifier ((COMMA)=> COMMA identifier)*)?
+	COMMA? KW_LATERAL KW_VIEW function tableAlias (KW_AS identifier ((COMMA)=> COMMA identifier)*)?
 	-> ^(TOK_LATERAL_VIEW ^(TOK_SELECT ^(TOK_SELEXPR function identifier* tableAlias)))
+    |
+    COMMA? KW_LATERAL KW_TABLE LPAREN valuesClause RPAREN KW_AS? tableAlias (LPAREN identifier (COMMA identifier)*)? RPAREN
+    -> ^(TOK_LATERAL_VIEW ^(TOK_SELECT ^(TOK_SELEXPR ^(TOK_FUNCTION Identifier["inline"] valuesClause) identifier* tableAlias)))
 	;
 
 tableAlias
@@ -290,51 +293,54 @@ searchCondition
 //in support of SELECT * FROM (VALUES(1,2,3),(4,5,6),...) as FOO(a,b,c) and
 // INSERT INTO <table> (col1,col2,...) VALUES(...),(...),...
 // INSERT INTO <table> (col1,col2,...) SELECT * FROM (VALUES(1,2,3),(4,5,6),...) as Foo(a,b,c)
-valueRowConstructor
-@init { gParent.pushMsg("value row constructor", state); }
+/*
+VALUES(1),(2) means 2 rows, 1 column each.
+VALUES(1,2),(3,4) means 2 rows, 2 columns each.
+VALUES(1,2,3) means 1 row, 3 columns
+*/
+valuesClause
+@init { gParent.pushMsg("values clause", state); }
 @after { gParent.popMsg(state); }
     :
-    expressionsInParenthesis[false] -> ^(TOK_VALUE_ROW expressionsInParenthesis)
+    KW_VALUES valuesTableConstructor -> ^(TOK_FUNCTION Identifier["array"] valuesTableConstructor)
     ;
 
 valuesTableConstructor
 @init { gParent.pushMsg("values table constructor", state); }
 @after { gParent.popMsg(state); }
     :
-    valueRowConstructor (COMMA valueRowConstructor)* -> ^(TOK_VALUES_TABLE valueRowConstructor+)
+    valueRowConstructor (COMMA! valueRowConstructor)*
     ;
 
-/*
-VALUES(1),(2) means 2 rows, 1 column each.
-VALUES(1,2),(3,4) means 2 rows, 2 columns each.
-VALUES(1,2,3) means 1 row, 3 columns
-*/
-valuesClause
-@init { gParent.pushMsg("values clause", state); }
+valueRowConstructor
+@init { gParent.pushMsg("value row constructor", state); }
 @after { gParent.popMsg(state); }
     :
-    KW_VALUES! valuesTableConstructor
+    expressionsInParenthesis[true, true]
     ;
 
 /*
 This represents a clause like this:
-(VALUES(1,2),(2,3)) as VirtTable(col1,col2)
+TABLE(VALUES(1,2),(2,3)) as VirtTable(col1,col2)
 */
 virtualTableSource
 @init { gParent.pushMsg("virtual table source", state); }
 @after { gParent.popMsg(state); }
-   :
-   LPAREN valuesClause RPAREN tableNameColList -> ^(TOK_VIRTUAL_TABLE tableNameColList valuesClause)
-   ;
-/*
-e.g. as VirtTable(col1,col2)
-Note that we only want literals as column names
-*/
-tableNameColList
-@init { gParent.pushMsg("from source", state); }
-@after { gParent.popMsg(state); }
     :
-    KW_AS? identifier LPAREN identifier (COMMA identifier)* RPAREN -> ^(TOK_VIRTUAL_TABREF ^(TOK_TABNAME identifier) ^(TOK_COL_NAME identifier+))
+    KW_TABLE LPAREN valuesClause RPAREN KW_AS? tabAlias=tableAlias (LPAREN identifier (COMMA identifier)*)? RPAREN
+    -> ^(TOK_SUBQUERY
+         ^(TOK_QUERY
+           ^(TOK_FROM
+             ^(TOK_SUBQUERY
+               ^(TOK_QUERY
+                 ^(TOK_INSERT
+                   ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
+                   ^(TOK_SELECT ^(TOK_SELEXPR IntegralLiteral["0"]))))
+               {adaptor.create(Identifier, $tabAlias.tree.getChild(0).getText())}))
+           ^(TOK_INSERT
+             ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
+             ^(TOK_SELECT ^(TOK_SELEXPR ^(TOK_FUNCTION Identifier["inline"] valuesClause) identifier*))))
+         {adaptor.create(Identifier, $tabAlias.tree.getChild(0).getText())})
     ;
 
 //-----------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 3360c0e..9073623 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -370,11 +370,6 @@ TOK_SHOW_TRANSACTIONS;
 TOK_DELETE_FROM;
 TOK_UPDATE_TABLE;
 TOK_SET_COLUMNS_CLAUSE;
-TOK_VALUE_ROW;
-TOK_VALUES_TABLE;
-TOK_VIRTUAL_TABLE;
-TOK_VIRTUAL_TABREF;
-TOK_ANONYMOUS;
 TOK_COL_NAME;
 TOK_URI_TYPE;
 TOK_SERVER_TYPE;
@@ -2682,8 +2677,7 @@ singleFromStatement
 The valuesClause rule below ensures that the parse tree for
 "insert into table FOO values (1,2),(3,4)" looks the same as
 "insert into table FOO select a,b from (values(1,2),(3,4)) as BAR(a,b)" which itself is made to look
-very similar to the tree for "insert into table FOO select a,b from BAR".  Since virtual table name
-is implicit, it's represented as TOK_ANONYMOUS.
+very similar to the tree for "insert into table FOO select a,b from BAR".
 */
 regularBody
    :
@@ -2694,10 +2688,7 @@ regularBody
      |
      valuesClause
       -> ^(TOK_QUERY
-            ^(TOK_FROM
-              ^(TOK_VIRTUAL_TABLE ^(TOK_VIRTUAL_TABREF ^(TOK_ANONYMOUS)) valuesClause)
-             )
-            ^(TOK_INSERT {$i.tree} ^(TOK_SELECT ^(TOK_SELEXPR TOK_ALLCOLREF)))
+            ^(TOK_INSERT {$i.tree} ^(TOK_SELECT ^(TOK_SELEXPR ^(TOK_FUNCTION Identifier["inline"] valuesClause))))
           )
    )
    |

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/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 2355c9a..35f9edf 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
@@ -82,7 +82,7 @@ rollupOldSyntax
 @init { gParent.pushMsg("rollup old syntax", state); }
 @after { gParent.popMsg(state); }
     :
-    expr=expressionsNotInParenthesis[false]
+    expr=expressionsNotInParenthesis[false, false]
     ((rollup=KW_WITH KW_ROLLUP) | (cube=KW_WITH KW_CUBE)) ?
     (sets=KW_GROUPING KW_SETS
     LPAREN groupingSetExpression ( COMMA groupingSetExpression)*  RPAREN ) ?
@@ -133,14 +133,16 @@ havingCondition
     expression
     ;
 
-expressionsInParenthesis[boolean isStruct]
+expressionsInParenthesis[boolean isStruct, boolean forceStruct]
     :
-    LPAREN! expressionsNotInParenthesis[isStruct] RPAREN!
+    LPAREN! expressionsNotInParenthesis[isStruct, forceStruct] RPAREN!
     ;
 
-expressionsNotInParenthesis[boolean isStruct]
+expressionsNotInParenthesis[boolean isStruct, boolean forceStruct]
     :
     first=expression more=expressionPart[$expression.tree, isStruct]?
+    -> {forceStruct && more==null}?
+       ^(TOK_FUNCTION Identifier["struct"] {$first.tree})
     -> {more==null}?
        {$first.tree}
     -> {$more.tree}
@@ -155,9 +157,9 @@ expressionPart[CommonTree t, boolean isStruct]
 
 expressions
     :
-    (expressionsInParenthesis[false]) => expressionsInParenthesis[false]
+    (expressionsInParenthesis[false, false]) => expressionsInParenthesis[false, false]
     |
-    expressionsNotInParenthesis[false]
+    expressionsNotInParenthesis[false, false]
     ;
 
 columnRefOrderInParenthesis
@@ -431,7 +433,7 @@ atomExpression
         -> ^(TOK_SUBQUERY_EXPR TOK_SUBQUERY_OP subQueryExpression)
     | (function) => function
     | tableOrColumn
-    | expressionsInParenthesis[true]
+    | expressionsInParenthesis[true, false]
     ;
 
 precedenceFieldExpression
@@ -580,10 +582,10 @@ precedenceSimilarExpressionAtom[CommonTree t]
     KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression)
     -> ^(TOK_FUNCTION Identifier["between"] KW_FALSE {$t} $min $max)
     |
-    KW_LIKE KW_ANY (expr=expressionsInParenthesis[false])
+    KW_LIKE KW_ANY (expr=expressionsInParenthesis[false, false])
     -> ^(TOK_FUNCTION Identifier["likeany"] {$t} {$expr.tree})
     |
-    KW_LIKE KW_ALL (expr=expressionsInParenthesis[false])
+    KW_LIKE KW_ALL (expr=expressionsInParenthesis[false, false])
     -> ^(TOK_FUNCTION Identifier["likeall"] {$t} {$expr.tree})
     ;
 
@@ -591,7 +593,7 @@ precedenceSimilarExpressionIn[CommonTree t]
     :
     (subQueryExpression) => subQueryExpression -> ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_IN) subQueryExpression {$t})
     |
-    expr=expressionsInParenthesis[false]
+    expr=expressionsInParenthesis[false, false]
     -> ^(TOK_FUNCTION Identifier["in"] {$t} {$expr.tree})
     ;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java
index bfe2ab5..e4ac017 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java
@@ -60,7 +60,6 @@ public class QB {
   private boolean isAnalyzeRewrite;
   private CreateTableDesc tblDesc = null; // table descriptor of the final
   private CreateTableDesc directoryDesc = null ;
-  private List<Path> encryptedTargetTablePaths;
   private boolean insideView;
   private Set<String> aliasInsideView;
 
@@ -424,23 +423,6 @@ public class QB {
     return viewDesc != null && !viewDesc.isMaterialized();
   }
 
-  void addEncryptedTargetTablePath(Path p) {
-    if(encryptedTargetTablePaths == null) {
-      encryptedTargetTablePaths = new ArrayList<>();
-    }
-    encryptedTargetTablePaths.add(p);
-  }
-  /**
-   * List of dbName.tblName of encrypted target tables of insert statement
-   * Used to support Insert ... values(...)
-   */
-  List<Path> getEncryptedTargetTablePaths() {
-    if(encryptedTargetTablePaths == null) {
-      return Collections.emptyList();
-    }
-    return encryptedTargetTablePaths;
-  }
-
   public HashMap<String, Table> getViewToTabSchema() {
     return viewAliasToViewSchema;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/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 5799b07..c558356 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
@@ -874,168 +874,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     out.write(to.getBytes(), 0, to.getLength());
   }
 
-  /**
-   * Generate a temp table out of a values clause
-   * See also {@link #preProcessForInsert(ASTNode, QB)}
-   */
-  private ASTNode genValuesTempTable(ASTNode originalFrom, QB qb) throws SemanticException {
-    Path dataDir = null;
-    if(!qb.getEncryptedTargetTablePaths().isEmpty()) {
-      //currently only Insert into T values(...) is supported thus only 1 values clause
-      //and only 1 target table are possible.  If/when support for
-      //select ... from values(...) is added an insert statement may have multiple
-      //encrypted target tables.
-      dataDir = ctx.getMRTmpPath(qb.getEncryptedTargetTablePaths().get(0).toUri());
-    }
-    // Pick a name for the table
-    SessionState ss = SessionState.get();
-    String tableName = VALUES_TMP_TABLE_NAME_PREFIX + ss.getNextValuesTempTableSuffix();
-
-    // Step 1, parse the values clause we were handed
-    List<? extends Node> fromChildren = originalFrom.getChildren();
-    // First child should be the virtual table ref
-    ASTNode virtualTableRef = (ASTNode)fromChildren.get(0);
-    assert virtualTableRef.getToken().getType() == HiveParser.TOK_VIRTUAL_TABREF :
-        "Expected first child of TOK_VIRTUAL_TABLE to be TOK_VIRTUAL_TABREF but was " +
-            virtualTableRef.getName();
-
-    List<? extends Node> virtualTableRefChildren = virtualTableRef.getChildren();
-    // First child of this should be the table name.  If it's anonymous,
-    // then we don't have a table name.
-    ASTNode tabName = (ASTNode)virtualTableRefChildren.get(0);
-    if (tabName.getToken().getType() != HiveParser.TOK_ANONYMOUS) {
-      // TODO, if you want to make select ... from (values(...) as foo(...) work,
-      // you need to parse this list of columns names and build it into the table
-      throw new SemanticException(ErrorMsg.VALUES_TABLE_CONSTRUCTOR_NOT_SUPPORTED.getMsg());
-    }
-
-    // The second child of the TOK_VIRTUAL_TABLE should be TOK_VALUES_TABLE
-    ASTNode valuesTable = (ASTNode)fromChildren.get(1);
-    assert valuesTable.getToken().getType() == HiveParser.TOK_VALUES_TABLE :
-        "Expected second child of TOK_VIRTUAL_TABLE to be TOK_VALUE_TABLE but was " +
-            valuesTable.getName();
-    // Each of the children of TOK_VALUES_TABLE will be a TOK_VALUE_ROW
-    List<? extends Node> valuesTableChildren = valuesTable.getChildren();
-
-    // Now that we're going to start reading through the rows, open a file to write the rows too
-    // If we leave this method before creating the temporary table we need to be sure to clean up
-    // this file.
-    Path tablePath = null;
-    FileSystem fs = null;
-    FSDataOutputStream out = null;
-    try {
-      if(dataDir == null) {
-        tablePath = Warehouse.getDnsPath(new Path(ss.getTempTableSpace(), tableName), conf);
-      }
-      else {
-        //if target table of insert is encrypted, make sure temporary table data is stored
-        //similarly encrypted
-        tablePath = Warehouse.getDnsPath(new Path(dataDir, tableName), conf);
-      }
-      fs = tablePath.getFileSystem(conf);
-      fs.mkdirs(tablePath);
-      Path dataFile = new Path(tablePath, "data_file");
-      out = fs.create(dataFile);
-      List<FieldSchema> fields = new ArrayList<FieldSchema>();
-
-      boolean firstRow = true;
-      for (Node n : valuesTableChildren) {
-        ASTNode valuesRow = (ASTNode) n;
-        assert valuesRow.getToken().getType() == HiveParser.TOK_VALUE_ROW :
-            "Expected child of TOK_VALUE_TABLE to be TOK_VALUE_ROW but was " + valuesRow.getName();
-        // Each of the children of this should be a literal
-        List<? extends Node> valuesRowChildren = valuesRow.getChildren();
-        boolean isFirst = true;
-        int nextColNum = 1;
-        for (Node n1 : valuesRowChildren) {
-          ASTNode value = (ASTNode) n1;
-          if (firstRow) {
-            fields.add(new FieldSchema("tmp_values_col" + nextColNum++, "string", ""));
-          }
-          if (isFirst) {
-            isFirst = false;
-          } else {
-            writeAsText("\u0001", out);
-          }
-          writeAsText(unparseExprForValuesClause(value), out);
-        }
-        writeAsText("\n", out);
-        firstRow = false;
-      }
-
-      // Step 2, create a temp table, using the created file as the data
-      StorageFormat format = new StorageFormat(conf);
-      format.processStorageFormat("TextFile");
-      Table table = db.newTable(tableName);
-      table.setSerializationLib(format.getSerde());
-      table.setFields(fields);
-      table.setDataLocation(tablePath);
-      table.getTTable().setTemporary(true);
-      table.setStoredAsSubDirectories(false);
-      table.setInputFormatClass(format.getInputFormat());
-      table.setOutputFormatClass(format.getOutputFormat());
-      db.createTable(table, false);
-    } catch (Exception e) {
-      String errMsg = ErrorMsg.INSERT_CANNOT_CREATE_TEMP_FILE.getMsg() + e.getMessage();
-      LOG.error(errMsg);
-      // Try to delete the file
-      if (fs != null && tablePath != null) {
-        try {
-          fs.delete(tablePath, false);
-        } catch (IOException swallowIt) {}
-      }
-      throw new SemanticException(errMsg, e);
-    } finally {
-        IOUtils.closeStream(out);
-    }
-
-    // Step 3, return a new subtree with a from clause built around that temp table
-    // The form of the tree is TOK_TABREF->TOK_TABNAME->identifier(tablename)
-    Token t = new ClassicToken(HiveParser.TOK_TABREF);
-    ASTNode tabRef = new ASTNode(t);
-    t = new ClassicToken(HiveParser.TOK_TABNAME);
-    ASTNode tabNameNode = new ASTNode(t);
-    tabRef.addChild(tabNameNode);
-    t = new ClassicToken(HiveParser.Identifier, tableName);
-    ASTNode identifier = new ASTNode(t);
-    tabNameNode.addChild(identifier);
-    return tabRef;
-  }
-
-  // Take an expression in the values clause and turn it back into a string.  This is far from
-  // comprehensive.  At the moment it only supports:
-  // * literals (all types)
-  // * unary negatives
-  // * true/false
-  private String unparseExprForValuesClause(ASTNode expr) throws SemanticException {
-    switch (expr.getToken().getType()) {
-      case HiveParser.Number:
-        return expr.getText();
-
-      case HiveParser.StringLiteral:
-        return BaseSemanticAnalyzer.unescapeSQLString(expr.getText());
-
-      case HiveParser.KW_FALSE:
-        // UDFToBoolean casts any non-empty string to true, so set this to false
-        return "";
-
-      case HiveParser.KW_TRUE:
-        return "TRUE";
-
-      case HiveParser.MINUS:
-        return "-" + unparseExprForValuesClause((ASTNode)expr.getChildren().get(0));
-
-      case HiveParser.TOK_NULL:
-        // Hive's text input will translate this as a null
-        return "\\N";
-
-      default:
-        throw new SemanticException("Expression of type " + expr.getText() +
-            " not supported in insert/values");
-    }
-
-  }
-
   private void assertCombineInputFormat(Tree numerator, String message) throws SemanticException {
     String inputFormat = conf.getVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez") ?
       HiveConf.getVar(conf, HiveConf.ConfVars.HIVETEZINPUTFORMAT):
@@ -1562,12 +1400,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         ASTNode frm = (ASTNode) ast.getChild(0);
         if (frm.getToken().getType() == HiveParser.TOK_TABREF) {
           processTable(qb, frm);
-        } else if (frm.getToken().getType() == HiveParser.TOK_VIRTUAL_TABLE) {
-          // Create a temp table with the passed values in it then rewrite this portion of the
-          // tree to be from that table.
-          ASTNode newFrom = genValuesTempTable(frm, qb);
-          ast.setChild(0, newFrom);
-          processTable(qb, newFrom);
         } else if (frm.getToken().getType() == HiveParser.TOK_SUBQUERY) {
           processSubQuery(qb, frm);
         } else if (frm.getToken().getType() == HiveParser.TOK_LATERAL_VIEW ||
@@ -11426,7 +11258,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
     // 4. continue analyzing from the child ASTNode.
     Phase1Ctx ctx_1 = initPhase1Ctx();
-    preProcessForInsert(child, qb);
     if (!doPhase1(child, qb, ctx_1, plannerCtx)) {
       // if phase1Result false return
       return false;
@@ -11443,49 +11274,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     return true;
   }
 
-  /**
-   * This will walk AST of an INSERT statement and assemble a list of target tables
-   * which are in an HDFS encryption zone.  This is needed to make sure that so that
-   * the data from values clause of Insert ... select values(...) is stored securely.
-   * See also {@link #genValuesTempTable(ASTNode, QB)}
-   * @throws SemanticException
-   */
-  private void preProcessForInsert(ASTNode node, QB qb) throws SemanticException {
-    try {
-      if(!(node != null && node.getToken() != null && node.getToken().getType() == HiveParser.TOK_QUERY)) {
-        return;
-      }
-      for (Node child : node.getChildren()) {
-        //each insert of multi insert looks like
-        //(TOK_INSERT (TOK_INSERT_INTO (TOK_TAB (TOK_TABNAME T1)))
-        if (((ASTNode) child).getToken().getType() != HiveParser.TOK_INSERT) {
-          continue;
-        }
-        ASTNode n = (ASTNode) ((ASTNode) child).getFirstChildWithType(HiveParser.TOK_INSERT_INTO);
-        if (n == null) {
-          continue;
-        }
-        n = (ASTNode) n.getFirstChildWithType(HiveParser.TOK_TAB);
-        if (n == null) {
-          continue;
-        }
-        n = (ASTNode) n.getFirstChildWithType(HiveParser.TOK_TABNAME);
-        if (n == null) {
-          continue;
-        }
-        String[] dbTab = getQualifiedTableName(n);
-        Table t = db.getTable(dbTab[0], dbTab[1]);
-        Path tablePath = t.getPath();
-        if (isPathEncrypted(tablePath)) {
-          qb.addEncryptedTargetTablePath(tablePath);
-        }
-      }
-    }
-    catch(Exception ex) {
-      throw new SemanticException(ex);
-    }
-  }
-
   public void getHintsFromQB(QB qb, List<ASTNode> hints) {
     if (qb.getParseInfo().getHints() != null) {
       hints.add(qb.getParseInfo().getHints());

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
index 28131a0..ec9c3b3 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
@@ -125,9 +125,10 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(driver.compileAndRespond("insert into T values (1,2)"));
     txnMgr.acquireLocks(driver.getPlan(), ctx, "Fifer");
     List<ShowLocksResponseElement> locks = getLocks();
-    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
     //since LM is using non strict mode we get shared lock
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "T", null, locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
 
     //simulate concurrent session
     HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
@@ -135,8 +136,9 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(driver.compileAndRespond("alter table T SET TBLPROPERTIES ('transactional'='true')"));
     ((DbTxnManager)txnMgr2).acquireLocks(driver.getPlan(), ctx, "Fiddler", false);
     locks = getLocks();
-    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    Assert.assertEquals("Unexpected lock count", 3, locks.size());
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "T", null, locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
     checkLock(LockType.EXCLUSIVE, LockState.WAITING, "default", "T", null, locks);
     txnMgr2.rollbackTxn();
     txnMgr.commitTxn();
@@ -630,32 +632,36 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(cpr);
     LockState lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
     List<ShowLocksResponseElement> locks = getLocks();
-    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
     checkLock(LockType.EXCLUSIVE, LockState.ACQUIRED, "default", "nonAcidPart", null, locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
     txnMgr.rollbackTxn();;
 
     cpr = driver.compileAndRespond("insert into nonAcidPart partition(p=1) values(5,6)");
     checkCmdOnDriver(cpr);
     lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
     locks = getLocks();
-    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
     checkLock(LockType.EXCLUSIVE, LockState.ACQUIRED, "default", "nonAcidPart", "p=1", locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
     txnMgr.rollbackTxn();
 
     cpr = driver.compileAndRespond("insert into acidPart partition(p) values(1,2,3)");
     checkCmdOnDriver(cpr);
     lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
     locks = getLocks();
-    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "acidPart", null, locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
     txnMgr.rollbackTxn();
 
     cpr = driver.compileAndRespond("insert into acidPart partition(p=1) values(5,6)");
     checkCmdOnDriver(cpr);
     lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
     locks = getLocks();
-    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "acidPart", "p=1", locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
     txnMgr.rollbackTxn();
     
     cpr = driver.compileAndRespond("update acidPart set b = 17 where a = 1");
@@ -703,7 +709,7 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(driver.compileAndRespond("insert into tab_not_acid partition(np='doh') values(5,6)"));
     LockState ls = ((DbTxnManager)txnMgr2).acquireLocks(driver.getPlan(), ctx, "T2", false);
     locks = getLocks(txnMgr2);
-    Assert.assertEquals("Unexpected lock count", 7, locks.size());
+    Assert.assertEquals("Unexpected lock count", 8, locks.size());
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "tab_acid", null, locks);
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "tab_acid", "p=bar", locks);
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "tab_acid", "p=foo", locks);
@@ -711,6 +717,7 @@ public class TestDbTxnManager2 {
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "tab_not_acid", "np=blah", locks);
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "tab_not_acid", "np=doh", locks);
     checkLock(LockType.EXCLUSIVE, LockState.WAITING, "default", "tab_not_acid", "np=doh", locks);
+    checkLock(LockType.SHARED_READ, LockState.WAITING, "_dummy_database", "_dummy_table", null, locks);
 
     // Test strict locking mode, i.e. backward compatible locking mode for non-ACID resources.
     // With non-strict mode, INSERT got SHARED_READ lock, instead of EXCLUSIVE with ACID semantics
@@ -720,7 +727,7 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(driver.compileAndRespond("insert into tab_not_acid partition(np='blah') values(7,8)"));
     ((DbTxnManager)txnMgr3).acquireLocks(driver.getPlan(), ctx, "T3", false);
     locks = getLocks(txnMgr3);
-    Assert.assertEquals("Unexpected lock count", 8, locks.size());
+    Assert.assertEquals("Unexpected lock count", 10, locks.size());
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "tab_acid", null, locks);
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "tab_acid", "p=bar", locks);
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "tab_acid", "p=foo", locks);
@@ -728,7 +735,9 @@ public class TestDbTxnManager2 {
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "tab_not_acid", "np=blah", locks);
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "tab_not_acid", "np=doh", locks);
     checkLock(LockType.EXCLUSIVE, LockState.WAITING, "default", "tab_not_acid", "np=doh", locks);
+    checkLock(LockType.SHARED_READ, LockState.WAITING, "_dummy_database", "_dummy_table", null, locks);
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "tab_not_acid", "np=blah", locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
     conf.setBoolVar(HiveConf.ConfVars.HIVE_TXN_STRICT_LOCKING_MODE, true);
   }
 
@@ -823,12 +832,13 @@ public class TestDbTxnManager2 {
 
     // SHOW LOCKS (no filter)
     List<ShowLocksResponseElement> locks = getLocks();
-    Assert.assertEquals("Unexpected lock count", 5, locks.size());
+    Assert.assertEquals("Unexpected lock count", 7, locks.size());
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "db1", "t14", "ds=today", locks);
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "db1", "t14", "ds=tomorrow", locks);
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "db2", "t15", null, locks);
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "db2", "t16", null, locks);
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "db2", "t14", null, locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
 
     // SHOW LOCKS db2
     locks = getLocksWithFilterOptions(txnMgr3, "db2", null, null);
@@ -1822,9 +1832,10 @@ public class TestDbTxnManager2 {
       TxnDbUtil.countQueryAgent(conf, "select count(*) from TXN_COMPONENTS where tc_txnid=" + txnid1));
 
     List<ShowLocksResponseElement> locks = getLocks(txnMgr);
-    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
     checkLock(causeConflict ? LockType.SHARED_WRITE : LockType.SHARED_READ,
       LockState.ACQUIRED, "default", "target", null, locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
 
     DbTxnManager txnMgr2 = (DbTxnManager) TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
     swapTxnManager(txnMgr2);
@@ -1837,8 +1848,9 @@ public class TestDbTxnManager2 {
     txnMgr2.acquireLocks(driver.getPlan(), ctx, "T2", false);
     locks = getLocks();
 
-    Assert.assertEquals("Unexpected lock count", 3, locks.size());
+    Assert.assertEquals("Unexpected lock count", 4, locks.size());
     checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "target", null, locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
     checkLock(LockType.SHARED_READ, causeConflict ? LockState.WAITING : LockState.ACQUIRED,
       "default", "source", null, locks);
     long extLockId = checkLock(LockType.SHARED_WRITE, causeConflict ? LockState.WAITING : LockState.ACQUIRED,
@@ -1909,13 +1921,14 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(driver.compileAndRespond("insert into target partition(p=1,q) values (1,2,2), (3,4,2), (5,6,3), (7,8,2)"));
     txnMgr.acquireLocks(driver.getPlan(), ctx, "T1");
     List<ShowLocksResponseElement> locks = getLocks(txnMgr);
-    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
     //table is empty, so can only lock the table
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "target", null, locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
     Assert.assertEquals(
       "HIVE_LOCKS mismatch(" + JavaUtils.txnIdToString(txnid1) + "): " +
         TxnDbUtil.queryToString(conf, "select * from HIVE_LOCKS"),
-      1,
+      2,
       TxnDbUtil.countQueryAgent(conf, "select count(*) from HIVE_LOCKS where hl_txnid=" + txnid1));
     txnMgr.rollbackTxn();
     Assert.assertEquals(
@@ -1941,9 +1954,10 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(driver.compileAndRespond("insert into target partition(p=1,q) values (10,2,2), (30,4,2), (50,6,3), (70,8,2)"));
     txnMgr.acquireLocks(driver.getPlan(), ctx, "T1");
     locks = getLocks(txnMgr);
-    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
     //Plan is using DummyPartition, so can only lock the table... unfortunately
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "target", null, locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
     AddDynamicPartitions adp = new AddDynamicPartitions(txnid2, "default", "target", Arrays.asList("p=1/q=2","p=1/q=2"));
     adp.setOperationType(DataOperationType.INSERT);
     txnHandler.addDynamicPartitions(adp);
@@ -2118,8 +2132,9 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(driver.compileAndRespond("insert into T values(1,3)"));
     txnMgr.acquireLocks(driver.getPlan(), ctx, "Fifer");
     List<ShowLocksResponseElement> locks = getLocks();
-    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
     checkLock(LockType.EXCLUSIVE, LockState.ACQUIRED, "default", "t", null, locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
 
     DbTxnManager txnMgr2 = (DbTxnManager) TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
     txnMgr2.openTxn(ctx, "Fidler");
@@ -2127,8 +2142,9 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(driver.compileAndRespond("show tables"));
     txnMgr2.acquireLocks(driver.getPlan(), ctx, "Fidler");
     locks = getLocks();
-    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    Assert.assertEquals("Unexpected lock count", 3, locks.size());
     checkLock(LockType.EXCLUSIVE, LockState.ACQUIRED, "default", "t", null, locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", null, null, locks);
     txnMgr.commitTxn();
     txnMgr2.rollbackTxn();
@@ -2145,8 +2161,9 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(driver.compileAndRespond("insert into T2 partition(p=1) values(1,3)"));
     txnMgr.acquireLocks(driver.getPlan(), ctx, "Fifer");
     locks = getLocks();
-    Assert.assertEquals("Unexpected lock count", 1, locks.size());
+    Assert.assertEquals("Unexpected lock count", 2, locks.size());
     checkLock(LockType.EXCLUSIVE, LockState.ACQUIRED, "default", "t2", "p=1", locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
 
     txnMgr2 = (DbTxnManager) TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
     txnMgr2.openTxn(ctx, "Fidler");
@@ -2154,8 +2171,9 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(driver.compileAndRespond("show tables"));
     txnMgr2.acquireLocks(driver.getPlan(), ctx, "Fidler", false);
     locks = getLocks();
-    Assert.assertEquals("Unexpected lock count", 2, locks.size());
+    Assert.assertEquals("Unexpected lock count", 3, locks.size());
     checkLock(LockType.EXCLUSIVE, LockState.ACQUIRED, "default", "t2", "p=1", locks);
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "_dummy_database", "_dummy_table", null, locks);
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", null, null, locks);
     txnMgr.commitTxn();
     txnMgr2.commitTxn();

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/org/apache/hadoop/hive/ql/parse/TestIUD.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestIUD.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestIUD.java
index 5f8ed93..bda057d 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestIUD.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestIUD.java
@@ -166,142 +166,6 @@ public class TestIUD {
     }
   }
   @Test
-  public void testSelectStarFromVirtTable1Row() throws ParseException {
-    ASTNode ast = parse("select * from (values (3,4)) as vc(a,b)");
-    Assert.assertEquals("AST doesn't match",
-      "(tok_query " +
-        "(tok_from " +
-          "(tok_virtual_table " +
-            "(tok_virtual_tabref (tok_tabname vc) (tok_col_name a b)) " +
-            "(tok_values_table (tok_value_row 3 4)))) " +
-        "(tok_insert (tok_destination (tok_dir tok_tmp_file)) (tok_select (tok_selexpr tok_allcolref))))",
-      ast.toStringTree());
-  }
-  @Test
-  public void testSelectStarFromVirtTable2Row() throws ParseException {
-    ASTNode ast = parse("select * from (values (1,2),(3,4)) as vc(a,b)");
-    Assert.assertEquals("AST doesn't match",
-      "(tok_query " +
-        "(tok_from " +
-          "(tok_virtual_table " +
-            "(tok_virtual_tabref (tok_tabname vc) (tok_col_name a b)) " +
-            "(tok_values_table (tok_value_row 1 2) (tok_value_row 3 4)))) " +
-        "(tok_insert (tok_destination (tok_dir tok_tmp_file)) (tok_select (tok_selexpr tok_allcolref))))",
-      ast.toStringTree());
-  }
-  @Test
-  public void testSelectStarFromVirtTable2RowNamedProjections() throws ParseException {
-    ASTNode ast = parse("select a as c, b as d from (values (1,2),(3,4)) as vc(a,b)");
-    Assert.assertEquals("AST doesn't match",
-      "(tok_query " +
-        "(tok_from " +
-        "(tok_virtual_table " +
-          "(tok_virtual_tabref (tok_tabname vc) (tok_col_name a b)) " +
-          "(tok_values_table (tok_value_row 1 2) (tok_value_row 3 4)))) " +
-        "(tok_insert (tok_destination (tok_dir tok_tmp_file)) " +
-          "(tok_select (tok_selexpr (tok_table_or_col a) c) (tok_selexpr (tok_table_or_col b) d))))",
-      ast.toStringTree());
-  }
-  @Test
-  public void testInsertIntoTableAsSelectFromNamedVirtTable() throws ParseException {
-    ASTNode ast = parse("insert into page_view select a,b as c from (values (1,2),(3,4)) as vc(a,b) where b = 9");
-    Assert.assertEquals("AST doesn't match",
-      "(tok_query " +
-        "(tok_from " +
-          "(tok_virtual_table " +
-            "(tok_virtual_tabref (tok_tabname vc) (tok_col_name a b)) " +
-            "(tok_values_table (tok_value_row 1 2) (tok_value_row 3 4)))) " +
-        "(tok_insert (tok_insert_into (tok_tab (tok_tabname page_view))) " +
-          "(tok_select " +
-            "(tok_selexpr (tok_table_or_col a)) " +
-            "(tok_selexpr (tok_table_or_col b) c)) " +
-          "(tok_where (= (tok_table_or_col b) 9))))",
-      ast.toStringTree());
-  }
-  /**
-   * same as testInsertIntoTableAsSelectFromNamedVirtTable but with column list on target table
-   * @throws ParseException
-   */
-  @Test
-  public void testInsertIntoTableAsSelectFromNamedVirtTableNamedCol() throws ParseException {
-    ASTNode ast = parse("insert into page_view(c1,c2) select a,b as c from (values (1,2),(3,4)) as vc(a,b) where b = 9");
-    Assert.assertEquals("AST doesn't match",
-      "(tok_query " +
-        "(tok_from " +
-        "(tok_virtual_table " +
-        "(tok_virtual_tabref (tok_tabname vc) (tok_col_name a b)) " +
-        "(tok_values_table (tok_value_row 1 2) (tok_value_row 3 4)))) " +
-        "(tok_insert (tok_insert_into (tok_tab (tok_tabname page_view)) (tok_tabcolname c1 c2)) " +
-        "(tok_select " +
-        "(tok_selexpr (tok_table_or_col a)) " +
-        "(tok_selexpr (tok_table_or_col b) c)) " +
-        "(tok_where (= (tok_table_or_col b) 9))))",
-      ast.toStringTree());
-  }
-  @Test
-  public void testInsertIntoTableFromAnonymousTable1Row() throws ParseException {
-    ASTNode ast = parse("insert into page_view values(1,2)");
-    Assert.assertEquals("AST doesn't match",
-      "(tok_query " +
-        "(tok_from " +
-        "(tok_virtual_table " +
-        "(tok_virtual_tabref tok_anonymous) " +
-        "(tok_values_table (tok_value_row 1 2)))) " +
-        "(tok_insert (tok_insert_into (tok_tab (tok_tabname page_view))) " +
-        "(tok_select (tok_selexpr tok_allcolref))))",
-      ast.toStringTree());
-  }
-  /**
-   * Same as testInsertIntoTableFromAnonymousTable1Row but with column list on target table
-   * @throws ParseException
-   */
-  @Test
-  public void testInsertIntoTableFromAnonymousTable1RowNamedCol() throws ParseException {
-    ASTNode ast = parse("insert into page_view(a,b) values(1,2)");
-    Assert.assertEquals("AST doesn't match",
-      "(tok_query " +
-        "(tok_from " +
-          "(tok_virtual_table " +
-            "(tok_virtual_tabref tok_anonymous) " +
-            "(tok_values_table (tok_value_row 1 2))" +
-          ")" +
-        ") " +
-        "(tok_insert " +
-          "(tok_insert_into " +
-            "(tok_tab (tok_tabname page_view)) " +
-            "(tok_tabcolname a b)" +//this is "extra" piece we get vs previous query
-          ") " +
-          "(tok_select " +
-            "(tok_selexpr tok_allcolref)" +
-          ")" +
-        ")" +
-      ")", ast.toStringTree());
-  }
-  @Test
-  public void testInsertIntoTableFromAnonymousTable() throws ParseException {
-    ASTNode ast = parse("insert into table page_view values(-1,2),(3,+4)");
-    Assert.assertEquals("AST doesn't match",
-      "(tok_query " +
-        "(tok_from " +
-          "(tok_virtual_table " +
-          "(tok_virtual_tabref tok_anonymous) " +
-          "(tok_values_table (tok_value_row (- 1) 2) (tok_value_row 3 (+ 4))))) " +
-        "(tok_insert (tok_insert_into (tok_tab (tok_tabname page_view))) " +
-          "(tok_select (tok_selexpr tok_allcolref))))",
-      ast.toStringTree());
-    //same query as above less the "table" keyword KW_table
-    ast = parse("insert into page_view values(-1,2),(3,+4)");
-    Assert.assertEquals("AST doesn't match",
-      "(tok_query " +
-        "(tok_from " +
-        "(tok_virtual_table " +
-        "(tok_virtual_tabref tok_anonymous) " +
-        "(tok_values_table (tok_value_row (- 1) 2) (tok_value_row 3 (+ 4))))) " +
-        "(tok_insert (tok_insert_into (tok_tab (tok_tabname page_view))) " +
-        "(tok_select (tok_selexpr tok_allcolref))))",
-      ast.toStringTree());
-  }
-  @Test
   public void testMultiInsert() throws ParseException {
     ASTNode ast = parse("from S insert into T1 select a, b insert into T2 select c, d");
     Assert.assertEquals("AST doesn't match", "(tok_query (tok_from (tok_tabref (tok_tabname s))) " +

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMergeStatement.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMergeStatement.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMergeStatement.java
index 5bdffb5..396d344 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMergeStatement.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMergeStatement.java
@@ -126,7 +126,7 @@ public class TestMergeStatement {
         "(tok_tabref (tok_tabname target)) (tok_tabref (tok_tabname source)) (= (. (tok_table_or_col target) pk) (. (tok_table_or_col source) pk)) " +
         "(tok_not_matched " +
           "(tok_insert " +
-            "(tok_value_row " +
+            "(tok_function struct " +
               "(. (tok_table_or_col source) a) " +
                 "(tok_function when " +
                   "(tok_function isnull (. (tok_table_or_col source) b)) (. (tok_table_or_col target) b) " +
@@ -157,7 +157,7 @@ public class TestMergeStatement {
         ") " +
         "(tok_not_matched " +
           "(tok_insert " +
-            "(tok_value_row " +
+            "(tok_function struct " +
               "(. (tok_table_or_col source) a) " +
               "2 " +
               "(tok_function current_date)" +
@@ -251,6 +251,6 @@ public class TestMergeStatement {
             "(tok_set_columns_clause (= (tok_table_or_col b) (. (tok_table_or_col source) b2))))) " +
         "(tok_not_matched " +
           "(tok_insert " +
-            "(tok_value_row (. (tok_table_or_col source) a2) (. (tok_table_or_col source) b2)))))");
+            "(tok_function struct (. (tok_table_or_col source) a2) (. (tok_table_or_col source) b2)))))");
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/queries/clientpositive/encryption_insert_values.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/encryption_insert_values.q b/ql/src/test/queries/clientpositive/encryption_insert_values.q
index c8d1d51..a712f35 100644
--- a/ql/src/test/queries/clientpositive/encryption_insert_values.q
+++ b/ql/src/test/queries/clientpositive/encryption_insert_values.q
@@ -10,7 +10,4 @@ INSERT INTO encrypted_table values(1,'foo'),(2,'bar');
 
 select * from encrypted_table;
 
--- this checks that we've actually created temp table data under encrypted_table folder 
-describe formatted values__tmp__table__1;
-
 CRYPTO DELETE_KEY --keyName key_128;

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/queries/clientpositive/materialized_view_create.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_create.q b/ql/src/test/queries/clientpositive/materialized_view_create.q
index 030627a..38dcdeb 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_create.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_create.q
@@ -26,7 +26,7 @@ describe extended cmv_mat_view4;
 
 create table cmv_basetable2 (d int, e varchar(256), f decimal(10,2));
 
-insert into cmv_basetable2 values (4, 'alfred', 100.30),(4, 'bob', 6133,14),(5, 'bonnie', 172.2),(6, 'calvin', 8.76),(17, 'charlie', 13144339.8);
+insert into cmv_basetable2 values (4, 'alfred', 100.30),(4, 'bob', 6133.14),(5, 'bonnie', 172.2),(6, 'calvin', 8.76),(17, 'charlie', 13144339.8);
 
 create materialized view cmv_mat_view5 tblproperties ('key'='value') as select a, b, d, c, f from cmv_basetable t1 join cmv_basetable2 t2 on (t1.b = t2.e);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/queries/clientpositive/tablevalues.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/tablevalues.q b/ql/src/test/queries/clientpositive/tablevalues.q
new file mode 100644
index 0000000..501f532
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/tablevalues.q
@@ -0,0 +1,94 @@
+-- VALUES -> array(struct(),struct())
+-- TABLE -> LATERAL VIEW INLINE
+
+CREATE TABLE mytbl AS
+SELECT key, value
+FROM src
+ORDER BY key
+LIMIT 5;
+
+EXPLAIN
+INSERT INTO mytbl(key,value)
+SELECT a,b as c FROM TABLE(VALUES(1,2),(3,4)) AS vc(a,b)
+WHERE b = 9;
+
+INSERT INTO mytbl(key,value)
+SELECT a,b as c FROM TABLE(VALUES(1,2),(3,4)) AS vc(a,b)
+WHERE b = 9;
+
+EXPLAIN
+SELECT tf.col1, tf.col2, tf.col3
+FROM
+  (SELECT key, value FROM mytbl) t
+  LATERAL VIEW
+  INLINE(array(struct('A', 10, t.key),struct('B', 20, t.key))) tf AS col1, col2, col3;
+
+SELECT tf.col1, tf.col2, tf.col3
+FROM
+  (SELECT key, value FROM mytbl) t
+  LATERAL VIEW
+  INLINE(array(struct('A', 10, t.key),struct('B', 20, t.key))) tf AS col1, col2, col3;
+
+EXPLAIN
+SELECT INLINE(array(struct('A', 10, 30),struct('B', 20, 30))) AS (col1, col2, col3);
+
+SELECT INLINE(array(struct('A', 10, 30),struct('B', 20, 30))) AS (col1, col2, col3);
+
+EXPLAIN
+SELECT tf.col1, tf.col2, tf.col3
+FROM
+  TABLE(VALUES('A', 10, 30.0),('B', 20, 30.0)) AS tf(col1, col2, col3);
+
+SELECT tf.col1, tf.col2, tf.col3
+FROM
+  TABLE(VALUES('A', 10, 30),('B', 20, 30)) AS tf(col1, col2, col3);
+
+-- CROSS PRODUCT (CANNOT BE EXPRESSED WITH LVJ)
+EXPLAIN
+SELECT tf.col1, tf.col2, tf.col3
+FROM
+  TABLE(VALUES('A', 10, 30),('B', 20, 30)) AS tf(col1, col2, col3),
+  (SELECT key, value FROM mytbl) t;
+
+SELECT tf.col1, tf.col2, tf.col3
+FROM
+  TABLE(VALUES('A', 10, 30),('B', 20, 30)) AS tf(col1, col2, col3),
+  (SELECT key, value FROM mytbl) t;
+
+-- CROSS PRODUCT (FIRST CANNOT BE EXPRESSED WITH LVJ, SECOND CAN
+-- BUT IT IS NOT NEEDED)
+EXPLAIN
+SELECT tf.col1, tf.col2, tf.col3
+FROM
+  TABLE(VALUES('A', 10, 30),('B', 20, 30)) AS tf(col1, col2, col3),
+  TABLE(VALUES('A', 10),('B', 20)) AS tf2(col1, col2);
+
+SELECT tf.col1, tf.col2, tf.col3
+FROM
+  TABLE(VALUES('A', 10, 30),('B', 20, 30)) AS tf(col1, col2, col3),
+  TABLE(VALUES('A', 10),('B', 20)) AS tf2(col1, col2);
+
+-- CROSS PRODUCT (CAN BE EXPRESSED WITH LVJ BUT IT IS NOT NEEDED)
+EXPLAIN
+SELECT tf.col1, tf.col2, tf.col3
+FROM
+  (SELECT key, value FROM mytbl) t,
+  TABLE(VALUES('A', 10, 30),('B', 20, 30)) AS tf(col1, col2, col3);
+
+SELECT tf.col1, tf.col2, tf.col3
+FROM
+  (SELECT key, value FROM mytbl) t,
+  TABLE(VALUES('A', 10, 30),('B', 20, 30)) AS tf(col1, col2, col3);
+
+-- LVJ (CORRELATED). LATERAL COULD BE OPTIONAL, BUT IF WE MAKE IT
+-- MANDATORY, IT HELPS US DISTINGUISHING FROM PREVIOUS CASE
+EXPLAIN
+SELECT tf.col1, tf.col2, tf.col3
+FROM
+  (SELECT key, value FROM mytbl) t,
+  LATERAL TABLE(VALUES('A', 10, t.key),('B', 20, t.key)) AS tf(col1, col2, col3);
+
+SELECT tf.col1, tf.col2, tf.col3
+FROM
+  (SELECT key, value FROM mytbl) t,
+  LATERAL TABLE(VALUES('A', 10, t.key),('B', 20, t.key)) AS tf(col1, col2, col3);

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/results/clientnegative/authorization_insertpart_noinspriv.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_insertpart_noinspriv.q.out b/ql/src/test/results/clientnegative/authorization_insertpart_noinspriv.q.out
index afe21fd..a524b97 100644
--- a/ql/src/test/results/clientnegative/authorization_insertpart_noinspriv.q.out
+++ b/ql/src/test/results/clientnegative/authorization_insertpart_noinspriv.q.out
@@ -25,6 +25,7 @@ PREHOOK: type: QUERY
 POSTHOOK: query: explain authorization insert into table testp partition (dt = '2012')  values (1)
 POSTHOOK: type: QUERY
 INPUTS: 
+  _dummy_database@_dummy_table
 OUTPUTS: 
   default@testp@dt=2012
 CURRENT_USER: 
@@ -38,6 +39,7 @@ PREHOOK: type: QUERY
 POSTHOOK: query: explain authorization insert overwrite table testp partition (dt = '2012')  values (1)
 POSTHOOK: type: QUERY
 INPUTS: 
+  _dummy_database@_dummy_table
 OUTPUTS: 
   default@testp@dt=2012
 CURRENT_USER: 

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/results/clientnegative/distinct_windowing_failure2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/distinct_windowing_failure2.q.out b/ql/src/test/results/clientnegative/distinct_windowing_failure2.q.out
index 9641e2b..0493e20 100644
--- a/ql/src/test/results/clientnegative/distinct_windowing_failure2.q.out
+++ b/ql/src/test/results/clientnegative/distinct_windowing_failure2.q.out
@@ -44,4 +44,4 @@ POSTHOOK: query: load data local inpath '../../data/files/over10k' into table ov
 POSTHOOK: type: LOAD
 #### A masked pattern was here ####
 POSTHOOK: Output: default@over10k
-FAILED: ParseException line 4:7 cannot recognize input near 'distinct' 'first_value' '(' in selection target
+FAILED: ParseException line 3:65 missing EOF at ',' near ')'

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/results/clientnegative/insert_into_with_schema.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/insert_into_with_schema.q.out b/ql/src/test/results/clientnegative/insert_into_with_schema.q.out
index c94f5d8..47ef238 100644
--- a/ql/src/test/results/clientnegative/insert_into_with_schema.q.out
+++ b/ql/src/test/results/clientnegative/insert_into_with_schema.q.out
@@ -22,4 +22,4 @@ POSTHOOK: query: create table source(s1 int, s2 int)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:x314n
 POSTHOOK: Output: x314n@source
-FAILED: SemanticException 0:0 Expected 1 columns for insclause-0/x314n@source; select produces 2 columns. Error encountered near token 'values__tmp__table__1.tmp_values_col2'
+FAILED: SemanticException 0:0 Expected 1 columns for insclause-0/x314n@source; select produces 2 columns. Error encountered near token 'col2'

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/results/clientnegative/insert_into_with_schema1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/insert_into_with_schema1.q.out b/ql/src/test/results/clientnegative/insert_into_with_schema1.q.out
index 1e8f5a3..ae729ea 100644
--- a/ql/src/test/results/clientnegative/insert_into_with_schema1.q.out
+++ b/ql/src/test/results/clientnegative/insert_into_with_schema1.q.out
@@ -22,4 +22,4 @@ POSTHOOK: query: create table source(s1 int, s2 int)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:x314n
 POSTHOOK: Output: x314n@source
-FAILED: SemanticException 0:0 Expected 2 columns for insclause-0/x314n@source; select produces 1 columns. Error encountered near token 'values__tmp__table__1.tmp_values_col1'
+FAILED: SemanticException 0:0 Expected 2 columns for insclause-0/x314n@source; select produces 1 columns. Error encountered near token 'col1'

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/results/clientnegative/insert_sorted.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/insert_sorted.q.out b/ql/src/test/results/clientnegative/insert_sorted.q.out
index 0b1d253..bb3c7e3 100644
--- a/ql/src/test/results/clientnegative/insert_sorted.q.out
+++ b/ql/src/test/results/clientnegative/insert_sorted.q.out
@@ -8,12 +8,14 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@mm_insertsort
 PREHOOK: query: insert into mm_insertsort values (1, '1'),(2, '2')
 PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@mm_insertsort
 POSTHOOK: query: insert into mm_insertsort values (1, '1'),(2, '2')
 POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@mm_insertsort
-POSTHOOK: Lineage: mm_insertsort.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-POSTHOOK: Lineage: mm_insertsort.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: mm_insertsort.a SCRIPT []
+POSTHOOK: Lineage: mm_insertsort.b SCRIPT []
 PREHOOK: query: create table acid_insertsort(a int, b varchar(128)) clustered by (a) sorted by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/results/clientnegative/materialized_view_authorization_create_no_grant.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_authorization_create_no_grant.q.out b/ql/src/test/results/clientnegative/materialized_view_authorization_create_no_grant.q.out
index 648396e..0891015 100644
--- a/ql/src/test/results/clientnegative/materialized_view_authorization_create_no_grant.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_authorization_create_no_grant.q.out
@@ -8,13 +8,15 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@amvcng_gtable
 PREHOOK: query: insert into amvcng_gtable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@amvcng_gtable
 POSTHOOK: query: insert into amvcng_gtable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@amvcng_gtable
-POSTHOOK: Lineage: amvcng_gtable.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-POSTHOOK: Lineage: amvcng_gtable.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
-POSTHOOK: Lineage: amvcng_gtable.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: amvcng_gtable.a SCRIPT []
+POSTHOOK: Lineage: amvcng_gtable.b SCRIPT []
+POSTHOOK: Lineage: amvcng_gtable.c SCRIPT []
 PREHOOK: query: grant select on table amvcng_gtable to user user2
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@amvcng_gtable

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/results/clientnegative/materialized_view_authorization_create_no_select_perm.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_authorization_create_no_select_perm.q.out b/ql/src/test/results/clientnegative/materialized_view_authorization_create_no_select_perm.q.out
index 0fd5144..d58ebec 100644
--- a/ql/src/test/results/clientnegative/materialized_view_authorization_create_no_select_perm.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_authorization_create_no_select_perm.q.out
@@ -8,11 +8,13 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@amvnsp_table
 PREHOOK: query: insert into amvnsp_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@amvnsp_table
 POSTHOOK: query: insert into amvnsp_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@amvnsp_table
-POSTHOOK: Lineage: amvnsp_table.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-POSTHOOK: Lineage: amvnsp_table.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
-POSTHOOK: Lineage: amvnsp_table.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: amvnsp_table.a SCRIPT []
+POSTHOOK: Lineage: amvnsp_table.b SCRIPT []
+POSTHOOK: Lineage: amvnsp_table.c SCRIPT []
 FAILED: HiveAccessControlException Permission denied: Principal [name=user2, type=USER] does not have following privileges for operation CREATE_MATERIALIZED_VIEW [[SELECT with grant] on Object [type=TABLE_OR_VIEW, name=default.amvnsp_table]]

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/results/clientnegative/materialized_view_authorization_drop_other.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_authorization_drop_other.q.out b/ql/src/test/results/clientnegative/materialized_view_authorization_drop_other.q.out
index 322762c..fcd84d3 100644
--- a/ql/src/test/results/clientnegative/materialized_view_authorization_drop_other.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_authorization_drop_other.q.out
@@ -8,13 +8,15 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@amvdo_table
 PREHOOK: query: insert into amvdo_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@amvdo_table
 POSTHOOK: query: insert into amvdo_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@amvdo_table
-POSTHOOK: Lineage: amvdo_table.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-POSTHOOK: Lineage: amvdo_table.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
-POSTHOOK: Lineage: amvdo_table.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: amvdo_table.a SCRIPT []
+POSTHOOK: Lineage: amvdo_table.b SCRIPT []
+POSTHOOK: Lineage: amvdo_table.c SCRIPT []
 PREHOOK: query: create materialized view amvdo_mat_view as select a, c from amvdo_table
 PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@amvdo_table

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/results/clientnegative/materialized_view_authorization_no_select_perm.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_authorization_no_select_perm.q.out b/ql/src/test/results/clientnegative/materialized_view_authorization_no_select_perm.q.out
index 45f4b4f..33d1aa0 100644
--- a/ql/src/test/results/clientnegative/materialized_view_authorization_no_select_perm.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_authorization_no_select_perm.q.out
@@ -8,13 +8,15 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@amvnsp_table
 PREHOOK: query: insert into amvnsp_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@amvnsp_table
 POSTHOOK: query: insert into amvnsp_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@amvnsp_table
-POSTHOOK: Lineage: amvnsp_table.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-POSTHOOK: Lineage: amvnsp_table.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
-POSTHOOK: Lineage: amvnsp_table.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: amvnsp_table.a SCRIPT []
+POSTHOOK: Lineage: amvnsp_table.b SCRIPT []
+POSTHOOK: Lineage: amvnsp_table.c SCRIPT []
 PREHOOK: query: create materialized view amvnsp_mat_view as select a, c from amvnsp_table
 PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@amvnsp_table

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_no_grant.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_no_grant.q.out b/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_no_grant.q.out
index 28f0a72..341fcea 100644
--- a/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_no_grant.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_no_grant.q.out
@@ -8,13 +8,15 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@amvrng_table
 PREHOOK: query: insert into amvrng_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@amvrng_table
 POSTHOOK: query: insert into amvrng_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@amvrng_table
-POSTHOOK: Lineage: amvrng_table.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-POSTHOOK: Lineage: amvrng_table.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
-POSTHOOK: Lineage: amvrng_table.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: amvrng_table.a SCRIPT []
+POSTHOOK: Lineage: amvrng_table.b SCRIPT []
+POSTHOOK: Lineage: amvrng_table.c SCRIPT []
 PREHOOK: query: grant select on table amvrng_table to user user2 with grant option
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@amvrng_table

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_other.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_other.q.out b/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_other.q.out
index 72244aa..97345bf 100644
--- a/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_other.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_authorization_rebuild_other.q.out
@@ -8,13 +8,15 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@amvro_table
 PREHOOK: query: insert into amvro_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@amvro_table
 POSTHOOK: query: insert into amvro_table values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@amvro_table
-POSTHOOK: Lineage: amvro_table.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-POSTHOOK: Lineage: amvro_table.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
-POSTHOOK: Lineage: amvro_table.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: amvro_table.a SCRIPT []
+POSTHOOK: Lineage: amvro_table.b SCRIPT []
+POSTHOOK: Lineage: amvro_table.c SCRIPT []
 PREHOOK: query: create materialized view amvro_mat_view as select a, c from amvro_table
 PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@amvro_table

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/results/clientnegative/materialized_view_drop.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_drop.q.out b/ql/src/test/results/clientnegative/materialized_view_drop.q.out
index ee2cb6b..4714597 100644
--- a/ql/src/test/results/clientnegative/materialized_view_drop.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_drop.q.out
@@ -8,13 +8,15 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@cmv_basetable
 PREHOOK: query: insert into cmv_basetable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@cmv_basetable
 POSTHOOK: query: insert into cmv_basetable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@cmv_basetable
-POSTHOOK: Lineage: cmv_basetable.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-POSTHOOK: Lineage: cmv_basetable.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
-POSTHOOK: Lineage: cmv_basetable.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: cmv_basetable.a SCRIPT []
+POSTHOOK: Lineage: cmv_basetable.b SCRIPT []
+POSTHOOK: Lineage: cmv_basetable.c SCRIPT []
 PREHOOK: query: create materialized view cmv_mat_view as select a, b, c from cmv_basetable
 PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@cmv_basetable

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/results/clientnegative/materialized_view_drop2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_drop2.q.out b/ql/src/test/results/clientnegative/materialized_view_drop2.q.out
index 58b87f2..6dfdfa9 100644
--- a/ql/src/test/results/clientnegative/materialized_view_drop2.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_drop2.q.out
@@ -8,13 +8,15 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@cmv_basetable
 PREHOOK: query: insert into cmv_basetable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@cmv_basetable
 POSTHOOK: query: insert into cmv_basetable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@cmv_basetable
-POSTHOOK: Lineage: cmv_basetable.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-POSTHOOK: Lineage: cmv_basetable.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
-POSTHOOK: Lineage: cmv_basetable.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: cmv_basetable.a SCRIPT []
+POSTHOOK: Lineage: cmv_basetable.b SCRIPT []
+POSTHOOK: Lineage: cmv_basetable.c SCRIPT []
 PREHOOK: query: create materialized view cmv_mat_view as select a, b, c from cmv_basetable
 PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@cmv_basetable

http://git-wip-us.apache.org/repos/asf/hive/blob/7e64114d/ql/src/test/results/clientnegative/materialized_view_replace_with_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/materialized_view_replace_with_view.q.out b/ql/src/test/results/clientnegative/materialized_view_replace_with_view.q.out
index fb5776b..e2e7132 100644
--- a/ql/src/test/results/clientnegative/materialized_view_replace_with_view.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_replace_with_view.q.out
@@ -8,13 +8,15 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@rmvwv_basetable
 PREHOOK: query: insert into rmvwv_basetable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@rmvwv_basetable
 POSTHOOK: query: insert into rmvwv_basetable values (1, 'alfred', 10.30),(2, 'bob', 3.14),(2, 'bonnie', 172342.2),(3, 'calvin', 978.76),(3, 'charlie', 9.8)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@rmvwv_basetable
-POSTHOOK: Lineage: rmvwv_basetable.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-POSTHOOK: Lineage: rmvwv_basetable.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
-POSTHOOK: Lineage: rmvwv_basetable.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: rmvwv_basetable.a SCRIPT []
+POSTHOOK: Lineage: rmvwv_basetable.b SCRIPT []
+POSTHOOK: Lineage: rmvwv_basetable.c SCRIPT []
 PREHOOK: query: create materialized view rmvwv_mat_view as select a, b, c from rmvwv_basetable
 PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@rmvwv_basetable