You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ek...@apache.org on 2018/10/04 21:40:03 UTC

[1/2] hive git commit: HIVE-20556: Expose an API to retrieve the TBL_ID from TBLS in the metastore tables (Jaume Marhuenda via Eugene Koifman)

Repository: hive
Updated Branches:
  refs/heads/master 97f0513c4 -> fe3a457d7


http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
index 4937d9d..d6f0d8c 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
@@ -222,6 +222,9 @@ public abstract class TestHiveMetaStore {
         tbl = client.getTable(dbName, tblName);
       }
 
+      Assert.assertTrue(tbl.isSetId());
+      tbl.unsetId();
+
       Partition part = makePartitionObject(dbName, tblName, vals, tbl, "/part1");
       Partition part2 = makePartitionObject(dbName, tblName, vals2, tbl, "/part2");
       Partition part3 = makePartitionObject(dbName, tblName, vals3, tbl, "/part3");
@@ -1272,6 +1275,7 @@ public abstract class TestHiveMetaStore {
 
       Table tbl2 = client.getTable(dbName, tblName);
       assertNotNull(tbl2);
+      Assert.assertTrue(tbl2.isSetId());
       assertEquals(tbl2.getDbName(), dbName);
       assertEquals(tbl2.getTableName(), tblName);
       assertEquals(tbl2.getSd().getCols().size(), typ1.getFields().size());
@@ -1305,6 +1309,7 @@ public abstract class TestHiveMetaStore {
         assertTrue(fieldSchemasFull.contains(fs));
       }
 
+      tbl2.unsetId();
       client.createTable(tbl2);
       if (isThriftClient) {
         tbl2 = client.getTable(tbl2.getDbName(), tbl2.getTableName());
@@ -1664,6 +1669,56 @@ public abstract class TestHiveMetaStore {
   }
 
   @Test
+  public void testCreateAndGetTableWithDriver() throws Exception {
+    String dbName = "createDb";
+    String tblName = "createTbl";
+
+    client.dropTable(dbName, tblName);
+    silentDropDatabase(dbName);
+    new DatabaseBuilder()
+        .setName(dbName)
+        .create(client, conf);
+
+    createTable(dbName, tblName);
+    Table tblRead = client.getTable(dbName, tblName);
+    Assert.assertTrue(tblRead.isSetId());
+    long firstTableId = tblRead.getId();
+
+    createTable(dbName, tblName + "_2");
+    Table tblRead2 = client.getTable(dbName, tblName + "_2");
+    Assert.assertTrue(tblRead2.isSetId());
+    Assert.assertNotEquals(firstTableId, tblRead2.getId());
+  }
+
+  @Test
+  public void testCreateTableSettingId() throws Exception {
+    String dbName = "createDb";
+    String tblName = "createTbl";
+
+    client.dropTable(dbName, tblName);
+    silentDropDatabase(dbName);
+    new DatabaseBuilder()
+        .setName(dbName)
+        .create(client, conf);
+
+    Table table = new TableBuilder()
+        .setDbName(dbName)
+        .setTableName(tblName)
+        .addCol("foo", "string")
+        .addCol("bar", "string")
+        .build(conf);
+    table.setId(1);
+    try {
+      client.createTable(table);
+      Assert.fail("An error should happen when setting the id"
+          + " to create a table");
+    } catch (InvalidObjectException e) {
+      Assert.assertTrue(e.getMessage().contains("Id shouldn't be set"));
+      Assert.assertTrue(e.getMessage().contains(tblName));
+    }
+  }
+
+  @Test
   public void testAlterTable() throws Exception {
     String dbName = "alterdb";
     String invTblName = "alter-tbl";

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java
index df83171..ebbd1c7 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java
@@ -222,6 +222,8 @@ public class TestHiveMetaStorePartitionSpecs {
       clearAndRecreateDB(hmsc);
       createTable(hmsc, true);
       Table table = hmsc.getTable(dbName, tableName);
+      Assert.assertTrue(table.isSetId());
+      table.unsetId();
       populatePartitions(hmsc, table, Arrays.asList("isLocatedInTablePath", "isLocatedOutsideTablePath"));
 
       // Clone the table,

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
index bf302ed..6c8c943 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
@@ -225,6 +225,8 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
     // Try to create a table with all of the parameters set
     Table table = getTableWithAllParametersSet();
     client.createTable(table);
+    table.unsetId();
+
     Table createdTable = client.getTable(table.getDbName(), table.getTableName());
     // The createTime will be set on the server side, so the comparison should skip it
     table.setCreateTime(createdTable.getCreateTime());
@@ -238,6 +240,9 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
     table.setParameters(createdTable.getParameters());
     table.setCreationMetadata(createdTable.getCreationMetadata());
     table.setWriteId(createdTable.getWriteId());
+
+    Assert.assertTrue(createdTable.isSetId());
+    createdTable.unsetId();
     Assert.assertEquals("create/get table data", table, createdTable);
 
     // Check that the directory is created
@@ -457,6 +462,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
   @Test(expected = AlreadyExistsException.class)
   public void testCreateTableAlreadyExists() throws Exception {
     Table table = testTables[0];
+    table.unsetId();
 
     client.createTable(table);
   }
@@ -524,6 +530,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
       // Expected exception
     }
 
+    table.unsetId();
     // Test in mixed case
     client.createTable(table);
     client.dropTable("DeFaUlt", "TeST_tAbLE");
@@ -546,6 +553,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
     Assert.assertFalse("Table path should be removed",
         metaStore.isPathExists(new Path(table.getSd().getLocation())));
 
+    table.unsetId();
     client.createTable(table);
     client.dropTable(table.getDbName(), table.getTableName(), false, false);
 
@@ -705,6 +713,9 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
     newTable.setCreateTime(alteredTable.getCreateTime());
     newTable.setCreationMetadata(alteredTable.getCreationMetadata());
     newTable.setWriteId(alteredTable.getWriteId());
+
+    Assert.assertTrue(alteredTable.isSetId());
+    alteredTable.unsetId();
     Assert.assertEquals("The table data should be the same", newTable, alteredTable);
   }
 


[2/2] hive git commit: HIVE-20556: Expose an API to retrieve the TBL_ID from TBLS in the metastore tables (Jaume Marhuenda via Eugene Koifman)

Posted by ek...@apache.org.
HIVE-20556: Expose an API to retrieve the TBL_ID from TBLS in the metastore tables (Jaume Marhuenda via Eugene Koifman)


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

Branch: refs/heads/master
Commit: fe3a457d7e6b8b9387b8fc0165d2e9efd9c6b364
Parents: 97f0513
Author: Jaume Marhuenda <ja...@gmail.com>
Authored: Thu Oct 4 14:39:04 2018 -0700
Committer: Eugene Koifman <ek...@apache.org>
Committed: Thu Oct 4 14:39:04 2018 -0700

----------------------------------------------------------------------
 data/files/exported_table/_metadata             |   2 +-
 .../TestAuthorizationPreEventListener.java      |   3 +
 .../TestMetastoreAuthorizationProvider.java     |   3 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   1 +
 .../hadoop/hive/ql/metadata/TestHive.java       |   5 +
 .../ql/metadata/TestHiveMetaStoreChecker.java   |   5 +
 .../apache/hadoop/hive/metastore/api/Table.java | 357 ++++++++++++-------
 .../src/gen/thrift/gen-php/metastore/Types.php  | 139 +++++---
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  | 133 +++----
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |  46 +--
 .../src/main/thrift/hive_metastore.thrift       |  42 +--
 .../hadoop/hive/metastore/HiveMetaStore.java    |   5 +
 .../hadoop/hive/metastore/ObjectStore.java      |   2 +
 .../hadoop/hive/metastore/model/MTable.java     |  10 +-
 .../src/main/resources/package.jdo              |   8 +-
 .../hive/metastore/TestHiveMetaStore.java       |  55 +++
 .../TestHiveMetaStorePartitionSpecs.java        |   2 +
 .../TestTablesCreateDropAlterTruncate.java      |  11 +
 18 files changed, 534 insertions(+), 295 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/data/files/exported_table/_metadata
----------------------------------------------------------------------
diff --git a/data/files/exported_table/_metadata b/data/files/exported_table/_metadata
index 81fbf63..8d8f8bb 100644
--- a/data/files/exported_table/_metadata
+++ b/data/files/exported_table/_metadata
@@ -1 +1 @@
-{"partitions":[],"table":"{\"1\":{\"str\":\"j1_41\"},\"2\":{\"str\":\"default\"},\"3\":{\"str\":\"johndee\"},\"4\":{\"i32\":1371900915},\"5\":{\"i32\":0},\"6\":{\"i32\":0},\"7\":{\"rec\":{\"1\":{\"lst\":[\"rec\",2,{\"1\":{\"str\":\"a\"},\"2\":{\"str\":\"string\"}},{\"1\":{\"str\":\"b\"},\"2\":{\"str\":\"int\"}}]},\"2\":{\"str\":\"hdfs://hivebase01:8020/user/hive/warehouse/j1_41\"},\"3\":{\"str\":\"org.apache.hadoop.mapred.TextInputFormat\"},\"4\":{\"str\":\"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat\"},\"5\":{\"tf\":0},\"6\":{\"i32\":-1},\"7\":{\"rec\":{\"2\":{\"str\":\"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe\"},\"3\":{\"map\":[\"str\",\"str\",2,{\"serialization.format\":\",\",\"field.delim\":\",\"}]}}},\"8\":{\"lst\":[\"str\",0]},\"9\":{\"lst\":[\"rec\",0]},\"10\":{\"map\":[\"str\",\"str\",0,{}]}}},\"8\":{\"lst\":[\"rec\",0]},\"9\":{\"map\":[\"str\",\"str\",1,{\"transient_lastDdlTime\":\"1371900931\"}]},\"12\":{\"str\":\"MANAGED_TABLE\"}}","version":"0.1"
 }
\ No newline at end of file
+{"partitions":[],"table":"{\"2\":{\"str\":\"j1_41\"},\"3\":{\"str\":\"default\"},\"4\":{\"str\":\"johndee\"},\"5\":{\"i32\":1371900915},\"6\":{\"i32\":0},\"7\":{\"i32\":0},\"8\":{\"rec\":{\"1\":{\"lst\":[\"rec\",2,{\"1\":{\"str\":\"a\"},\"2\":{\"str\":\"string\"}},{\"1\":{\"str\":\"b\"},\"2\":{\"str\":\"int\"}}]},\"2\":{\"str\":\"hdfs://hivebase01:8020/user/hive/warehouse/j1_41\"},\"3\":{\"str\":\"org.apache.hadoop.mapred.TextInputFormat\"},\"4\":{\"str\":\"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat\"},\"5\":{\"tf\":0},\"6\":{\"i32\":-1},\"7\":{\"rec\":{\"2\":{\"str\":\"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe\"},\"3\":{\"map\":[\"str\",\"str\",2,{\"serialization.format\":\",\",\"field.delim\":\",\"}]}}},\"8\":{\"lst\":[\"str\",0]},\"9\":{\"lst\":[\"rec\",0]},\"10\":{\"map\":[\"str\",\"str\",0,{}]}}},\"9\":{\"lst\":[\"rec\",0]},\"10\":{\"map\":[\"str\",\"str\",1,{\"transient_lastDdlTime\":\"1371900931\"}]},\"13\":{\"str\":\"MANAGED_TABLE\"}}","version":"0.1
 "}

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
index 05c0009..1f6ec27 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.security.DummyHiveMetastoreAuthorizationProvider.AuthCallContext;
 import org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventListener;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.Assert;
 
 /**
  * TestAuthorizationPreEventListener. Test case for
@@ -189,6 +190,8 @@ public class TestAuthorizationPreEventListener extends TestCase {
             DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.TABLE))
             .getTTable();
     Table tbl = msc.getTable(dbName, tblName);
+    Assert.assertTrue(tbl.isSetId());
+    tbl.unsetId();
     validateCreateTable(tbl, tblFromEvent);
 
     driver.run("alter table tmptbl add partition (b='2011')");

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
index 7673213..0e08e81 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -238,6 +239,8 @@ public class TestMetastoreAuthorizationProvider extends TestCase {
 
     assertEquals(0,ret.getResponseCode()); // now it succeeds.
     Table tbl = msc.getTable(dbName, tblName);
+    Assert.assertTrue(tbl.isSetId());
+    tbl.unsetId();
 
     validateCreateTable(tbl,tblName, dbName);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/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 f72e08c..e913462 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
@@ -5103,6 +5103,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       }
 
       tbl.getTTable().setTemporary(crtTbl.isTemporary());
+      tbl.getTTable().unsetId();
 
       if (crtTbl.isExternal()) {
         tbl.setProperty("EXTERNAL", "TRUE");

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
index ca4d36f..e57db93 100755
--- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
@@ -307,6 +307,9 @@ public class TestHive extends TestCase {
       tbl.getTTable().setPrivilegesIsSet(false);
 
       ft = hm.getTable(Warehouse.DEFAULT_DATABASE_NAME, tableName);
+      Assert.assertTrue(ft.getTTable().isSetId());
+      ft.getTTable().unsetId();
+
       assertNotNull("Unable to fetch table", ft);
       ft.checkValidity(hiveConf);
       assertEquals("Table names didn't match for table: " + tableName, tbl
@@ -330,6 +333,8 @@ public class TestHive extends TestCase {
         ft.getTTable().setWriteId(0);
         tbl.getTTable().setWriteId(0);
       }
+
+      tbl.getTTable().unsetId();
       assertTrue("Tables  doesn't match: " + tableName + " (" + ft.getTTable()
           + "; " + tbl.getTTable() + ")", ft.getTTable().equals(tbl.getTTable()));
       assertEquals("SerializationLib is not set correctly", tbl

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java
index ff411f6..a2a0583 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.thrift.TException;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -136,6 +137,8 @@ public class TestHiveMetaStoreChecker {
     table.setOutputFormatClass(HiveIgnoreKeyTextOutputFormat.class);
 
     hive.createTable(table);
+    Assert.assertTrue(table.getTTable().isSetId());
+    table.getTTable().unsetId();
     // now we've got a table, check that it works
     // first check all (1) tables
     result = new CheckResult();
@@ -276,6 +279,8 @@ public class TestHiveMetaStoreChecker {
 
     hive.createTable(table);
     table = hive.getTable(dbName, tableName);
+    Assert.assertTrue(table.getTTable().isSetId());
+    table.getTTable().unsetId();
 
     for (Map<String, String> partSpec : parts) {
       hive.createPartition(table, partSpec);

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
index 78ac909..8c2988c 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
@@ -38,26 +38,27 @@ import org.slf4j.LoggerFactory;
 @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, java.io.Serializable, Cloneable, Comparable<Table> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Table");
 
-  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)2);
-  private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)3);
-  private static final org.apache.thrift.protocol.TField CREATE_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("createTime", org.apache.thrift.protocol.TType.I32, (short)4);
-  private static final org.apache.thrift.protocol.TField LAST_ACCESS_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("lastAccessTime", org.apache.thrift.protocol.TType.I32, (short)5);
-  private static final org.apache.thrift.protocol.TField RETENTION_FIELD_DESC = new org.apache.thrift.protocol.TField("retention", org.apache.thrift.protocol.TType.I32, (short)6);
-  private static final org.apache.thrift.protocol.TField SD_FIELD_DESC = new org.apache.thrift.protocol.TField("sd", org.apache.thrift.protocol.TType.STRUCT, (short)7);
-  private static final org.apache.thrift.protocol.TField PARTITION_KEYS_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionKeys", org.apache.thrift.protocol.TType.LIST, (short)8);
-  private static final org.apache.thrift.protocol.TField PARAMETERS_FIELD_DESC = new org.apache.thrift.protocol.TField("parameters", org.apache.thrift.protocol.TType.MAP, (short)9);
-  private static final org.apache.thrift.protocol.TField VIEW_ORIGINAL_TEXT_FIELD_DESC = new org.apache.thrift.protocol.TField("viewOriginalText", org.apache.thrift.protocol.TType.STRING, (short)10);
-  private static final org.apache.thrift.protocol.TField VIEW_EXPANDED_TEXT_FIELD_DESC = new org.apache.thrift.protocol.TField("viewExpandedText", org.apache.thrift.protocol.TType.STRING, (short)11);
-  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 org.apache.thrift.protocol.TField CREATION_METADATA_FIELD_DESC = new org.apache.thrift.protocol.TField("creationMetadata", org.apache.thrift.protocol.TType.STRUCT, (short)16);
-  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)17);
-  private static final org.apache.thrift.protocol.TField OWNER_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("ownerType", org.apache.thrift.protocol.TType.I32, (short)18);
-  private static final org.apache.thrift.protocol.TField WRITE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("writeId", org.apache.thrift.protocol.TType.I64, (short)19);
-  private static final org.apache.thrift.protocol.TField IS_STATS_COMPLIANT_FIELD_DESC = new org.apache.thrift.protocol.TField("isStatsCompliant", org.apache.thrift.protocol.TType.BOOL, (short)20);
+  private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField CREATE_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("createTime", org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField LAST_ACCESS_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("lastAccessTime", org.apache.thrift.protocol.TType.I32, (short)6);
+  private static final org.apache.thrift.protocol.TField RETENTION_FIELD_DESC = new org.apache.thrift.protocol.TField("retention", org.apache.thrift.protocol.TType.I32, (short)7);
+  private static final org.apache.thrift.protocol.TField SD_FIELD_DESC = new org.apache.thrift.protocol.TField("sd", org.apache.thrift.protocol.TType.STRUCT, (short)8);
+  private static final org.apache.thrift.protocol.TField PARTITION_KEYS_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionKeys", org.apache.thrift.protocol.TType.LIST, (short)9);
+  private static final org.apache.thrift.protocol.TField PARAMETERS_FIELD_DESC = new org.apache.thrift.protocol.TField("parameters", org.apache.thrift.protocol.TType.MAP, (short)10);
+  private static final org.apache.thrift.protocol.TField VIEW_ORIGINAL_TEXT_FIELD_DESC = new org.apache.thrift.protocol.TField("viewOriginalText", org.apache.thrift.protocol.TType.STRING, (short)11);
+  private static final org.apache.thrift.protocol.TField VIEW_EXPANDED_TEXT_FIELD_DESC = new org.apache.thrift.protocol.TField("viewExpandedText", org.apache.thrift.protocol.TType.STRING, (short)12);
+  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)13);
+  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)14);
+  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)15);
+  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)16);
+  private static final org.apache.thrift.protocol.TField CREATION_METADATA_FIELD_DESC = new org.apache.thrift.protocol.TField("creationMetadata", org.apache.thrift.protocol.TType.STRUCT, (short)17);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)18);
+  private static final org.apache.thrift.protocol.TField OWNER_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("ownerType", org.apache.thrift.protocol.TType.I32, (short)19);
+  private static final org.apache.thrift.protocol.TField WRITE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("writeId", org.apache.thrift.protocol.TType.I64, (short)20);
+  private static final org.apache.thrift.protocol.TField IS_STATS_COMPLIANT_FIELD_DESC = new org.apache.thrift.protocol.TField("isStatsCompliant", org.apache.thrift.protocol.TType.BOOL, (short)21);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -65,6 +66,7 @@ import org.slf4j.LoggerFactory;
     schemes.put(TupleScheme.class, new TableTupleSchemeFactory());
   }
 
+  private long id; // optional
   private String tableName; // required
   private String dbName; // required
   private String owner; // required
@@ -88,30 +90,31 @@ import org.slf4j.LoggerFactory;
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    TABLE_NAME((short)1, "tableName"),
-    DB_NAME((short)2, "dbName"),
-    OWNER((short)3, "owner"),
-    CREATE_TIME((short)4, "createTime"),
-    LAST_ACCESS_TIME((short)5, "lastAccessTime"),
-    RETENTION((short)6, "retention"),
-    SD((short)7, "sd"),
-    PARTITION_KEYS((short)8, "partitionKeys"),
-    PARAMETERS((short)9, "parameters"),
-    VIEW_ORIGINAL_TEXT((short)10, "viewOriginalText"),
-    VIEW_EXPANDED_TEXT((short)11, "viewExpandedText"),
-    TABLE_TYPE((short)12, "tableType"),
-    PRIVILEGES((short)13, "privileges"),
-    TEMPORARY((short)14, "temporary"),
-    REWRITE_ENABLED((short)15, "rewriteEnabled"),
-    CREATION_METADATA((short)16, "creationMetadata"),
-    CAT_NAME((short)17, "catName"),
+    ID((short)1, "id"),
+    TABLE_NAME((short)2, "tableName"),
+    DB_NAME((short)3, "dbName"),
+    OWNER((short)4, "owner"),
+    CREATE_TIME((short)5, "createTime"),
+    LAST_ACCESS_TIME((short)6, "lastAccessTime"),
+    RETENTION((short)7, "retention"),
+    SD((short)8, "sd"),
+    PARTITION_KEYS((short)9, "partitionKeys"),
+    PARAMETERS((short)10, "parameters"),
+    VIEW_ORIGINAL_TEXT((short)11, "viewOriginalText"),
+    VIEW_EXPANDED_TEXT((short)12, "viewExpandedText"),
+    TABLE_TYPE((short)13, "tableType"),
+    PRIVILEGES((short)14, "privileges"),
+    TEMPORARY((short)15, "temporary"),
+    REWRITE_ENABLED((short)16, "rewriteEnabled"),
+    CREATION_METADATA((short)17, "creationMetadata"),
+    CAT_NAME((short)18, "catName"),
     /**
      * 
      * @see PrincipalType
      */
-    OWNER_TYPE((short)18, "ownerType"),
-    WRITE_ID((short)19, "writeId"),
-    IS_STATS_COMPLIANT((short)20, "isStatsCompliant");
+    OWNER_TYPE((short)19, "ownerType"),
+    WRITE_ID((short)20, "writeId"),
+    IS_STATS_COMPLIANT((short)21, "isStatsCompliant");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -126,45 +129,47 @@ import org.slf4j.LoggerFactory;
      */
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 1: // TABLE_NAME
+        case 1: // ID
+          return ID;
+        case 2: // TABLE_NAME
           return TABLE_NAME;
-        case 2: // DB_NAME
+        case 3: // DB_NAME
           return DB_NAME;
-        case 3: // OWNER
+        case 4: // OWNER
           return OWNER;
-        case 4: // CREATE_TIME
+        case 5: // CREATE_TIME
           return CREATE_TIME;
-        case 5: // LAST_ACCESS_TIME
+        case 6: // LAST_ACCESS_TIME
           return LAST_ACCESS_TIME;
-        case 6: // RETENTION
+        case 7: // RETENTION
           return RETENTION;
-        case 7: // SD
+        case 8: // SD
           return SD;
-        case 8: // PARTITION_KEYS
+        case 9: // PARTITION_KEYS
           return PARTITION_KEYS;
-        case 9: // PARAMETERS
+        case 10: // PARAMETERS
           return PARAMETERS;
-        case 10: // VIEW_ORIGINAL_TEXT
+        case 11: // VIEW_ORIGINAL_TEXT
           return VIEW_ORIGINAL_TEXT;
-        case 11: // VIEW_EXPANDED_TEXT
+        case 12: // VIEW_EXPANDED_TEXT
           return VIEW_EXPANDED_TEXT;
-        case 12: // TABLE_TYPE
+        case 13: // TABLE_TYPE
           return TABLE_TYPE;
-        case 13: // PRIVILEGES
+        case 14: // PRIVILEGES
           return PRIVILEGES;
-        case 14: // TEMPORARY
+        case 15: // TEMPORARY
           return TEMPORARY;
-        case 15: // REWRITE_ENABLED
+        case 16: // REWRITE_ENABLED
           return REWRITE_ENABLED;
-        case 16: // CREATION_METADATA
+        case 17: // CREATION_METADATA
           return CREATION_METADATA;
-        case 17: // CAT_NAME
+        case 18: // CAT_NAME
           return CAT_NAME;
-        case 18: // OWNER_TYPE
+        case 19: // OWNER_TYPE
           return OWNER_TYPE;
-        case 19: // WRITE_ID
+        case 20: // WRITE_ID
           return WRITE_ID;
-        case 20: // IS_STATS_COMPLIANT
+        case 21: // IS_STATS_COMPLIANT
           return IS_STATS_COMPLIANT;
         default:
           return null;
@@ -206,18 +211,21 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
-  private static final int __CREATETIME_ISSET_ID = 0;
-  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 static final int __WRITEID_ISSET_ID = 5;
-  private static final int __ISSTATSCOMPLIANT_ISSET_ID = 6;
+  private static final int __ID_ISSET_ID = 0;
+  private static final int __CREATETIME_ISSET_ID = 1;
+  private static final int __LASTACCESSTIME_ISSET_ID = 2;
+  private static final int __RETENTION_ISSET_ID = 3;
+  private static final int __TEMPORARY_ISSET_ID = 4;
+  private static final int __REWRITEENABLED_ISSET_ID = 5;
+  private static final int __WRITEID_ISSET_ID = 6;
+  private static final int __ISSTATSCOMPLIANT_ISSET_ID = 7;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.TEMPORARY,_Fields.REWRITE_ENABLED,_Fields.CREATION_METADATA,_Fields.CAT_NAME,_Fields.OWNER_TYPE,_Fields.WRITE_ID,_Fields.IS_STATS_COMPLIANT};
+  private static final _Fields optionals[] = {_Fields.ID,_Fields.PRIVILEGES,_Fields.TEMPORARY,_Fields.REWRITE_ENABLED,_Fields.CREATION_METADATA,_Fields.CAT_NAME,_Fields.OWNER_TYPE,_Fields.WRITE_ID,_Fields.IS_STATS_COMPLIANT};
   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);
+    tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -311,6 +319,7 @@ import org.slf4j.LoggerFactory;
    */
   public Table(Table other) {
     __isset_bitfield = other.__isset_bitfield;
+    this.id = other.id;
     if (other.isSetTableName()) {
       this.tableName = other.tableName;
     }
@@ -370,6 +379,8 @@ import org.slf4j.LoggerFactory;
 
   @Override
   public void clear() {
+    setIdIsSet(false);
+    this.id = 0;
     this.tableName = null;
     this.dbName = null;
     this.owner = null;
@@ -400,6 +411,28 @@ import org.slf4j.LoggerFactory;
     this.isStatsCompliant = false;
   }
 
+  public long getId() {
+    return this.id;
+  }
+
+  public void setId(long id) {
+    this.id = id;
+    setIdIsSet(true);
+  }
+
+  public void unsetId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ID_ISSET_ID);
+  }
+
+  /** Returns true if field id is set (has been assigned a value) and false otherwise */
+  public boolean isSetId() {
+    return EncodingUtils.testBit(__isset_bitfield, __ID_ISSET_ID);
+  }
+
+  public void setIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ID_ISSET_ID, value);
+  }
+
   public String getTableName() {
     return this.tableName;
   }
