You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by px...@apache.org on 2017/01/23 19:30:25 UTC

[3/3] hive git commit: HIVE-15646: Column level lineage is not available for table Views (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

HIVE-15646: Column level lineage is not available for table Views (Pengcheng Xiong, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master
Commit: c7ac340564352405eb512b7eada4fc2a1b74acf2
Parents: 588c391
Author: Pengcheng Xiong <px...@apache.org>
Authored: Mon Jan 23 11:30:04 2017 -0800
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Mon Jan 23 11:30:04 2017 -0800

----------------------------------------------------------------------
 .../test/results/positive/hbase_ppd_join.q.out  |  12 +
 .../test/results/positive/hbase_viewjoins.q.out |   6 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   4 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  18 +-
 .../hadoop/hive/ql/session/LineageState.java    |  19 +-
 .../alter_view_as_select_with_partition.q.out   |   1 +
 .../clientnegative/alter_view_failure.q.out     |   2 +
 .../clientnegative/alter_view_failure2.q.out    |   1 +
 .../clientnegative/alter_view_failure4.q.out    |   1 +
 .../clientnegative/alter_view_failure5.q.out    |   1 +
 .../clientnegative/alter_view_failure6.q.out    |   1 +
 .../clientnegative/alter_view_failure7.q.out    |   1 +
 .../clientnegative/alter_view_failure9.q.out    |   2 +
 .../results/clientnegative/analyze_view.q.out   |   2 +
 .../authorization_not_owner_drop_view.q.out     |   1 +
 .../authorization_select_view.q.out             |   1 +
 .../clientnegative/authorization_view_1.q.out   |   2 +
 .../clientnegative/authorization_view_2.q.out   |   2 +
 .../clientnegative/authorization_view_3.q.out   |   2 +
 .../clientnegative/authorization_view_4.q.out   |   6 +
 .../clientnegative/authorization_view_5.q.out   |   4 +
 .../clientnegative/authorization_view_6.q.out   |   4 +
 .../clientnegative/authorization_view_7.q.out   |   4 +
 .../authorization_view_disable_cbo_1.q.out      |   2 +
 .../authorization_view_disable_cbo_2.q.out      |   2 +
 .../authorization_view_disable_cbo_3.q.out      |   2 +
 .../authorization_view_disable_cbo_4.q.out      |   6 +
 .../authorization_view_disable_cbo_5.q.out      |   4 +
 .../authorization_view_disable_cbo_6.q.out      |   4 +
 .../authorization_view_disable_cbo_7.q.out      |   4 +
 .../create_or_replace_view1.q.out               |   2 +
 .../create_or_replace_view2.q.out               |   2 +
 .../create_or_replace_view4.q.out               |   2 +
 .../create_or_replace_view5.q.out               |   2 +
 .../create_or_replace_view6.q.out               |   2 +
 .../create_or_replace_view7.q.out               |   6 +
 .../create_or_replace_view8.q.out               |   2 +
 .../clientnegative/create_view_failure2.q.out   |   1 +
 .../clientnegative/drop_table_failure2.q.out    |   1 +
 .../clientnegative/insert_view_failure.q.out    |   2 +
 .../clientnegative/invalidate_view1.q.out       |   4 +
 .../clientnegative/load_view_failure.q.out      |   2 +
 .../results/clientnegative/recursive_view.q.out |   4 +
 .../clientnegative/unset_view_property.q.out    |   1 +
 .../results/clientnegative/view_delete.q.out    |   3 +
 .../results/clientnegative/view_update.q.out    |   3 +
 .../clientpositive/alter_view_as_select.q.out   |   4 +
 .../clientpositive/alter_view_rename.q.out      |   3 +
 .../clientpositive/authorization_8.q.out        |   3 +
 .../authorization_cli_createtab.q.out           |   1 +
 ...authorization_cli_createtab_noauthzapi.q.out |   1 +
 .../authorization_owner_actions.q.out           |   1 +
 .../clientpositive/authorization_view_1.q.out   |   6 +
 .../clientpositive/authorization_view_2.q.out   |   4 +
 .../clientpositive/authorization_view_3.q.out   |   4 +
 .../clientpositive/authorization_view_4.q.out   |   4 +
 .../authorization_view_disable_cbo_1.q.out      |   6 +
 .../authorization_view_disable_cbo_2.q.out      |   4 +
 .../authorization_view_disable_cbo_3.q.out      |   4 +
 .../authorization_view_disable_cbo_4.q.out      |   4 +
 .../test/results/clientpositive/cbo_const.q.out |   2 +
 .../clientpositive/cbo_subq_exists.q.out        |   2 +
 .../results/clientpositive/cbo_union_view.q.out |   2 +
 .../test/results/clientpositive/concat_op.q.out |   1 +
 .../clientpositive/create_big_view.q.out        | 235 +++++++++++++++++++
 .../clientpositive/create_like_tbl_props.q.out  |   2 +
 .../clientpositive/create_like_view.q.out       |   4 +
 .../clientpositive/create_or_replace_view.q.out |   6 +
 .../results/clientpositive/create_view.q.out    |  27 +++
 .../create_view_defaultformats.q.out            |   4 +
 .../create_view_partitioned.q.out               |   5 +
 .../clientpositive/create_view_translate.q.out  |   5 +
 .../test/results/clientpositive/ctas_char.q.out |   2 +
 .../test/results/clientpositive/ctas_date.q.out |   8 +
 .../results/clientpositive/ctas_varchar.q.out   |   2 +
 .../test/results/clientpositive/cteViews.q.out  |   5 +
 ql/src/test/results/clientpositive/cte_2.q.out  |   6 +
 ql/src/test/results/clientpositive/cte_4.q.out  |   2 +
 .../results/clientpositive/database_drop.q.out  |   4 +
 .../results/clientpositive/dbtxnmgr_ddl1.q.out  |   1 +
 .../clientpositive/dbtxnmgr_query5.q.out        |   1 +
 .../describe_formatted_view_partitioned.q.out   |   1 +
 ...scribe_formatted_view_partitioned_json.q.out |   1 +
 .../encrypted/encryption_drop_view.q.out        |   2 +
 .../clientpositive/escape_comments.q.out        |   2 +
 .../results/clientpositive/explain_ddl.q.out    |   2 +
 .../clientpositive/explain_dependency.q.out     |  14 ++
 .../clientpositive/explain_logical.q.out        |  14 ++
 .../test/results/clientpositive/join_view.q.out |   3 +
 .../clientpositive/lateral_view_noalias.q.out   |   2 +
 .../clientpositive/lateral_view_onview.q.out    |   4 +
 .../llap/cbo_rp_unionDistinct_2.q.out           |   6 +
 .../clientpositive/llap/cbo_rp_views.q.out      |   9 +
 .../llap/cbo_rp_windowing_2.q.out               |   6 +
 .../clientpositive/llap/cbo_subq_exists.q.out   |   2 +
 .../results/clientpositive/llap/cbo_views.q.out |   9 +
 .../results/clientpositive/llap/cte_2.q.out     |   6 +
 .../results/clientpositive/llap/cte_4.q.out     |   2 +
 .../clientpositive/llap/explainuser_1.q.out     |   2 +
 .../clientpositive/llap/multi_column_in.q.out   |   3 +
 .../llap/multi_column_in_single.q.out           |   3 +
 .../clientpositive/llap/ppd_union_view.q.out    |   2 +
 .../test/results/clientpositive/llap/ptf.q.out  |   3 +
 .../llap/selectDistinctStar.q.out               |   4 +
 .../llap/special_character_in_tabnames_1.q.out  |  11 +
 .../clientpositive/llap/subquery_exists.q.out   |   2 +
 .../clientpositive/llap/subquery_notin.q.out    |   2 +
 .../clientpositive/llap/subquery_views.q.out    |   8 +
 .../results/clientpositive/llap/tez_union.q.out |   2 +
 .../clientpositive/llap/unionDistinct_2.q.out   |   6 +
 .../clientpositive/llap/union_top_level.q.out   |   2 +
 .../clientpositive/llap/vectorized_ptf.q.out    |   3 +
 .../results/clientpositive/llap/windowing.q.out |   6 +
 .../test/results/clientpositive/masking_2.q.out |   2 +
 .../test/results/clientpositive/masking_6.q.out |  15 ++
 .../test/results/clientpositive/masking_7.q.out |  17 ++
 .../clientpositive/masking_disablecbo_2.q.out   |   2 +
 .../clientpositive/ppd_field_garbage.q.out      |   3 +
 .../results/clientpositive/quotedid_basic.q.out |   2 +
 .../clientpositive/show_create_table_view.q.out |   2 +
 .../results/clientpositive/show_views.q.out     |  16 ++
 .../clientpositive/spark/join_view.q.out        |   3 +
 .../test/results/clientpositive/spark/ptf.q.out |   3 +
 .../clientpositive/spark/subquery_exists.q.out  |   2 +
 .../clientpositive/spark/union_top_level.q.out  |   2 +
 .../clientpositive/spark/vectorized_ptf.q.out   |   3 +
 .../clientpositive/spark/windowing.q.out        |   6 +
 .../results/clientpositive/struct_in_view.q.out |   6 +
 .../clientpositive/subquery_exists.q.out        |   2 +
 .../clientpositive/subquery_exists_having.q.out |   2 +
 .../clientpositive/tez/unionDistinct_2.q.out    |   6 +
 .../clientpositive/unicode_comments.q.out       |   1 +
 .../unionall_join_nullconstant.q.out            |   2 +
 .../clientpositive/unionall_unbalancedppd.q.out |   2 +
 .../unset_table_view_property.q.out             |   1 +
 ql/src/test/results/clientpositive/view.q.out   |  16 ++
 .../results/clientpositive/view_alias.q.out     |  19 ++
 .../view_authorization_sqlstd.q.out             |   7 +
 .../test/results/clientpositive/view_cast.q.out |  16 ++
 .../results/clientpositive/view_inputs.q.out    |   4 +
 140 files changed, 828 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/hbase-handler/src/test/results/positive/hbase_ppd_join.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_ppd_join.q.out b/hbase-handler/src/test/results/positive/hbase_ppd_join.q.out
index ee459e2..0744caf 100644
--- a/hbase-handler/src/test/results/positive/hbase_ppd_join.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_ppd_join.q.out
@@ -50,6 +50,12 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@hive1_tbl_data_hbase1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@hive1_view_data_hbase1
+POSTHOOK: Lineage: hive1_view_data_hbase1.col_updated_date SIMPLE [(hive1_tbl_data_hbase1)hive1_tbl_data_hbase1.FieldSchema(name:col_updated_date, type:timestamp, comment:), ]
+POSTHOOK: Lineage: hive1_view_data_hbase1.columid SIMPLE [(hive1_tbl_data_hbase1)hive1_tbl_data_hbase1.FieldSchema(name:columid, type:string, comment:), ]
+POSTHOOK: Lineage: hive1_view_data_hbase1.column_fn SIMPLE [(hive1_tbl_data_hbase1)hive1_tbl_data_hbase1.FieldSchema(name:column_fn, type:string, comment:), ]
+POSTHOOK: Lineage: hive1_view_data_hbase1.column_ln SIMPLE [(hive1_tbl_data_hbase1)hive1_tbl_data_hbase1.FieldSchema(name:column_ln, type:string, comment:), ]
+POSTHOOK: Lineage: hive1_view_data_hbase1.email SIMPLE [(hive1_tbl_data_hbase1)hive1_tbl_data_hbase1.FieldSchema(name:email, type:string, comment:), ]
+POSTHOOK: Lineage: hive1_view_data_hbase1.pk_colum SIMPLE [(hive1_tbl_data_hbase1)hive1_tbl_data_hbase1.FieldSchema(name:pk_colum, type:string, comment:), ]
 PREHOOK: query: insert into table hive1_tbl_data_hbase1 select '00001','john','doe','john@hotmail.com','2014-01-01 12:01:02','4000-10000' from src where key = 100
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -90,6 +96,12 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@hive1_tbl_data_hbase2
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@hive1_view_data_hbase2
+POSTHOOK: Lineage: hive1_view_data_hbase2.col_updated_date SIMPLE [(hive1_tbl_data_hbase2)hive1_tbl_data_hbase2.FieldSchema(name:col_updated_date, type:timestamp, comment:), ]
+POSTHOOK: Lineage: hive1_view_data_hbase2.columid SIMPLE [(hive1_tbl_data_hbase2)hive1_tbl_data_hbase2.FieldSchema(name:columid, type:string, comment:), ]
+POSTHOOK: Lineage: hive1_view_data_hbase2.column_fn SIMPLE [(hive1_tbl_data_hbase2)hive1_tbl_data_hbase2.FieldSchema(name:column_fn, type:string, comment:), ]
+POSTHOOK: Lineage: hive1_view_data_hbase2.column_ln SIMPLE [(hive1_tbl_data_hbase2)hive1_tbl_data_hbase2.FieldSchema(name:column_ln, type:string, comment:), ]
+POSTHOOK: Lineage: hive1_view_data_hbase2.email SIMPLE [(hive1_tbl_data_hbase2)hive1_tbl_data_hbase2.FieldSchema(name:email, type:string, comment:), ]
+POSTHOOK: Lineage: hive1_view_data_hbase2.pk_colum SIMPLE [(hive1_tbl_data_hbase2)hive1_tbl_data_hbase2.FieldSchema(name:pk_colum, type:string, comment:), ]
 PREHOOK: query: insert into table hive1_tbl_data_hbase2 select '00001','john','doe','john@hotmail.com','2014-01-01 12:01:02','00001' from src where key = 100
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/hbase-handler/src/test/results/positive/hbase_viewjoins.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_viewjoins.q.out b/hbase-handler/src/test/results/positive/hbase_viewjoins.q.out
index 908024c..95fcaa0 100644
--- a/hbase-handler/src/test/results/positive/hbase_viewjoins.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_viewjoins.q.out
@@ -62,6 +62,9 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@hbase_table_test_1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@VIEW_HBASE_TABLE_TEST_1
+POSTHOOK: Lineage: VIEW_HBASE_TABLE_TEST_1.ccount SIMPLE [(hbase_table_test_1)hbase_table_test_1.FieldSchema(name:ccount, type:int, comment:), ]
+POSTHOOK: Lineage: VIEW_HBASE_TABLE_TEST_1.cvalue SIMPLE [(hbase_table_test_1)hbase_table_test_1.FieldSchema(name:cvalue, type:string, comment:), ]
+POSTHOOK: Lineage: VIEW_HBASE_TABLE_TEST_1.pk SIMPLE [(hbase_table_test_1)hbase_table_test_1.FieldSchema(name:pk, type:string, comment:), ]
 PREHOOK: query: CREATE TABLE HBASE_TABLE_TEST_2(
   cvalue string ,
     pk string ,
@@ -114,6 +117,9 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@hbase_table_test_2
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@VIEW_HBASE_TABLE_TEST_2
+POSTHOOK: Lineage: VIEW_HBASE_TABLE_TEST_2.ccount SIMPLE [(hbase_table_test_2)hbase_table_test_2.FieldSchema(name:ccount, type:int, comment:), ]
+POSTHOOK: Lineage: VIEW_HBASE_TABLE_TEST_2.cvalue SIMPLE [(hbase_table_test_2)hbase_table_test_2.FieldSchema(name:cvalue, type:string, comment:), ]
+POSTHOOK: Lineage: VIEW_HBASE_TABLE_TEST_2.pk SIMPLE [(hbase_table_test_2)hbase_table_test_2.FieldSchema(name:pk, type:string, comment:), ]
 PREHOOK: query: SELECT  p.cvalue cvalue
 FROM `VIEW_HBASE_TABLE_TEST_1` `p`
 LEFT OUTER JOIN `VIEW_HBASE_TABLE_TEST_2` `A1`

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index a930408..9511b46 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -4434,6 +4434,10 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         HiveMaterializedViewsRegistry.get().addMaterializedView(tbl);
       }
       addIfAbsentByName(new WriteEntity(tbl, WriteEntity.WriteType.DDL_NO_LOCK));
+      
+      //set lineage info
+      DataContainer dc = new DataContainer(tbl.getTTable());
+      SessionState.get().getLineageState().setLineage(new Path(crtView.getViewName()), dc, tbl.getCols());
     }
     return 0;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/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 266bc7d..e5d0101 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
@@ -237,6 +237,8 @@ import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.OutputFormat;
 import org.apache.hadoop.security.UserGroupInformation;
 
+import com.google.common.base.Splitter;
+import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
 import com.google.common.math.IntMath;
 
@@ -7143,7 +7145,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
     if (ltd != null && SessionState.get() != null) {
       SessionState.get().getLineageState()
-          .mapDirToFop(ltd.getSourcePath(), (FileSinkOperator) output);
+          .mapDirToOp(ltd.getSourcePath(), (FileSinkOperator) output);
     } else if ( queryState.getCommandType().equals(HiveOperation.CREATETABLE_AS_SELECT.getOperationName())) {
 
       Path tlocation = null;
@@ -7156,7 +7158,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       }
 
       SessionState.get().getLineageState()
-              .mapDirToFop(tlocation, (FileSinkOperator) output);
+              .mapDirToOp(tlocation, (FileSinkOperator) output);
     }
 
     if (LOG.isDebugEnabled()) {
@@ -11054,14 +11056,22 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
         // Generate lineage info for create view statements
         // if LineageLogger hook is configured.
-        if (HiveConf.getVar(conf, HiveConf.ConfVars.POSTEXECHOOKS).contains(
-            "org.apache.hadoop.hive.ql.hooks.LineageLogger")) {
+        // Add the transformation that computes the lineage information.
+        Set<String> postExecHooks = Sets.newHashSet(Splitter.on(",").trimResults()
+            .omitEmptyStrings()
+            .split(Strings.nullToEmpty(HiveConf.getVar(conf, HiveConf.ConfVars.POSTEXECHOOKS))));
+        if (postExecHooks.contains("org.apache.hadoop.hive.ql.hooks.PostExecutePrinter")
+            || postExecHooks.contains("org.apache.hadoop.hive.ql.hooks.LineageLogger")
+            || postExecHooks.contains("org.apache.atlas.hive.hook.HiveHook")) {
           ArrayList<Transform> transformations = new ArrayList<Transform>();
           transformations.add(new HiveOpConverterPostProc());
           transformations.add(new Generator());
           for (Transform t : transformations) {
             pCtx = t.transform(pCtx);
           }
+          // we just use view name as location.
+          SessionState.get().getLineageState()
+              .mapDirToOp(new Path(createVwDesc.getViewName()), sinkOp);
         }
         return;
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/java/org/apache/hadoop/hive/ql/session/LineageState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/LineageState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/LineageState.java
index 223f0ea..0f95063 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/LineageState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/LineageState.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.hooks.LineageInfo;
 import org.apache.hadoop.hive.ql.hooks.LineageInfo.DataContainer;
 import org.apache.hadoop.hive.ql.optimizer.lineage.LineageCtx.Index;
@@ -38,12 +39,12 @@ import org.apache.hadoop.hive.ql.optimizer.lineage.LineageCtx.Index;
 public class LineageState {
 
   /**
-   * Mapping from the directory name to FileSinkOperator. This
+   * Mapping from the directory name to FileSinkOperator (may not be FileSinkOperator for views). This
    * mapping is generated at the filesink operator creation
    * time and is then later used to created the mapping from
    * movetask to the set of filesink operators.
    */
-  private final Map<Path, FileSinkOperator> dirToFop;
+  private final Map<Path, Operator> dirToFop;
 
   /**
    * The lineage context index for this query.
@@ -60,7 +61,7 @@ public class LineageState {
    * Constructor.
    */
   public LineageState() {
-    dirToFop = new HashMap<Path, FileSinkOperator>();
+    dirToFop = new HashMap<Path, Operator>();
     linfo = new LineageInfo();
     index = new Index();
   }
@@ -69,9 +70,9 @@ public class LineageState {
    * Adds a mapping from the load work to the file sink operator.
    *
    * @param dir The directory name.
-   * @param fop The file sink operator.
+   * @param fop The sink operator.
    */
-  public void mapDirToFop(Path dir, FileSinkOperator fop) {
+  public void mapDirToOp(Path dir, Operator fop) {
     dirToFop.put(dir, fop);
   }
 
@@ -85,18 +86,18 @@ public class LineageState {
   public void setLineage(Path dir, DataContainer dc,
       List<FieldSchema> cols) {
     // First lookup the file sink operator from the load work.
-    FileSinkOperator fop = dirToFop.get(dir);
+    Operator<?> op = dirToFop.get(dir);
 
     // Go over the associated fields and look up the dependencies
     // by position in the row schema of the filesink operator.
-    if (fop == null) {
+    if (op == null) {
       return;
     }
 
-    List<ColumnInfo> signature = fop.getSchema().getSignature();
+    List<ColumnInfo> signature = op.getSchema().getSignature();
     int i = 0;
     for (FieldSchema fs : cols) {
-      linfo.putDependency(dc, fs, index.getDependency(fop, signature.get(i++)));
+      linfo.putDependency(dc, fs, index.getDependency(op, signature.get(i++)));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out b/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
index adbc7a8..9e07416 100644
--- a/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
+++ b/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
@@ -16,6 +16,7 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@testViewPart
+POSTHOOK: Lineage: testViewPart.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 PREHOOK: query: ALTER VIEW testViewPart 
 ADD PARTITION (value='val_86') PARTITION (value='val_xyz')
 PREHOOK: type: ALTERTABLE_ADDPARTS

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/alter_view_failure.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_view_failure.q.out b/ql/src/test/results/clientnegative/alter_view_failure.q.out
index b1a56b4..4b318a0 100644
--- a/ql/src/test/results/clientnegative/alter_view_failure.q.out
+++ b/ql/src/test/results/clientnegative/alter_view_failure.q.out
@@ -12,4 +12,6 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@xxx3
+POSTHOOK: Lineage: xxx3.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: xxx3.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 FAILED: SemanticException [Error 10131]: To alter a view you need to use the ALTER VIEW command.

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/alter_view_failure2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_view_failure2.q.out b/ql/src/test/results/clientnegative/alter_view_failure2.q.out
index 755299d..dfee098 100644
--- a/ql/src/test/results/clientnegative/alter_view_failure2.q.out
+++ b/ql/src/test/results/clientnegative/alter_view_failure2.q.out
@@ -18,4 +18,5 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@xxx4
+POSTHOOK: Lineage: xxx4.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 FAILED: SemanticException [Error 10131]: To alter a view you need to use the ALTER VIEW command.

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/alter_view_failure4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_view_failure4.q.out b/ql/src/test/results/clientnegative/alter_view_failure4.q.out
index 2efa54c..7a7def1 100644
--- a/ql/src/test/results/clientnegative/alter_view_failure4.q.out
+++ b/ql/src/test/results/clientnegative/alter_view_failure4.q.out
@@ -18,4 +18,5 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@xxx5
+POSTHOOK: Lineage: xxx5.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 FAILED: SemanticException LOCATION clause illegal for view partition

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/alter_view_failure5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_view_failure5.q.out b/ql/src/test/results/clientnegative/alter_view_failure5.q.out
index 2891fb0..6629df4 100644
--- a/ql/src/test/results/clientnegative/alter_view_failure5.q.out
+++ b/ql/src/test/results/clientnegative/alter_view_failure5.q.out
@@ -18,4 +18,5 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@xxx6
+POSTHOOK: Lineage: xxx6.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 FAILED: ValidationFailureSemanticException Partition spec {v=val_86} contains non-partition columns

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/alter_view_failure6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_view_failure6.q.out b/ql/src/test/results/clientnegative/alter_view_failure6.q.out
index 78416ad..1ada547 100644
--- a/ql/src/test/results/clientnegative/alter_view_failure6.q.out
+++ b/ql/src/test/results/clientnegative/alter_view_failure6.q.out
@@ -18,5 +18,6 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@srcpart
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@xxx7
+POSTHOOK: Lineage: xxx7.hr SIMPLE [(srcpart)srcpart.FieldSchema(name:hr, type:string, comment:null), ]
 FAILED: SemanticException Queries against partitioned tables without a partition filter are disabled for safety reasons. If you know what you are doing, please sethive.strict.checks.large.query to false and that hive.mapred.mode is not set to 'strict' to proceed. Note that if you may get errors or incorrect results if you make a mistake while using some of the unsafe features. No partition predicate for Alias "default.xxx7:srcpart" Table "srcpart"
 FAILED: SemanticException [Error 10056]: The query does not reference any valid partition. To run this query, set hive.mapred.mode=nonstrict

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/alter_view_failure7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_view_failure7.q.out b/ql/src/test/results/clientnegative/alter_view_failure7.q.out
index 97d2b83..f96f3cc 100644
--- a/ql/src/test/results/clientnegative/alter_view_failure7.q.out
+++ b/ql/src/test/results/clientnegative/alter_view_failure7.q.out
@@ -18,4 +18,5 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@srcpart
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@xxx8
+POSTHOOK: Lineage: xxx8.key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
 FAILED: ValidationFailureSemanticException partition spec {ds=2011-01-01} doesn't contain all (2) partition columns

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/alter_view_failure9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_view_failure9.q.out b/ql/src/test/results/clientnegative/alter_view_failure9.q.out
index 0afae11..85ed74e 100644
--- a/ql/src/test/results/clientnegative/alter_view_failure9.q.out
+++ b/ql/src/test/results/clientnegative/alter_view_failure9.q.out
@@ -16,4 +16,6 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@xxx4
+POSTHOOK: Lineage: xxx4.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: xxx4.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 FAILED: SemanticException [Error 10131]: To alter a view you need to use the ALTER VIEW command.

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/analyze_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/analyze_view.q.out b/ql/src/test/results/clientnegative/analyze_view.q.out
index 95bdde8..f185dcf 100644
--- a/ql/src/test/results/clientnegative/analyze_view.q.out
+++ b/ql/src/test/results/clientnegative/analyze_view.q.out
@@ -12,4 +12,6 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@av
+POSTHOOK: Lineage: av.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: av.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 FAILED: SemanticException [Error 10091]: ANALYZE is not supported for views

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_not_owner_drop_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_not_owner_drop_view.q.out b/ql/src/test/results/clientnegative/authorization_not_owner_drop_view.q.out
index b1dc969..4920517 100644
--- a/ql/src/test/results/clientnegative/authorization_not_owner_drop_view.q.out
+++ b/ql/src/test/results/clientnegative/authorization_not_owner_drop_view.q.out
@@ -16,4 +16,5 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@t1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@vt1
+POSTHOOK: Lineage: vt1.i SIMPLE [(t1)t1.FieldSchema(name:i, type:int, comment:null), ]
 FAILED: HiveAccessControlException Permission denied: Principal [name=user2, type=USER] does not have following privileges for operation DROPVIEW [[OBJECT OWNERSHIP] on Object [type=TABLE_OR_VIEW, name=default.vt1], [OBJECT OWNERSHIP] on Object [type=TABLE_OR_VIEW, name=default.vt1]]

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_select_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_select_view.q.out b/ql/src/test/results/clientnegative/authorization_select_view.q.out
index 707757e..7e8fa16 100644
--- a/ql/src/test/results/clientnegative/authorization_select_view.q.out
+++ b/ql/src/test/results/clientnegative/authorization_select_view.q.out
@@ -16,4 +16,5 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@t1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.i SIMPLE [(t1)t1.FieldSchema(name:i, type:int, comment:null), ]
 FAILED: HiveAccessControlException Permission denied: Principal [name=user1, type=USER] does not have following privileges for operation QUERY [[SELECT] on Object [type=TABLE_OR_VIEW, name=default.v1]]

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_view_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_view_1.q.out b/ql/src/test/results/clientnegative/authorization_view_1.q.out
index 2a36dc6..277726b 100644
--- a/ql/src/test/results/clientnegative/authorization_view_1.q.out
+++ b/ql/src/test/results/clientnegative/authorization_view_1.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select(key) on table src_autho_test to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@src_autho_test

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_view_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_view_2.q.out b/ql/src/test/results/clientnegative/authorization_view_2.q.out
index 099f42d..2b0f000 100644
--- a/ql/src/test/results/clientnegative/authorization_view_2.q.out
+++ b/ql/src/test/results/clientnegative/authorization_view_2.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select(key) on table src_autho_test to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@src_autho_test

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_view_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_view_3.q.out b/ql/src/test/results/clientnegative/authorization_view_3.q.out
index a43f902..f414d19 100644
--- a/ql/src/test/results/clientnegative/authorization_view_3.q.out
+++ b/ql/src/test/results/clientnegative/authorization_view_3.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select(key) on table src_autho_test to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@src_autho_test

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_view_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_view_4.q.out b/ql/src/test/results/clientnegative/authorization_view_4.q.out
index 3214508..ab49f86 100644
--- a/ql/src/test/results/clientnegative/authorization_view_4.q.out
+++ b/ql/src/test/results/clientnegative/authorization_view_4.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v1 as select * from src_autho_test
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -30,6 +32,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v2 as select * from src_autho_test
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -40,6 +44,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select on table src_autho_test to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@src_autho_test

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_view_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_view_5.q.out b/ql/src/test/results/clientnegative/authorization_view_5.q.out
index a185146..fb98ca3 100644
--- a/ql/src/test/results/clientnegative/authorization_view_5.q.out
+++ b/ql/src/test/results/clientnegative/authorization_view_5.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v2 as select * from v1
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -32,4 +34,6 @@ POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Input: default@v1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 Authorization failed:No privilege 'Select' found for inputs { database:default, table:v2, columnName:key}. Use SHOW GRANT to get more details.

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_view_6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_view_6.q.out b/ql/src/test/results/clientnegative/authorization_view_6.q.out
index d3e7378..b373c59 100644
--- a/ql/src/test/results/clientnegative/authorization_view_6.q.out
+++ b/ql/src/test/results/clientnegative/authorization_view_6.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v2 as select * from v1
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -32,6 +34,8 @@ POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Input: default@v1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select on table v2 to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@v2

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_view_7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_view_7.q.out b/ql/src/test/results/clientnegative/authorization_view_7.q.out
index 98186f0..c405a9a 100644
--- a/ql/src/test/results/clientnegative/authorization_view_7.q.out
+++ b/ql/src/test/results/clientnegative/authorization_view_7.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: create view v2 as select * from v1
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src
@@ -32,6 +34,8 @@ POSTHOOK: Input: default@src
 POSTHOOK: Input: default@v1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: grant select on table v2 to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@v2

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_view_disable_cbo_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_view_disable_cbo_1.q.out b/ql/src/test/results/clientnegative/authorization_view_disable_cbo_1.q.out
index 2a36dc6..277726b 100644
--- a/ql/src/test/results/clientnegative/authorization_view_disable_cbo_1.q.out
+++ b/ql/src/test/results/clientnegative/authorization_view_disable_cbo_1.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select(key) on table src_autho_test to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@src_autho_test

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_view_disable_cbo_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_view_disable_cbo_2.q.out b/ql/src/test/results/clientnegative/authorization_view_disable_cbo_2.q.out
index 099f42d..2b0f000 100644
--- a/ql/src/test/results/clientnegative/authorization_view_disable_cbo_2.q.out
+++ b/ql/src/test/results/clientnegative/authorization_view_disable_cbo_2.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select(key) on table src_autho_test to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@src_autho_test

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_view_disable_cbo_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_view_disable_cbo_3.q.out b/ql/src/test/results/clientnegative/authorization_view_disable_cbo_3.q.out
index a43f902..f414d19 100644
--- a/ql/src/test/results/clientnegative/authorization_view_disable_cbo_3.q.out
+++ b/ql/src/test/results/clientnegative/authorization_view_disable_cbo_3.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select(key) on table src_autho_test to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@src_autho_test

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_view_disable_cbo_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_view_disable_cbo_4.q.out b/ql/src/test/results/clientnegative/authorization_view_disable_cbo_4.q.out
index 3214508..ab49f86 100644
--- a/ql/src/test/results/clientnegative/authorization_view_disable_cbo_4.q.out
+++ b/ql/src/test/results/clientnegative/authorization_view_disable_cbo_4.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v1 as select * from src_autho_test
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -30,6 +32,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v2 as select * from src_autho_test
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -40,6 +44,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select on table src_autho_test to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@src_autho_test

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_view_disable_cbo_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_view_disable_cbo_5.q.out b/ql/src/test/results/clientnegative/authorization_view_disable_cbo_5.q.out
index a185146..fb98ca3 100644
--- a/ql/src/test/results/clientnegative/authorization_view_disable_cbo_5.q.out
+++ b/ql/src/test/results/clientnegative/authorization_view_disable_cbo_5.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v2 as select * from v1
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -32,4 +34,6 @@ POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Input: default@v1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 Authorization failed:No privilege 'Select' found for inputs { database:default, table:v2, columnName:key}. Use SHOW GRANT to get more details.

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_view_disable_cbo_6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_view_disable_cbo_6.q.out b/ql/src/test/results/clientnegative/authorization_view_disable_cbo_6.q.out
index d3e7378..b373c59 100644
--- a/ql/src/test/results/clientnegative/authorization_view_disable_cbo_6.q.out
+++ b/ql/src/test/results/clientnegative/authorization_view_disable_cbo_6.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v2 as select * from v1
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -32,6 +34,8 @@ POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Input: default@v1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select on table v2 to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@v2

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/authorization_view_disable_cbo_7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/authorization_view_disable_cbo_7.q.out b/ql/src/test/results/clientnegative/authorization_view_disable_cbo_7.q.out
index 98186f0..c405a9a 100644
--- a/ql/src/test/results/clientnegative/authorization_view_disable_cbo_7.q.out
+++ b/ql/src/test/results/clientnegative/authorization_view_disable_cbo_7.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: create view v2 as select * from v1
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src
@@ -32,6 +34,8 @@ POSTHOOK: Input: default@src
 POSTHOOK: Input: default@v1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: grant select on table v2 to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@v2

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/create_or_replace_view1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_or_replace_view1.q.out b/ql/src/test/results/clientnegative/create_or_replace_view1.q.out
index 9ec57e3..4610ff3 100644
--- a/ql/src/test/results/clientnegative/create_or_replace_view1.q.out
+++ b/ql/src/test/results/clientnegative/create_or_replace_view1.q.out
@@ -12,6 +12,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@srcpart
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v.value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: alter view v add partition (ds='1',hr='2')
 PREHOOK: type: ALTERTABLE_ADDPARTS
 PREHOOK: Input: default@srcpart

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/create_or_replace_view2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_or_replace_view2.q.out b/ql/src/test/results/clientnegative/create_or_replace_view2.q.out
index 9ec57e3..4610ff3 100644
--- a/ql/src/test/results/clientnegative/create_or_replace_view2.q.out
+++ b/ql/src/test/results/clientnegative/create_or_replace_view2.q.out
@@ -12,6 +12,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@srcpart
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v.value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: alter view v add partition (ds='1',hr='2')
 PREHOOK: type: ALTERTABLE_ADDPARTS
 PREHOOK: Input: default@srcpart

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/create_or_replace_view4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_or_replace_view4.q.out b/ql/src/test/results/clientnegative/create_or_replace_view4.q.out
index 71a3d80..b83939f 100644
--- a/ql/src/test/results/clientnegative/create_or_replace_view4.q.out
+++ b/ql/src/test/results/clientnegative/create_or_replace_view4.q.out
@@ -12,4 +12,6 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@srcpart
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v.value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
 FAILED: SemanticException [Error 10092]: At least one non-partitioning column must be present in view

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/create_or_replace_view5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_or_replace_view5.q.out b/ql/src/test/results/clientnegative/create_or_replace_view5.q.out
index 669a228..6d8df2b 100644
--- a/ql/src/test/results/clientnegative/create_or_replace_view5.q.out
+++ b/ql/src/test/results/clientnegative/create_or_replace_view5.q.out
@@ -12,4 +12,6 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@srcpart
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v.value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
 FAILED: SemanticException Can't combine IF NOT EXISTS and OR REPLACE.

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/create_or_replace_view6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_or_replace_view6.q.out b/ql/src/test/results/clientnegative/create_or_replace_view6.q.out
index a5702ab..a4b3d38 100644
--- a/ql/src/test/results/clientnegative/create_or_replace_view6.q.out
+++ b/ql/src/test/results/clientnegative/create_or_replace_view6.q.out
@@ -12,4 +12,6 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@srcpart
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v.value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
 FAILED: ParseException line 2:52 cannot recognize input near 'blah' '<EOF>' '<EOF>' in create view statement

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/create_or_replace_view7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_or_replace_view7.q.out b/ql/src/test/results/clientnegative/create_or_replace_view7.q.out
index 2d73a1f..7cea130 100644
--- a/ql/src/test/results/clientnegative/create_or_replace_view7.q.out
+++ b/ql/src/test/results/clientnegative/create_or_replace_view7.q.out
@@ -12,6 +12,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@srcpart
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: create view v2 partitioned on (ds, hr) as select * from v1
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@srcpart
@@ -24,6 +26,8 @@ POSTHOOK: Input: default@srcpart
 POSTHOOK: Input: default@v1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: create view v3 partitioned on (ds, hr) as select * from v2
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@srcpart
@@ -38,4 +42,6 @@ POSTHOOK: Input: default@v1
 POSTHOOK: Input: default@v2
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v3
+POSTHOOK: Lineage: v3.key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v3.value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
 FAILED: SemanticException Recursive view default.v1 detected (cycle: default.v1 -> default.v3 -> default.v2 -> default.v1).

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/create_or_replace_view8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_or_replace_view8.q.out b/ql/src/test/results/clientnegative/create_or_replace_view8.q.out
index 2d8d5b0..20b979b 100644
--- a/ql/src/test/results/clientnegative/create_or_replace_view8.q.out
+++ b/ql/src/test/results/clientnegative/create_or_replace_view8.q.out
@@ -12,4 +12,6 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@srcpart
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
 FAILED: SemanticException Recursive view default.v1 detected (cycle: default.v1 -> default.v1).

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/create_view_failure2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_view_failure2.q.out b/ql/src/test/results/clientnegative/create_view_failure2.q.out
index 58da497..3342a23 100644
--- a/ql/src/test/results/clientnegative/create_view_failure2.q.out
+++ b/ql/src/test/results/clientnegative/create_view_failure2.q.out
@@ -12,6 +12,7 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@xxx4
+POSTHOOK: Lineage: xxx4.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 PREHOOK: query: CREATE TABLE xxx4(key int)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/drop_table_failure2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/drop_table_failure2.q.out b/ql/src/test/results/clientnegative/drop_table_failure2.q.out
index bcb4bfe..f0097cd 100644
--- a/ql/src/test/results/clientnegative/drop_table_failure2.q.out
+++ b/ql/src/test/results/clientnegative/drop_table_failure2.q.out
@@ -8,6 +8,7 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@xxx6
+POSTHOOK: Lineage: xxx6.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 PREHOOK: query: DROP TABLE xxx6
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@xxx6

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/insert_view_failure.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/insert_view_failure.q.out b/ql/src/test/results/clientnegative/insert_view_failure.q.out
index 7bc52cb..6bbaa10 100644
--- a/ql/src/test/results/clientnegative/insert_view_failure.q.out
+++ b/ql/src/test/results/clientnegative/insert_view_failure.q.out
@@ -12,4 +12,6 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@xxx2
+POSTHOOK: Lineage: xxx2.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: xxx2.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 FAILED: SemanticException [Error 10090]: A view cannot be used as target table for LOAD or INSERT

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/invalidate_view1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/invalidate_view1.q.out b/ql/src/test/results/clientnegative/invalidate_view1.q.out
index 2da46bb..16e5173 100644
--- a/ql/src/test/results/clientnegative/invalidate_view1.q.out
+++ b/ql/src/test/results/clientnegative/invalidate_view1.q.out
@@ -24,6 +24,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@xxx10
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@xxx9
+POSTHOOK: Lineage: xxx9.key SIMPLE [(xxx10)xxx10.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: xxx9.value SIMPLE [(xxx10)xxx10.FieldSchema(name:value, type:int, comment:null), ]
 PREHOOK: query: CREATE VIEW xxx8 AS SELECT * FROM xxx9 xxx
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@xxx10
@@ -36,6 +38,8 @@ POSTHOOK: Input: default@xxx10
 POSTHOOK: Input: default@xxx9
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@xxx8
+POSTHOOK: Lineage: xxx8.key SIMPLE [(xxx10)xxx10.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: xxx8.value SIMPLE [(xxx10)xxx10.FieldSchema(name:value, type:int, comment:null), ]
 PREHOOK: query: ALTER TABLE xxx10 REPLACE COLUMNS (key int)
 PREHOOK: type: ALTERTABLE_REPLACECOLS
 PREHOOK: Input: default@xxx10

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/load_view_failure.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_view_failure.q.out b/ql/src/test/results/clientnegative/load_view_failure.q.out
index ba1e507..23b3bfc 100644
--- a/ql/src/test/results/clientnegative/load_view_failure.q.out
+++ b/ql/src/test/results/clientnegative/load_view_failure.q.out
@@ -12,4 +12,6 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@xxx11
+POSTHOOK: Lineage: xxx11.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: xxx11.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 FAILED: SemanticException [Error 10090]: A view cannot be used as target table for LOAD or INSERT

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/recursive_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/recursive_view.q.out b/ql/src/test/results/clientnegative/recursive_view.q.out
index ca785f1..75c37a5 100644
--- a/ql/src/test/results/clientnegative/recursive_view.q.out
+++ b/ql/src/test/results/clientnegative/recursive_view.q.out
@@ -36,6 +36,7 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@t
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@r0
+POSTHOOK: Lineage: r0.id SIMPLE [(t)t.FieldSchema(name:id, type:int, comment:null), ]
 PREHOOK: query: create view r1 as select * from r0
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@r0
@@ -48,6 +49,7 @@ POSTHOOK: Input: default@r0
 POSTHOOK: Input: default@t
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@r1
+POSTHOOK: Lineage: r1.id SIMPLE [(t)t.FieldSchema(name:id, type:int, comment:null), ]
 PREHOOK: query: create view r2 as select * from r1
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@r0
@@ -62,6 +64,7 @@ POSTHOOK: Input: default@r1
 POSTHOOK: Input: default@t
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@r2
+POSTHOOK: Lineage: r2.id SIMPLE [(t)t.FieldSchema(name:id, type:int, comment:null), ]
 PREHOOK: query: create view r3 as select * from r2
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@r0
@@ -78,6 +81,7 @@ POSTHOOK: Input: default@r2
 POSTHOOK: Input: default@t
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@r3
+POSTHOOK: Lineage: r3.id SIMPLE [(t)t.FieldSchema(name:id, type:int, comment:null), ]
 PREHOOK: query: drop view r0
 PREHOOK: type: DROPVIEW
 PREHOOK: Input: default@r0

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/unset_view_property.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/unset_view_property.q.out b/ql/src/test/results/clientnegative/unset_view_property.q.out
index 4c94e25..ae7f4f6 100644
--- a/ql/src/test/results/clientnegative/unset_view_property.q.out
+++ b/ql/src/test/results/clientnegative/unset_view_property.q.out
@@ -8,6 +8,7 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@testView
+POSTHOOK: Lineage: testView.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: ALTER VIEW testView SET TBLPROPERTIES ('propA'='100', 'propB'='200')
 PREHOOK: type: ALTERVIEW_PROPERTIES
 PREHOOK: Input: default@testview

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/view_delete.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/view_delete.q.out b/ql/src/test/results/clientnegative/view_delete.q.out
index 24e7ba5..29f4a5f 100644
--- a/ql/src/test/results/clientnegative/view_delete.q.out
+++ b/ql/src/test/results/clientnegative/view_delete.q.out
@@ -16,4 +16,7 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@dv_basetable
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@dv_view
+POSTHOOK: Lineage: dv_view.a SIMPLE [(dv_basetable)dv_basetable.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: dv_view.b SIMPLE [(dv_basetable)dv_basetable.FieldSchema(name:b, type:varchar(256), comment:null), ]
+POSTHOOK: Lineage: dv_view.c SIMPLE [(dv_basetable)dv_basetable.FieldSchema(name:c, type:decimal(10,2), comment:null), ]
 FAILED: SemanticException [Error 10402]: You cannot update or delete records in a view

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientnegative/view_update.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/view_update.q.out b/ql/src/test/results/clientnegative/view_update.q.out
index aaee442..a33fdf7 100644
--- a/ql/src/test/results/clientnegative/view_update.q.out
+++ b/ql/src/test/results/clientnegative/view_update.q.out
@@ -16,4 +16,7 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@uv_basetable
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@uv_view
+POSTHOOK: Lineage: uv_view.a SIMPLE [(uv_basetable)uv_basetable.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: uv_view.b SIMPLE [(uv_basetable)uv_basetable.FieldSchema(name:b, type:varchar(256), comment:null), ]
+POSTHOOK: Lineage: uv_view.c SIMPLE [(uv_basetable)uv_basetable.FieldSchema(name:c, type:decimal(10,2), comment:null), ]
 FAILED: SemanticException [Error 10402]: You cannot update or delete records in a view

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/alter_view_as_select.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_view_as_select.q.out b/ql/src/test/results/clientpositive/alter_view_as_select.q.out
index 9cbaa24..4eddf94 100644
--- a/ql/src/test/results/clientpositive/alter_view_as_select.q.out
+++ b/ql/src/test/results/clientpositive/alter_view_as_select.q.out
@@ -14,6 +14,10 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@srcpart
 POSTHOOK: Output: database:tv
 POSTHOOK: Output: tv@testView
+POSTHOOK: Lineage: testView.ds SIMPLE [(srcpart)srcpart.FieldSchema(name:ds, type:string, comment:null), ]
+POSTHOOK: Lineage: testView.hr SIMPLE [(srcpart)srcpart.FieldSchema(name:hr, type:string, comment:null), ]
+POSTHOOK: Lineage: testView.key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: testView.value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: DESCRIBE FORMATTED tv.testView
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: tv@testview

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/alter_view_rename.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_view_rename.q.out b/ql/src/test/results/clientpositive/alter_view_rename.q.out
index 300b96c..307c0f6 100644
--- a/ql/src/test/results/clientpositive/alter_view_rename.q.out
+++ b/ql/src/test/results/clientpositive/alter_view_rename.q.out
@@ -28,6 +28,9 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@invites
 POSTHOOK: Output: database:tv1
 POSTHOOK: Output: tv1@view1
+POSTHOOK: Lineage: view1.bar SIMPLE [(invites)invites.FieldSchema(name:bar, type:string, comment:null), ]
+POSTHOOK: Lineage: view1.ds SIMPLE [(invites)invites.FieldSchema(name:ds, type:string, comment:null), ]
+POSTHOOK: Lineage: view1.foo SIMPLE [(invites)invites.FieldSchema(name:foo, type:int, comment:null), ]
 PREHOOK: query: DESCRIBE EXTENDED tv1.view1
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: tv1@view1

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/authorization_8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_8.q.out b/ql/src/test/results/clientpositive/authorization_8.q.out
index 9fed5dc..eef4611 100644
--- a/ql/src/test/results/clientpositive/authorization_8.q.out
+++ b/ql/src/test/results/clientpositive/authorization_8.q.out
@@ -20,6 +20,9 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@tbl_j5jbymsx8e
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@view_j5jbymsx8e_1
+POSTHOOK: Lineage: view_j5jbymsx8e_1.ds SIMPLE [(tbl_j5jbymsx8e)tbl_j5jbymsx8e.FieldSchema(name:ds, type:string, comment:null), ]
+POSTHOOK: Lineage: view_j5jbymsx8e_1.key SIMPLE [(tbl_j5jbymsx8e)tbl_j5jbymsx8e.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: view_j5jbymsx8e_1.value SIMPLE [(tbl_j5jbymsx8e)tbl_j5jbymsx8e.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: DESCRIBE view_j5jbymsx8e_1
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view_j5jbymsx8e_1

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/authorization_cli_createtab.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_cli_createtab.q.out b/ql/src/test/results/clientpositive/authorization_cli_createtab.q.out
index 79456ce..eeebc85 100644
--- a/ql/src/test/results/clientpositive/authorization_cli_createtab.q.out
+++ b/ql/src/test/results/clientpositive/authorization_cli_createtab.q.out
@@ -24,6 +24,7 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@t_cli
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v_cli
+POSTHOOK: Lineage: v_cli.i SIMPLE [(t_cli)t_cli.FieldSchema(name:i, type:int, comment:null), ]
 PREHOOK: query: show grant user hive_test_user on v_cli
 PREHOOK: type: SHOW_GRANT
 POSTHOOK: query: show grant user hive_test_user on v_cli

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/authorization_cli_createtab_noauthzapi.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_cli_createtab_noauthzapi.q.out b/ql/src/test/results/clientpositive/authorization_cli_createtab_noauthzapi.q.out
index 7c8909a..d5a79c6 100644
--- a/ql/src/test/results/clientpositive/authorization_cli_createtab_noauthzapi.q.out
+++ b/ql/src/test/results/clientpositive/authorization_cli_createtab_noauthzapi.q.out
@@ -16,3 +16,4 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@t_cli
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v_cli
+POSTHOOK: Lineage: v_cli.i SIMPLE [(t_cli)t_cli.FieldSchema(name:i, type:int, comment:null), ]

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/authorization_owner_actions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_owner_actions.q.out b/ql/src/test/results/clientpositive/authorization_owner_actions.q.out
index 5edce56..779c3ba 100644
--- a/ql/src/test/results/clientpositive/authorization_owner_actions.q.out
+++ b/ql/src/test/results/clientpositive/authorization_owner_actions.q.out
@@ -40,6 +40,7 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@t1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@vt1
+POSTHOOK: Lineage: vt1.i SIMPLE [(t1)t1.FieldSchema(name:i, type:int, comment:null), ]
 PREHOOK: query: drop view vt1
 PREHOOK: type: DROPVIEW
 PREHOOK: Input: default@vt1

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/authorization_view_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_view_1.q.out b/ql/src/test/results/clientpositive/authorization_view_1.q.out
index 89adf19..31cc0c8 100644
--- a/ql/src/test/results/clientpositive/authorization_view_1.q.out
+++ b/ql/src/test/results/clientpositive/authorization_view_1.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v1 as select * from src_autho_test
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -30,6 +32,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v2 as select * from src_autho_test
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -40,6 +44,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select on table src_autho_test to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@src_autho_test

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/authorization_view_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_view_2.q.out b/ql/src/test/results/clientpositive/authorization_view_2.q.out
index 9f909be..e4dd059 100644
--- a/ql/src/test/results/clientpositive/authorization_view_2.q.out
+++ b/ql/src/test/results/clientpositive/authorization_view_2.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v2 as select * from v1
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -32,6 +34,8 @@ POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Input: default@v1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select on table v2 to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@v2

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/authorization_view_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_view_3.q.out b/ql/src/test/results/clientpositive/authorization_view_3.q.out
index c9334e3..b2d3b1f 100644
--- a/ql/src/test/results/clientpositive/authorization_view_3.q.out
+++ b/ql/src/test/results/clientpositive/authorization_view_3.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v2 as select * from v1
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -32,6 +34,8 @@ POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Input: default@v1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select on table v2 to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@v2

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/authorization_view_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_view_4.q.out b/ql/src/test/results/clientpositive/authorization_view_4.q.out
index 68fdfb6..c832f11 100644
--- a/ql/src/test/results/clientpositive/authorization_view_4.q.out
+++ b/ql/src/test/results/clientpositive/authorization_view_4.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: create view v2 as select * from v1
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src
@@ -32,6 +34,8 @@ POSTHOOK: Input: default@src
 POSTHOOK: Input: default@v1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: grant select on table v2 to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@v2

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/authorization_view_disable_cbo_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_view_disable_cbo_1.q.out b/ql/src/test/results/clientpositive/authorization_view_disable_cbo_1.q.out
index 8395782..edc0d897 100644
--- a/ql/src/test/results/clientpositive/authorization_view_disable_cbo_1.q.out
+++ b/ql/src/test/results/clientpositive/authorization_view_disable_cbo_1.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
+POSTHOOK: Lineage: v.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v1 as select * from src_autho_test
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -30,6 +32,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v2 as select * from src_autho_test
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -40,6 +44,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select on table src_autho_test to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@src_autho_test

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/authorization_view_disable_cbo_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_view_disable_cbo_2.q.out b/ql/src/test/results/clientpositive/authorization_view_disable_cbo_2.q.out
index 9f909be..e4dd059 100644
--- a/ql/src/test/results/clientpositive/authorization_view_disable_cbo_2.q.out
+++ b/ql/src/test/results/clientpositive/authorization_view_disable_cbo_2.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v2 as select * from v1
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -32,6 +34,8 @@ POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Input: default@v1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select on table v2 to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@v2

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/authorization_view_disable_cbo_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_view_disable_cbo_3.q.out b/ql/src/test/results/clientpositive/authorization_view_disable_cbo_3.q.out
index c9334e3..b2d3b1f 100644
--- a/ql/src/test/results/clientpositive/authorization_view_disable_cbo_3.q.out
+++ b/ql/src/test/results/clientpositive/authorization_view_disable_cbo_3.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: create view v2 as select * from v1
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src_autho_test
@@ -32,6 +34,8 @@ POSTHOOK: Input: default@src_autho_test
 POSTHOOK: Input: default@v1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src_autho_test)src_autho_test.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: grant select on table v2 to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@v2

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/authorization_view_disable_cbo_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_view_disable_cbo_4.q.out b/ql/src/test/results/clientpositive/authorization_view_disable_cbo_4.q.out
index 68fdfb6..c832f11 100644
--- a/ql/src/test/results/clientpositive/authorization_view_disable_cbo_4.q.out
+++ b/ql/src/test/results/clientpositive/authorization_view_disable_cbo_4.q.out
@@ -20,6 +20,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v1
+POSTHOOK: Lineage: v1.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: create view v2 as select * from v1
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src
@@ -32,6 +34,8 @@ POSTHOOK: Input: default@src
 POSTHOOK: Input: default@v1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v2
+POSTHOOK: Lineage: v2.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: v2.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: grant select on table v2 to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@v2

http://git-wip-us.apache.org/repos/asf/hive/blob/c7ac3405/ql/src/test/results/clientpositive/cbo_const.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_const.q.out b/ql/src/test/results/clientpositive/cbo_const.q.out
index ecf0269..07a89e5 100644
--- a/ql/src/test/results/clientpositive/cbo_const.q.out
+++ b/ql/src/test/results/clientpositive/cbo_const.q.out
@@ -91,6 +91,8 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@t1_new
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@t1
+POSTHOOK: Lineage: t1.key EXPRESSION [(t1_new)t1_new.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: t1.value EXPRESSION [(t1_new)t1_new.FieldSchema(name:value, type:string, comment:null), ]
 PREHOOK: query: select * from t1 where ds = '2011-10-15'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t1