You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2016/12/16 13:58:32 UTC

[6/6] hive git commit: HIVE-14496: Enable Calcite rewriting with materialized views (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

HIVE-14496: Enable Calcite rewriting with materialized views (Jesus Camacho Rodriguez, 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/3da29fe7
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3da29fe7
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3da29fe7

Branch: refs/heads/master
Commit: 3da29fe7e11b849b1cf5ee8f49a867b2b691ae8d
Parents: bf37d78
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Mon Dec 12 09:22:47 2016 +0000
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Fri Dec 16 13:58:06 2016 +0000

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   14 +-
 data/conf/hive-site.xml                         |    6 +
 .../hive/hcatalog/common/TestHCatUtil.java      |    5 +-
 .../listener/TestDbNotificationListener.java    |    5 +-
 .../hive/metastore/TestHiveMetaStore.java       |    1 +
 .../metastore/hbase/TestHBaseSchemaTool.java    |    8 +-
 .../org/apache/hive/beeline/TestSchemaTool.java |   11 +-
 metastore/if/hive_metastore.thrift              |    5 +-
 .../upgrade/derby/037-HIVE-14496.derby.sql      |    8 +
 .../upgrade/derby/hive-schema-2.2.0.derby.sql   |    2 +-
 .../derby/upgrade-2.1.0-to-2.2.0.derby.sql      |    1 +
 .../upgrade/mssql/022-HIVE-14496.mssql.sql      |    1 +
 .../upgrade/mssql/hive-schema-2.2.0.mssql.sql   |    3 +-
 .../mssql/upgrade-2.1.0-to-2.2.0.mssql.sql      |    2 +
 .../upgrade/mysql/037-HIVE-14496.mysql.sql      |    8 +
 .../upgrade/mysql/hive-schema-2.2.0.mysql.sql   |    1 +
 .../mysql/upgrade-2.1.0-to-2.2.0.mysql.sql      |    2 +
 .../upgrade/oracle/037-HIVE-14496.oracle.sql    |    9 +
 .../upgrade/oracle/hive-schema-2.2.0.oracle.sql |    3 +-
 .../oracle/upgrade-2.1.0-to-2.2.0.oracle.sql    |    2 +
 .../postgres/036-HIVE-14496.postgres.sql        |    8 +
 .../postgres/hive-schema-2.2.0.postgres.sql     |    3 +-
 .../upgrade-2.1.0-to-2.2.0.postgres.sql         |    2 +
 .../metastore/hbase/HbaseMetastoreProto.java    |  177 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |   22 +
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   12 +-
 .../apache/hadoop/hive/metastore/api/Table.java |  107 +-
 .../src/gen/thrift/gen-php/metastore/Types.php  |   23 +
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |   15 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |    4 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |  218 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   64 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |   12 +-
 .../hadoop/hive/metastore/ObjectStore.java      |   17 +-
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |    2 +
 .../hadoop/hive/metastore/model/MTable.java     |   20 +-
 metastore/src/model/package.jdo                 |    3 +
 .../metastore/hbase/hbase_metastore_proto.proto |    1 +
 .../hadoop/hive/metastore/TestObjectStore.java  |    6 +-
 .../hbase/TestHBaseAggregateStatsCache.java     |   20 +-
 ...stHBaseAggregateStatsCacheWithBitVector.java |   21 +-
 .../TestHBaseAggregateStatsExtrapolation.java   |   21 +-
 .../TestHBaseAggregateStatsNDVUniformDist.java  |   21 +-
 .../hive/metastore/hbase/TestHBaseStore.java    |    2 -
 .../metastore/hbase/TestHBaseStoreCached.java   |   43 +-
 .../org/apache/hadoop/hive/ql/QueryState.java   |    2 -
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   24 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   89 +-
 .../metadata/HiveMaterializedViewsRegistry.java |  393 +++
 .../apache/hadoop/hive/ql/metadata/Table.java   |   27 +-
 .../formatting/MetaDataFormatUtils.java         |    3 +-
 .../optimizer/calcite/HiveRexExecutorImpl.java  |   10 +-
 .../ql/optimizer/calcite/RelOptHiveTable.java   |   14 +
 .../calcite/reloperators/HiveAggregate.java     |    7 -
 .../calcite/reloperators/HiveFilter.java        |    8 -
 .../calcite/reloperators/HiveJoin.java          |    9 -
 .../calcite/reloperators/HiveProject.java       |    8 -
 .../calcite/reloperators/HiveSemiJoin.java      |    8 -
 .../calcite/reloperators/HiveTableScan.java     |    6 -
 .../HiveMaterializedViewFilterScanRule.java     |   91 +
 .../MaterializedViewSubstitutionVisitor.java    |  292 +++
 .../rules/views/SubstitutionVisitor.java        | 2458 ++++++++++++++++++
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |  157 +-
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |   19 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   24 +-
 .../hadoop/hive/ql/parse/StorageFormat.java     |    4 +-
 .../hadoop/hive/ql/plan/CreateViewDesc.java     |   16 +-
 .../hadoop/hive/ql/session/SessionState.java    |    6 +-
 .../hadoop/hive/ql/metadata/TestHive.java       |    4 +
 .../materialized_view_create_rewrite.q          |   59 +
 .../materialized_view_create_rewrite_multi_db.q |   40 +
 .../alter_view_as_select_with_partition.q.out   |    1 +
 .../clientpositive/alter_view_as_select.q.out   |    3 +
 .../clientpositive/create_or_replace_view.q.out |    5 +
 .../results/clientpositive/create_view.q.out    |   31 +-
 .../create_view_defaultformats.q.out            |    2 +
 .../create_view_partitioned.q.out               |    5 +-
 .../clientpositive/create_view_translate.q.out  |    3 +
 .../test/results/clientpositive/cteViews.q.out  |    8 +-
 .../clientpositive/escape_comments.q.out        |    1 +
 .../results/clientpositive/explain_ddl.q.out    |    1 +
 .../llap/cbo_rp_unionDistinct_2.q.out           |    6 +-
 .../llap/selectDistinctStar.q.out               |    4 +
 .../clientpositive/llap/subquery_views.q.out    |    6 +-
 .../clientpositive/llap/unionDistinct_2.q.out   |    6 +-
 .../clientpositive/llap/union_top_level.q.out   |    1 +
 .../materialized_view_create_rewrite.q.out      |  322 +++
 ...erialized_view_create_rewrite_multi_db.q.out |  157 ++
 .../materialized_view_describe.q.out            |   20 +
 .../clientpositive/spark/union_top_level.q.out  |    1 +
 .../results/clientpositive/subquery_views.q.out |    6 +-
 .../clientpositive/tez/unionDistinct_2.q.out    |    6 +-
 .../clientpositive/unicode_comments.q.out       |    1 +
 .../results/clientpositive/view_alias.q.out     |    6 +
 .../apache/hive/service/server/HiveServer2.java |   10 +
 95 files changed, 4791 insertions(+), 540 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index dcb383d..b4e89b0 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1070,6 +1070,15 @@ public class HiveConf extends Configuration {
     HIVE_COLUMN_ALIGNMENT("hive.order.columnalignment", true, "Flag to control whether we want to try to align" +
         "columns in operators such as Aggregate or Join so that we try to reduce the number of shuffling stages"),
 
+    // materialized views
+    HIVE_MATERIALIZED_VIEW_ENABLE_AUTO_REWRITING("hive.materializedview.rewriting", false,
+        "Whether to try to rewrite queries using the materialized views enabled for rewriting"),
+    HIVE_MATERIALIZED_VIEW_FILE_FORMAT("hive.materializedview.fileformat", "ORC",
+        new StringSet("none", "TextFile", "SequenceFile", "RCfile", "ORC"),
+        "Default file format for CREATE MATERIALIZED VIEW statement"),
+    HIVE_MATERIALIZED_VIEW_SERDE("hive.materializedview.serde",
+        "org.apache.hadoop.hive.ql.io.orc.OrcSerde", "Default SerDe used for materialized views"),
+
     // hive.mapjoin.bucket.cache.size has been replaced by hive.smbjoin.cache.row,
     // need to remove by hive .13. Also, do not change default (see SMB operator)
     HIVEMAPJOINBUCKETCACHESIZE("hive.mapjoin.bucket.cache.size", 100, ""),
@@ -1149,11 +1158,6 @@ public class HiveConf extends Configuration {
         "Default file format for CREATE TABLE statement applied to managed tables only. External tables will be \n" +
         "created with format specified by hive.default.fileformat. Leaving this null will result in using hive.default.fileformat \n" +
         "for all tables."),
-    HIVEMATERIALIZEDVIEWFILEFORMAT("hive.materializedview.fileformat", "ORC",
-        new StringSet("none", "TextFile", "SequenceFile", "RCfile", "ORC"),
-        "Default file format for CREATE MATERIALIZED VIEW statement"),
-    HIVEMATERIALIZEDVIEWSERDE("hive.materializedview.serde",
-        "org.apache.hadoop.hive.ql.io.orc.OrcSerde", "Default SerDe used for materialized views"),
     HIVEQUERYRESULTFILEFORMAT("hive.query.result.fileformat", "SequenceFile", new StringSet("TextFile", "SequenceFile", "RCfile", "Llap"),
         "Default file format for storing result of the query."),
     HIVECHECKFILEFORMAT("hive.fileformat.check", true, "Whether to check file format or not when loading data files"),

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/data/conf/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/hive-site.xml b/data/conf/hive-site.xml
index b09c159..7a69711 100644
--- a/data/conf/hive-site.xml
+++ b/data/conf/hive-site.xml
@@ -309,4 +309,10 @@
 </property>
 
 
+<property>
+  <name>hive.materializedview.rewriting</name>
+  <value>true</value>
+</property>
+
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/hcatalog/core/src/test/java/org/apache/hive/hcatalog/common/TestHCatUtil.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/common/TestHCatUtil.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/common/TestHCatUtil.java
index 102d6d2..61add91 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/common/TestHCatUtil.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/common/TestHCatUtil.java
@@ -24,8 +24,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hive.metastore.TableType;
@@ -40,6 +38,9 @@ import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
 public class TestHCatUtil {
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
index 690616d..0b691b1 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
@@ -188,9 +188,8 @@ public class TestDbNotificationListener {
     StorageDescriptor sd =
         new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0, serde, null, null,
             emptyParameters);
-    Table table =
-        new Table("mytable", "default", "me", startTime, startTime, 0, sd, null, emptyParameters,
-            null, null, null);
+    Table table = new Table("mytable", "default", "me", startTime, startTime, 0, sd, null,
+        emptyParameters, null, null, null);
     msClient.createTable(table);
     // Get the event
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
index 21d1b46..af125c3 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
@@ -738,6 +738,7 @@ public abstract class TestHiveMetaStore extends TestCase {
     view.setViewOriginalText("SELECT income, name FROM " + tblName);
     view.setViewExpandedText("SELECT `" + tblName + "`.`income`, `" + tblName +
         "`.`name` FROM `" + dbName + "`.`" + tblName + "`");
+    view.setRewriteEnabled(false);
     StorageDescriptor viewSd = new StorageDescriptor();
     view.setSd(viewSd);
     viewSd.setCols(viewCols);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java
index b131163..c98911a 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java
@@ -455,7 +455,7 @@ public class TestHBaseSchemaTool extends HBaseIntegrationTests {
         "\"createTime\":0,\"lastAccessTime\":0,\"retention\":0," +
         "\"partitionKeys\":[{\"name\":\"pcol1\",\"type\":\"string\",\"comment\":\"\"}," +
         "{\"name\":\"pcol2\",\"type\":\"string\",\"comment\":\"\"}],\"parameters\":{}," +
-        "\"tableType\":\"\"} sdHash: qQTgZAi5VzgpozzFGmIVTQ stats:" + lsep,
+        "\"tableType\":\"\",\"rewriteEnabled\":0} sdHash: qQTgZAi5VzgpozzFGmIVTQ stats:" + lsep,
         outStr.toString());
 
     outStr = new ByteArrayOutputStream();
@@ -465,7 +465,7 @@ public class TestHBaseSchemaTool extends HBaseIntegrationTests {
         "\"createTime\":0,\"lastAccessTime\":0,\"retention\":0," +
         "\"partitionKeys\":[{\"name\":\"pcol1\",\"type\":\"string\",\"comment\":\"\"}," +
         "{\"name\":\"pcol2\",\"type\":\"string\",\"comment\":\"\"}],\"parameters\":{\"COLUMN_STATS_ACCURATE\":\"{\\\"COLUMN_STATS\\\":{\\\"col1\\\":\\\"true\\\",\\\"col2\\\":\\\"true\\\"}}\"}," +
-            "\"tableType\":\"\"} sdHash: qQTgZAi5VzgpozzFGmIVTQ stats: column " +
+            "\"tableType\":\"\",\"rewriteEnabled\":0} sdHash: qQTgZAi5VzgpozzFGmIVTQ stats: column " +
             "col1: {\"colName\":\"col1\",\"colType\":\"int\"," +
             "\"statsData\":{\"longStats\":{\"lowValue\":-95,\"highValue\":95,\"numNulls\":1," +
             "\"numDVs\":2,\"bitVectors\":\"\"}}} column col2: {\"colName\":\"col2\",\"colType\":\"varchar(32)\"," +
@@ -474,12 +474,12 @@ public class TestHBaseSchemaTool extends HBaseIntegrationTests {
         "{\"tableName\":\"tab1\",\"dbName\":\"db0\",\"owner\":\"me\",\"createTime\":0," +
         "\"lastAccessTime\":0,\"retention\":0,\"partitionKeys\":[{\"name\":\"pcol1\"," +
             "\"type\":\"string\",\"comment\":\"\"},{\"name\":\"pcol2\",\"type\":\"string\"," +
-            "\"comment\":\"\"}],\"parameters\":{},\"tableType\":\"\"} sdHash: " +
+            "\"comment\":\"\"}],\"parameters\":{},\"tableType\":\"\",\"rewriteEnabled\":0} sdHash: " +
             "qQTgZAi5VzgpozzFGmIVTQ stats:" + lsep +
         "{\"tableName\":\"tab2\",\"dbName\":\"db0\",\"owner\":\"me\",\"createTime\":0," +
         "\"lastAccessTime\":0,\"retention\":0,\"partitionKeys\":[{\"name\":\"pcol1\"," +
         "\"type\":\"string\",\"comment\":\"\"},{\"name\":\"pcol2\",\"type\":\"string\"," +
-        "\"comment\":\"\"}],\"parameters\":{},\"tableType\":\"\"} sdHash: " +
+        "\"comment\":\"\"}],\"parameters\":{},\"tableType\":\"\",\"rewriteEnabled\":0} sdHash: " +
         "qQTgZAi5VzgpozzFGmIVTQ stats:" + lsep, outStr.toString());
 
     List<List<String>> partVals = Arrays.asList(Arrays.asList("a", "b"), Arrays.asList("c", "d"));

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
index 17a4bd9..724f990 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
@@ -613,10 +613,9 @@ public class TestSchemaTool extends TestCase {
          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (3,null,'org.apache.hadoop.mapred.TextInputFormat','N','N',null,-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL)",
+         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
          "insert into PARTITiONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)",
-         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT) values (3 ,1435255431,2,0 ,'hive',0,3,'myView','VIRTUAL_VIEW','select a.col1,a.col2 from foo','select * from foo')"
-
+         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3 ,1435255431,2,0 ,'hive',0,3,'myView','VIRTUAL_VIEW','select a.col1,a.col2 from foo','select * from foo','n')"
        };
     File scriptFile = generateTestScript(scripts);
     schemaTool.runBeeLine(scriptFile.getPath());
@@ -632,11 +631,11 @@ public class TestSchemaTool extends TestCase {
         "insert into DBS values(2, 'my db', '/user/hive/warehouse/mydb', 'mydb', 'public', 'role')",
         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://yourhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','file:///user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL)",
+        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
         "insert into PARTITiONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)",
         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (3000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','yourhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (5000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','file:///user/admin/2016_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT) values (3000 ,1435255431,2,0 ,'hive',0,3000,'mytal3000','MANAGED_TABLE',NULL,NULL)",
+        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3000 ,1435255431,2,0 ,'hive',0,3000,'mytal3000','MANAGED_TABLE',NULL,NULL,'n')",
         "insert into PARTITiONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(5000, 1441402388,0, 'd1=1/d2=5000',5000,2)"
     };
     scriptFile = generateTestScript(scripts);
@@ -686,7 +685,7 @@ public class TestSchemaTool extends TestCase {
           "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb', 'mydb', 'public', 'role')",
           "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
           "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-          "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL)",
+          "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
           "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)"
         };
      File scriptFile = generateTestScript(scripts);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index baab31b..6f77156 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -302,9 +302,10 @@ struct Table {
   9: map<string, string> parameters,   // to store comments or any other user level parameters
   10: string viewOriginalText,         // original view text, null for non-view
   11: string viewExpandedText,         // expanded view text, null for non-view
-  12: string tableType,                 // table type enum, e.g. EXTERNAL_TABLE
+  12: string tableType,                // table type enum, e.g. EXTERNAL_TABLE
   13: optional PrincipalPrivilegeSet privileges,
-  14: optional bool temporary=false
+  14: optional bool temporary=false,
+  15: optional bool rewriteEnabled     // rewrite enabled or not
 }
 
 struct Partition {

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/derby/037-HIVE-14496.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/037-HIVE-14496.derby.sql b/metastore/scripts/upgrade/derby/037-HIVE-14496.derby.sql
new file mode 100644
index 0000000..0c294ce
--- /dev/null
+++ b/metastore/scripts/upgrade/derby/037-HIVE-14496.derby.sql
@@ -0,0 +1,8 @@
+-- Step 1: Add the column allowing null
+ALTER TABLE "APP"."TBLS" ADD "IS_REWRITE_ENABLED" CHAR(1);
+
+ -- Step 2: Replace the null with default value (false)
+UPDATE "APP"."TBLS" SET "IS_REWRITE_ENABLED" = 'N';
+
+-- Step 3: Alter the column to disallow null values
+ALTER TABLE "APP"."TBLS" ALTER COLUMN "IS_REWRITE_ENABLED" NOT NULL;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql b/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql
index ae980e0..fe18089 100644
--- a/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql
@@ -60,7 +60,7 @@ CREATE TABLE "APP"."COLUMNS" ("SD_ID" BIGINT NOT NULL, "COMMENT" VARCHAR(256), "
 
 CREATE TABLE "APP"."ROLES" ("ROLE_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "OWNER_NAME" VARCHAR(128), "ROLE_NAME" VARCHAR(128));
 
-CREATE TABLE "APP"."TBLS" ("TBL_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "OWNER" VARCHAR(767), "RETENTION" INTEGER NOT NULL, "SD_ID" BIGINT, "TBL_NAME" VARCHAR(128), "TBL_TYPE" VARCHAR(128), "VIEW_EXPANDED_TEXT" LONG VARCHAR, "VIEW_ORIGINAL_TEXT" LONG VARCHAR);
+CREATE TABLE "APP"."TBLS" ("TBL_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "OWNER" VARCHAR(767), "RETENTION" INTEGER NOT NULL, "SD_ID" BIGINT, "TBL_NAME" VARCHAR(128), "TBL_TYPE" VARCHAR(128), "VIEW_EXPANDED_TEXT" LONG VARCHAR, "VIEW_ORIGINAL_TEXT" LONG VARCHAR, "IS_REWRITE_ENABLED" CHAR(1) NOT NULL);
 
 CREATE TABLE "APP"."PARTITION_KEYS" ("TBL_ID" BIGINT NOT NULL, "PKEY_COMMENT" VARCHAR(4000), "PKEY_NAME" VARCHAR(128) NOT NULL, "PKEY_TYPE" VARCHAR(767) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/derby/upgrade-2.1.0-to-2.2.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/upgrade-2.1.0-to-2.2.0.derby.sql b/metastore/scripts/upgrade/derby/upgrade-2.1.0-to-2.2.0.derby.sql
index 25a5e37..699a619 100644
--- a/metastore/scripts/upgrade/derby/upgrade-2.1.0-to-2.2.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/upgrade-2.1.0-to-2.2.0.derby.sql
@@ -1,3 +1,4 @@
 -- Upgrade MetaStore schema from 2.1.0 to 2.2.0
+RUN '037-HIVE-14496.derby.sql';
 
 UPDATE "APP".VERSION SET SCHEMA_VERSION='2.2.0', VERSION_COMMENT='Hive release version 2.2.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/mssql/022-HIVE-14496.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/022-HIVE-14496.mssql.sql b/metastore/scripts/upgrade/mssql/022-HIVE-14496.mssql.sql
new file mode 100644
index 0000000..0c59467
--- /dev/null
+++ b/metastore/scripts/upgrade/mssql/022-HIVE-14496.mssql.sql
@@ -0,0 +1 @@
+ALTER TABLE TBLS ADD IS_REWRITE_ENABLED bit NOT NULL DEFAULT 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql b/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql
index fdb4004..7ff881c 100644
--- a/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql
@@ -358,7 +358,8 @@ CREATE TABLE TBLS
     TBL_NAME nvarchar(128) NULL,
     TBL_TYPE nvarchar(128) NULL,
     VIEW_EXPANDED_TEXT text NULL,
-    VIEW_ORIGINAL_TEXT text NULL
+    VIEW_ORIGINAL_TEXT text NULL,
+    IS_REWRITE_ENABLED bit NOT NULL
 );
 
 ALTER TABLE TBLS ADD CONSTRAINT TBLS_PK PRIMARY KEY (TBL_ID);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql b/metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql
index df97206..55d8e9b 100644
--- a/metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql
@@ -1,4 +1,6 @@
 SELECT 'Upgrading MetaStore schema from 2.1.0 to 2.2.0' AS MESSAGE;
 
+:r 022-HIVE-14496.mssql.sql
+
 UPDATE VERSION SET SCHEMA_VERSION='2.2.0', VERSION_COMMENT='Hive release version 2.2.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.1.0 to 2.2.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/mysql/037-HIVE-14496.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/037-HIVE-14496.mysql.sql b/metastore/scripts/upgrade/mysql/037-HIVE-14496.mysql.sql
new file mode 100644
index 0000000..6cccefe
--- /dev/null
+++ b/metastore/scripts/upgrade/mysql/037-HIVE-14496.mysql.sql
@@ -0,0 +1,8 @@
+-- Step 1: Add the column allowing null
+ALTER TABLE `TBLS` ADD `IS_REWRITE_ENABLED` bit(1);
+
+ -- Step 2: Replace the null with default value (false)
+UPDATE `TBLS` SET `IS_REWRITE_ENABLED` = false;
+
+-- Step 3: Alter the column to disallow null values
+ALTER TABLE `TBLS` MODIFY COLUMN `IS_REWRITE_ENABLED` bit(1) NOT NULL;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql
index 91e221d..2009f1f 100644
--- a/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql
@@ -587,6 +587,7 @@ CREATE TABLE IF NOT EXISTS `TBLS` (
   `TBL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `VIEW_EXPANDED_TEXT` mediumtext,
   `VIEW_ORIGINAL_TEXT` mediumtext,
+  `IS_REWRITE_ENABLED` bit(1) NOT NULL,
   PRIMARY KEY (`TBL_ID`),
   UNIQUE KEY `UNIQUETABLE` (`TBL_NAME`,`DB_ID`),
   KEY `TBLS_N50` (`SD_ID`),

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql b/metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql
index de38b58..07a002f 100644
--- a/metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql
@@ -1,5 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.1.0 to 2.2.0' AS ' ';
 
+SOURCE 037-HIVE-14496.mysql.sql;
+
 UPDATE VERSION SET SCHEMA_VERSION='2.2.0', VERSION_COMMENT='Hive release version 2.2.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.1.0 to 2.2.0' AS ' ';
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/oracle/037-HIVE-14496.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/037-HIVE-14496.oracle.sql b/metastore/scripts/upgrade/oracle/037-HIVE-14496.oracle.sql
new file mode 100644
index 0000000..2b3bb77
--- /dev/null
+++ b/metastore/scripts/upgrade/oracle/037-HIVE-14496.oracle.sql
@@ -0,0 +1,9 @@
+-- Step 1: Add the column allowing null
+ALTER TABLE TBLS ADD IS_REWRITE_ENABLED NUMBER(1) NULL;
+
+ -- Step 2: Replace the null with default value (false)
+UPDATE TBLS SET IS_REWRITE_ENABLED = 0;
+
+-- Step 3: Alter the column to disallow null values
+ALTER TABLE TBLS MODIFY(IS_REWRITE_ENABLED NOT NULL);
+ALTER TABLE TBLS ADD CONSTRAINT REWRITE_CHECK CHECK (IS_REWRITE_ENABLED IN (1,0));

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql b/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql
index 39ba7cb..bb5a934 100644
--- a/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql
@@ -375,7 +375,8 @@ CREATE TABLE TBLS
     TBL_NAME VARCHAR2(128) NULL,
     TBL_TYPE VARCHAR2(128) NULL,
     VIEW_EXPANDED_TEXT CLOB NULL,
-    VIEW_ORIGINAL_TEXT CLOB NULL
+    VIEW_ORIGINAL_TEXT CLOB NULL,
+    IS_REWRITE_ENABLED NUMBER(1) NOT NULL CHECK (IS_REWRITE_ENABLED IN (1,0))
 );
 
 ALTER TABLE TBLS ADD CONSTRAINT TBLS_PK PRIMARY KEY (TBL_ID);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql b/metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql
index 66784a4..b5e65b9 100644
--- a/metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql
@@ -1,4 +1,6 @@
 SELECT 'Upgrading MetaStore schema from 2.1.0 to 2.2.0' AS Status from dual;
 
+@037-HIVE-14496.oracle.sql;
+
 UPDATE VERSION SET SCHEMA_VERSION='2.2.0', VERSION_COMMENT='Hive release version 2.2.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.1.0 to 2.2.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/postgres/036-HIVE-14496.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/036-HIVE-14496.postgres.sql b/metastore/scripts/upgrade/postgres/036-HIVE-14496.postgres.sql
new file mode 100644
index 0000000..1910cc3
--- /dev/null
+++ b/metastore/scripts/upgrade/postgres/036-HIVE-14496.postgres.sql
@@ -0,0 +1,8 @@
+-- Step 1: Add the column allowing null
+ALTER TABLE "TBLS" ADD COLUMN "IS_REWRITE_ENABLED" boolean NULL;
+
+ -- Step 2: Replace the null with default value (false)
+UPDATE "TBLS" SET "IS_REWRITE_ENABLED" = false;
+
+-- Step 3: Alter the column to disallow null values
+ALTER TABLE "TBLS" ALTER COLUMN "IS_REWRITE_ENABLED" SET NOT NULL;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql b/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql
index 63ac3be..0021df0 100644
--- a/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql
@@ -372,7 +372,8 @@ CREATE TABLE "TBLS" (
     "TBL_NAME" character varying(128) DEFAULT NULL::character varying,
     "TBL_TYPE" character varying(128) DEFAULT NULL::character varying,
     "VIEW_EXPANDED_TEXT" text,
-    "VIEW_ORIGINAL_TEXT" text
+    "VIEW_ORIGINAL_TEXT" text,
+    "IS_REWRITE_ENABLED" boolean NOT NULL
 );
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql b/metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql
index 0b4591d..0f7139a 100644
--- a/metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql
@@ -1,5 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.1.0 to 2.2.0';
 
+\i 036-HIVE-14496.postgres.sql;
+
 UPDATE "VERSION" SET "SCHEMA_VERSION"='2.2.0', "VERSION_COMMENT"='Hive release version 2.2.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 2.1.0 to 2.2.0';
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
index b15b0de..03e492e 100644
--- a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
+++ b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
@@ -28422,6 +28422,16 @@ public final class HbaseMetastoreProto {
      * <code>optional bool is_temporary = 14;</code>
      */
     boolean getIsTemporary();
+
+    // optional bool is_rewrite_enabled = 15;
+    /**
+     * <code>optional bool is_rewrite_enabled = 15;</code>
+     */
+    boolean hasIsRewriteEnabled();
+    /**
+     * <code>optional bool is_rewrite_enabled = 15;</code>
+     */
+    boolean getIsRewriteEnabled();
   }
   /**
    * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.Table}
@@ -28571,6 +28581,11 @@ public final class HbaseMetastoreProto {
               isTemporary_ = input.readBool();
               break;
             }
+            case 120: {
+              bitField0_ |= 0x00002000;
+              isRewriteEnabled_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -29023,6 +29038,22 @@ public final class HbaseMetastoreProto {
       return isTemporary_;
     }
 
+    // optional bool is_rewrite_enabled = 15;
+    public static final int IS_REWRITE_ENABLED_FIELD_NUMBER = 15;
+    private boolean isRewriteEnabled_;
+    /**
+     * <code>optional bool is_rewrite_enabled = 15;</code>
+     */
+    public boolean hasIsRewriteEnabled() {
+      return ((bitField0_ & 0x00002000) == 0x00002000);
+    }
+    /**
+     * <code>optional bool is_rewrite_enabled = 15;</code>
+     */
+    public boolean getIsRewriteEnabled() {
+      return isRewriteEnabled_;
+    }
+
     private void initFields() {
       owner_ = "";
       createTime_ = 0L;
@@ -29038,6 +29069,7 @@ public final class HbaseMetastoreProto {
       tableType_ = "";
       privileges_ = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrincipalPrivilegeSet.getDefaultInstance();
       isTemporary_ = false;
+      isRewriteEnabled_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -29121,6 +29153,9 @@ public final class HbaseMetastoreProto {
       if (((bitField0_ & 0x00001000) == 0x00001000)) {
         output.writeBool(14, isTemporary_);
       }
+      if (((bitField0_ & 0x00002000) == 0x00002000)) {
+        output.writeBool(15, isRewriteEnabled_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -29186,6 +29221,10 @@ public final class HbaseMetastoreProto {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(14, isTemporary_);
       }
+      if (((bitField0_ & 0x00002000) == 0x00002000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(15, isRewriteEnabled_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -29350,6 +29389,8 @@ public final class HbaseMetastoreProto {
         bitField0_ = (bitField0_ & ~0x00001000);
         isTemporary_ = false;
         bitField0_ = (bitField0_ & ~0x00002000);
+        isRewriteEnabled_ = false;
+        bitField0_ = (bitField0_ & ~0x00004000);
         return this;
       }
 
@@ -29451,6 +29492,10 @@ public final class HbaseMetastoreProto {
           to_bitField0_ |= 0x00001000;
         }
         result.isTemporary_ = isTemporary_;
+        if (((from_bitField0_ & 0x00004000) == 0x00004000)) {
+          to_bitField0_ |= 0x00002000;
+        }
+        result.isRewriteEnabled_ = isRewriteEnabled_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -29542,6 +29587,9 @@ public final class HbaseMetastoreProto {
         if (other.hasIsTemporary()) {
           setIsTemporary(other.getIsTemporary());
         }
+        if (other.hasIsRewriteEnabled()) {
+          setIsRewriteEnabled(other.getIsRewriteEnabled());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -30762,6 +30810,39 @@ public final class HbaseMetastoreProto {
         return this;
       }
 
+      // optional bool is_rewrite_enabled = 15;
+      private boolean isRewriteEnabled_ ;
+      /**
+       * <code>optional bool is_rewrite_enabled = 15;</code>
+       */
+      public boolean hasIsRewriteEnabled() {
+        return ((bitField0_ & 0x00004000) == 0x00004000);
+      }
+      /**
+       * <code>optional bool is_rewrite_enabled = 15;</code>
+       */
+      public boolean getIsRewriteEnabled() {
+        return isRewriteEnabled_;
+      }
+      /**
+       * <code>optional bool is_rewrite_enabled = 15;</code>
+       */
+      public Builder setIsRewriteEnabled(boolean value) {
+        bitField0_ |= 0x00004000;
+        isRewriteEnabled_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool is_rewrite_enabled = 15;</code>
+       */
+      public Builder clearIsRewriteEnabled() {
+        bitField0_ = (bitField0_ & ~0x00004000);
+        isRewriteEnabled_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hive.metastore.hbase.Table)
     }
 
@@ -41582,7 +41663,7 @@ public final class HbaseMetastoreProto {
       "Info.SkewedColValueLocationMap\032.\n\022Skewed" +
       "ColValueList\022\030\n\020skewed_col_value\030\001 \003(\t\0327" +
       "\n\031SkewedColValueLocationMap\022\013\n\003key\030\001 \003(\t" +
-      "\022\r\n\005value\030\002 \002(\t\"\220\004\n\005Table\022\r\n\005owner\030\001 \001(\t" +
+      "\022\r\n\005value\030\002 \002(\t\"\254\004\n\005Table\022\r\n\005owner\030\001 \001(\t" +
       "\022\023\n\013create_time\030\002 \001(\003\022\030\n\020last_access_tim",
       "e\030\003 \001(\003\022\021\n\tretention\030\004 \001(\003\022\020\n\010location\030\005" +
       " \001(\t\022I\n\rsd_parameters\030\006 \001(\01322.org.apache" +
@@ -41595,52 +41676,52 @@ public final class HbaseMetastoreProto {
       "w_expanded_text\030\013 \001(\t\022\022\n\ntable_type\030\014 \001(" +
       "\t\022Q\n\nprivileges\030\r \001(\0132=.org.apache.hadoo",
       "p.hive.metastore.hbase.PrincipalPrivileg" +
-      "eSet\022\024\n\014is_temporary\030\016 \001(\010\"\334\002\n\005Index\022\031\n\021" +
-      "indexHandlerClass\030\001 \001(\t\022\016\n\006dbName\030\002 \002(\t\022" +
-      "\025\n\rorigTableName\030\003 \002(\t\022\020\n\010location\030\004 \001(\t" +
-      "\022I\n\rsd_parameters\030\005 \001(\01322.org.apache.had" +
-      "oop.hive.metastore.hbase.Parameters\022\022\n\nc" +
-      "reateTime\030\006 \001(\005\022\026\n\016lastAccessTime\030\007 \001(\005\022" +
-      "\026\n\016indexTableName\030\010 \001(\t\022\017\n\007sd_hash\030\t \001(\014" +
-      "\022F\n\nparameters\030\n \001(\01322.org.apache.hadoop" +
-      ".hive.metastore.hbase.Parameters\022\027\n\017defe",
-      "rredRebuild\030\013 \001(\010\"\353\004\n\026PartitionKeyCompar" +
-      "ator\022\r\n\005names\030\001 \002(\t\022\r\n\005types\030\002 \002(\t\022S\n\002op" +
-      "\030\003 \003(\0132G.org.apache.hadoop.hive.metastor" +
-      "e.hbase.PartitionKeyComparator.Operator\022" +
-      "S\n\005range\030\004 \003(\0132D.org.apache.hadoop.hive." +
-      "metastore.hbase.PartitionKeyComparator.R" +
-      "ange\032(\n\004Mark\022\r\n\005value\030\001 \002(\t\022\021\n\tinclusive" +
-      "\030\002 \002(\010\032\272\001\n\005Range\022\013\n\003key\030\001 \002(\t\022R\n\005start\030\002" +
-      " \001(\0132C.org.apache.hadoop.hive.metastore." +
-      "hbase.PartitionKeyComparator.Mark\022P\n\003end",
-      "\030\003 \001(\0132C.org.apache.hadoop.hive.metastor" +
-      "e.hbase.PartitionKeyComparator.Mark\032\241\001\n\010" +
-      "Operator\022Z\n\004type\030\001 \002(\0162L.org.apache.hado" +
-      "op.hive.metastore.hbase.PartitionKeyComp" +
-      "arator.Operator.Type\022\013\n\003key\030\002 \002(\t\022\013\n\003val" +
-      "\030\003 \002(\t\"\037\n\004Type\022\010\n\004LIKE\020\000\022\r\n\tNOTEQUALS\020\001\"" +
-      "\373\001\n\nPrimaryKey\022\017\n\007pk_name\030\001 \002(\t\022Q\n\004cols\030" +
-      "\002 \003(\0132C.org.apache.hadoop.hive.metastore" +
-      ".hbase.PrimaryKey.PrimaryKeyColumn\022\031\n\021en" +
-      "able_constraint\030\003 \001(\010\022\033\n\023validate_constr",
-      "aint\030\004 \001(\010\022\027\n\017rely_constraint\030\005 \001(\010\0328\n\020P" +
-      "rimaryKeyColumn\022\023\n\013column_name\030\001 \002(\t\022\017\n\007" +
-      "key_seq\030\002 \002(\021\"\205\004\n\013ForeignKeys\022K\n\003fks\030\001 \003" +
-      "(\0132>.org.apache.hadoop.hive.metastore.hb" +
-      "ase.ForeignKeys.ForeignKey\032\250\003\n\nForeignKe" +
-      "y\022\017\n\007fk_name\030\001 \002(\t\022\032\n\022referenced_db_name" +
-      "\030\002 \002(\t\022\035\n\025referenced_table_name\030\003 \002(\t\022\032\n" +
-      "\022referenced_pk_name\030\004 \001(\t\022\023\n\013update_rule" +
-      "\030\005 \001(\005\022\023\n\013delete_rule\030\006 \001(\005\022]\n\004cols\030\007 \003(" +
-      "\0132O.org.apache.hadoop.hive.metastore.hba",
-      "se.ForeignKeys.ForeignKey.ForeignKeyColu" +
-      "mn\022\031\n\021enable_constraint\030\010 \001(\010\022\033\n\023validat" +
-      "e_constraint\030\t \001(\010\022\027\n\017rely_constraint\030\n " +
-      "\001(\010\032X\n\020ForeignKeyColumn\022\023\n\013column_name\030\001" +
-      " \002(\t\022\036\n\026referenced_column_name\030\002 \002(\t\022\017\n\007" +
-      "key_seq\030\003 \002(\021*#\n\rPrincipalType\022\010\n\004USER\020\000" +
-      "\022\010\n\004ROLE\020\001"
+      "eSet\022\024\n\014is_temporary\030\016 \001(\010\022\032\n\022is_rewrite" +
+      "_enabled\030\017 \001(\010\"\334\002\n\005Index\022\031\n\021indexHandler" +
+      "Class\030\001 \001(\t\022\016\n\006dbName\030\002 \002(\t\022\025\n\rorigTable" +
+      "Name\030\003 \002(\t\022\020\n\010location\030\004 \001(\t\022I\n\rsd_param" +
+      "eters\030\005 \001(\01322.org.apache.hadoop.hive.met" +
+      "astore.hbase.Parameters\022\022\n\ncreateTime\030\006 " +
+      "\001(\005\022\026\n\016lastAccessTime\030\007 \001(\005\022\026\n\016indexTabl" +
+      "eName\030\010 \001(\t\022\017\n\007sd_hash\030\t \001(\014\022F\n\nparamete" +
+      "rs\030\n \001(\01322.org.apache.hadoop.hive.metast",
+      "ore.hbase.Parameters\022\027\n\017deferredRebuild\030" +
+      "\013 \001(\010\"\353\004\n\026PartitionKeyComparator\022\r\n\005name" +
+      "s\030\001 \002(\t\022\r\n\005types\030\002 \002(\t\022S\n\002op\030\003 \003(\0132G.org" +
+      ".apache.hadoop.hive.metastore.hbase.Part" +
+      "itionKeyComparator.Operator\022S\n\005range\030\004 \003" +
+      "(\0132D.org.apache.hadoop.hive.metastore.hb" +
+      "ase.PartitionKeyComparator.Range\032(\n\004Mark" +
+      "\022\r\n\005value\030\001 \002(\t\022\021\n\tinclusive\030\002 \002(\010\032\272\001\n\005R" +
+      "ange\022\013\n\003key\030\001 \002(\t\022R\n\005start\030\002 \001(\0132C.org.a" +
+      "pache.hadoop.hive.metastore.hbase.Partit",
+      "ionKeyComparator.Mark\022P\n\003end\030\003 \001(\0132C.org" +
+      ".apache.hadoop.hive.metastore.hbase.Part" +
+      "itionKeyComparator.Mark\032\241\001\n\010Operator\022Z\n\004" +
+      "type\030\001 \002(\0162L.org.apache.hadoop.hive.meta" +
+      "store.hbase.PartitionKeyComparator.Opera" +
+      "tor.Type\022\013\n\003key\030\002 \002(\t\022\013\n\003val\030\003 \002(\t\"\037\n\004Ty" +
+      "pe\022\010\n\004LIKE\020\000\022\r\n\tNOTEQUALS\020\001\"\373\001\n\nPrimaryK" +
+      "ey\022\017\n\007pk_name\030\001 \002(\t\022Q\n\004cols\030\002 \003(\0132C.org." +
+      "apache.hadoop.hive.metastore.hbase.Prima" +
+      "ryKey.PrimaryKeyColumn\022\031\n\021enable_constra",
+      "int\030\003 \001(\010\022\033\n\023validate_constraint\030\004 \001(\010\022\027" +
+      "\n\017rely_constraint\030\005 \001(\010\0328\n\020PrimaryKeyCol" +
+      "umn\022\023\n\013column_name\030\001 \002(\t\022\017\n\007key_seq\030\002 \002(" +
+      "\021\"\205\004\n\013ForeignKeys\022K\n\003fks\030\001 \003(\0132>.org.apa" +
+      "che.hadoop.hive.metastore.hbase.ForeignK" +
+      "eys.ForeignKey\032\250\003\n\nForeignKey\022\017\n\007fk_name" +
+      "\030\001 \002(\t\022\032\n\022referenced_db_name\030\002 \002(\t\022\035\n\025re" +
+      "ferenced_table_name\030\003 \002(\t\022\032\n\022referenced_" +
+      "pk_name\030\004 \001(\t\022\023\n\013update_rule\030\005 \001(\005\022\023\n\013de" +
+      "lete_rule\030\006 \001(\005\022]\n\004cols\030\007 \003(\0132O.org.apac",
+      "he.hadoop.hive.metastore.hbase.ForeignKe" +
+      "ys.ForeignKey.ForeignKeyColumn\022\031\n\021enable" +
+      "_constraint\030\010 \001(\010\022\033\n\023validate_constraint" +
+      "\030\t \001(\010\022\027\n\017rely_constraint\030\n \001(\010\032X\n\020Forei" +
+      "gnKeyColumn\022\023\n\013column_name\030\001 \002(\t\022\036\n\026refe" +
+      "renced_column_name\030\002 \002(\t\022\017\n\007key_seq\030\003 \002(" +
+      "\021*#\n\rPrincipalType\022\010\n\004USER\020\000\022\010\n\004ROLE\020\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -41856,7 +41937,7 @@ public final class HbaseMetastoreProto {
           internal_static_org_apache_hadoop_hive_metastore_hbase_Table_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_org_apache_hadoop_hive_metastore_hbase_Table_descriptor,
-              new java.lang.String[] { "Owner", "CreateTime", "LastAccessTime", "Retention", "Location", "SdParameters", "SdHash", "PartitionKeys", "Parameters", "ViewOriginalText", "ViewExpandedText", "TableType", "Privileges", "IsTemporary", });
+              new java.lang.String[] { "Owner", "CreateTime", "LastAccessTime", "Retention", "Location", "SdParameters", "SdHash", "PartitionKeys", "Parameters", "ViewOriginalText", "ViewExpandedText", "TableType", "Privileges", "IsTemporary", "IsRewriteEnabled", });
           internal_static_org_apache_hadoop_hive_metastore_hbase_Index_descriptor =
             getDescriptor().getMessageTypes().get(21);
           internal_static_org_apache_hadoop_hive_metastore_hbase_Index_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 1fae3bc..a74e28b 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -4479,6 +4479,11 @@ void Table::__set_temporary(const bool val) {
 __isset.temporary = true;
 }
 
+void Table::__set_rewriteEnabled(const bool val) {
+  this->rewriteEnabled = val;
+__isset.rewriteEnabled = true;
+}
+
 uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -4639,6 +4644,14 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 15:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->rewriteEnabled);
+          this->__isset.rewriteEnabled = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -4731,6 +4744,11 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
     xfer += oprot->writeBool(this->temporary);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.rewriteEnabled) {
+    xfer += oprot->writeFieldBegin("rewriteEnabled", ::apache::thrift::protocol::T_BOOL, 15);
+    xfer += oprot->writeBool(this->rewriteEnabled);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -4752,6 +4770,7 @@ void swap(Table &a, Table &b) {
   swap(a.tableType, b.tableType);
   swap(a.privileges, b.privileges);
   swap(a.temporary, b.temporary);
+  swap(a.rewriteEnabled, b.rewriteEnabled);
   swap(a.__isset, b.__isset);
 }
 
@@ -4770,6 +4789,7 @@ Table::Table(const Table& other221) {
   tableType = other221.tableType;
   privileges = other221.privileges;
   temporary = other221.temporary;
+  rewriteEnabled = other221.rewriteEnabled;
   __isset = other221.__isset;
 }
 Table& Table::operator=(const Table& other222) {
@@ -4787,6 +4807,7 @@ Table& Table::operator=(const Table& other222) {
   tableType = other222.tableType;
   privileges = other222.privileges;
   temporary = other222.temporary;
+  rewriteEnabled = other222.rewriteEnabled;
   __isset = other222.__isset;
   return *this;
 }
@@ -4807,6 +4828,7 @@ void Table::printTo(std::ostream& out) const {
   out << ", " << "tableType=" << to_string(tableType);
   out << ", " << "privileges="; (__isset.privileges ? (out << to_string(privileges)) : (out << "<null>"));
   out << ", " << "temporary="; (__isset.temporary ? (out << to_string(temporary)) : (out << "<null>"));
+  out << ", " << "rewriteEnabled="; (__isset.rewriteEnabled ? (out << to_string(rewriteEnabled)) : (out << "<null>"));
   out << ")";
 }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index 6838133..0104f6e 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -2062,7 +2062,7 @@ inline std::ostream& operator<<(std::ostream& out, const StorageDescriptor& obj)
 }
 
 typedef struct _Table__isset {
-  _Table__isset() : tableName(false), dbName(false), owner(false), createTime(false), lastAccessTime(false), retention(false), sd(false), partitionKeys(false), parameters(false), viewOriginalText(false), viewExpandedText(false), tableType(false), privileges(false), temporary(true) {}
+  _Table__isset() : tableName(false), dbName(false), owner(false), createTime(false), lastAccessTime(false), retention(false), sd(false), partitionKeys(false), parameters(false), viewOriginalText(false), viewExpandedText(false), tableType(false), privileges(false), temporary(true), rewriteEnabled(false) {}
   bool tableName :1;
   bool dbName :1;
   bool owner :1;
@@ -2077,6 +2077,7 @@ typedef struct _Table__isset {
   bool tableType :1;
   bool privileges :1;
   bool temporary :1;
+  bool rewriteEnabled :1;
 } _Table__isset;
 
 class Table {
@@ -2084,7 +2085,7 @@ class Table {
 
   Table(const Table&);
   Table& operator=(const Table&);
-  Table() : tableName(), dbName(), owner(), createTime(0), lastAccessTime(0), retention(0), viewOriginalText(), viewExpandedText(), tableType(), temporary(false) {
+  Table() : tableName(), dbName(), owner(), createTime(0), lastAccessTime(0), retention(0), viewOriginalText(), viewExpandedText(), tableType(), temporary(false), rewriteEnabled(0) {
   }
 
   virtual ~Table() throw();
@@ -2102,6 +2103,7 @@ class Table {
   std::string tableType;
   PrincipalPrivilegeSet privileges;
   bool temporary;
+  bool rewriteEnabled;
 
   _Table__isset __isset;
 
@@ -2133,6 +2135,8 @@ class Table {
 
   void __set_temporary(const bool val);
 
+  void __set_rewriteEnabled(const bool val);
+
   bool operator == (const Table & rhs) const
   {
     if (!(tableName == rhs.tableName))
@@ -2167,6 +2171,10 @@ class Table {
       return false;
     else if (__isset.temporary && !(temporary == rhs.temporary))
       return false;
+    if (__isset.rewriteEnabled != rhs.__isset.rewriteEnabled)
+      return false;
+    else if (__isset.rewriteEnabled && !(rewriteEnabled == rhs.rewriteEnabled))
+      return false;
     return true;
   }
   bool operator != (const Table &rhs) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
index 5d683fb..800219f 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
@@ -52,6 +52,7 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
   private static final org.apache.thrift.protocol.TField TABLE_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tableType", org.apache.thrift.protocol.TType.STRING, (short)12);
   private static final org.apache.thrift.protocol.TField PRIVILEGES_FIELD_DESC = new org.apache.thrift.protocol.TField("privileges", org.apache.thrift.protocol.TType.STRUCT, (short)13);
   private static final org.apache.thrift.protocol.TField TEMPORARY_FIELD_DESC = new org.apache.thrift.protocol.TField("temporary", org.apache.thrift.protocol.TType.BOOL, (short)14);
+  private static final org.apache.thrift.protocol.TField REWRITE_ENABLED_FIELD_DESC = new org.apache.thrift.protocol.TField("rewriteEnabled", org.apache.thrift.protocol.TType.BOOL, (short)15);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -73,6 +74,7 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
   private String tableType; // required
   private PrincipalPrivilegeSet privileges; // optional
   private boolean temporary; // optional
+  private boolean rewriteEnabled; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -89,7 +91,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     VIEW_EXPANDED_TEXT((short)11, "viewExpandedText"),
     TABLE_TYPE((short)12, "tableType"),
     PRIVILEGES((short)13, "privileges"),
-    TEMPORARY((short)14, "temporary");
+    TEMPORARY((short)14, "temporary"),
+    REWRITE_ENABLED((short)15, "rewriteEnabled");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -132,6 +135,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
           return PRIVILEGES;
         case 14: // TEMPORARY
           return TEMPORARY;
+        case 15: // REWRITE_ENABLED
+          return REWRITE_ENABLED;
         default:
           return null;
       }
@@ -176,8 +181,9 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
   private static final int __LASTACCESSTIME_ISSET_ID = 1;
   private static final int __RETENTION_ISSET_ID = 2;
   private static final int __TEMPORARY_ISSET_ID = 3;
+  private static final int __REWRITEENABLED_ISSET_ID = 4;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.TEMPORARY};
+  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.TEMPORARY,_Fields.REWRITE_ENABLED};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -212,6 +218,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, PrincipalPrivilegeSet.class)));
     tmpMap.put(_Fields.TEMPORARY, new org.apache.thrift.meta_data.FieldMetaData("temporary", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.REWRITE_ENABLED, new org.apache.thrift.meta_data.FieldMetaData("rewriteEnabled", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Table.class, metaDataMap);
   }
@@ -297,6 +305,7 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       this.privileges = new PrincipalPrivilegeSet(other.privileges);
     }
     this.temporary = other.temporary;
+    this.rewriteEnabled = other.rewriteEnabled;
   }
 
   public Table deepCopy() {
@@ -323,6 +332,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     this.privileges = null;
     this.temporary = false;
 
+    setRewriteEnabledIsSet(false);
+    this.rewriteEnabled = false;
   }
 
   public String getTableName() {
@@ -669,6 +680,28 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TEMPORARY_ISSET_ID, value);
   }
 
+  public boolean isRewriteEnabled() {
+    return this.rewriteEnabled;
+  }
+
+  public void setRewriteEnabled(boolean rewriteEnabled) {
+    this.rewriteEnabled = rewriteEnabled;
+    setRewriteEnabledIsSet(true);
+  }
+
+  public void unsetRewriteEnabled() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REWRITEENABLED_ISSET_ID);
+  }
+
+  /** Returns true if field rewriteEnabled is set (has been assigned a value) and false otherwise */
+  public boolean isSetRewriteEnabled() {
+    return EncodingUtils.testBit(__isset_bitfield, __REWRITEENABLED_ISSET_ID);
+  }
+
+  public void setRewriteEnabledIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REWRITEENABLED_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case TABLE_NAME:
@@ -783,6 +816,14 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       }
       break;
 
+    case REWRITE_ENABLED:
+      if (value == null) {
+        unsetRewriteEnabled();
+      } else {
+        setRewriteEnabled((Boolean)value);
+      }
+      break;
+
     }
   }
 
@@ -830,6 +871,9 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     case TEMPORARY:
       return isTemporary();
 
+    case REWRITE_ENABLED:
+      return isRewriteEnabled();
+
     }
     throw new IllegalStateException();
   }
@@ -869,6 +913,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       return isSetPrivileges();
     case TEMPORARY:
       return isSetTemporary();
+    case REWRITE_ENABLED:
+      return isSetRewriteEnabled();
     }
     throw new IllegalStateException();
   }
@@ -1012,6 +1058,15 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         return false;
     }
 
+    boolean this_present_rewriteEnabled = true && this.isSetRewriteEnabled();
+    boolean that_present_rewriteEnabled = true && that.isSetRewriteEnabled();
+    if (this_present_rewriteEnabled || that_present_rewriteEnabled) {
+      if (!(this_present_rewriteEnabled && that_present_rewriteEnabled))
+        return false;
+      if (this.rewriteEnabled != that.rewriteEnabled)
+        return false;
+    }
+
     return true;
   }
 
@@ -1089,6 +1144,11 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     if (present_temporary)
       list.add(temporary);
 
+    boolean present_rewriteEnabled = true && (isSetRewriteEnabled());
+    list.add(present_rewriteEnabled);
+    if (present_rewriteEnabled)
+      list.add(rewriteEnabled);
+
     return list.hashCode();
   }
 
@@ -1240,6 +1300,16 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetRewriteEnabled()).compareTo(other.isSetRewriteEnabled());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetRewriteEnabled()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rewriteEnabled, other.rewriteEnabled);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1359,6 +1429,12 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       sb.append(this.temporary);
       first = false;
     }
+    if (isSetRewriteEnabled()) {
+      if (!first) sb.append(", ");
+      sb.append("rewriteEnabled:");
+      sb.append(this.rewriteEnabled);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1547,6 +1623,14 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 15: // REWRITE_ENABLED
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.rewriteEnabled = iprot.readBool();
+              struct.setRewriteEnabledIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1641,6 +1725,11 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         oprot.writeBool(struct.temporary);
         oprot.writeFieldEnd();
       }
+      if (struct.isSetRewriteEnabled()) {
+        oprot.writeFieldBegin(REWRITE_ENABLED_FIELD_DESC);
+        oprot.writeBool(struct.rewriteEnabled);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1701,7 +1790,10 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       if (struct.isSetTemporary()) {
         optionals.set(13);
       }
-      oprot.writeBitSet(optionals, 14);
+      if (struct.isSetRewriteEnabled()) {
+        optionals.set(14);
+      }
+      oprot.writeBitSet(optionals, 15);
       if (struct.isSetTableName()) {
         oprot.writeString(struct.tableName);
       }
@@ -1757,12 +1849,15 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       if (struct.isSetTemporary()) {
         oprot.writeBool(struct.temporary);
       }
+      if (struct.isSetRewriteEnabled()) {
+        oprot.writeBool(struct.rewriteEnabled);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, Table struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(14);
+      BitSet incoming = iprot.readBitSet(15);
       if (incoming.get(0)) {
         struct.tableName = iprot.readString();
         struct.setTableNameIsSet(true);
@@ -1842,6 +1937,10 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         struct.temporary = iprot.readBool();
         struct.setTemporaryIsSet(true);
       }
+      if (incoming.get(14)) {
+        struct.rewriteEnabled = iprot.readBool();
+        struct.setRewriteEnabledIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/Types.php b/metastore/src/gen/thrift/gen-php/metastore/Types.php
index b9af4ef..595c448 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -4563,6 +4563,10 @@ class Table {
    * @var bool
    */
   public $temporary = false;
+  /**
+   * @var bool
+   */
+  public $rewriteEnabled = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -4638,6 +4642,10 @@ class Table {
           'var' => 'temporary',
           'type' => TType::BOOL,
           ),
+        15 => array(
+          'var' => 'rewriteEnabled',
+          'type' => TType::BOOL,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -4683,6 +4691,9 @@ class Table {
       if (isset($vals['temporary'])) {
         $this->temporary = $vals['temporary'];
       }
+      if (isset($vals['rewriteEnabled'])) {
+        $this->rewriteEnabled = $vals['rewriteEnabled'];
+      }
     }
   }
 
@@ -4829,6 +4840,13 @@ class Table {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 15:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->rewriteEnabled);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -4943,6 +4961,11 @@ class Table {
       $xfer += $output->writeBool($this->temporary);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->rewriteEnabled !== null) {
+      $xfer += $output->writeFieldBegin('rewriteEnabled', TType::BOOL, 15);
+      $xfer += $output->writeBool($this->rewriteEnabled);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 21c0390..7927a46 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -3133,6 +3133,7 @@ class Table:
    - tableType
    - privileges
    - temporary
+   - rewriteEnabled
   """
 
   thrift_spec = (
@@ -3151,9 +3152,10 @@ class Table:
     (12, TType.STRING, 'tableType', None, None, ), # 12
     (13, TType.STRUCT, 'privileges', (PrincipalPrivilegeSet, PrincipalPrivilegeSet.thrift_spec), None, ), # 13
     (14, TType.BOOL, 'temporary', None, False, ), # 14
+    (15, TType.BOOL, 'rewriteEnabled', None, None, ), # 15
   )
 
-  def __init__(self, tableName=None, dbName=None, owner=None, createTime=None, lastAccessTime=None, retention=None, sd=None, partitionKeys=None, parameters=None, viewOriginalText=None, viewExpandedText=None, tableType=None, privileges=None, temporary=thrift_spec[14][4],):
+  def __init__(self, tableName=None, dbName=None, owner=None, createTime=None, lastAccessTime=None, retention=None, sd=None, partitionKeys=None, parameters=None, viewOriginalText=None, viewExpandedText=None, tableType=None, privileges=None, temporary=thrift_spec[14][4], rewriteEnabled=None,):
     self.tableName = tableName
     self.dbName = dbName
     self.owner = owner
@@ -3168,6 +3170,7 @@ class Table:
     self.tableType = tableType
     self.privileges = privileges
     self.temporary = temporary
+    self.rewriteEnabled = rewriteEnabled
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -3262,6 +3265,11 @@ class Table:
           self.temporary = iprot.readBool()
         else:
           iprot.skip(ftype)
+      elif fid == 15:
+        if ftype == TType.BOOL:
+          self.rewriteEnabled = iprot.readBool()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -3335,6 +3343,10 @@ class Table:
       oprot.writeFieldBegin('temporary', TType.BOOL, 14)
       oprot.writeBool(self.temporary)
       oprot.writeFieldEnd()
+    if self.rewriteEnabled is not None:
+      oprot.writeFieldBegin('rewriteEnabled', TType.BOOL, 15)
+      oprot.writeBool(self.rewriteEnabled)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -3358,6 +3370,7 @@ class Table:
     value = (value * 31) ^ hash(self.tableType)
     value = (value * 31) ^ hash(self.privileges)
     value = (value * 31) ^ hash(self.temporary)
+    value = (value * 31) ^ hash(self.rewriteEnabled)
     return value
 
   def __repr__(self):

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index c735932..c82edd6 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -734,6 +734,7 @@ class Table
   TABLETYPE = 12
   PRIVILEGES = 13
   TEMPORARY = 14
+  REWRITEENABLED = 15
 
   FIELDS = {
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'},
@@ -749,7 +750,8 @@ class Table
     VIEWEXPANDEDTEXT => {:type => ::Thrift::Types::STRING, :name => 'viewExpandedText'},
     TABLETYPE => {:type => ::Thrift::Types::STRING, :name => 'tableType'},
     PRIVILEGES => {:type => ::Thrift::Types::STRUCT, :name => 'privileges', :class => ::PrincipalPrivilegeSet, :optional => true},
-    TEMPORARY => {:type => ::Thrift::Types::BOOL, :name => 'temporary', :default => false, :optional => true}
+    TEMPORARY => {:type => ::Thrift::Types::BOOL, :name => 'temporary', :default => false, :optional => true},
+    REWRITEENABLED => {:type => ::Thrift::Types::BOOL, :name => 'rewriteEnabled', :optional => true}
   }
 
   def struct_fields; FIELDS; end