@@ -889,6 +922,14 @@ import org.slf4j.LoggerFactory;
 
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
+    case ID:
+      if (value == null) {
+        unsetId();
+      } else {
+        setId((Long)value);
+      }
+      break;
+
     case TABLE_NAME:
       if (value == null) {
         unsetTableName();
@@ -1054,6 +1095,9 @@ import org.slf4j.LoggerFactory;
 
   public Object getFieldValue(_Fields field) {
     switch (field) {
+    case ID:
+      return getId();
+
     case TABLE_NAME:
       return getTableName();
 
@@ -1125,6 +1169,8 @@ import org.slf4j.LoggerFactory;
     }
 
     switch (field) {
+    case ID:
+      return isSetId();
     case TABLE_NAME:
       return isSetTableName();
     case DB_NAME:
@@ -1182,6 +1228,15 @@ import org.slf4j.LoggerFactory;
     if (that == null)
       return false;
 
+    boolean this_present_id = true && this.isSetId();
+    boolean that_present_id = true && that.isSetId();
+    if (this_present_id || that_present_id) {
+      if (!(this_present_id && that_present_id))
+        return false;
+      if (this.id != that.id)
+        return false;
+    }
+
     boolean this_present_tableName = true && this.isSetTableName();
     boolean that_present_tableName = true && that.isSetTableName();
     if (this_present_tableName || that_present_tableName) {
@@ -1369,6 +1424,11 @@ import org.slf4j.LoggerFactory;
   public int hashCode() {
     List<Object> list = new ArrayList<Object>();
 
+    boolean present_id = true && (isSetId());
+    list.add(present_id);
+    if (present_id)
+      list.add(id);
+
     boolean present_tableName = true && (isSetTableName());
     list.add(present_tableName);
     if (present_tableName)
@@ -1480,6 +1540,16 @@ import org.slf4j.LoggerFactory;
 
     int lastComparison = 0;
 
+    lastComparison = Boolean.valueOf(isSetId()).compareTo(other.isSetId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName());
     if (lastComparison != 0) {
       return lastComparison;
@@ -1700,6 +1770,12 @@ import org.slf4j.LoggerFactory;
     StringBuilder sb = new StringBuilder("Table(");
     boolean first = true;
 
+    if (isSetId()) {
+      sb.append("id:");
+      sb.append(this.id);
+      first = false;
+    }
+    if (!first) sb.append(", ");
     sb.append("tableName:");
     if (this.tableName == null) {
       sb.append("null");
@@ -1898,7 +1974,15 @@ import org.slf4j.LoggerFactory;
           break;
         }
         switch (schemeField.id) {
-          case 1: // TABLE_NAME
+          case 1: // ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.id = iprot.readI64();
+              struct.setIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TABLE_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.tableName = iprot.readString();
               struct.setTableNameIsSet(true);
@@ -1906,7 +1990,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 2: // DB_NAME
+          case 3: // DB_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.dbName = iprot.readString();
               struct.setDbNameIsSet(true);
@@ -1914,7 +1998,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 3: // OWNER
+          case 4: // OWNER
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.owner = iprot.readString();
               struct.setOwnerIsSet(true);
@@ -1922,7 +2006,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 4: // CREATE_TIME
+          case 5: // CREATE_TIME
             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
               struct.createTime = iprot.readI32();
               struct.setCreateTimeIsSet(true);
@@ -1930,7 +2014,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 5: // LAST_ACCESS_TIME
+          case 6: // LAST_ACCESS_TIME
             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
               struct.lastAccessTime = iprot.readI32();
               struct.setLastAccessTimeIsSet(true);
@@ -1938,7 +2022,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 6: // RETENTION
+          case 7: // RETENTION
             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
               struct.retention = iprot.readI32();
               struct.setRetentionIsSet(true);
@@ -1946,7 +2030,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 7: // SD
+          case 8: // SD
             if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
               struct.sd = new StorageDescriptor();
               struct.sd.read(iprot);
@@ -1955,7 +2039,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 8: // PARTITION_KEYS
+          case 9: // PARTITION_KEYS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
                 org.apache.thrift.protocol.TList _list206 = iprot.readListBegin();
@@ -1974,7 +2058,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 9: // PARAMETERS
+          case 10: // PARAMETERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
                 org.apache.thrift.protocol.TMap _map209 = iprot.readMapBegin();
@@ -1994,7 +2078,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 10: // VIEW_ORIGINAL_TEXT
+          case 11: // VIEW_ORIGINAL_TEXT
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.viewOriginalText = iprot.readString();
               struct.setViewOriginalTextIsSet(true);
@@ -2002,7 +2086,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 11: // VIEW_EXPANDED_TEXT
+          case 12: // VIEW_EXPANDED_TEXT
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.viewExpandedText = iprot.readString();
               struct.setViewExpandedTextIsSet(true);
@@ -2010,7 +2094,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 12: // TABLE_TYPE
+          case 13: // TABLE_TYPE
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.tableType = iprot.readString();
               struct.setTableTypeIsSet(true);
@@ -2018,7 +2102,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 13: // PRIVILEGES
+          case 14: // PRIVILEGES
             if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
               struct.privileges = new PrincipalPrivilegeSet();
               struct.privileges.read(iprot);
@@ -2027,7 +2111,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 14: // TEMPORARY
+          case 15: // TEMPORARY
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.temporary = iprot.readBool();
               struct.setTemporaryIsSet(true);
@@ -2035,7 +2119,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 15: // REWRITE_ENABLED
+          case 16: // REWRITE_ENABLED
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.rewriteEnabled = iprot.readBool();
               struct.setRewriteEnabledIsSet(true);
@@ -2043,7 +2127,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 16: // CREATION_METADATA
+          case 17: // CREATION_METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
               struct.creationMetadata = new CreationMetadata();
               struct.creationMetadata.read(iprot);
@@ -2052,7 +2136,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 17: // CAT_NAME
+          case 18: // CAT_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.catName = iprot.readString();
               struct.setCatNameIsSet(true);
@@ -2060,7 +2144,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 18: // OWNER_TYPE
+          case 19: // OWNER_TYPE
             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
               struct.ownerType = org.apache.hadoop.hive.metastore.api.PrincipalType.findByValue(iprot.readI32());
               struct.setOwnerTypeIsSet(true);
@@ -2068,7 +2152,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 19: // WRITE_ID
+          case 20: // WRITE_ID
             if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
               struct.writeId = iprot.readI64();
               struct.setWriteIdIsSet(true);
@@ -2076,7 +2160,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 20: // IS_STATS_COMPLIANT
+          case 21: // IS_STATS_COMPLIANT
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.isStatsCompliant = iprot.readBool();
               struct.setIsStatsCompliantIsSet(true);
@@ -2097,6 +2181,11 @@ import org.slf4j.LoggerFactory;
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.isSetId()) {
+        oprot.writeFieldBegin(ID_FIELD_DESC);
+        oprot.writeI64(struct.id);
+        oprot.writeFieldEnd();
+      }
       if (struct.tableName != null) {
         oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
         oprot.writeString(struct.tableName);
@@ -2232,67 +2321,73 @@ import org.slf4j.LoggerFactory;
     public void write(org.apache.thrift.protocol.TProtocol prot, Table struct) throws org.apache.thrift.TException {
       TTupleProtocol oprot = (TTupleProtocol) prot;
       BitSet optionals = new BitSet();
-      if (struct.isSetTableName()) {
+      if (struct.isSetId()) {
         optionals.set(0);
       }
-      if (struct.isSetDbName()) {
+      if (struct.isSetTableName()) {
         optionals.set(1);
       }
-      if (struct.isSetOwner()) {
+      if (struct.isSetDbName()) {
         optionals.set(2);
       }
-      if (struct.isSetCreateTime()) {
+      if (struct.isSetOwner()) {
         optionals.set(3);
       }
-      if (struct.isSetLastAccessTime()) {
+      if (struct.isSetCreateTime()) {
         optionals.set(4);
       }
-      if (struct.isSetRetention()) {
+      if (struct.isSetLastAccessTime()) {
         optionals.set(5);
       }
-      if (struct.isSetSd()) {
+      if (struct.isSetRetention()) {
         optionals.set(6);
       }
-      if (struct.isSetPartitionKeys()) {
+      if (struct.isSetSd()) {
         optionals.set(7);
       }
-      if (struct.isSetParameters()) {
+      if (struct.isSetPartitionKeys()) {
         optionals.set(8);
       }
-      if (struct.isSetViewOriginalText()) {
+      if (struct.isSetParameters()) {
         optionals.set(9);
       }
-      if (struct.isSetViewExpandedText()) {
+      if (struct.isSetViewOriginalText()) {
         optionals.set(10);
       }
-      if (struct.isSetTableType()) {
+      if (struct.isSetViewExpandedText()) {
         optionals.set(11);
       }
-      if (struct.isSetPrivileges()) {
+      if (struct.isSetTableType()) {
         optionals.set(12);
       }
-      if (struct.isSetTemporary()) {
+      if (struct.isSetPrivileges()) {
         optionals.set(13);
       }
-      if (struct.isSetRewriteEnabled()) {
+      if (struct.isSetTemporary()) {
         optionals.set(14);
       }
-      if (struct.isSetCreationMetadata()) {
+      if (struct.isSetRewriteEnabled()) {
         optionals.set(15);
       }
-      if (struct.isSetCatName()) {
+      if (struct.isSetCreationMetadata()) {
         optionals.set(16);
       }
-      if (struct.isSetOwnerType()) {
+      if (struct.isSetCatName()) {
         optionals.set(17);
       }
-      if (struct.isSetWriteId()) {
+      if (struct.isSetOwnerType()) {
         optionals.set(18);
       }
-      if (struct.isSetIsStatsCompliant()) {
+      if (struct.isSetWriteId()) {
         optionals.set(19);
       }
-      oprot.writeBitSet(optionals, 20);
+      if (struct.isSetIsStatsCompliant()) {
+        optionals.set(20);
+      }
+      oprot.writeBitSet(optionals, 21);
+      if (struct.isSetId()) {
+        oprot.writeI64(struct.id);
+      }
       if (struct.isSetTableName()) {
         oprot.writeString(struct.tableName);
       }
@@ -2371,37 +2466,41 @@ import org.slf4j.LoggerFactory;
     @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(20);
+      BitSet incoming = iprot.readBitSet(21);
       if (incoming.get(0)) {
+        struct.id = iprot.readI64();
+        struct.setIdIsSet(true);
+      }
+      if (incoming.get(1)) {
         struct.tableName = iprot.readString();
         struct.setTableNameIsSet(true);
       }
-      if (incoming.get(1)) {
+      if (incoming.get(2)) {
         struct.dbName = iprot.readString();
         struct.setDbNameIsSet(true);
       }
-      if (incoming.get(2)) {
+      if (incoming.get(3)) {
         struct.owner = iprot.readString();
         struct.setOwnerIsSet(true);
       }
-      if (incoming.get(3)) {
+      if (incoming.get(4)) {
         struct.createTime = iprot.readI32();
         struct.setCreateTimeIsSet(true);
       }
-      if (incoming.get(4)) {
+      if (incoming.get(5)) {
         struct.lastAccessTime = iprot.readI32();
         struct.setLastAccessTimeIsSet(true);
       }
-      if (incoming.get(5)) {
+      if (incoming.get(6)) {
         struct.retention = iprot.readI32();
         struct.setRetentionIsSet(true);
       }
-      if (incoming.get(6)) {
+      if (incoming.get(7)) {
         struct.sd = new StorageDescriptor();
         struct.sd.read(iprot);
         struct.setSdIsSet(true);
       }
-      if (incoming.get(7)) {
+      if (incoming.get(8)) {
         {
           org.apache.thrift.protocol.TList _list217 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
           struct.partitionKeys = new ArrayList<FieldSchema>(_list217.size);
@@ -2415,7 +2514,7 @@ import org.slf4j.LoggerFactory;
         }
         struct.setPartitionKeysIsSet(true);
       }
-      if (incoming.get(8)) {
+      if (incoming.get(9)) {
         {
           org.apache.thrift.protocol.TMap _map220 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
           struct.parameters = new HashMap<String,String>(2*_map220.size);
@@ -2430,49 +2529,49 @@ import org.slf4j.LoggerFactory;
         }
         struct.setParametersIsSet(true);
       }
-      if (incoming.get(9)) {
+      if (incoming.get(10)) {
         struct.viewOriginalText = iprot.readString();
         struct.setViewOriginalTextIsSet(true);
       }
-      if (incoming.get(10)) {
+      if (incoming.get(11)) {
         struct.viewExpandedText = iprot.readString();
         struct.setViewExpandedTextIsSet(true);
       }
-      if (incoming.get(11)) {
+      if (incoming.get(12)) {
         struct.tableType = iprot.readString();
         struct.setTableTypeIsSet(true);
       }
-      if (incoming.get(12)) {
+      if (incoming.get(13)) {
         struct.privileges = new PrincipalPrivilegeSet();
         struct.privileges.read(iprot);
         struct.setPrivilegesIsSet(true);
       }
-      if (incoming.get(13)) {
+      if (incoming.get(14)) {
         struct.temporary = iprot.readBool();
         struct.setTemporaryIsSet(true);
       }
-      if (incoming.get(14)) {
+      if (incoming.get(15)) {
         struct.rewriteEnabled = iprot.readBool();
         struct.setRewriteEnabledIsSet(true);
       }
-      if (incoming.get(15)) {
+      if (incoming.get(16)) {
         struct.creationMetadata = new CreationMetadata();
         struct.creationMetadata.read(iprot);
         struct.setCreationMetadataIsSet(true);
       }
-      if (incoming.get(16)) {
+      if (incoming.get(17)) {
         struct.catName = iprot.readString();
         struct.setCatNameIsSet(true);
       }
-      if (incoming.get(17)) {
+      if (incoming.get(18)) {
         struct.ownerType = org.apache.hadoop.hive.metastore.api.PrincipalType.findByValue(iprot.readI32());
         struct.setOwnerTypeIsSet(true);
       }
-      if (incoming.get(18)) {
+      if (incoming.get(19)) {
         struct.writeId = iprot.readI64();
         struct.setWriteIdIsSet(true);
       }
-      if (incoming.get(19)) {
+      if (incoming.get(20)) {
         struct.isStatsCompliant = iprot.readBool();
         struct.setIsStatsCompliantIsSet(true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
index 22deffe..936f7c5 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
@@ -6689,6 +6689,10 @@ class Table {
   static $_TSPEC;
 
   /**
+   * @var int
+   */
+  public $id = null;
+  /**
    * @var string
    */
   public $tableName = null;
@@ -6773,35 +6777,39 @@ class Table {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
+          'var' => 'id',
+          'type' => TType::I64,
+          ),
+        2 => array(
           'var' => 'tableName',
           'type' => TType::STRING,
           ),
-        2 => array(
+        3 => array(
           'var' => 'dbName',
           'type' => TType::STRING,
           ),
-        3 => array(
+        4 => array(
           'var' => 'owner',
           'type' => TType::STRING,
           ),
-        4 => array(
+        5 => array(
           'var' => 'createTime',
           'type' => TType::I32,
           ),
-        5 => array(
+        6 => array(
           'var' => 'lastAccessTime',
           'type' => TType::I32,
           ),
-        6 => array(
+        7 => array(
           'var' => 'retention',
           'type' => TType::I32,
           ),
-        7 => array(
+        8 => array(
           'var' => 'sd',
           'type' => TType::STRUCT,
           'class' => '\metastore\StorageDescriptor',
           ),
-        8 => array(
+        9 => array(
           'var' => 'partitionKeys',
           'type' => TType::LST,
           'etype' => TType::STRUCT,
@@ -6810,7 +6818,7 @@ class Table {
             'class' => '\metastore\FieldSchema',
             ),
           ),
-        9 => array(
+        10 => array(
           'var' => 'parameters',
           'type' => TType::MAP,
           'ktype' => TType::STRING,
@@ -6822,55 +6830,58 @@ class Table {
             'type' => TType::STRING,
             ),
           ),
-        10 => array(
+        11 => array(
           'var' => 'viewOriginalText',
           'type' => TType::STRING,
           ),
-        11 => array(
+        12 => array(
           'var' => 'viewExpandedText',
           'type' => TType::STRING,
           ),
-        12 => array(
+        13 => array(
           'var' => 'tableType',
           'type' => TType::STRING,
           ),
-        13 => array(
+        14 => array(
           'var' => 'privileges',
           'type' => TType::STRUCT,
           'class' => '\metastore\PrincipalPrivilegeSet',
           ),
-        14 => array(
+        15 => array(
           'var' => 'temporary',
           'type' => TType::BOOL,
           ),
-        15 => array(
+        16 => array(
           'var' => 'rewriteEnabled',
           'type' => TType::BOOL,
           ),
-        16 => array(
+        17 => array(
           'var' => 'creationMetadata',
           'type' => TType::STRUCT,
           'class' => '\metastore\CreationMetadata',
           ),
-        17 => array(
+        18 => array(
           'var' => 'catName',
           'type' => TType::STRING,
           ),
-        18 => array(
+        19 => array(
           'var' => 'ownerType',
           'type' => TType::I32,
           ),
-        19 => array(
+        20 => array(
           'var' => 'writeId',
           'type' => TType::I64,
           ),
-        20 => array(
+        21 => array(
           'var' => 'isStatsCompliant',
           'type' => TType::BOOL,
           ),
         );
     }
     if (is_array($vals)) {
+      if (isset($vals['id'])) {
+        $this->id = $vals['id'];
+      }
       if (isset($vals['tableName'])) {
         $this->tableName = $vals['tableName'];
       }
@@ -6954,48 +6965,55 @@ class Table {
       switch ($fid)
       {
         case 1:
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->id);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
           if ($ftype == TType::STRING) {
             $xfer += $input->readString($this->tableName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 2:
+        case 3:
           if ($ftype == TType::STRING) {
             $xfer += $input->readString($this->dbName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 3:
+        case 4:
           if ($ftype == TType::STRING) {
             $xfer += $input->readString($this->owner);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 4:
+        case 5:
           if ($ftype == TType::I32) {
             $xfer += $input->readI32($this->createTime);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 5:
+        case 6:
           if ($ftype == TType::I32) {
             $xfer += $input->readI32($this->lastAccessTime);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 6:
+        case 7:
           if ($ftype == TType::I32) {
             $xfer += $input->readI32($this->retention);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 7:
+        case 8:
           if ($ftype == TType::STRUCT) {
             $this->sd = new \metastore\StorageDescriptor();
             $xfer += $this->sd->read($input);
@@ -7003,7 +7021,7 @@ class Table {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 8:
+        case 9:
           if ($ftype == TType::LST) {
             $this->partitionKeys = array();
             $_size182 = 0;
@@ -7021,7 +7039,7 @@ class Table {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 9:
+        case 10:
           if ($ftype == TType::MAP) {
             $this->parameters = array();
             $_size188 = 0;
@@ -7041,28 +7059,28 @@ class Table {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 10:
+        case 11:
           if ($ftype == TType::STRING) {
             $xfer += $input->readString($this->viewOriginalText);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 11:
+        case 12:
           if ($ftype == TType::STRING) {
             $xfer += $input->readString($this->viewExpandedText);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 12:
+        case 13:
           if ($ftype == TType::STRING) {
             $xfer += $input->readString($this->tableType);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 13:
+        case 14:
           if ($ftype == TType::STRUCT) {
             $this->privileges = new \metastore\PrincipalPrivilegeSet();
             $xfer += $this->privileges->read($input);
@@ -7070,21 +7088,21 @@ class Table {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 14:
+        case 15:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->temporary);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 15:
+        case 16:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->rewriteEnabled);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 16:
+        case 17:
           if ($ftype == TType::STRUCT) {
             $this->creationMetadata = new \metastore\CreationMetadata();
             $xfer += $this->creationMetadata->read($input);
@@ -7092,28 +7110,28 @@ class Table {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 17:
+        case 18:
           if ($ftype == TType::STRING) {
             $xfer += $input->readString($this->catName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 18:
+        case 19:
           if ($ftype == TType::I32) {
             $xfer += $input->readI32($this->ownerType);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 19:
+        case 20:
           if ($ftype == TType::I64) {
             $xfer += $input->readI64($this->writeId);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 20:
+        case 21:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->isStatsCompliant);
           } else {
@@ -7133,33 +7151,38 @@ class Table {
   public function write($output) {
     $xfer = 0;
     $xfer += $output->writeStructBegin('Table');
+    if ($this->id !== null) {
+      $xfer += $output->writeFieldBegin('id', TType::I64, 1);
+      $xfer += $output->writeI64($this->id);
+      $xfer += $output->writeFieldEnd();
+    }
     if ($this->tableName !== null) {
-      $xfer += $output->writeFieldBegin('tableName', TType::STRING, 1);
+      $xfer += $output->writeFieldBegin('tableName', TType::STRING, 2);
       $xfer += $output->writeString($this->tableName);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->dbName !== null) {
-      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 2);
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 3);
       $xfer += $output->writeString($this->dbName);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->owner !== null) {
-      $xfer += $output->writeFieldBegin('owner', TType::STRING, 3);
+      $xfer += $output->writeFieldBegin('owner', TType::STRING, 4);
       $xfer += $output->writeString($this->owner);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->createTime !== null) {
-      $xfer += $output->writeFieldBegin('createTime', TType::I32, 4);
+      $xfer += $output->writeFieldBegin('createTime', TType::I32, 5);
       $xfer += $output->writeI32($this->createTime);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->lastAccessTime !== null) {
-      $xfer += $output->writeFieldBegin('lastAccessTime', TType::I32, 5);
+      $xfer += $output->writeFieldBegin('lastAccessTime', TType::I32, 6);
       $xfer += $output->writeI32($this->lastAccessTime);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->retention !== null) {
-      $xfer += $output->writeFieldBegin('retention', TType::I32, 6);
+      $xfer += $output->writeFieldBegin('retention', TType::I32, 7);
       $xfer += $output->writeI32($this->retention);
       $xfer += $output->writeFieldEnd();
     }
@@ -7167,7 +7190,7 @@ class Table {
       if (!is_object($this->sd)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('sd', TType::STRUCT, 7);
+      $xfer += $output->writeFieldBegin('sd', TType::STRUCT, 8);
       $xfer += $this->sd->write($output);
       $xfer += $output->writeFieldEnd();
     }
@@ -7175,7 +7198,7 @@ class Table {
       if (!is_array($this->partitionKeys)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('partitionKeys', TType::LST, 8);
+      $xfer += $output->writeFieldBegin('partitionKeys', TType::LST, 9);
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitionKeys));
         {
@@ -7192,7 +7215,7 @@ class Table {
       if (!is_array($this->parameters)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('parameters', TType::MAP, 9);
+      $xfer += $output->writeFieldBegin('parameters', TType::MAP, 10);
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->parameters));
         {
@@ -7207,17 +7230,17 @@ class Table {
       $xfer += $output->writeFieldEnd();
     }
     if ($this->viewOriginalText !== null) {
-      $xfer += $output->writeFieldBegin('viewOriginalText', TType::STRING, 10);
+      $xfer += $output->writeFieldBegin('viewOriginalText', TType::STRING, 11);
       $xfer += $output->writeString($this->viewOriginalText);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->viewExpandedText !== null) {
-      $xfer += $output->writeFieldBegin('viewExpandedText', TType::STRING, 11);
+      $xfer += $output->writeFieldBegin('viewExpandedText', TType::STRING, 12);
       $xfer += $output->writeString($this->viewExpandedText);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->tableType !== null) {
-      $xfer += $output->writeFieldBegin('tableType', TType::STRING, 12);
+      $xfer += $output->writeFieldBegin('tableType', TType::STRING, 13);
       $xfer += $output->writeString($this->tableType);
       $xfer += $output->writeFieldEnd();
     }
@@ -7225,17 +7248,17 @@ class Table {
       if (!is_object($this->privileges)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('privileges', TType::STRUCT, 13);
+      $xfer += $output->writeFieldBegin('privileges', TType::STRUCT, 14);
       $xfer += $this->privileges->write($output);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->temporary !== null) {
-      $xfer += $output->writeFieldBegin('temporary', TType::BOOL, 14);
+      $xfer += $output->writeFieldBegin('temporary', TType::BOOL, 15);
       $xfer += $output->writeBool($this->temporary);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->rewriteEnabled !== null) {
-      $xfer += $output->writeFieldBegin('rewriteEnabled', TType::BOOL, 15);
+      $xfer += $output->writeFieldBegin('rewriteEnabled', TType::BOOL, 16);
       $xfer += $output->writeBool($this->rewriteEnabled);
       $xfer += $output->writeFieldEnd();
     }
@@ -7243,27 +7266,27 @@ class Table {
       if (!is_object($this->creationMetadata)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('creationMetadata', TType::STRUCT, 16);
+      $xfer += $output->writeFieldBegin('creationMetadata', TType::STRUCT, 17);
       $xfer += $this->creationMetadata->write($output);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->catName !== null) {
-      $xfer += $output->writeFieldBegin('catName', TType::STRING, 17);
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 18);
       $xfer += $output->writeString($this->catName);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->ownerType !== null) {
-      $xfer += $output->writeFieldBegin('ownerType', TType::I32, 18);
+      $xfer += $output->writeFieldBegin('ownerType', TType::I32, 19);
       $xfer += $output->writeI32($this->ownerType);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->writeId !== null) {
-      $xfer += $output->writeFieldBegin('writeId', TType::I64, 19);
+      $xfer += $output->writeFieldBegin('writeId', TType::I64, 20);
       $xfer += $output->writeI64($this->writeId);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->isStatsCompliant !== null) {
-      $xfer += $output->writeFieldBegin('isStatsCompliant', TType::BOOL, 20);
+      $xfer += $output->writeFieldBegin('isStatsCompliant', TType::BOOL, 21);
       $xfer += $output->writeBool($this->isStatsCompliant);
       $xfer += $output->writeFieldEnd();
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 38fac46..958f13c 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -4707,6 +4707,7 @@ class StorageDescriptor:
 class Table:
   """
   Attributes:
+   - id
    - tableName
    - dbName
    - owner
@@ -4731,29 +4732,31 @@ class Table:
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRING, 'tableName', None, None, ), # 1
-    (2, TType.STRING, 'dbName', None, None, ), # 2
-    (3, TType.STRING, 'owner', None, None, ), # 3
-    (4, TType.I32, 'createTime', None, None, ), # 4
-    (5, TType.I32, 'lastAccessTime', None, None, ), # 5
-    (6, TType.I32, 'retention', None, None, ), # 6
-    (7, TType.STRUCT, 'sd', (StorageDescriptor, StorageDescriptor.thrift_spec), None, ), # 7
-    (8, TType.LIST, 'partitionKeys', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 8
-    (9, TType.MAP, 'parameters', (TType.STRING,None,TType.STRING,None), None, ), # 9
-    (10, TType.STRING, 'viewOriginalText', None, None, ), # 10
-    (11, TType.STRING, 'viewExpandedText', None, None, ), # 11
-    (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
-    (16, TType.STRUCT, 'creationMetadata', (CreationMetadata, CreationMetadata.thrift_spec), None, ), # 16
-    (17, TType.STRING, 'catName', None, None, ), # 17
-    (18, TType.I32, 'ownerType', None,     1, ), # 18
-    (19, TType.I64, 'writeId', None, -1, ), # 19
-    (20, TType.BOOL, 'isStatsCompliant', None, None, ), # 20
+    (1, TType.I64, 'id', None, None, ), # 1
+    (2, TType.STRING, 'tableName', None, None, ), # 2
+    (3, TType.STRING, 'dbName', None, None, ), # 3
+    (4, TType.STRING, 'owner', None, None, ), # 4
+    (5, TType.I32, 'createTime', None, None, ), # 5
+    (6, TType.I32, 'lastAccessTime', None, None, ), # 6
+    (7, TType.I32, 'retention', None, None, ), # 7
+    (8, TType.STRUCT, 'sd', (StorageDescriptor, StorageDescriptor.thrift_spec), None, ), # 8
+    (9, TType.LIST, 'partitionKeys', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 9
+    (10, TType.MAP, 'parameters', (TType.STRING,None,TType.STRING,None), None, ), # 10
+    (11, TType.STRING, 'viewOriginalText', None, None, ), # 11
+    (12, TType.STRING, 'viewExpandedText', None, None, ), # 12
+    (13, TType.STRING, 'tableType', None, None, ), # 13
+    (14, TType.STRUCT, 'privileges', (PrincipalPrivilegeSet, PrincipalPrivilegeSet.thrift_spec), None, ), # 14
+    (15, TType.BOOL, 'temporary', None, False, ), # 15
+    (16, TType.BOOL, 'rewriteEnabled', None, None, ), # 16
+    (17, TType.STRUCT, 'creationMetadata', (CreationMetadata, CreationMetadata.thrift_spec), None, ), # 17
+    (18, TType.STRING, 'catName', None, None, ), # 18
+    (19, TType.I32, 'ownerType', None,     1, ), # 19
+    (20, TType.I64, 'writeId', None, -1, ), # 20
+    (21, TType.BOOL, 'isStatsCompliant', None, None, ), # 21
   )
 
-  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, creationMetadata=None, catName=None, ownerType=thrift_spec[18][4], writeId=thrift_spec[19][4], isStatsCompliant=None,):
+  def __init__(self, id=None, 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[15][4], rewriteEnabled=None, creationMetadata=None, catName=None, ownerType=thrift_spec[19][4], writeId=thrift_spec[20][4], isStatsCompliant=None,):
+    self.id = id
     self.tableName = tableName
     self.dbName = dbName
     self.owner = owner
@@ -4785,42 +4788,47 @@ class Table:
       if ftype == TType.STOP:
         break
       if fid == 1:
+        if ftype == TType.I64:
+          self.id = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
         if ftype == TType.STRING:
           self.tableName = iprot.readString()
         else:
           iprot.skip(ftype)
-      elif fid == 2:
+      elif fid == 3:
         if ftype == TType.STRING:
           self.dbName = iprot.readString()
         else:
           iprot.skip(ftype)
-      elif fid == 3:
+      elif fid == 4:
         if ftype == TType.STRING:
           self.owner = iprot.readString()
         else:
           iprot.skip(ftype)
-      elif fid == 4:
+      elif fid == 5:
         if ftype == TType.I32:
           self.createTime = iprot.readI32()
         else:
           iprot.skip(ftype)
-      elif fid == 5:
+      elif fid == 6:
         if ftype == TType.I32:
           self.lastAccessTime = iprot.readI32()
         else:
           iprot.skip(ftype)
-      elif fid == 6:
+      elif fid == 7:
         if ftype == TType.I32:
           self.retention = iprot.readI32()
         else:
           iprot.skip(ftype)
-      elif fid == 7:
+      elif fid == 8:
         if ftype == TType.STRUCT:
           self.sd = StorageDescriptor()
           self.sd.read(iprot)
         else:
           iprot.skip(ftype)
-      elif fid == 8:
+      elif fid == 9:
         if ftype == TType.LIST:
           self.partitionKeys = []
           (_etype185, _size182) = iprot.readListBegin()
@@ -4831,7 +4839,7 @@ class Table:
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
-      elif fid == 9:
+      elif fid == 10:
         if ftype == TType.MAP:
           self.parameters = {}
           (_ktype189, _vtype190, _size188 ) = iprot.readMapBegin()
@@ -4842,59 +4850,59 @@ class Table:
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
-      elif fid == 10:
+      elif fid == 11:
         if ftype == TType.STRING:
           self.viewOriginalText = iprot.readString()
         else:
           iprot.skip(ftype)
-      elif fid == 11:
+      elif fid == 12:
         if ftype == TType.STRING:
           self.viewExpandedText = iprot.readString()
         else:
           iprot.skip(ftype)
-      elif fid == 12:
+      elif fid == 13:
         if ftype == TType.STRING:
           self.tableType = iprot.readString()
         else:
           iprot.skip(ftype)
-      elif fid == 13:
+      elif fid == 14:
         if ftype == TType.STRUCT:
           self.privileges = PrincipalPrivilegeSet()
           self.privileges.read(iprot)
         else:
           iprot.skip(ftype)
-      elif fid == 14:
+      elif fid == 15:
         if ftype == TType.BOOL:
           self.temporary = iprot.readBool()
         else:
           iprot.skip(ftype)
-      elif fid == 15:
+      elif fid == 16:
         if ftype == TType.BOOL:
           self.rewriteEnabled = iprot.readBool()
         else:
           iprot.skip(ftype)
-      elif fid == 16:
+      elif fid == 17:
         if ftype == TType.STRUCT:
           self.creationMetadata = CreationMetadata()
           self.creationMetadata.read(iprot)
         else:
           iprot.skip(ftype)
-      elif fid == 17:
+      elif fid == 18:
         if ftype == TType.STRING:
           self.catName = iprot.readString()
         else:
           iprot.skip(ftype)
-      elif fid == 18:
+      elif fid == 19:
         if ftype == TType.I32:
           self.ownerType = iprot.readI32()
         else:
           iprot.skip(ftype)
-      elif fid == 19:
+      elif fid == 20:
         if ftype == TType.I64:
           self.writeId = iprot.readI64()
         else:
           iprot.skip(ftype)
-      elif fid == 20:
+      elif fid == 21:
         if ftype == TType.BOOL:
           self.isStatsCompliant = iprot.readBool()
         else:
@@ -4909,43 +4917,47 @@ class Table:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
     oprot.writeStructBegin('Table')
+    if self.id is not None:
+      oprot.writeFieldBegin('id', TType.I64, 1)
+      oprot.writeI64(self.id)
+      oprot.writeFieldEnd()
     if self.tableName is not None:
-      oprot.writeFieldBegin('tableName', TType.STRING, 1)
+      oprot.writeFieldBegin('tableName', TType.STRING, 2)
       oprot.writeString(self.tableName)
       oprot.writeFieldEnd()
     if self.dbName is not None:
-      oprot.writeFieldBegin('dbName', TType.STRING, 2)
+      oprot.writeFieldBegin('dbName', TType.STRING, 3)
       oprot.writeString(self.dbName)
       oprot.writeFieldEnd()
     if self.owner is not None:
-      oprot.writeFieldBegin('owner', TType.STRING, 3)
+      oprot.writeFieldBegin('owner', TType.STRING, 4)
       oprot.writeString(self.owner)
       oprot.writeFieldEnd()
     if self.createTime is not None:
-      oprot.writeFieldBegin('createTime', TType.I32, 4)
+      oprot.writeFieldBegin('createTime', TType.I32, 5)
       oprot.writeI32(self.createTime)
       oprot.writeFieldEnd()
     if self.lastAccessTime is not None:
-      oprot.writeFieldBegin('lastAccessTime', TType.I32, 5)
+      oprot.writeFieldBegin('lastAccessTime', TType.I32, 6)
       oprot.writeI32(self.lastAccessTime)
       oprot.writeFieldEnd()
     if self.retention is not None:
-      oprot.writeFieldBegin('retention', TType.I32, 6)
+      oprot.writeFieldBegin('retention', TType.I32, 7)
       oprot.writeI32(self.retention)
       oprot.writeFieldEnd()
     if self.sd is not None:
-      oprot.writeFieldBegin('sd', TType.STRUCT, 7)
+      oprot.writeFieldBegin('sd', TType.STRUCT, 8)
       self.sd.write(oprot)
       oprot.writeFieldEnd()
     if self.partitionKeys is not None:
-      oprot.writeFieldBegin('partitionKeys', TType.LIST, 8)
+      oprot.writeFieldBegin('partitionKeys', TType.LIST, 9)
       oprot.writeListBegin(TType.STRUCT, len(self.partitionKeys))
       for iter195 in self.partitionKeys:
         iter195.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.parameters is not None:
-      oprot.writeFieldBegin('parameters', TType.MAP, 9)
+      oprot.writeFieldBegin('parameters', TType.MAP, 10)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.parameters))
       for kiter196,viter197 in self.parameters.items():
         oprot.writeString(kiter196)
@@ -4953,47 +4965,47 @@ class Table:
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.viewOriginalText is not None:
-      oprot.writeFieldBegin('viewOriginalText', TType.STRING, 10)
+      oprot.writeFieldBegin('viewOriginalText', TType.STRING, 11)
       oprot.writeString(self.viewOriginalText)
       oprot.writeFieldEnd()
     if self.viewExpandedText is not None:
-      oprot.writeFieldBegin('viewExpandedText', TType.STRING, 11)
+      oprot.writeFieldBegin('viewExpandedText', TType.STRING, 12)
       oprot.writeString(self.viewExpandedText)
       oprot.writeFieldEnd()
     if self.tableType is not None:
-      oprot.writeFieldBegin('tableType', TType.STRING, 12)
+      oprot.writeFieldBegin('tableType', TType.STRING, 13)
       oprot.writeString(self.tableType)
       oprot.writeFieldEnd()
     if self.privileges is not None:
-      oprot.writeFieldBegin('privileges', TType.STRUCT, 13)
+      oprot.writeFieldBegin('privileges', TType.STRUCT, 14)
       self.privileges.write(oprot)
       oprot.writeFieldEnd()
     if self.temporary is not None:
-      oprot.writeFieldBegin('temporary', TType.BOOL, 14)
+      oprot.writeFieldBegin('temporary', TType.BOOL, 15)
       oprot.writeBool(self.temporary)
       oprot.writeFieldEnd()
     if self.rewriteEnabled is not None:
-      oprot.writeFieldBegin('rewriteEnabled', TType.BOOL, 15)
+      oprot.writeFieldBegin('rewriteEnabled', TType.BOOL, 16)
       oprot.writeBool(self.rewriteEnabled)
       oprot.writeFieldEnd()
     if self.creationMetadata is not None:
-      oprot.writeFieldBegin('creationMetadata', TType.STRUCT, 16)
+      oprot.writeFieldBegin('creationMetadata', TType.STRUCT, 17)
       self.creationMetadata.write(oprot)
       oprot.writeFieldEnd()
     if self.catName is not None:
-      oprot.writeFieldBegin('catName', TType.STRING, 17)
+      oprot.writeFieldBegin('catName', TType.STRING, 18)
       oprot.writeString(self.catName)
       oprot.writeFieldEnd()
     if self.ownerType is not None:
-      oprot.writeFieldBegin('ownerType', TType.I32, 18)
+      oprot.writeFieldBegin('ownerType', TType.I32, 19)
       oprot.writeI32(self.ownerType)
       oprot.writeFieldEnd()
     if self.writeId is not None:
-      oprot.writeFieldBegin('writeId', TType.I64, 19)
+      oprot.writeFieldBegin('writeId', TType.I64, 20)
       oprot.writeI64(self.writeId)
       oprot.writeFieldEnd()
     if self.isStatsCompliant is not None:
-      oprot.writeFieldBegin('isStatsCompliant', TType.BOOL, 20)
+      oprot.writeFieldBegin('isStatsCompliant', TType.BOOL, 21)
       oprot.writeBool(self.isStatsCompliant)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -5005,6 +5017,7 @@ class Table:
 
   def __hash__(self):
     value = 17
+    value = (value * 31) ^ hash(self.id)
     value = (value * 31) ^ hash(self.tableName)
     value = (value * 31) ^ hash(self.dbName)
     value = (value * 31) ^ hash(self.owner)

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 0192c6d..a3dddf5 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -1085,28 +1085,30 @@ end
 
 class Table
   include ::Thrift::Struct, ::Thrift::Struct_Union
-  TABLENAME = 1
-  DBNAME = 2
-  OWNER = 3
-  CREATETIME = 4
-  LASTACCESSTIME = 5
-  RETENTION = 6
-  SD = 7
-  PARTITIONKEYS = 8
-  PARAMETERS = 9
-  VIEWORIGINALTEXT = 10
-  VIEWEXPANDEDTEXT = 11
-  TABLETYPE = 12
-  PRIVILEGES = 13
-  TEMPORARY = 14
-  REWRITEENABLED = 15
-  CREATIONMETADATA = 16
-  CATNAME = 17
-  OWNERTYPE = 18
-  WRITEID = 19
-  ISSTATSCOMPLIANT = 20
-
-  FIELDS = {
+  ID = 1
+  TABLENAME = 2
+  DBNAME = 3
+  OWNER = 4
+  CREATETIME = 5
+  LASTACCESSTIME = 6
+  RETENTION = 7
+  SD = 8
+  PARTITIONKEYS = 9
+  PARAMETERS = 10
+  VIEWORIGINALTEXT = 11
+  VIEWEXPANDEDTEXT = 12
+  TABLETYPE = 13
+  PRIVILEGES = 14
+  TEMPORARY = 15
+  REWRITEENABLED = 16
+  CREATIONMETADATA = 17
+  CATNAME = 18
+  OWNERTYPE = 19
+  WRITEID = 20
+  ISSTATSCOMPLIANT = 21
+
+  FIELDS = {
+    ID => {:type => ::Thrift::Types::I64, :name => 'id', :optional => true},
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'},
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     OWNER => {:type => ::Thrift::Types::STRING, :name => 'owner'},

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
index 85a5c60..ad83162 100644
--- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
@@ -424,26 +424,28 @@ struct StorageDescriptor {
 
 // table information
 struct Table {
-  1: string tableName,                // name of the table
-  2: string dbName,                   // database name ('default')
-  3: string owner,                    // owner of this table
-  4: i32    createTime,               // creation time of the table
-  5: i32    lastAccessTime,           // last access time (usually this will be filled from HDFS and shouldn't be relied on)
-  6: i32    retention,                // retention time
-  7: StorageDescriptor sd,            // storage descriptor of the table
-  8: list<FieldSchema> partitionKeys, // partition keys of the table. only primitive types are supported
-  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
-  13: optional PrincipalPrivilegeSet privileges,
-  14: optional bool temporary=false,
-  15: optional bool rewriteEnabled,     // rewrite enabled or not
-  16: optional CreationMetadata creationMetadata,   // only for MVs, it stores table names used and txn list at MV creation
-  17: optional string catName,          // Name of the catalog the table is in
-  18: optional PrincipalType ownerType = PrincipalType.USER, // owner type of this table (default to USER for backward compatibility)
-  19: optional i64 writeId=-1,
-  20: optional bool isStatsCompliant
+  1: optional i64 id,                 // id of the table. It will be ignored if set. It's only for
+                                      // read purposed
+  2: string tableName,                // name of the table
+  3: string dbName,                   // database name ('default')
+  4: string owner,                    // owner of this table
+  5: i32    createTime,               // creation time of the table
+  6: i32    lastAccessTime,           // last access time (usually this will be filled from HDFS and shouldn't be relied on)
+  7: i32    retention,                // retention time
+  8: StorageDescriptor sd,            // storage descriptor of the table
+  9: list<FieldSchema> partitionKeys, // partition keys of the table. only primitive types are supported
+  10: map<string, string> parameters,   // to store comments or any other user level parameters
+  11: string viewOriginalText,         // original view text, null for non-view
+  12: string viewExpandedText,         // expanded view text, null for non-view
+  13: string tableType,                // table type enum, e.g. EXTERNAL_TABLE
+  14: optional PrincipalPrivilegeSet privileges,
+  15: optional bool temporary=false,
+  16: optional bool rewriteEnabled,     // rewrite enabled or not
+  17: optional CreationMetadata creationMetadata,   // only for MVs, it stores table names used and txn list at MV creation
+  18: optional string catName,          // Name of the catalog the table is in
+  19: optional PrincipalType ownerType = PrincipalType.USER, // owner type of this table (default to USER for backward compatibility)
+  20: optional i64 writeId=-1,
+  21: optional bool isStatsCompliant
 }
 
 struct Partition {

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index ba82a93..b5200bd 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -1805,6 +1805,11 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           throw new InvalidObjectException("Invalid partition column " + validate);
         }
       }
+      if (tbl.isSetId()) {
+        throw new InvalidObjectException("Id shouldn't be set but table "
+            + tbl.getDbName() + "." + tbl.getTableName() + "has the Id set to "
+            + tbl.getId() + ". It's a read-only option");
+      }
       SkewedInfo skew = tbl.getSd().getSkewedInfo();
       if (skew != null) {
         validate = MetaStoreServerUtils.validateSkewedColNames(skew.getSkewedColNames());

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 6494506..466902d 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -1209,6 +1209,7 @@ public class ObjectStore implements RawStore, Configurable {
         MCreationMetadata mcm = convertToMCreationMetadata(tbl.getCreationMetadata());
         pm.makePersistent(mcm);
       }
+      tbl.setId(mtbl.getId());
 
       PrincipalPrivilegeSet principalPrivs = tbl.getPrivileges();
       List<Object> toPersistPrivObjs = new ArrayList<>();
@@ -1932,6 +1933,7 @@ public class ObjectStore implements RawStore, Configurable {
       t.setOwnerType(PrincipalType.valueOf(mtbl.getOwnerType()));
     }
 
+    t.setId(mtbl.getId());
     t.setRewriteEnabled(mtbl.isRewriteEnabled());
     t.setCatName(mtbl.getDatabase().getCatalogName());
     t.setWriteId(mtbl.getWriteId());

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MTable.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MTable.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MTable.java
index deeb971..92182ae 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MTable.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MTable.java
@@ -23,7 +23,8 @@ import java.util.List;
 import java.util.Map;
 
 public class MTable {
-  
+
+  private long id;
   private String tableName;
   private MDatabase database;
   private MStorageDescriptor sd;
@@ -280,4 +281,11 @@ public class MTable {
   public void setWriteId(long writeId) {
     this.writeId = writeId;
   }
+
+  /**
+   * @return the id.
+   */
+  public long getId() {
+    return id;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/fe3a457d/standalone-metastore/metastore-server/src/main/resources/package.jdo
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/resources/package.jdo b/standalone-metastore/metastore-server/src/main/resources/package.jdo
index 2a5f016..fef6a42 100644
--- a/standalone-metastore/metastore-server/src/main/resources/package.jdo
+++ b/standalone-metastore/metastore-server/src/main/resources/package.jdo
@@ -131,10 +131,10 @@
       </field>
     </class>
 
-    <class name="MTable" table="TBLS" identity-type="datastore" detachable="true">
-      <datastore-identity>
-        <column name="TBL_ID"/>
-      </datastore-identity>
+    <class name="MTable" table="TBLS" identity-type="application" detachable="true">
+      <field name="id" primary-key="true" value-strategy="native">
+        <column name="TBL_ID" jdbc-type="BIGINT" />
+      </field>
       <index name="UniqueTable" unique="true">
         <column name="TBL_NAME"/>
         <column name="DB_ID"/>