You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ai...@apache.org on 2015/11/10 21:38:36 UTC

[1/7] hive git commit: HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)

Repository: hive
Updated Branches:
  refs/heads/master 16521c400 -> b678ed85d


http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index a100e9f..9a1d159 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
@@ -220,6 +221,12 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
+  public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
+      throws MetaException {
+    return objectStore.getTableMeta(dbNames, tableNames, tableTypes);
+  }
+
+  @Override
   public List<Table> getTableObjectsByName(String dbName, List<String> tableNames)
       throws MetaException, UnknownDBException {
     return objectStore.getTableObjectsByName(dbName, tableNames);

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index f6100e6..8dde0af 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
@@ -224,6 +225,12 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
+  public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
+      throws MetaException {
+    return Collections.emptyList();
+  }
+
+  @Override
   public List<Table> getTableObjectsByName(String dbname, List<String> tableNames)
       throws MetaException, UnknownDBException {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
index 7af9d85..581a919 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -161,11 +162,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
     Matcher matcher = pattern.matcher("");
     Set<String> combinedTableNames = new HashSet<String>();
     for (String tableName : tables.keySet()) {
-      if (matcher == null) {
-        matcher = pattern.matcher(tableName);
-      } else {
-        matcher.reset(tableName);
-      }
+      matcher.reset(tableName);
       if (matcher.matches()) {
         combinedTableNames.add(tableName);
       }
@@ -177,6 +174,55 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
     Collections.sort(tableNames);
     return tableNames;
   }
+  
+  @Override
+  public List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)
+      throws MetaException {
+    List<TableMeta> tableMetas = super.getTableMeta(dbPatterns, tablePatterns, tableTypes);
+    Map<String, Map<String, Table>> tmpTables = getTempTables();
+    if (tmpTables.isEmpty()) {
+      return tableMetas;
+    }
+
+    List<Matcher> dbPatternList = new ArrayList<>();
+    for (String element : dbPatterns.split("\\|")) {
+      dbPatternList.add(Pattern.compile(element.replaceAll("\\*", ".*")).matcher(""));
+    }
+    List<Matcher> tblPatternList = new ArrayList<>();
+    for (String element : tablePatterns.split("\\|")) {
+      tblPatternList.add(Pattern.compile(element.replaceAll("\\*", ".*")).matcher(""));
+    }
+    StringBuilder builder = new StringBuilder();
+    for (Map.Entry<String, Map<String, Table>> outer : tmpTables.entrySet()) {
+      if (!matchesAny(outer.getKey(), dbPatternList)) {
+        continue;
+      }
+      for (Map.Entry<String, Table> inner : outer.getValue().entrySet()) {
+        Table table = inner.getValue();
+        String tableName = table.getTableName();
+        String typeString = table.getTableType().name();
+        if (tableTypes != null && !tableTypes.contains(typeString)) {
+          continue;
+        }
+        if (!matchesAny(inner.getKey(), tblPatternList)) {
+          continue;
+        }
+        TableMeta tableMeta = new TableMeta(table.getDbName(), tableName, typeString);
+        tableMeta.setComments(table.getProperty("comment"));
+        tableMetas.add(tableMeta);
+      }
+    }
+    return tableMetas;
+  }
+  
+  private boolean matchesAny(String string, List<Matcher> matchers) {
+    for (Matcher matcher : matchers) {
+      if (matcher.reset(string).matches()) {
+        return true;
+      }
+    }
+    return matchers.isEmpty();
+  }
 
   @Override
   public List<org.apache.hadoop.hive.metastore.api.Table> getTableObjectsByName(String dbName,
@@ -508,12 +554,16 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
   }
 
   public static Map<String, Table> getTempTablesForDatabase(String dbName) {
+    return getTempTables().get(dbName);
+  }
+  
+  public static Map<String, Map<String, Table>> getTempTables() {
     SessionState ss = SessionState.get();
     if (ss == null) {
       LOG.debug("No current SessionState, skipping temp tables");
-      return null;
+      return Collections.emptyMap();
     }
-    return ss.getTempTables().get(dbName);
+    return ss.getTempTables();
   }
 
   private Map<String, ColumnStatisticsObj> getTempTableColumnStatsForTable(String dbName,

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
index 296280f..65bbc1c 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
@@ -22,14 +22,11 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.ql.metadata.TableIterable;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils;
-import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationState;
@@ -48,7 +45,7 @@ public class GetTablesOperation extends MetadataOperation {
   private final String catalogName;
   private final String schemaName;
   private final String tableName;
-  private final List<String> tableTypes = new ArrayList<String>();
+  private final List<String> tableTypeList;
   private final RowSet rowSet;
   private final TableTypeMapping tableTypeMapping;
 
@@ -58,7 +55,14 @@ public class GetTablesOperation extends MetadataOperation {
   .addStringColumn("TABLE_SCHEM", "Schema name.")
   .addStringColumn("TABLE_NAME", "Table name.")
   .addStringColumn("TABLE_TYPE", "The table type, e.g. \"TABLE\", \"VIEW\", etc.")
-  .addStringColumn("REMARKS", "Comments about the table.");
+  .addStringColumn("REMARKS", "Comments about the table.")
+  .addStringColumn("TYPE_CAT", "The types catalog.")
+  .addStringColumn("TYPE_SCHEM", "The types schema.")
+  .addStringColumn("TYPE_NAME", "Type name.")
+  .addStringColumn("SELF_REFERENCING_COL_NAME", 
+      "Name of the designated \"identifier\" column of a typed table.")
+  .addStringColumn("REF_GENERATION", 
+      "Specifies how values in SELF_REFERENCING_COL_NAME are created.");
 
   protected GetTablesOperation(HiveSession parentSession,
       String catalogName, String schemaName, String tableName,
@@ -72,7 +76,12 @@ public class GetTablesOperation extends MetadataOperation {
     tableTypeMapping =
         TableTypeMappingFactory.getTableTypeMapping(tableMappingStr);
     if (tableTypes != null) {
-      this.tableTypes.addAll(tableTypes);
+      tableTypeList = new ArrayList<String>();
+      for (String tableType : tableTypes) {
+        tableTypeList.add(tableTypeMapping.mapToHiveType(tableType.trim()));
+      }
+    } else {
+      tableTypeList = null;
     }
     this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion());
   }
@@ -91,23 +100,17 @@ public class GetTablesOperation extends MetadataOperation {
       }
 
       String tablePattern = convertIdentifierPattern(tableName, true);
-      int maxBatchSize = SessionState.get().getConf().getIntVar(ConfVars.METASTORE_BATCH_RETRIEVE_MAX);
 
-      for (String dbName : metastoreClient.getDatabases(schemaPattern)) {
-        List<String> tableNames = metastoreClient.getTables(dbName, tablePattern);
-        for (Table table : new TableIterable(metastoreClient, dbName, tableNames, maxBatchSize)) {
-          Object[] rowData = new Object[] {
+      for (TableMeta tableMeta : 
+          metastoreClient.getTableMeta(schemaPattern, tablePattern, tableTypeList)) {
+        rowSet.addRow(new Object[] {
               DEFAULT_HIVE_CATALOG,
-              table.getDbName(),
-              table.getTableName(),
-              tableTypeMapping.mapToClientType(table.getTableType()),
-              table.getParameters().get("comment")
-              };
-          if (tableTypes.isEmpty() || tableTypes.contains(
-                tableTypeMapping.mapToClientType(table.getTableType()))) {
-            rowSet.addRow(rowData);
-          }
-        }
+              tableMeta.getDbName(),
+              tableMeta.getTableName(),
+              tableTypeMapping.mapToClientType(tableMeta.getTableType()),
+              tableMeta.getComments(),
+              null, null, null, null, null
+              });
       }
       setState(OperationState.FINISHED);
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
index 4595ef5..285b4f9 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
@@ -18,7 +18,6 @@
 
 package org.apache.hive.service.cli.operation;
 
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -95,16 +94,30 @@ public abstract class MetadataOperation extends Operation {
    * other hand is done locally inside the hive code and that requires the regex wildchar
    * format '.*'  This is driven by the datanucleusFormat flag.
    */
-  private String convertPattern(final String pattern, boolean datanucleusFormat) {
+  private String convertPattern(String pattern, boolean datanucleusFormat) {
     String wStr;
     if (datanucleusFormat) {
       wStr = "*";
     } else {
       wStr = ".*";
     }
-    return pattern
-        .replaceAll("([^\\\\])%", "$1" + wStr).replaceAll("\\\\%", "%").replaceAll("^%", wStr)
-        .replaceAll("([^\\\\])_", "$1.").replaceAll("\\\\_", "_").replaceAll("^_", ".");
+    pattern = replaceAll(pattern, "([^\\\\])%", "$1" + wStr);
+    pattern = replaceAll(pattern, "\\\\%", "%");
+    pattern = replaceAll(pattern, "^%", wStr);
+    pattern = replaceAll(pattern, "([^\\\\])_", "$1.");
+    pattern = replaceAll(pattern, "\\\\_", "_");
+    pattern = replaceAll(pattern, "^_", ".");
+    return pattern;
+  }
+  
+  private String replaceAll(String input, final String pattern, final String replace) {
+    while (true) {
+      String replaced = input.replaceAll(pattern, replace);
+      if (replaced.equals(input)) {
+        return replaced;
+      }
+      input = replaced;
+    }
   }
 
   protected boolean isAuthV2Enabled(){


[7/7] hive git commit: HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)

Posted by ai...@apache.org.
 HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)


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

Branch: refs/heads/master
Commit: b678ed85da7b25c0883426b4ee53a014857d0e1b
Parents: 16521c4
Author: aihuaxu <ai...@apache.org>
Authored: Tue Nov 10 15:37:08 2015 -0500
Committer: aihuaxu <ai...@apache.org>
Committed: Tue Nov 10 15:37:08 2015 -0500

----------------------------------------------------------------------
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |  208 +-
 .../apache/hive/jdbc/HiveDatabaseMetaData.java  |    4 +-
 metastore/if/hive_metastore.thrift              |    9 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2323 +++++++-----
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  148 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  349 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   65 +
 .../hadoop/hive/metastore/api/TableMeta.java    |  701 ++++
 .../hive/metastore/api/ThriftHiveMetastore.java | 3406 ++++++++++++------
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1275 ++++---
 .../src/gen/thrift/gen-php/metastore/Types.php  |  144 +
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       |  883 +++--
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  110 +
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   25 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   65 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   21 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   32 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |    7 +
 .../hadoop/hive/metastore/ObjectStore.java      |  112 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |    4 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |   44 +-
 .../DummyRawStoreControlledCommit.java          |    7 +
 .../DummyRawStoreForJdoConnection.java          |    7 +
 .../ql/metadata/SessionHiveMetaStoreClient.java |   64 +-
 .../cli/operation/GetTablesOperation.java       |   47 +-
 .../cli/operation/MetadataOperation.java        |   23 +-
 28 files changed, 7128 insertions(+), 2967 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
index 2b3fdf1..5450eaa 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
@@ -42,6 +42,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.InputStream;
+import java.lang.Exception;
+import java.lang.Object;
+import java.lang.String;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
@@ -56,8 +59,11 @@ import java.sql.Timestamp;
 import java.sql.Types;
 import java.text.ParseException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Date;
+import java.util.HashSet;
 import java.util.HashMap;
+import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -134,6 +140,7 @@ public class TestJdbcDriver2 {
         stmt1.execute("DROP DATABASE " + db + " CASCADE");
       }
     }
+    stmt1.execute("create database testdb");
     stmt1.close();
     con1.close();
   }
@@ -147,12 +154,20 @@ public class TestJdbcDriver2 {
 
     stmt.execute("set hive.support.concurrency = false");
 
-    // drop table. ignore error.
-    try {
-      stmt.execute("drop table " + tableName);
-    } catch (Exception ex) {
-      fail(ex.toString());
-    }
+    createTestTables(stmt, "", true);
+    createTestTables(stmt, "testdb.", false);
+  }
+
+  private void createTestTables(Statement stmt, String prefix, boolean loadData)
+      throws SQLException {
+
+    // drop test tables/views
+    dropTestTables(stmt, prefix);
+
+    String tableName = prefix + this.tableName;
+    String partitionedTableName = prefix + this.partitionedTableName;
+    String dataTypeTableName = prefix + this.dataTypeTableName;
+    String viewName = prefix + this.viewName;
 
     // create table
     stmt.execute("create table " + tableName
@@ -160,35 +175,25 @@ public class TestJdbcDriver2 {
         + tableComment + "'");
 
     // load data
-    stmt.execute("load data local inpath '"
-        + dataFilePath.toString() + "' into table " + tableName);
-
-    // also initialize a paritioned table to test against.
-
-    // drop table. ignore error.
-    try {
-      stmt.execute("drop table " + partitionedTableName);
-    } catch (Exception ex) {
-      fail(ex.toString());
+    if (loadData) {
+      stmt.execute("load data local inpath '"
+          + dataFilePath.toString() + "' into table " + tableName);
     }
 
+    // also initialize a paritioned table to test against.
     stmt.execute("create table " + partitionedTableName
         + " (under_col int, value string) comment '"+partitionedTableComment
         +"' partitioned by (" + partitionedColumnName + " STRING)");
 
     // load data
-    stmt.execute("load data local inpath '"
-        + dataFilePath.toString() + "' into table " + partitionedTableName
-        + " PARTITION (" + partitionedColumnName + "="
-        + partitionedColumnValue + ")");
-
-    // drop table. ignore error.
-    try {
-      stmt.execute("drop table " + dataTypeTableName);
-    } catch (Exception ex) {
-      fail(ex.toString());
+    if (loadData) {
+      stmt.execute("load data local inpath '"
+          + dataFilePath.toString() + "' into table " + partitionedTableName
+          + " PARTITION (" + partitionedColumnName + "="
+          + partitionedColumnValue + ")");
     }
 
+    // tables with various types
     stmt.execute("create table " + dataTypeTableName
         + " (c1 int, c2 boolean, c3 double, c4 string,"
         + " c5 array<int>, c6 map<int,string>, c7 map<string,string>,"
@@ -208,15 +213,10 @@ public class TestJdbcDriver2 {
         + ") comment'" + dataTypeTableComment
         +"' partitioned by (dt STRING)");
 
-    stmt.execute("load data local inpath '"
-        + dataTypeDataFilePath.toString() + "' into table " + dataTypeTableName
-        + " PARTITION (dt='20090619')");
-
-    // drop view. ignore error.
-    try {
-      stmt.execute("drop view " + viewName);
-    } catch (Exception ex) {
-      fail(ex.toString());
+    if (loadData) {
+      stmt.execute("load data local inpath '"
+          + dataTypeDataFilePath.toString() + "' into table " + dataTypeTableName
+          + " PARTITION (dt='20090619')");
     }
 
     // create view
@@ -224,6 +224,28 @@ public class TestJdbcDriver2 {
         +"' as select * from "+ tableName);
   }
 
+  // drop test tables/views. ignore error.
+  private void dropTestTables(Statement stmt, String prefix) throws SQLException {
+    String tableName = prefix + this.tableName;
+    String partitionedTableName = prefix + this.partitionedTableName;
+    String dataTypeTableName = prefix + this.dataTypeTableName;
+    String viewName = prefix + this.viewName;
+
+    executeWithIgnore(stmt, "drop table " + tableName);
+    executeWithIgnore(stmt, "drop table " + partitionedTableName);
+    executeWithIgnore(stmt, "drop table " + dataTypeTableName);
+    executeWithIgnore(stmt, "drop view " + viewName);
+  }
+
+  private void executeWithIgnore(Statement stmt, String sql) throws SQLException {
+  // drop table. ignore error.
+    try {
+      stmt.execute(sql);
+    } catch (Exception ex) {
+      fail(ex.toString());
+    }
+  }
+
   private static Connection getConnection(String postfix) throws SQLException {
     Connection con1;
     if (standAloneServer) {
@@ -244,9 +266,8 @@ public class TestJdbcDriver2 {
     // drop table
     Statement stmt = con.createStatement();
     assertNotNull("Statement is null", stmt);
-    stmt.execute("drop table " + tableName);
-    stmt.execute("drop table " + partitionedTableName);
-    stmt.execute("drop table " + dataTypeTableName);
+    dropTestTables(stmt, "");
+    dropTestTables(stmt, "testdb.");
 
     con.close();
     assertTrue("Connection should be closed", con.isClosed());
@@ -1123,25 +1144,77 @@ public class TestJdbcDriver2 {
    * @throws SQLException
    */
   private void getTablesTest(String tableTypeName, String viewTypeName) throws SQLException {
-    Map<String, Object[]> tests = new HashMap<String, Object[]>();
-    tests.put("test%jdbc%", new Object[]{"testhivejdbcdriver_table"
-        , "testhivejdbcdriverpartitionedtable"
-        , "testhivejdbcdriverview"});
-    tests.put("%jdbcdriver\\_table", new Object[]{"testhivejdbcdriver_table"});
-    tests.put("testhivejdbcdriver\\_table", new Object[]{"testhivejdbcdriver_table"});
-    tests.put("test_ivejdbcdri_er\\_table", new Object[]{"testhivejdbcdriver_table"});
-    tests.put("test_ivejdbcdri_er_table", new Object[]{"testhivejdbcdriver_table"});
-    tests.put("test_ivejdbcdri_er%table", new Object[]{
-        "testhivejdbcdriver_table", "testhivejdbcdriverpartitionedtable" });
-    tests.put("%jdbc%", new Object[]{ "testhivejdbcdriver_table"
-        , "testhivejdbcdriverpartitionedtable"
-        , "testhivejdbcdriverview"});
-    tests.put("", new Object[]{});
-
-    for (String checkPattern: tests.keySet()) {
-      ResultSet rs = con.getMetaData().getTables("default", null, checkPattern, null);
+    String[] ALL = null;
+    String[] VIEW_ONLY = {viewTypeName};
+    String[] TABLE_ONLY = {tableTypeName};
+    String[] VIEWORTABLE = {tableTypeName, viewTypeName};
+
+    Map<Object[], String[]> tests = new IdentityHashMap<Object[], String[]>();
+    tests.put(new Object[] { null, "test%jdbc%", ALL}, new String[]{
+        "default.testhivejdbcdriver_table",
+        "default.testhivejdbcdriverpartitionedtable",
+        "default.testhivejdbcdriverview",
+        "testdb.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriverpartitionedtable",
+        "testdb.testhivejdbcdriverview"});
+    tests.put(new Object[] { "test%", "test%jdbc%", ALL}, new String[]{
+        "testdb.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriverpartitionedtable",
+        "testdb.testhivejdbcdriverview"});
+    tests.put(new Object[] { "test%", "test%jdbc%", VIEW_ONLY}, new String[]{
+        "testdb.testhivejdbcdriverview"});
+
+    tests.put(new Object[] { null, "%jdbcdriver\\_table", VIEWORTABLE}, new String[]{
+        "default.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriver_table"});
+    tests.put(new Object[] { "def%", "%jdbcdriver\\_table", VIEWORTABLE}, new String[]{
+        "default.testhivejdbcdriver_table"});
+    tests.put(new Object[] { "def%", "%jdbcdriver\\_table", VIEW_ONLY}, new String[0]);
+
+    tests.put(new Object[] { null, "testhivejdbcdriver\\_table", ALL}, new String[]{
+        "default.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriver_table"});
+    tests.put(new Object[] { "%faul%", "testhivejdbcdriver\\_table", ALL}, new String[]{
+        "default.testhivejdbcdriver_table"});
+    tests.put(new Object[] { "%faul%", "testhivejdbcdriver\\_table", TABLE_ONLY}, new String[]{
+        "default.testhivejdbcdriver_table"});
+
+    tests.put(new Object[] { null, "test_ivejdbcdri_er\\_table", ALL}, new String[]{
+        "default.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriver_table"});
+    tests.put(new Object[] { "test__", "test_ivejdbcdri_er\\_table", ALL}, new String[]{
+        "testdb.testhivejdbcdriver_table"});
+
+    tests.put(new Object[] { null, "test_ivejdbcdri_er_table", ALL}, new String[]{
+        "default.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriver_table"});
+    tests.put(new Object[] { null, "test_ivejdbcdri_er%table", ALL}, new String[]{
+        "default.testhivejdbcdriver_table",
+        "default.testhivejdbcdriverpartitionedtable",
+        "testdb.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriverpartitionedtable"});
+    tests.put(new Object[] { null, "%jdbc%", ALL}, new String[]{
+        "default.testhivejdbcdriver_table",
+        "default.testhivejdbcdriverpartitionedtable",
+        "default.testhivejdbcdriverview",
+        "testdb.testhivejdbcdriver_table",
+        "testdb.testhivejdbcdriverpartitionedtable",
+        "testdb.testhivejdbcdriverview"});
+    tests.put(new Object[] { "%", "%jdbc%", VIEW_ONLY}, new String[]{
+        "default.testhivejdbcdriverview",
+        "testdb.testhivejdbcdriverview"});
+    tests.put(new Object[] { null, "", ALL}, new String[]{});
+
+    for (Map.Entry<Object[], String[]> entry : tests.entrySet()) {
+      Object[] checkPattern = entry.getKey();
+      String debugString = checkPattern[0] + ", " + checkPattern[1] + ", " +
+          Arrays.toString((String[]) checkPattern[2]);
+
+      Set<String> expectedTables = new HashSet<String>(Arrays.asList(entry.getValue()));
+      ResultSet rs = con.getMetaData().getTables(null,
+          (String)checkPattern[0], (String)checkPattern[1], (String[])checkPattern[2]);
       ResultSetMetaData resMeta = rs.getMetaData();
-      assertEquals(5, resMeta.getColumnCount());
+      assertEquals(10, resMeta.getColumnCount());
       assertEquals("TABLE_CAT", resMeta.getColumnName(1));
       assertEquals("TABLE_SCHEM", resMeta.getColumnName(2));
       assertEquals("TABLE_NAME", resMeta.getColumnName(3));
@@ -1150,9 +1223,11 @@ public class TestJdbcDriver2 {
 
       int cnt = 0;
       while (rs.next()) {
+        String resultDbName = rs.getString("TABLE_SCHEM");
         String resultTableName = rs.getString("TABLE_NAME");
-        assertEquals("Get by index different from get by name.", rs.getString(3), resultTableName);
-        assertEquals("Excpected a different table.", tests.get(checkPattern)[cnt], resultTableName);
+        assertTrue("Invalid table " + resultDbName + "." + resultTableName + " for test " + debugString,
+            expectedTables.contains(resultDbName + "." + resultTableName));
+
         String resultTableComment = rs.getString("REMARKS");
         assertTrue("Missing comment on the table.", resultTableComment.length()>0);
         String tableType = rs.getString("TABLE_TYPE");
@@ -1164,18 +1239,9 @@ public class TestJdbcDriver2 {
         cnt++;
       }
       rs.close();
-      assertEquals("Received an incorrect number of tables.", tests.get(checkPattern).length, cnt);
+      assertEquals("Received an incorrect number of tables for test " + debugString,
+          expectedTables.size(), cnt);
     }
-
-    // only ask for the views.
-    ResultSet rs = con.getMetaData().getTables("default", null, null
-        , new String[]{viewTypeName});
-    int cnt=0;
-    while (rs.next()) {
-      cnt++;
-    }
-    rs.close();
-    assertEquals("Incorrect number of views found.", 1, cnt);
   }
 
   @Test
@@ -1198,6 +1264,8 @@ public class TestJdbcDriver2 {
 
     assertTrue(rs.next());
     assertEquals("default", rs.getString(1));
+    assertTrue(rs.next());
+    assertEquals("testdb", rs.getString(1));
 
     assertFalse(rs.next());
     rs.close();
@@ -1270,7 +1338,7 @@ public class TestJdbcDriver2 {
     tests.put(new String[]{"%jdbcdriver\\_table%", "_%"}, 2);
 
     for (String[] checkPattern: tests.keySet()) {
-      ResultSet rs = con.getMetaData().getColumns(null, null, checkPattern[0],
+      ResultSet rs = con.getMetaData().getColumns(null, "default", checkPattern[0],
           checkPattern[1]);
 
       // validate the metadata for the getColumns result set

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java b/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java
index 13e42b5..a73f443 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java
@@ -646,9 +646,7 @@ public class HiveDatabaseMetaData implements DatabaseMetaData {
     if (types != null) {
       getTableReq.setTableTypes(Arrays.asList(types));
     }
-    if (schemaPattern != null) {
-      getTableReq.setSchemaName(schemaPattern);
-    }
+    getTableReq.setSchemaName(schemaPattern);
 
     try {
       getTableResp = client.GetTables(getTableReq);

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index 98fd42b..bb754f1 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -772,6 +772,13 @@ struct GetAllFunctionsResponse {
   1: optional list<Function> functions
 }
 
+struct TableMeta {
+  1: required string dbName;
+  2: required string tableName;
+  3: required string tableType;
+  4: optional string comments;
+}
+
 exception MetaException {
   1: string message
 }
@@ -890,6 +897,8 @@ service ThriftHiveMetastore extends fb303.FacebookService
       4:EnvironmentContext environment_context)
                        throws(1:NoSuchObjectException o1, 2:MetaException o3)
   list<string> get_tables(1: string db_name, 2: string pattern) throws (1: MetaException o1)
+  list<TableMeta> get_table_meta(1: string db_patterns, 2: string tbl_patterns, 3: list<string> tbl_types)
+                       throws (1: MetaException o1)
   list<string> get_all_tables(1: string db_name) throws (1: MetaException o1)
 
   Table get_table(1:string dbname, 2:string tbl_name)


[5/7] hive git commit: HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)

Posted by ai...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index 3d7cb18..cea9000 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -43,6 +43,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) = 0;
   virtual void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) = 0;
   virtual void get_tables(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern) = 0;
+  virtual void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types) = 0;
   virtual void get_all_tables(std::vector<std::string> & _return, const std::string& db_name) = 0;
   virtual void get_table(Table& _return, const std::string& dbname, const std::string& tbl_name) = 0;
   virtual void get_table_objects_by_name(std::vector<Table> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names) = 0;
@@ -246,6 +247,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void get_tables(std::vector<std::string> & /* _return */, const std::string& /* db_name */, const std::string& /* pattern */) {
     return;
   }
+  void get_table_meta(std::vector<TableMeta> & /* _return */, const std::string& /* db_patterns */, const std::string& /* tbl_patterns */, const std::vector<std::string> & /* tbl_types */) {
+    return;
+  }
   void get_all_tables(std::vector<std::string> & /* _return */, const std::string& /* db_name */) {
     return;
   }
@@ -3199,6 +3203,132 @@ class ThriftHiveMetastore_get_tables_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_get_table_meta_args__isset {
+  _ThriftHiveMetastore_get_table_meta_args__isset() : db_patterns(false), tbl_patterns(false), tbl_types(false) {}
+  bool db_patterns :1;
+  bool tbl_patterns :1;
+  bool tbl_types :1;
+} _ThriftHiveMetastore_get_table_meta_args__isset;
+
+class ThriftHiveMetastore_get_table_meta_args {
+ public:
+
+  ThriftHiveMetastore_get_table_meta_args(const ThriftHiveMetastore_get_table_meta_args&);
+  ThriftHiveMetastore_get_table_meta_args& operator=(const ThriftHiveMetastore_get_table_meta_args&);
+  ThriftHiveMetastore_get_table_meta_args() : db_patterns(), tbl_patterns() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_table_meta_args() throw();
+  std::string db_patterns;
+  std::string tbl_patterns;
+  std::vector<std::string>  tbl_types;
+
+  _ThriftHiveMetastore_get_table_meta_args__isset __isset;
+
+  void __set_db_patterns(const std::string& val);
+
+  void __set_tbl_patterns(const std::string& val);
+
+  void __set_tbl_types(const std::vector<std::string> & val);
+
+  bool operator == (const ThriftHiveMetastore_get_table_meta_args & rhs) const
+  {
+    if (!(db_patterns == rhs.db_patterns))
+      return false;
+    if (!(tbl_patterns == rhs.tbl_patterns))
+      return false;
+    if (!(tbl_types == rhs.tbl_types))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_table_meta_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_table_meta_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_table_meta_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_table_meta_pargs() throw();
+  const std::string* db_patterns;
+  const std::string* tbl_patterns;
+  const std::vector<std::string> * tbl_types;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_table_meta_result__isset {
+  _ThriftHiveMetastore_get_table_meta_result__isset() : success(false), o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_get_table_meta_result__isset;
+
+class ThriftHiveMetastore_get_table_meta_result {
+ public:
+
+  ThriftHiveMetastore_get_table_meta_result(const ThriftHiveMetastore_get_table_meta_result&);
+  ThriftHiveMetastore_get_table_meta_result& operator=(const ThriftHiveMetastore_get_table_meta_result&);
+  ThriftHiveMetastore_get_table_meta_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_table_meta_result() throw();
+  std::vector<TableMeta>  success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_get_table_meta_result__isset __isset;
+
+  void __set_success(const std::vector<TableMeta> & val);
+
+  void __set_o1(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_table_meta_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_table_meta_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_table_meta_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_table_meta_presult__isset {
+  _ThriftHiveMetastore_get_table_meta_presult__isset() : success(false), o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_get_table_meta_presult__isset;
+
+class ThriftHiveMetastore_get_table_meta_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_table_meta_presult() throw();
+  std::vector<TableMeta> * success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_get_table_meta_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_get_all_tables_args__isset {
   _ThriftHiveMetastore_get_all_tables_args__isset() : db_name(false) {}
   bool db_name :1;
@@ -16967,6 +17097,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void get_tables(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern);
   void send_get_tables(const std::string& db_name, const std::string& pattern);
   void recv_get_tables(std::vector<std::string> & _return);
+  void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
+  void send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
+  void recv_get_table_meta(std::vector<TableMeta> & _return);
   void get_all_tables(std::vector<std::string> & _return, const std::string& db_name);
   void send_get_all_tables(const std::string& db_name);
   void recv_get_all_tables(std::vector<std::string> & _return);
@@ -17325,6 +17458,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_drop_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_drop_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_get_table_meta(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_all_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_table_objects_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -17459,6 +17593,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["drop_table"] = &ThriftHiveMetastoreProcessor::process_drop_table;
     processMap_["drop_table_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_drop_table_with_environment_context;
     processMap_["get_tables"] = &ThriftHiveMetastoreProcessor::process_get_tables;
+    processMap_["get_table_meta"] = &ThriftHiveMetastoreProcessor::process_get_table_meta;
     processMap_["get_all_tables"] = &ThriftHiveMetastoreProcessor::process_get_all_tables;
     processMap_["get_table"] = &ThriftHiveMetastoreProcessor::process_get_table;
     processMap_["get_table_objects_by_name"] = &ThriftHiveMetastoreProcessor::process_get_table_objects_by_name;
@@ -17801,6 +17936,16 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     return;
   }
 
+  void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_table_meta(_return, db_patterns, tbl_patterns, tbl_types);
+    }
+    ifaces_[i]->get_table_meta(_return, db_patterns, tbl_patterns, tbl_types);
+    return;
+  }
+
   void get_all_tables(std::vector<std::string> & _return, const std::string& db_name) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -18929,6 +19074,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void get_tables(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern);
   int32_t send_get_tables(const std::string& db_name, const std::string& pattern);
   void recv_get_tables(std::vector<std::string> & _return, const int32_t seqid);
+  void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
+  int32_t send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
+  void recv_get_table_meta(std::vector<TableMeta> & _return, const int32_t seqid);
   void get_all_tables(std::vector<std::string> & _return, const std::string& db_name);
   int32_t send_get_all_tables(const std::string& db_name);
   void recv_get_all_tables(std::vector<std::string> & _return, const int32_t seqid);

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index a395729..c0d9401 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -127,6 +127,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("get_tables\n");
   }
 
+  void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types) {
+    // Your implementation goes here
+    printf("get_table_meta\n");
+  }
+
   void get_all_tables(std::vector<std::string> & _return, const std::string& db_name) {
     // Your implementation goes here
     printf("get_all_tables\n");

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 5fd4a90..ee28d0d 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -15157,6 +15157,163 @@ void GetAllFunctionsResponse::printTo(std::ostream& out) const {
 }
 
 
+TableMeta::~TableMeta() throw() {
+}
+
+
+void TableMeta::__set_dbName(const std::string& val) {
+  this->dbName = val;
+}
+
+void TableMeta::__set_tableName(const std::string& val) {
+  this->tableName = val;
+}
+
+void TableMeta::__set_tableType(const std::string& val) {
+  this->tableType = val;
+}
+
+void TableMeta::__set_comments(const std::string& val) {
+  this->comments = val;
+__isset.comments = true;
+}
+
+uint32_t TableMeta::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_dbName = false;
+  bool isset_tableName = false;
+  bool isset_tableType = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->dbName);
+          isset_dbName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->tableName);
+          isset_tableName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->tableType);
+          isset_tableType = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->comments);
+          this->__isset.comments = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_dbName)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_tableName)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_tableType)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t TableMeta::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("TableMeta");
+
+  xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->dbName);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->tableName);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->tableType);
+  xfer += oprot->writeFieldEnd();
+
+  if (this->__isset.comments) {
+    xfer += oprot->writeFieldBegin("comments", ::apache::thrift::protocol::T_STRING, 4);
+    xfer += oprot->writeString(this->comments);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(TableMeta &a, TableMeta &b) {
+  using ::std::swap;
+  swap(a.dbName, b.dbName);
+  swap(a.tableName, b.tableName);
+  swap(a.tableType, b.tableType);
+  swap(a.comments, b.comments);
+  swap(a.__isset, b.__isset);
+}
+
+TableMeta::TableMeta(const TableMeta& other682) {
+  dbName = other682.dbName;
+  tableName = other682.tableName;
+  tableType = other682.tableType;
+  comments = other682.comments;
+  __isset = other682.__isset;
+}
+TableMeta& TableMeta::operator=(const TableMeta& other683) {
+  dbName = other683.dbName;
+  tableName = other683.tableName;
+  tableType = other683.tableType;
+  comments = other683.comments;
+  __isset = other683.__isset;
+  return *this;
+}
+void TableMeta::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "TableMeta(";
+  out << "dbName=" << to_string(dbName);
+  out << ", " << "tableName=" << to_string(tableName);
+  out << ", " << "tableType=" << to_string(tableType);
+  out << ", " << "comments="; (__isset.comments ? (out << to_string(comments)) : (out << "<null>"));
+  out << ")";
+}
+
+
 MetaException::~MetaException() throw() {
 }
 
@@ -15226,13 +15383,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other682) : TException() {
-  message = other682.message;
-  __isset = other682.__isset;
+MetaException::MetaException(const MetaException& other684) : TException() {
+  message = other684.message;
+  __isset = other684.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other683) {
-  message = other683.message;
-  __isset = other683.__isset;
+MetaException& MetaException::operator=(const MetaException& other685) {
+  message = other685.message;
+  __isset = other685.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -15323,13 +15480,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other684) : TException() {
-  message = other684.message;
-  __isset = other684.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other686) : TException() {
+  message = other686.message;
+  __isset = other686.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other685) {
-  message = other685.message;
-  __isset = other685.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other687) {
+  message = other687.message;
+  __isset = other687.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -15420,13 +15577,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other686) : TException() {
-  message = other686.message;
-  __isset = other686.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other688) : TException() {
+  message = other688.message;
+  __isset = other688.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other687) {
-  message = other687.message;
-  __isset = other687.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other689) {
+  message = other689.message;
+  __isset = other689.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -15517,13 +15674,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other688) : TException() {
-  message = other688.message;
-  __isset = other688.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other690) : TException() {
+  message = other690.message;
+  __isset = other690.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other689) {
-  message = other689.message;
-  __isset = other689.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other691) {
+  message = other691.message;
+  __isset = other691.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -15614,13 +15771,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other690) : TException() {
-  message = other690.message;
-  __isset = other690.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other692) : TException() {
+  message = other692.message;
+  __isset = other692.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other691) {
-  message = other691.message;
-  __isset = other691.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other693) {
+  message = other693.message;
+  __isset = other693.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -15711,13 +15868,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other692) : TException() {
-  message = other692.message;
-  __isset = other692.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other694) : TException() {
+  message = other694.message;
+  __isset = other694.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other693) {
-  message = other693.message;
-  __isset = other693.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other695) {
+  message = other695.message;
+  __isset = other695.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -15808,13 +15965,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other694) : TException() {
-  message = other694.message;
-  __isset = other694.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other696) : TException() {
+  message = other696.message;
+  __isset = other696.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other695) {
-  message = other695.message;
-  __isset = other695.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other697) {
+  message = other697.message;
+  __isset = other697.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -15905,13 +16062,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other696) : TException() {
-  message = other696.message;
-  __isset = other696.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other698) : TException() {
+  message = other698.message;
+  __isset = other698.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other697) {
-  message = other697.message;
-  __isset = other697.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other699) {
+  message = other699.message;
+  __isset = other699.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -16002,13 +16159,13 @@ void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other698) : TException() {
-  message = other698.message;
-  __isset = other698.__isset;
+IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other700) : TException() {
+  message = other700.message;
+  __isset = other700.__isset;
 }
-IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other699) {
-  message = other699.message;
-  __isset = other699.__isset;
+IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other701) {
+  message = other701.message;
+  __isset = other701.__isset;
   return *this;
 }
 void IndexAlreadyExistsException::printTo(std::ostream& out) const {
@@ -16099,13 +16256,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other700) : TException() {
-  message = other700.message;
-  __isset = other700.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other702) : TException() {
+  message = other702.message;
+  __isset = other702.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other701) {
-  message = other701.message;
-  __isset = other701.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other703) {
+  message = other703.message;
+  __isset = other703.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -16196,13 +16353,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other702) : TException() {
-  message = other702.message;
-  __isset = other702.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other704) : TException() {
+  message = other704.message;
+  __isset = other704.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other703) {
-  message = other703.message;
-  __isset = other703.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other705) {
+  message = other705.message;
+  __isset = other705.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -16293,13 +16450,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other704) : TException() {
-  message = other704.message;
-  __isset = other704.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other706) : TException() {
+  message = other706.message;
+  __isset = other706.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other705) {
-  message = other705.message;
-  __isset = other705.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other707) {
+  message = other707.message;
+  __isset = other707.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -16390,13 +16547,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other706) : TException() {
-  message = other706.message;
-  __isset = other706.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other708) : TException() {
+  message = other708.message;
+  __isset = other708.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other707) {
-  message = other707.message;
-  __isset = other707.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other709) {
+  message = other709.message;
+  __isset = other709.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -16487,13 +16644,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other708) : TException() {
-  message = other708.message;
-  __isset = other708.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other710) : TException() {
+  message = other710.message;
+  __isset = other710.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other709) {
-  message = other709.message;
-  __isset = other709.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other711) {
+  message = other711.message;
+  __isset = other711.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -16584,13 +16741,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other710) : TException() {
-  message = other710.message;
-  __isset = other710.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other712) : TException() {
+  message = other712.message;
+  __isset = other712.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other711) {
-  message = other711.message;
-  __isset = other711.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other713) {
+  message = other713.message;
+  __isset = other713.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -16681,13 +16838,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other712) : TException() {
-  message = other712.message;
-  __isset = other712.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other714) : TException() {
+  message = other714.message;
+  __isset = other714.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other713) {
-  message = other713.message;
-  __isset = other713.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other715) {
+  message = other715.message;
+  __isset = other715.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index 53ab272..05c288c 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -349,6 +349,8 @@ class ClearFileMetadataRequest;
 
 class GetAllFunctionsResponse;
 
+class TableMeta;
+
 class MetaException;
 
 class UnknownTableException;
@@ -6158,6 +6160,69 @@ inline std::ostream& operator<<(std::ostream& out, const GetAllFunctionsResponse
   return out;
 }
 
+typedef struct _TableMeta__isset {
+  _TableMeta__isset() : comments(false) {}
+  bool comments :1;
+} _TableMeta__isset;
+
+class TableMeta {
+ public:
+
+  TableMeta(const TableMeta&);
+  TableMeta& operator=(const TableMeta&);
+  TableMeta() : dbName(), tableName(), tableType(), comments() {
+  }
+
+  virtual ~TableMeta() throw();
+  std::string dbName;
+  std::string tableName;
+  std::string tableType;
+  std::string comments;
+
+  _TableMeta__isset __isset;
+
+  void __set_dbName(const std::string& val);
+
+  void __set_tableName(const std::string& val);
+
+  void __set_tableType(const std::string& val);
+
+  void __set_comments(const std::string& val);
+
+  bool operator == (const TableMeta & rhs) const
+  {
+    if (!(dbName == rhs.dbName))
+      return false;
+    if (!(tableName == rhs.tableName))
+      return false;
+    if (!(tableType == rhs.tableType))
+      return false;
+    if (__isset.comments != rhs.__isset.comments)
+      return false;
+    else if (__isset.comments && !(comments == rhs.comments))
+      return false;
+    return true;
+  }
+  bool operator != (const TableMeta &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const TableMeta & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(TableMeta &a, TableMeta &b);
+
+inline std::ostream& operator<<(std::ostream& out, const TableMeta& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _MetaException__isset {
   _MetaException__isset() : message(false) {}
   bool message :1;

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableMeta.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableMeta.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableMeta.java
new file mode 100644
index 0000000..08a8e36
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableMeta.java
@@ -0,0 +1,701 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class TableMeta implements org.apache.thrift.TBase<TableMeta, TableMeta._Fields>, java.io.Serializable, Cloneable, Comparable<TableMeta> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TableMeta");
+
+  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)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 TABLE_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tableType", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField COMMENTS_FIELD_DESC = new org.apache.thrift.protocol.TField("comments", org.apache.thrift.protocol.TType.STRING, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TableMetaStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TableMetaTupleSchemeFactory());
+  }
+
+  private String dbName; // required
+  private String tableName; // required
+  private String tableType; // required
+  private String comments; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    DB_NAME((short)1, "dbName"),
+    TABLE_NAME((short)2, "tableName"),
+    TABLE_TYPE((short)3, "tableType"),
+    COMMENTS((short)4, "comments");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // DB_NAME
+          return DB_NAME;
+        case 2: // TABLE_NAME
+          return TABLE_NAME;
+        case 3: // TABLE_TYPE
+          return TABLE_TYPE;
+        case 4: // COMMENTS
+          return COMMENTS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.COMMENTS};
+  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.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TABLE_TYPE, new org.apache.thrift.meta_data.FieldMetaData("tableType", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.COMMENTS, new org.apache.thrift.meta_data.FieldMetaData("comments", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TableMeta.class, metaDataMap);
+  }
+
+  public TableMeta() {
+  }
+
+  public TableMeta(
+    String dbName,
+    String tableName,
+    String tableType)
+  {
+    this();
+    this.dbName = dbName;
+    this.tableName = tableName;
+    this.tableType = tableType;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TableMeta(TableMeta other) {
+    if (other.isSetDbName()) {
+      this.dbName = other.dbName;
+    }
+    if (other.isSetTableName()) {
+      this.tableName = other.tableName;
+    }
+    if (other.isSetTableType()) {
+      this.tableType = other.tableType;
+    }
+    if (other.isSetComments()) {
+      this.comments = other.comments;
+    }
+  }
+
+  public TableMeta deepCopy() {
+    return new TableMeta(this);
+  }
+
+  @Override
+  public void clear() {
+    this.dbName = null;
+    this.tableName = null;
+    this.tableType = null;
+    this.comments = null;
+  }
+
+  public String getDbName() {
+    return this.dbName;
+  }
+
+  public void setDbName(String dbName) {
+    this.dbName = dbName;
+  }
+
+  public void unsetDbName() {
+    this.dbName = null;
+  }
+
+  /** Returns true if field dbName is set (has been assigned a value) and false otherwise */
+  public boolean isSetDbName() {
+    return this.dbName != null;
+  }
+
+  public void setDbNameIsSet(boolean value) {
+    if (!value) {
+      this.dbName = null;
+    }
+  }
+
+  public String getTableName() {
+    return this.tableName;
+  }
+
+  public void setTableName(String tableName) {
+    this.tableName = tableName;
+  }
+
+  public void unsetTableName() {
+    this.tableName = null;
+  }
+
+  /** Returns true if field tableName is set (has been assigned a value) and false otherwise */
+  public boolean isSetTableName() {
+    return this.tableName != null;
+  }
+
+  public void setTableNameIsSet(boolean value) {
+    if (!value) {
+      this.tableName = null;
+    }
+  }
+
+  public String getTableType() {
+    return this.tableType;
+  }
+
+  public void setTableType(String tableType) {
+    this.tableType = tableType;
+  }
+
+  public void unsetTableType() {
+    this.tableType = null;
+  }
+
+  /** Returns true if field tableType is set (has been assigned a value) and false otherwise */
+  public boolean isSetTableType() {
+    return this.tableType != null;
+  }
+
+  public void setTableTypeIsSet(boolean value) {
+    if (!value) {
+      this.tableType = null;
+    }
+  }
+
+  public String getComments() {
+    return this.comments;
+  }
+
+  public void setComments(String comments) {
+    this.comments = comments;
+  }
+
+  public void unsetComments() {
+    this.comments = null;
+  }
+
+  /** Returns true if field comments is set (has been assigned a value) and false otherwise */
+  public boolean isSetComments() {
+    return this.comments != null;
+  }
+
+  public void setCommentsIsSet(boolean value) {
+    if (!value) {
+      this.comments = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case DB_NAME:
+      if (value == null) {
+        unsetDbName();
+      } else {
+        setDbName((String)value);
+      }
+      break;
+
+    case TABLE_NAME:
+      if (value == null) {
+        unsetTableName();
+      } else {
+        setTableName((String)value);
+      }
+      break;
+
+    case TABLE_TYPE:
+      if (value == null) {
+        unsetTableType();
+      } else {
+        setTableType((String)value);
+      }
+      break;
+
+    case COMMENTS:
+      if (value == null) {
+        unsetComments();
+      } else {
+        setComments((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case DB_NAME:
+      return getDbName();
+
+    case TABLE_NAME:
+      return getTableName();
+
+    case TABLE_TYPE:
+      return getTableType();
+
+    case COMMENTS:
+      return getComments();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case DB_NAME:
+      return isSetDbName();
+    case TABLE_NAME:
+      return isSetTableName();
+    case TABLE_TYPE:
+      return isSetTableType();
+    case COMMENTS:
+      return isSetComments();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TableMeta)
+      return this.equals((TableMeta)that);
+    return false;
+  }
+
+  public boolean equals(TableMeta that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_dbName = true && this.isSetDbName();
+    boolean that_present_dbName = true && that.isSetDbName();
+    if (this_present_dbName || that_present_dbName) {
+      if (!(this_present_dbName && that_present_dbName))
+        return false;
+      if (!this.dbName.equals(that.dbName))
+        return false;
+    }
+
+    boolean this_present_tableName = true && this.isSetTableName();
+    boolean that_present_tableName = true && that.isSetTableName();
+    if (this_present_tableName || that_present_tableName) {
+      if (!(this_present_tableName && that_present_tableName))
+        return false;
+      if (!this.tableName.equals(that.tableName))
+        return false;
+    }
+
+    boolean this_present_tableType = true && this.isSetTableType();
+    boolean that_present_tableType = true && that.isSetTableType();
+    if (this_present_tableType || that_present_tableType) {
+      if (!(this_present_tableType && that_present_tableType))
+        return false;
+      if (!this.tableType.equals(that.tableType))
+        return false;
+    }
+
+    boolean this_present_comments = true && this.isSetComments();
+    boolean that_present_comments = true && that.isSetComments();
+    if (this_present_comments || that_present_comments) {
+      if (!(this_present_comments && that_present_comments))
+        return false;
+      if (!this.comments.equals(that.comments))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_dbName = true && (isSetDbName());
+    list.add(present_dbName);
+    if (present_dbName)
+      list.add(dbName);
+
+    boolean present_tableName = true && (isSetTableName());
+    list.add(present_tableName);
+    if (present_tableName)
+      list.add(tableName);
+
+    boolean present_tableType = true && (isSetTableType());
+    list.add(present_tableType);
+    if (present_tableType)
+      list.add(tableType);
+
+    boolean present_comments = true && (isSetComments());
+    list.add(present_comments);
+    if (present_comments)
+      list.add(comments);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TableMeta other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetDbName()).compareTo(other.isSetDbName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDbName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbName, other.dbName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTableName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTableType()).compareTo(other.isSetTableType());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTableType()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableType, other.tableType);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetComments()).compareTo(other.isSetComments());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetComments()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.comments, other.comments);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TableMeta(");
+    boolean first = true;
+
+    sb.append("dbName:");
+    if (this.dbName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.dbName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("tableName:");
+    if (this.tableName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tableName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("tableType:");
+    if (this.tableType == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tableType);
+    }
+    first = false;
+    if (isSetComments()) {
+      if (!first) sb.append(", ");
+      sb.append("comments:");
+      if (this.comments == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.comments);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetDbName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'dbName' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTableName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTableType()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableType' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TableMetaStandardSchemeFactory implements SchemeFactory {
+    public TableMetaStandardScheme getScheme() {
+      return new TableMetaStandardScheme();
+    }
+  }
+
+  private static class TableMetaStandardScheme extends StandardScheme<TableMeta> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TableMeta struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // DB_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.dbName = iprot.readString();
+              struct.setDbNameIsSet(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);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TABLE_TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tableType = iprot.readString();
+              struct.setTableTypeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // COMMENTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.comments = iprot.readString();
+              struct.setCommentsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TableMeta struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.dbName != null) {
+        oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
+        oprot.writeString(struct.dbName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.tableName != null) {
+        oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
+        oprot.writeString(struct.tableName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.tableType != null) {
+        oprot.writeFieldBegin(TABLE_TYPE_FIELD_DESC);
+        oprot.writeString(struct.tableType);
+        oprot.writeFieldEnd();
+      }
+      if (struct.comments != null) {
+        if (struct.isSetComments()) {
+          oprot.writeFieldBegin(COMMENTS_FIELD_DESC);
+          oprot.writeString(struct.comments);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TableMetaTupleSchemeFactory implements SchemeFactory {
+    public TableMetaTupleScheme getScheme() {
+      return new TableMetaTupleScheme();
+    }
+  }
+
+  private static class TableMetaTupleScheme extends TupleScheme<TableMeta> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TableMeta struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.dbName);
+      oprot.writeString(struct.tableName);
+      oprot.writeString(struct.tableType);
+      BitSet optionals = new BitSet();
+      if (struct.isSetComments()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetComments()) {
+        oprot.writeString(struct.comments);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TableMeta struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.dbName = iprot.readString();
+      struct.setDbNameIsSet(true);
+      struct.tableName = iprot.readString();
+      struct.setTableNameIsSet(true);
+      struct.tableType = iprot.readString();
+      struct.setTableTypeIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.comments = iprot.readString();
+        struct.setCommentsIsSet(true);
+      }
+    }
+  }
+
+}
+


[2/7] hive git commit: HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)

Posted by ai...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 9873810..5c72a27 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -183,6 +183,15 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def get_table_meta(self, db_patterns, tbl_patterns, tbl_types):
+    """
+    Parameters:
+     - db_patterns
+     - tbl_patterns
+     - tbl_types
+    """
+    pass
+
   def get_all_tables(self, db_name):
     """
     Parameters:
@@ -1830,6 +1839,43 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o1
     raise TApplicationException(TApplicationException.MISSING_RESULT, "get_tables failed: unknown result")
 
+  def get_table_meta(self, db_patterns, tbl_patterns, tbl_types):
+    """
+    Parameters:
+     - db_patterns
+     - tbl_patterns
+     - tbl_types
+    """
+    self.send_get_table_meta(db_patterns, tbl_patterns, tbl_types)
+    return self.recv_get_table_meta()
+
+  def send_get_table_meta(self, db_patterns, tbl_patterns, tbl_types):
+    self._oprot.writeMessageBegin('get_table_meta', TMessageType.CALL, self._seqid)
+    args = get_table_meta_args()
+    args.db_patterns = db_patterns
+    args.tbl_patterns = tbl_patterns
+    args.tbl_types = tbl_types
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_table_meta(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_table_meta_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_table_meta failed: unknown result")
+
   def get_all_tables(self, db_name):
     """
     Parameters:
@@ -5852,6 +5898,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["drop_table"] = Processor.process_drop_table
     self._processMap["drop_table_with_environment_context"] = Processor.process_drop_table_with_environment_context
     self._processMap["get_tables"] = Processor.process_get_tables
+    self._processMap["get_table_meta"] = Processor.process_get_table_meta
     self._processMap["get_all_tables"] = Processor.process_get_all_tables
     self._processMap["get_table"] = Processor.process_get_table
     self._processMap["get_table_objects_by_name"] = Processor.process_get_table_objects_by_name
@@ -6517,6 +6564,28 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_get_table_meta(self, seqid, iprot, oprot):
+    args = get_table_meta_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_table_meta_result()
+    try:
+      result.success = self._handler.get_table_meta(args.db_patterns, args.tbl_patterns, args.tbl_types)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except MetaException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_table_meta", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_get_all_tables(self, seqid, iprot, oprot):
     args = get_all_tables_args()
     args.read(iprot)
@@ -12752,6 +12821,192 @@ class get_tables_result:
   def __ne__(self, other):
     return not (self == other)
 
+class get_table_meta_args:
+  """
+  Attributes:
+   - db_patterns
+   - tbl_patterns
+   - tbl_types
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'db_patterns', None, None, ), # 1
+    (2, TType.STRING, 'tbl_patterns', None, None, ), # 2
+    (3, TType.LIST, 'tbl_types', (TType.STRING,None), None, ), # 3
+  )
+
+  def __init__(self, db_patterns=None, tbl_patterns=None, tbl_types=None,):
+    self.db_patterns = db_patterns
+    self.tbl_patterns = tbl_patterns
+    self.tbl_types = tbl_types
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.db_patterns = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.tbl_patterns = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.LIST:
+          self.tbl_types = []
+          (_etype583, _size580) = iprot.readListBegin()
+          for _i584 in xrange(_size580):
+            _elem585 = iprot.readString()
+            self.tbl_types.append(_elem585)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_table_meta_args')
+    if self.db_patterns is not None:
+      oprot.writeFieldBegin('db_patterns', TType.STRING, 1)
+      oprot.writeString(self.db_patterns)
+      oprot.writeFieldEnd()
+    if self.tbl_patterns is not None:
+      oprot.writeFieldBegin('tbl_patterns', TType.STRING, 2)
+      oprot.writeString(self.tbl_patterns)
+      oprot.writeFieldEnd()
+    if self.tbl_types is not None:
+      oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
+      oprot.writeListBegin(TType.STRING, len(self.tbl_types))
+      for iter586 in self.tbl_types:
+        oprot.writeString(iter586)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.db_patterns)
+    value = (value * 31) ^ hash(self.tbl_patterns)
+    value = (value * 31) ^ hash(self.tbl_types)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class get_table_meta_result:
+  """
+  Attributes:
+   - success
+   - o1
+  """
+
+  thrift_spec = (
+    (0, TType.LIST, 'success', (TType.STRUCT,(TableMeta, TableMeta.thrift_spec)), None, ), # 0
+    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, success=None, o1=None,):
+    self.success = success
+    self.o1 = o1
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.LIST:
+          self.success = []
+          (_etype590, _size587) = iprot.readListBegin()
+          for _i591 in xrange(_size587):
+            _elem592 = TableMeta()
+            _elem592.read(iprot)
+            self.success.append(_elem592)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = MetaException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_table_meta_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.LIST, 0)
+      oprot.writeListBegin(TType.STRUCT, len(self.success))
+      for iter593 in self.success:
+        iter593.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    value = (value * 31) ^ hash(self.o1)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class get_all_tables_args:
   """
   Attributes:
@@ -12845,10 +13100,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype583, _size580) = iprot.readListBegin()
-          for _i584 in xrange(_size580):
-            _elem585 = iprot.readString()
-            self.success.append(_elem585)
+          (_etype597, _size594) = iprot.readListBegin()
+          for _i598 in xrange(_size594):
+            _elem599 = iprot.readString()
+            self.success.append(_elem599)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12871,8 +13126,8 @@ class get_all_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter586 in self.success:
-        oprot.writeString(iter586)
+      for iter600 in self.success:
+        oprot.writeString(iter600)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13108,10 +13363,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype590, _size587) = iprot.readListBegin()
-          for _i591 in xrange(_size587):
-            _elem592 = iprot.readString()
-            self.tbl_names.append(_elem592)
+          (_etype604, _size601) = iprot.readListBegin()
+          for _i605 in xrange(_size601):
+            _elem606 = iprot.readString()
+            self.tbl_names.append(_elem606)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13132,8 +13387,8 @@ class get_table_objects_by_name_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter593 in self.tbl_names:
-        oprot.writeString(iter593)
+      for iter607 in self.tbl_names:
+        oprot.writeString(iter607)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13194,11 +13449,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype597, _size594) = iprot.readListBegin()
-          for _i598 in xrange(_size594):
-            _elem599 = Table()
-            _elem599.read(iprot)
-            self.success.append(_elem599)
+          (_etype611, _size608) = iprot.readListBegin()
+          for _i612 in xrange(_size608):
+            _elem613 = Table()
+            _elem613.read(iprot)
+            self.success.append(_elem613)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13233,8 +13488,8 @@ class get_table_objects_by_name_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter600 in self.success:
-        iter600.write(oprot)
+      for iter614 in self.success:
+        iter614.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13400,10 +13655,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype604, _size601) = iprot.readListBegin()
-          for _i605 in xrange(_size601):
-            _elem606 = iprot.readString()
-            self.success.append(_elem606)
+          (_etype618, _size615) = iprot.readListBegin()
+          for _i619 in xrange(_size615):
+            _elem620 = iprot.readString()
+            self.success.append(_elem620)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13438,8 +13693,8 @@ class get_table_names_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter607 in self.success:
-        oprot.writeString(iter607)
+      for iter621 in self.success:
+        oprot.writeString(iter621)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14409,11 +14664,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype611, _size608) = iprot.readListBegin()
-          for _i612 in xrange(_size608):
-            _elem613 = Partition()
-            _elem613.read(iprot)
-            self.new_parts.append(_elem613)
+          (_etype625, _size622) = iprot.readListBegin()
+          for _i626 in xrange(_size622):
+            _elem627 = Partition()
+            _elem627.read(iprot)
+            self.new_parts.append(_elem627)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14430,8 +14685,8 @@ class add_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter614 in self.new_parts:
-        iter614.write(oprot)
+      for iter628 in self.new_parts:
+        iter628.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14589,11 +14844,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype618, _size615) = iprot.readListBegin()
-          for _i619 in xrange(_size615):
-            _elem620 = PartitionSpec()
-            _elem620.read(iprot)
-            self.new_parts.append(_elem620)
+          (_etype632, _size629) = iprot.readListBegin()
+          for _i633 in xrange(_size629):
+            _elem634 = PartitionSpec()
+            _elem634.read(iprot)
+            self.new_parts.append(_elem634)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14610,8 +14865,8 @@ class add_partitions_pspec_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter621 in self.new_parts:
-        iter621.write(oprot)
+      for iter635 in self.new_parts:
+        iter635.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14785,10 +15040,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype625, _size622) = iprot.readListBegin()
-          for _i626 in xrange(_size622):
-            _elem627 = iprot.readString()
-            self.part_vals.append(_elem627)
+          (_etype639, _size636) = iprot.readListBegin()
+          for _i640 in xrange(_size636):
+            _elem641 = iprot.readString()
+            self.part_vals.append(_elem641)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14813,8 +15068,8 @@ class append_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter628 in self.part_vals:
-        oprot.writeString(iter628)
+      for iter642 in self.part_vals:
+        oprot.writeString(iter642)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15167,10 +15422,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype632, _size629) = iprot.readListBegin()
-          for _i633 in xrange(_size629):
-            _elem634 = iprot.readString()
-            self.part_vals.append(_elem634)
+          (_etype646, _size643) = iprot.readListBegin()
+          for _i647 in xrange(_size643):
+            _elem648 = iprot.readString()
+            self.part_vals.append(_elem648)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15201,8 +15456,8 @@ class append_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter635 in self.part_vals:
-        oprot.writeString(iter635)
+      for iter649 in self.part_vals:
+        oprot.writeString(iter649)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -15797,10 +16052,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype639, _size636) = iprot.readListBegin()
-          for _i640 in xrange(_size636):
-            _elem641 = iprot.readString()
-            self.part_vals.append(_elem641)
+          (_etype653, _size650) = iprot.readListBegin()
+          for _i654 in xrange(_size650):
+            _elem655 = iprot.readString()
+            self.part_vals.append(_elem655)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15830,8 +16085,8 @@ class drop_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter642 in self.part_vals:
-        oprot.writeString(iter642)
+      for iter656 in self.part_vals:
+        oprot.writeString(iter656)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -16004,10 +16259,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype646, _size643) = iprot.readListBegin()
-          for _i647 in xrange(_size643):
-            _elem648 = iprot.readString()
-            self.part_vals.append(_elem648)
+          (_etype660, _size657) = iprot.readListBegin()
+          for _i661 in xrange(_size657):
+            _elem662 = iprot.readString()
+            self.part_vals.append(_elem662)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16043,8 +16298,8 @@ class drop_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter649 in self.part_vals:
-        oprot.writeString(iter649)
+      for iter663 in self.part_vals:
+        oprot.writeString(iter663)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -16781,10 +17036,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype653, _size650) = iprot.readListBegin()
-          for _i654 in xrange(_size650):
-            _elem655 = iprot.readString()
-            self.part_vals.append(_elem655)
+          (_etype667, _size664) = iprot.readListBegin()
+          for _i668 in xrange(_size664):
+            _elem669 = iprot.readString()
+            self.part_vals.append(_elem669)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16809,8 +17064,8 @@ class get_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter656 in self.part_vals:
-        oprot.writeString(iter656)
+      for iter670 in self.part_vals:
+        oprot.writeString(iter670)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16969,11 +17224,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype658, _vtype659, _size657 ) = iprot.readMapBegin()
-          for _i661 in xrange(_size657):
-            _key662 = iprot.readString()
-            _val663 = iprot.readString()
-            self.partitionSpecs[_key662] = _val663
+          (_ktype672, _vtype673, _size671 ) = iprot.readMapBegin()
+          for _i675 in xrange(_size671):
+            _key676 = iprot.readString()
+            _val677 = iprot.readString()
+            self.partitionSpecs[_key676] = _val677
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -17010,9 +17265,9 @@ class exchange_partition_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter664,viter665 in self.partitionSpecs.items():
-        oprot.writeString(kiter664)
-        oprot.writeString(viter665)
+      for kiter678,viter679 in self.partitionSpecs.items():
+        oprot.writeString(kiter678)
+        oprot.writeString(viter679)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -17217,11 +17472,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype667, _vtype668, _size666 ) = iprot.readMapBegin()
-          for _i670 in xrange(_size666):
-            _key671 = iprot.readString()
-            _val672 = iprot.readString()
-            self.partitionSpecs[_key671] = _val672
+          (_ktype681, _vtype682, _size680 ) = iprot.readMapBegin()
+          for _i684 in xrange(_size680):
+            _key685 = iprot.readString()
+            _val686 = iprot.readString()
+            self.partitionSpecs[_key685] = _val686
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -17258,9 +17513,9 @@ class exchange_partitions_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter673,viter674 in self.partitionSpecs.items():
-        oprot.writeString(kiter673)
-        oprot.writeString(viter674)
+      for kiter687,viter688 in self.partitionSpecs.items():
+        oprot.writeString(kiter687)
+        oprot.writeString(viter688)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -17343,11 +17598,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype678, _size675) = iprot.readListBegin()
-          for _i679 in xrange(_size675):
-            _elem680 = Partition()
-            _elem680.read(iprot)
-            self.success.append(_elem680)
+          (_etype692, _size689) = iprot.readListBegin()
+          for _i693 in xrange(_size689):
+            _elem694 = Partition()
+            _elem694.read(iprot)
+            self.success.append(_elem694)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17388,8 +17643,8 @@ class exchange_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter681 in self.success:
-        iter681.write(oprot)
+      for iter695 in self.success:
+        iter695.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17483,10 +17738,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype685, _size682) = iprot.readListBegin()
-          for _i686 in xrange(_size682):
-            _elem687 = iprot.readString()
-            self.part_vals.append(_elem687)
+          (_etype699, _size696) = iprot.readListBegin()
+          for _i700 in xrange(_size696):
+            _elem701 = iprot.readString()
+            self.part_vals.append(_elem701)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17498,10 +17753,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype691, _size688) = iprot.readListBegin()
-          for _i692 in xrange(_size688):
-            _elem693 = iprot.readString()
-            self.group_names.append(_elem693)
+          (_etype705, _size702) = iprot.readListBegin()
+          for _i706 in xrange(_size702):
+            _elem707 = iprot.readString()
+            self.group_names.append(_elem707)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17526,8 +17781,8 @@ class get_partition_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter694 in self.part_vals:
-        oprot.writeString(iter694)
+      for iter708 in self.part_vals:
+        oprot.writeString(iter708)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -17537,8 +17792,8 @@ class get_partition_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter695 in self.group_names:
-        oprot.writeString(iter695)
+      for iter709 in self.group_names:
+        oprot.writeString(iter709)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17967,11 +18222,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype699, _size696) = iprot.readListBegin()
-          for _i700 in xrange(_size696):
-            _elem701 = Partition()
-            _elem701.read(iprot)
-            self.success.append(_elem701)
+          (_etype713, _size710) = iprot.readListBegin()
+          for _i714 in xrange(_size710):
+            _elem715 = Partition()
+            _elem715.read(iprot)
+            self.success.append(_elem715)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18000,8 +18255,8 @@ class get_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter702 in self.success:
-        iter702.write(oprot)
+      for iter716 in self.success:
+        iter716.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18095,10 +18350,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype706, _size703) = iprot.readListBegin()
-          for _i707 in xrange(_size703):
-            _elem708 = iprot.readString()
-            self.group_names.append(_elem708)
+          (_etype720, _size717) = iprot.readListBegin()
+          for _i721 in xrange(_size717):
+            _elem722 = iprot.readString()
+            self.group_names.append(_elem722)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18131,8 +18386,8 @@ class get_partitions_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter709 in self.group_names:
-        oprot.writeString(iter709)
+      for iter723 in self.group_names:
+        oprot.writeString(iter723)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18193,11 +18448,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype713, _size710) = iprot.readListBegin()
-          for _i714 in xrange(_size710):
-            _elem715 = Partition()
-            _elem715.read(iprot)
-            self.success.append(_elem715)
+          (_etype727, _size724) = iprot.readListBegin()
+          for _i728 in xrange(_size724):
+            _elem729 = Partition()
+            _elem729.read(iprot)
+            self.success.append(_elem729)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18226,8 +18481,8 @@ class get_partitions_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter716 in self.success:
-        iter716.write(oprot)
+      for iter730 in self.success:
+        iter730.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18385,11 +18640,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype720, _size717) = iprot.readListBegin()
-          for _i721 in xrange(_size717):
-            _elem722 = PartitionSpec()
-            _elem722.read(iprot)
-            self.success.append(_elem722)
+          (_etype734, _size731) = iprot.readListBegin()
+          for _i735 in xrange(_size731):
+            _elem736 = PartitionSpec()
+            _elem736.read(iprot)
+            self.success.append(_elem736)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18418,8 +18673,8 @@ class get_partitions_pspec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter723 in self.success:
-        iter723.write(oprot)
+      for iter737 in self.success:
+        iter737.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18574,10 +18829,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype727, _size724) = iprot.readListBegin()
-          for _i728 in xrange(_size724):
-            _elem729 = iprot.readString()
-            self.success.append(_elem729)
+          (_etype741, _size738) = iprot.readListBegin()
+          for _i742 in xrange(_size738):
+            _elem743 = iprot.readString()
+            self.success.append(_elem743)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18600,8 +18855,8 @@ class get_partition_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter730 in self.success:
-        oprot.writeString(iter730)
+      for iter744 in self.success:
+        oprot.writeString(iter744)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -18677,10 +18932,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype734, _size731) = iprot.readListBegin()
-          for _i735 in xrange(_size731):
-            _elem736 = iprot.readString()
-            self.part_vals.append(_elem736)
+          (_etype748, _size745) = iprot.readListBegin()
+          for _i749 in xrange(_size745):
+            _elem750 = iprot.readString()
+            self.part_vals.append(_elem750)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18710,8 +18965,8 @@ class get_partitions_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter737 in self.part_vals:
-        oprot.writeString(iter737)
+      for iter751 in self.part_vals:
+        oprot.writeString(iter751)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -18775,11 +19030,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype741, _size738) = iprot.readListBegin()
-          for _i742 in xrange(_size738):
-            _elem743 = Partition()
-            _elem743.read(iprot)
-            self.success.append(_elem743)
+          (_etype755, _size752) = iprot.readListBegin()
+          for _i756 in xrange(_size752):
+            _elem757 = Partition()
+            _elem757.read(iprot)
+            self.success.append(_elem757)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18808,8 +19063,8 @@ class get_partitions_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter744 in self.success:
-        iter744.write(oprot)
+      for iter758 in self.success:
+        iter758.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18896,10 +19151,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype748, _size745) = iprot.readListBegin()
-          for _i749 in xrange(_size745):
-            _elem750 = iprot.readString()
-            self.part_vals.append(_elem750)
+          (_etype762, _size759) = iprot.readListBegin()
+          for _i763 in xrange(_size759):
+            _elem764 = iprot.readString()
+            self.part_vals.append(_elem764)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18916,10 +19171,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype754, _size751) = iprot.readListBegin()
-          for _i755 in xrange(_size751):
-            _elem756 = iprot.readString()
-            self.group_names.append(_elem756)
+          (_etype768, _size765) = iprot.readListBegin()
+          for _i769 in xrange(_size765):
+            _elem770 = iprot.readString()
+            self.group_names.append(_elem770)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18944,8 +19199,8 @@ class get_partitions_ps_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter757 in self.part_vals:
-        oprot.writeString(iter757)
+      for iter771 in self.part_vals:
+        oprot.writeString(iter771)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -18959,8 +19214,8 @@ class get_partitions_ps_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter758 in self.group_names:
-        oprot.writeString(iter758)
+      for iter772 in self.group_names:
+        oprot.writeString(iter772)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19022,11 +19277,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype762, _size759) = iprot.readListBegin()
-          for _i763 in xrange(_size759):
-            _elem764 = Partition()
-            _elem764.read(iprot)
-            self.success.append(_elem764)
+          (_etype776, _size773) = iprot.readListBegin()
+          for _i777 in xrange(_size773):
+            _elem778 = Partition()
+            _elem778.read(iprot)
+            self.success.append(_elem778)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19055,8 +19310,8 @@ class get_partitions_ps_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter765 in self.success:
-        iter765.write(oprot)
+      for iter779 in self.success:
+        iter779.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19137,10 +19392,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype769, _size766) = iprot.readListBegin()
-          for _i770 in xrange(_size766):
-            _elem771 = iprot.readString()
-            self.part_vals.append(_elem771)
+          (_etype783, _size780) = iprot.readListBegin()
+          for _i784 in xrange(_size780):
+            _elem785 = iprot.readString()
+            self.part_vals.append(_elem785)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19170,8 +19425,8 @@ class get_partition_names_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter772 in self.part_vals:
-        oprot.writeString(iter772)
+      for iter786 in self.part_vals:
+        oprot.writeString(iter786)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -19235,10 +19490,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype776, _size773) = iprot.readListBegin()
-          for _i777 in xrange(_size773):
-            _elem778 = iprot.readString()
-            self.success.append(_elem778)
+          (_etype790, _size787) = iprot.readListBegin()
+          for _i791 in xrange(_size787):
+            _elem792 = iprot.readString()
+            self.success.append(_elem792)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19267,8 +19522,8 @@ class get_partition_names_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter779 in self.success:
-        oprot.writeString(iter779)
+      for iter793 in self.success:
+        oprot.writeString(iter793)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19439,11 +19694,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype783, _size780) = iprot.readListBegin()
-          for _i784 in xrange(_size780):
-            _elem785 = Partition()
-            _elem785.read(iprot)
-            self.success.append(_elem785)
+          (_etype797, _size794) = iprot.readListBegin()
+          for _i798 in xrange(_size794):
+            _elem799 = Partition()
+            _elem799.read(iprot)
+            self.success.append(_elem799)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19472,8 +19727,8 @@ class get_partitions_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter786 in self.success:
-        iter786.write(oprot)
+      for iter800 in self.success:
+        iter800.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19644,11 +19899,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype790, _size787) = iprot.readListBegin()
-          for _i791 in xrange(_size787):
-            _elem792 = PartitionSpec()
-            _elem792.read(iprot)
-            self.success.append(_elem792)
+          (_etype804, _size801) = iprot.readListBegin()
+          for _i805 in xrange(_size801):
+            _elem806 = PartitionSpec()
+            _elem806.read(iprot)
+            self.success.append(_elem806)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19677,8 +19932,8 @@ class get_part_specs_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter793 in self.success:
-        iter793.write(oprot)
+      for iter807 in self.success:
+        iter807.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19915,10 +20170,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype797, _size794) = iprot.readListBegin()
-          for _i798 in xrange(_size794):
-            _elem799 = iprot.readString()
-            self.names.append(_elem799)
+          (_etype811, _size808) = iprot.readListBegin()
+          for _i812 in xrange(_size808):
+            _elem813 = iprot.readString()
+            self.names.append(_elem813)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19943,8 +20198,8 @@ class get_partitions_by_names_args:
     if self.names is not None:
       oprot.writeFieldBegin('names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.names))
-      for iter800 in self.names:
-        oprot.writeString(iter800)
+      for iter814 in self.names:
+        oprot.writeString(iter814)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20003,11 +20258,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype804, _size801) = iprot.readListBegin()
-          for _i805 in xrange(_size801):
-            _elem806 = Partition()
-            _elem806.read(iprot)
-            self.success.append(_elem806)
+          (_etype818, _size815) = iprot.readListBegin()
+          for _i819 in xrange(_size815):
+            _elem820 = Partition()
+            _elem820.read(iprot)
+            self.success.append(_elem820)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20036,8 +20291,8 @@ class get_partitions_by_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter807 in self.success:
-        iter807.write(oprot)
+      for iter821 in self.success:
+        iter821.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20287,11 +20542,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype811, _size808) = iprot.readListBegin()
-          for _i812 in xrange(_size808):
-            _elem813 = Partition()
-            _elem813.read(iprot)
-            self.new_parts.append(_elem813)
+          (_etype825, _size822) = iprot.readListBegin()
+          for _i826 in xrange(_size822):
+            _elem827 = Partition()
+            _elem827.read(iprot)
+            self.new_parts.append(_elem827)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20316,8 +20571,8 @@ class alter_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter814 in self.new_parts:
-        iter814.write(oprot)
+      for iter828 in self.new_parts:
+        iter828.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20656,10 +20911,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype818, _size815) = iprot.readListBegin()
-          for _i819 in xrange(_size815):
-            _elem820 = iprot.readString()
-            self.part_vals.append(_elem820)
+          (_etype832, _size829) = iprot.readListBegin()
+          for _i833 in xrange(_size829):
+            _elem834 = iprot.readString()
+            self.part_vals.append(_elem834)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20690,8 +20945,8 @@ class rename_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter821 in self.part_vals:
-        oprot.writeString(iter821)
+      for iter835 in self.part_vals:
+        oprot.writeString(iter835)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -20833,10 +21088,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype825, _size822) = iprot.readListBegin()
-          for _i826 in xrange(_size822):
-            _elem827 = iprot.readString()
-            self.part_vals.append(_elem827)
+          (_etype839, _size836) = iprot.readListBegin()
+          for _i840 in xrange(_size836):
+            _elem841 = iprot.readString()
+            self.part_vals.append(_elem841)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20858,8 +21113,8 @@ class partition_name_has_valid_characters_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter828 in self.part_vals:
-        oprot.writeString(iter828)
+      for iter842 in self.part_vals:
+        oprot.writeString(iter842)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -21217,10 +21472,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype832, _size829) = iprot.readListBegin()
-          for _i833 in xrange(_size829):
-            _elem834 = iprot.readString()
-            self.success.append(_elem834)
+          (_etype846, _size843) = iprot.readListBegin()
+          for _i847 in xrange(_size843):
+            _elem848 = iprot.readString()
+            self.success.append(_elem848)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21243,8 +21498,8 @@ class partition_name_to_vals_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter835 in self.success:
-        oprot.writeString(iter835)
+      for iter849 in self.success:
+        oprot.writeString(iter849)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21368,11 +21623,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype837, _vtype838, _size836 ) = iprot.readMapBegin()
-          for _i840 in xrange(_size836):
-            _key841 = iprot.readString()
-            _val842 = iprot.readString()
-            self.success[_key841] = _val842
+          (_ktype851, _vtype852, _size850 ) = iprot.readMapBegin()
+          for _i854 in xrange(_size850):
+            _key855 = iprot.readString()
+            _val856 = iprot.readString()
+            self.success[_key855] = _val856
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21395,9 +21650,9 @@ class partition_name_to_spec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success))
-      for kiter843,viter844 in self.success.items():
-        oprot.writeString(kiter843)
-        oprot.writeString(viter844)
+      for kiter857,viter858 in self.success.items():
+        oprot.writeString(kiter857)
+        oprot.writeString(viter858)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21473,11 +21728,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype846, _vtype847, _size845 ) = iprot.readMapBegin()
-          for _i849 in xrange(_size845):
-            _key850 = iprot.readString()
-            _val851 = iprot.readString()
-            self.part_vals[_key850] = _val851
+          (_ktype860, _vtype861, _size859 ) = iprot.readMapBegin()
+          for _i863 in xrange(_size859):
+            _key864 = iprot.readString()
+            _val865 = iprot.readString()
+            self.part_vals[_key864] = _val865
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21507,9 +21762,9 @@ class markPartitionForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter852,viter853 in self.part_vals.items():
-        oprot.writeString(kiter852)
-        oprot.writeString(viter853)
+      for kiter866,viter867 in self.part_vals.items():
+        oprot.writeString(kiter866)
+        oprot.writeString(viter867)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -21723,11 +21978,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype855, _vtype856, _size854 ) = iprot.readMapBegin()
-          for _i858 in xrange(_size854):
-            _key859 = iprot.readString()
-            _val860 = iprot.readString()
-            self.part_vals[_key859] = _val860
+          (_ktype869, _vtype870, _size868 ) = iprot.readMapBegin()
+          for _i872 in xrange(_size868):
+            _key873 = iprot.readString()
+            _val874 = iprot.readString()
+            self.part_vals[_key873] = _val874
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21757,9 +22012,9 @@ class isPartitionMarkedForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter861,viter862 in self.part_vals.items():
-        oprot.writeString(kiter861)
-        oprot.writeString(viter862)
+      for kiter875,viter876 in self.part_vals.items():
+        oprot.writeString(kiter875)
+        oprot.writeString(viter876)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -22814,11 +23069,11 @@ class get_indexes_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype866, _size863) = iprot.readListBegin()
-          for _i867 in xrange(_size863):
-            _elem868 = Index()
-            _elem868.read(iprot)
-            self.success.append(_elem868)
+          (_etype880, _size877) = iprot.readListBegin()
+          for _i881 in xrange(_size877):
+            _elem882 = Index()
+            _elem882.read(iprot)
+            self.success.append(_elem882)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22847,8 +23102,8 @@ class get_indexes_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter869 in self.success:
-        iter869.write(oprot)
+      for iter883 in self.success:
+        iter883.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23003,10 +23258,10 @@ class get_index_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype873, _size870) = iprot.readListBegin()
-          for _i874 in xrange(_size870):
-            _elem875 = iprot.readString()
-            self.success.append(_elem875)
+          (_etype887, _size884) = iprot.readListBegin()
+          for _i888 in xrange(_size884):
+            _elem889 = iprot.readString()
+            self.success.append(_elem889)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23029,8 +23284,8 @@ class get_index_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter876 in self.success:
-        oprot.writeString(iter876)
+      for iter890 in self.success:
+        oprot.writeString(iter890)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -25578,10 +25833,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype880, _size877) = iprot.readListBegin()
-          for _i881 in xrange(_size877):
-            _elem882 = iprot.readString()
-            self.success.append(_elem882)
+          (_etype894, _size891) = iprot.readListBegin()
+          for _i895 in xrange(_size891):
+            _elem896 = iprot.readString()
+            self.success.append(_elem896)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25604,8 +25859,8 @@ class get_functions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter883 in self.success:
-        oprot.writeString(iter883)
+      for iter897 in self.success:
+        oprot.writeString(iter897)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26293,10 +26548,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype887, _size884) = iprot.readListBegin()
-          for _i888 in xrange(_size884):
-            _elem889 = iprot.readString()
-            self.success.append(_elem889)
+          (_etype901, _size898) = iprot.readListBegin()
+          for _i902 in xrange(_size898):
+            _elem903 = iprot.readString()
+            self.success.append(_elem903)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26319,8 +26574,8 @@ class get_role_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter890 in self.success:
-        oprot.writeString(iter890)
+      for iter904 in self.success:
+        oprot.writeString(iter904)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26834,11 +27089,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype894, _size891) = iprot.readListBegin()
-          for _i895 in xrange(_size891):
-            _elem896 = Role()
-            _elem896.read(iprot)
-            self.success.append(_elem896)
+          (_etype908, _size905) = iprot.readListBegin()
+          for _i909 in xrange(_size905):
+            _elem910 = Role()
+            _elem910.read(iprot)
+            self.success.append(_elem910)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26861,8 +27116,8 @@ class list_roles_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter897 in self.success:
-        iter897.write(oprot)
+      for iter911 in self.success:
+        iter911.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27371,10 +27626,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype901, _size898) = iprot.readListBegin()
-          for _i902 in xrange(_size898):
-            _elem903 = iprot.readString()
-            self.group_names.append(_elem903)
+          (_etype915, _size912) = iprot.readListBegin()
+          for _i916 in xrange(_size912):
+            _elem917 = iprot.readString()
+            self.group_names.append(_elem917)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27399,8 +27654,8 @@ class get_privilege_set_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter904 in self.group_names:
-        oprot.writeString(iter904)
+      for iter918 in self.group_names:
+        oprot.writeString(iter918)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -27627,11 +27882,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype908, _size905) = iprot.readListBegin()
-          for _i909 in xrange(_size905):
-            _elem910 = HiveObjectPrivilege()
-            _elem910.read(iprot)
-            self.success.append(_elem910)
+          (_etype922, _size919) = iprot.readListBegin()
+          for _i923 in xrange(_size919):
+            _elem924 = HiveObjectPrivilege()
+            _elem924.read(iprot)
+            self.success.append(_elem924)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27654,8 +27909,8 @@ class list_privileges_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter911 in self.success:
-        iter911.write(oprot)
+      for iter925 in self.success:
+        iter925.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28153,10 +28408,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype915, _size912) = iprot.readListBegin()
-          for _i916 in xrange(_size912):
-            _elem917 = iprot.readString()
-            self.group_names.append(_elem917)
+          (_etype929, _size926) = iprot.readListBegin()
+          for _i930 in xrange(_size926):
+            _elem931 = iprot.readString()
+            self.group_names.append(_elem931)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28177,8 +28432,8 @@ class set_ugi_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter918 in self.group_names:
-        oprot.writeString(iter918)
+      for iter932 in self.group_names:
+        oprot.writeString(iter932)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -28233,10 +28488,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype922, _size919) = iprot.readListBegin()
-          for _i923 in xrange(_size919):
-            _elem924 = iprot.readString()
-            self.success.append(_elem924)
+          (_etype936, _size933) = iprot.readListBegin()
+          for _i937 in xrange(_size933):
+            _elem938 = iprot.readString()
+            self.success.append(_elem938)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28259,8 +28514,8 @@ class set_ugi_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter925 in self.success:
-        oprot.writeString(iter925)
+      for iter939 in self.success:
+        oprot.writeString(iter939)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 8940dff..ba525ed 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -10554,6 +10554,116 @@ class GetAllFunctionsResponse:
   def __ne__(self, other):
     return not (self == other)
 
+class TableMeta:
+  """
+  Attributes:
+   - dbName
+   - tableName
+   - tableType
+   - comments
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'dbName', None, None, ), # 1
+    (2, TType.STRING, 'tableName', None, None, ), # 2
+    (3, TType.STRING, 'tableType', None, None, ), # 3
+    (4, TType.STRING, 'comments', None, None, ), # 4
+  )
+
+  def __init__(self, dbName=None, tableName=None, tableType=None, comments=None,):
+    self.dbName = dbName
+    self.tableName = tableName
+    self.tableType = tableType
+    self.comments = comments
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.dbName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.tableName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.tableType = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.comments = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('TableMeta')
+    if self.dbName is not None:
+      oprot.writeFieldBegin('dbName', TType.STRING, 1)
+      oprot.writeString(self.dbName)
+      oprot.writeFieldEnd()
+    if self.tableName is not None:
+      oprot.writeFieldBegin('tableName', TType.STRING, 2)
+      oprot.writeString(self.tableName)
+      oprot.writeFieldEnd()
+    if self.tableType is not None:
+      oprot.writeFieldBegin('tableType', TType.STRING, 3)
+      oprot.writeString(self.tableType)
+      oprot.writeFieldEnd()
+    if self.comments is not None:
+      oprot.writeFieldBegin('comments', TType.STRING, 4)
+      oprot.writeString(self.comments)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.dbName is None:
+      raise TProtocol.TProtocolException(message='Required field dbName is unset!')
+    if self.tableName is None:
+      raise TProtocol.TProtocolException(message='Required field tableName is unset!')
+    if self.tableType is None:
+      raise TProtocol.TProtocolException(message='Required field tableType is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.dbName)
+    value = (value * 31) ^ hash(self.tableName)
+    value = (value * 31) ^ hash(self.tableType)
+    value = (value * 31) ^ hash(self.comments)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class MetaException(TException):
   """
   Attributes:

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 08b9b06..f943f2d 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -2422,6 +2422,31 @@ class GetAllFunctionsResponse
   ::Thrift::Struct.generate_accessors self
 end
 
+class TableMeta
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  DBNAME = 1
+  TABLENAME = 2
+  TABLETYPE = 3
+  COMMENTS = 4
+
+  FIELDS = {
+    DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
+    TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'},
+    TABLETYPE => {:type => ::Thrift::Types::STRING, :name => 'tableType'},
+    COMMENTS => {:type => ::Thrift::Types::STRING, :name => 'comments', :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field dbName is unset!') unless @dbName
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tableName is unset!') unless @tableName
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tableType is unset!') unless @tableType
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class MetaException < ::Thrift::Exception
   include ::Thrift::Struct, ::Thrift::Struct_Union
   def initialize(message=nil)

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index c613e4b..5fe54b5 100644
--- a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -366,6 +366,22 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_tables failed: unknown result')
     end
 
+    def get_table_meta(db_patterns, tbl_patterns, tbl_types)
+      send_get_table_meta(db_patterns, tbl_patterns, tbl_types)
+      return recv_get_table_meta()
+    end
+
+    def send_get_table_meta(db_patterns, tbl_patterns, tbl_types)
+      send_message('get_table_meta', Get_table_meta_args, :db_patterns => db_patterns, :tbl_patterns => tbl_patterns, :tbl_types => tbl_types)
+    end
+
+    def recv_get_table_meta()
+      result = receive_message(Get_table_meta_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_table_meta failed: unknown result')
+    end
+
     def get_all_tables(db_name)
       send_get_all_tables(db_name)
       return recv_get_all_tables()
@@ -2471,6 +2487,17 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'get_tables', seqid)
     end
 
+    def process_get_table_meta(seqid, iprot, oprot)
+      args = read_args(iprot, Get_table_meta_args)
+      result = Get_table_meta_result.new()
+      begin
+        result.success = @handler.get_table_meta(args.db_patterns, args.tbl_patterns, args.tbl_types)
+      rescue ::MetaException => o1
+        result.o1 = o1
+      end
+      write_result(result, oprot, 'get_table_meta', seqid)
+    end
+
     def process_get_all_tables(seqid, iprot, oprot)
       args = read_args(iprot, Get_all_tables_args)
       result = Get_all_tables_result.new()
@@ -4613,6 +4640,44 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Get_table_meta_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    DB_PATTERNS = 1
+    TBL_PATTERNS = 2
+    TBL_TYPES = 3
+
+    FIELDS = {
+      DB_PATTERNS => {:type => ::Thrift::Types::STRING, :name => 'db_patterns'},
+      TBL_PATTERNS => {:type => ::Thrift::Types::STRING, :name => 'tbl_patterns'},
+      TBL_TYPES => {:type => ::Thrift::Types::LIST, :name => 'tbl_types', :element => {:type => ::Thrift::Types::STRING}}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_table_meta_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::LIST, :name => 'success', :element => {:type => ::Thrift::Types::STRUCT, :class => ::TableMeta}},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Get_all_tables_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     DB_NAME = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 2e9afaf..3c40d6e 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -110,7 +110,6 @@ import org.apache.thrift.transport.TTransportFactory;
 import javax.jdo.JDOException;
 
 import java.io.IOException;
-import java.lang.reflect.Field;
 import java.nio.ByteBuffer;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
@@ -1710,6 +1709,23 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       return t;
     }
 
+    @Override
+    public List<TableMeta> get_table_meta(String dbnames, String tblNames, List<String> tblTypes)
+        throws MetaException, NoSuchObjectException {
+      List<TableMeta> t = null;
+      startTableFunction("get_table_metas", dbnames, tblNames);
+      Exception ex = null;
+      try {
+        t = getMS().getTableMeta(dbnames, tblNames, tblTypes);
+      } catch (Exception e) {
+        ex = e;
+        throw newMetaException(e);
+      } finally {
+        endFunction("get_table_metas", t != null, ex);
+      }
+      return t;
+    }
+
     /**
      * Equivalent of get_table, but does not log audits and fire pre-event listener.
      * Meant to be used for calls made by other hive classes, that are not using the
@@ -5249,6 +5265,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     private static MetaException newMetaException(Exception e) {
+      if (e instanceof MetaException) {
+        return (MetaException)e;
+      }
       MetaException me = new MetaException(e.toString());
       me.initCause(e);
       return me;

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index f86ec45..c5e7a5f 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -131,6 +131,7 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.TableStatsRequest;
 import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore;
 import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
@@ -1314,6 +1315,37 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     return null;
   }
 
+  @Override
+  public List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)
+      throws MetaException {
+    try {
+      return filterNames(client.get_table_meta(dbPatterns, tablePatterns, tableTypes));
+    } catch (Exception e) {
+      MetaStoreUtils.logAndThrowMetaException(e);
+    }
+    return null;
+  }
+
+  private List<TableMeta> filterNames(List<TableMeta> metas) throws MetaException {
+    Map<String, TableMeta> sources = new LinkedHashMap<>();
+    Map<String, List<String>> dbTables = new LinkedHashMap<>();
+    for (TableMeta meta : metas) {
+      sources.put(meta.getDbName() + "." + meta.getTableName(), meta);
+      List<String> tables = dbTables.get(meta.getDbName());
+      if (tables == null) {
+        dbTables.put(meta.getDbName(), tables = new ArrayList<String>());
+      }
+      tables.add(meta.getTableName());
+    }
+    List<TableMeta> filtered = new ArrayList<>();
+    for (Map.Entry<String, List<String>> entry : dbTables.entrySet()) {
+      for (String table : filterHook.filterTableNames(entry.getKey(), entry.getValue())) {
+        filtered.add(sources.get(entry.getKey() + "." + table));
+      }
+    }
+    return filtered;
+  }
+
   /** {@inheritDoc} */
   @Override
   public List<String> getAllTables(String dbname) throws MetaException {

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 9279cf5..aa96f77 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 import org.apache.hadoop.hive.metastore.api.TxnOpenException;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
@@ -161,6 +162,12 @@ public interface IMetaStoreClient {
       throws MetaException, TException, UnknownDBException;
 
   /**
+   * For quick GetTablesOperation
+   */
+  List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)
+      throws MetaException, TException, UnknownDBException;
+
+  /**
    * Get the names of all tables in the specified database.
    * @param dbName
    * @return List of table names.

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 9f2f5f4..803c6e7 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -55,8 +55,6 @@ import javax.jdo.datastore.DataStoreCache;
 import javax.jdo.identity.IntIdentity;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.RecognitionException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configurable;
@@ -109,6 +107,7 @@ import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
@@ -143,12 +142,7 @@ import org.apache.hadoop.hive.metastore.model.MTablePrivilege;
 import org.apache.hadoop.hive.metastore.model.MType;
 import org.apache.hadoop.hive.metastore.model.MVersionTable;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree;
-import org.apache.hadoop.hive.metastore.parser.ExpressionTree.ANTLRNoCaseStringStream;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree.FilterBuilder;
-import org.apache.hadoop.hive.metastore.parser.ExpressionTree.LeafNode;
-import org.apache.hadoop.hive.metastore.parser.ExpressionTree.Operator;
-import org.apache.hadoop.hive.metastore.parser.FilterLexer;
-import org.apache.hadoop.hive.metastore.parser.FilterParser;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.shims.ShimLoader;
@@ -729,6 +723,9 @@ public class ObjectStore implements RawStore, Configurable {
 
   @Override
   public List<String> getDatabases(String pattern) throws MetaException {
+    if (pattern == null || pattern.equals("*")) {
+      return getAllDatabases();
+    }
     boolean commited = false;
     List<String> databases = null;
     Query query = null;
@@ -770,7 +767,28 @@ public class ObjectStore implements RawStore, Configurable {
 
   @Override
   public List<String> getAllDatabases() throws MetaException {
-    return getDatabases(".*");
+    boolean commited = false;
+    List<String> databases = null;
+
+    String queryStr = "select name from org.apache.hadoop.hive.metastore.model.MDatabase";
+    Query query = null;
+    
+    openTransaction();
+    try {
+      query = pm.newQuery(queryStr);
+      query.setResult("name");
+      databases = new ArrayList<String>((Collection<String>) query.execute());
+      commited = commitTransaction();
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+      if (query != null) {
+        query.closeAll();
+      }
+    }
+    Collections.sort(databases);
+    return databases;
   }
 
   private MType getMType(Type type) {
@@ -1050,6 +1068,84 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
+  public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
+      throws MetaException {
+
+    boolean commited = false;
+    Query query = null;
+    List<TableMeta> metas = new ArrayList<TableMeta>();
+    try {
+      openTransaction();
+      // Take the pattern and split it on the | to get all the composing
+      // patterns
+      StringBuilder builder = new StringBuilder();
+      if (dbNames != null && !dbNames.equals("*")) {
+        appendPatternCondition(builder, "database.name", dbNames);
+      }
+      if (tableNames != null && !tableNames.equals("*")) {
+        appendPatternCondition(builder, "tableName", tableNames);
+      }
+      if (tableTypes != null && !tableTypes.isEmpty()) {
+        appendSimpleCondition(builder, "tableType", tableTypes.toArray(new String[0]));
+      }
+
+      query = pm.newQuery(MTable.class, builder.toString());
+      Collection<MTable> tables = (Collection<MTable>) query.execute();
+      for (MTable table : tables) {
+        TableMeta metaData = new TableMeta(
+            table.getDatabase().getName(), table.getTableName(), table.getTableType());
+        metaData.setComments(table.getParameters().get("comment"));
+        metas.add(metaData);
+      }
+      commited = commitTransaction();
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+      if (query != null) {
+        query.closeAll();
+      }
+    }
+    return metas;
+  }
+
+  private StringBuilder appendPatternCondition(StringBuilder builder,
+      String fieldName, String elements) {
+      elements = HiveStringUtils.normalizeIdentifier(elements);
+    return appendCondition(builder, fieldName, elements.split("\\|"), true);
+  }
+
+  private StringBuilder appendSimpleCondition(StringBuilder builder,
+      String fieldName, String[] elements) {
+    return appendCondition(builder, fieldName, elements, false);
+  }
+
+  private StringBuilder appendCondition(StringBuilder builder,
+      String fieldName, String[] elements, boolean pattern) {
+    if (builder.length() > 0) {
+      builder.append(" && ");
+    }
+    builder.append(" (");
+    int length = builder.length();
+    for (String element : elements) {
+      if (pattern) {
+        element = "(?i)" + element.replaceAll("\\*", ".*");
+      }
+      if (builder.length() > length) {
+        builder.append(" || ");
+      }
+      builder.append(fieldName);
+      if (pattern) {
+        builder.append(".matches(\"").append(element).append("\")");
+      } else {
+        builder.append(" == \"").append(element).append("\"");
+      }
+    }
+    builder.append(" )");
+    return builder;
+  }
+
+  @Override
   public List<String> getAllTables(String dbName) throws MetaException {
     return getTables(dbName, ".*");
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
index 4aa17a5..5b36b03 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
@@ -151,6 +152,9 @@ public interface RawStore extends Configurable {
   public List<String> getTables(String dbName, String pattern)
       throws MetaException;
 
+  public List<TableMeta> getTableMeta(
+      String dbNames, String tableNames, List<String> tableTypes) throws MetaException;
+
   /**
    * @param dbname
    *        The name of the database from which to retrieve the tables

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index 5cc7c30..98e6c75 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -62,6 +62,7 @@ import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
@@ -71,7 +72,6 @@ import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.PlanResult;
 import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.ScanPlan;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hive.common.util.HiveStringUtils;
@@ -487,10 +487,7 @@ public class HBaseStore implements RawStore {
     boolean commit = false;
     openTransaction();
     try {
-      List<Table> tables = getHBase().scanTables(HiveStringUtils.normalizeIdentifier(dbName),
-          pattern==null?null:HiveStringUtils.normalizeIdentifier(likeToRegex(pattern)));
-      List<String> tableNames = new ArrayList<String>(tables.size());
-      for (Table table : tables) tableNames.add(table.getTableName());
+      List<String> tableNames = getTableNamesInTx(dbName, pattern);
       commit = true;
       return tableNames;
     } catch (IOException e) {
@@ -501,6 +498,41 @@ public class HBaseStore implements RawStore {
     }
   }
 
+  private List<String> getTableNamesInTx(String dbName, String pattern) throws IOException {
+    List<Table> tables = getHBase().scanTables(HiveStringUtils.normalizeIdentifier(dbName),
+        pattern==null?null:HiveStringUtils.normalizeIdentifier(likeToRegex(pattern)));
+    List<String> tableNames = new ArrayList<String>(tables.size());
+    for (Table table : tables) tableNames.add(table.getTableName());
+    return tableNames;
+  }
+
+  @Override
+  public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
+      throws MetaException {
+    boolean commit = false;
+    openTransaction();
+    try {
+      List<TableMeta> metas = new ArrayList<>();
+      for (String dbName : getDatabases(dbNames)) {
+        for (Table table : getTableObjectsByName(dbName, getTableNamesInTx(dbName, tableNames))) {
+          if (tableTypes == null || tableTypes.contains(table.getTableType())) {
+            TableMeta metaData = new TableMeta(
+              table.getDbName(), table.getTableName(), table.getTableType());
+            metaData.setComments(table.getParameters().get("comment"));
+            metas.add(metaData);
+          }
+        }
+      }
+      commit = true;
+      return metas;
+    } catch (Exception e) {
+      LOG.error("Unable to get tables ", e);
+      throw new MetaException("Unable to get tables, " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
+  }
+
   @Override
   public List<Table> getTableObjectsByName(String dbname, List<String> tableNames) throws
       MetaException, UnknownDBException {
@@ -1660,7 +1692,7 @@ public class HBaseStore implements RawStore {
     openTransaction();
     try {
       List<ColumnStatistics> cs =
-          getHBase().getPartitionStatistics(dbName, tblName, partNames,  partVals, colNames);
+          getHBase().getPartitionStatistics(dbName, tblName, partNames, partVals, colNames);
       commit = true;
       return cs;
     } catch (IOException e) {


[6/7] hive git commit: HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)

Posted by ai...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 6a80db7..0443f80 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -1240,14 +1240,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size714;
-            ::apache::thrift::protocol::TType _etype717;
-            xfer += iprot->readListBegin(_etype717, _size714);
-            this->success.resize(_size714);
-            uint32_t _i718;
-            for (_i718 = 0; _i718 < _size714; ++_i718)
+            uint32_t _size716;
+            ::apache::thrift::protocol::TType _etype719;
+            xfer += iprot->readListBegin(_etype719, _size716);
+            this->success.resize(_size716);
+            uint32_t _i720;
+            for (_i720 = 0; _i720 < _size716; ++_i720)
             {
-              xfer += iprot->readString(this->success[_i718]);
+              xfer += iprot->readString(this->success[_i720]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1286,10 +1286,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter719;
-      for (_iter719 = this->success.begin(); _iter719 != this->success.end(); ++_iter719)
+      std::vector<std::string> ::const_iterator _iter721;
+      for (_iter721 = this->success.begin(); _iter721 != this->success.end(); ++_iter721)
       {
-        xfer += oprot->writeString((*_iter719));
+        xfer += oprot->writeString((*_iter721));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1334,14 +1334,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size720;
-            ::apache::thrift::protocol::TType _etype723;
-            xfer += iprot->readListBegin(_etype723, _size720);
-            (*(this->success)).resize(_size720);
-            uint32_t _i724;
-            for (_i724 = 0; _i724 < _size720; ++_i724)
+            uint32_t _size722;
+            ::apache::thrift::protocol::TType _etype725;
+            xfer += iprot->readListBegin(_etype725, _size722);
+            (*(this->success)).resize(_size722);
+            uint32_t _i726;
+            for (_i726 = 0; _i726 < _size722; ++_i726)
             {
-              xfer += iprot->readString((*(this->success))[_i724]);
+              xfer += iprot->readString((*(this->success))[_i726]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1458,14 +1458,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size725;
-            ::apache::thrift::protocol::TType _etype728;
-            xfer += iprot->readListBegin(_etype728, _size725);
-            this->success.resize(_size725);
-            uint32_t _i729;
-            for (_i729 = 0; _i729 < _size725; ++_i729)
+            uint32_t _size727;
+            ::apache::thrift::protocol::TType _etype730;
+            xfer += iprot->readListBegin(_etype730, _size727);
+            this->success.resize(_size727);
+            uint32_t _i731;
+            for (_i731 = 0; _i731 < _size727; ++_i731)
             {
-              xfer += iprot->readString(this->success[_i729]);
+              xfer += iprot->readString(this->success[_i731]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1504,10 +1504,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter730;
-      for (_iter730 = this->success.begin(); _iter730 != this->success.end(); ++_iter730)
+      std::vector<std::string> ::const_iterator _iter732;
+      for (_iter732 = this->success.begin(); _iter732 != this->success.end(); ++_iter732)
       {
-        xfer += oprot->writeString((*_iter730));
+        xfer += oprot->writeString((*_iter732));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1552,14 +1552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size731;
-            ::apache::thrift::protocol::TType _etype734;
-            xfer += iprot->readListBegin(_etype734, _size731);
-            (*(this->success)).resize(_size731);
-            uint32_t _i735;
-            for (_i735 = 0; _i735 < _size731; ++_i735)
+            uint32_t _size733;
+            ::apache::thrift::protocol::TType _etype736;
+            xfer += iprot->readListBegin(_etype736, _size733);
+            (*(this->success)).resize(_size733);
+            uint32_t _i737;
+            for (_i737 = 0; _i737 < _size733; ++_i737)
             {
-              xfer += iprot->readString((*(this->success))[_i735]);
+              xfer += iprot->readString((*(this->success))[_i737]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2621,17 +2621,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size736;
-            ::apache::thrift::protocol::TType _ktype737;
-            ::apache::thrift::protocol::TType _vtype738;
-            xfer += iprot->readMapBegin(_ktype737, _vtype738, _size736);
-            uint32_t _i740;
-            for (_i740 = 0; _i740 < _size736; ++_i740)
+            uint32_t _size738;
+            ::apache::thrift::protocol::TType _ktype739;
+            ::apache::thrift::protocol::TType _vtype740;
+            xfer += iprot->readMapBegin(_ktype739, _vtype740, _size738);
+            uint32_t _i742;
+            for (_i742 = 0; _i742 < _size738; ++_i742)
             {
-              std::string _key741;
-              xfer += iprot->readString(_key741);
-              Type& _val742 = this->success[_key741];
-              xfer += _val742.read(iprot);
+              std::string _key743;
+              xfer += iprot->readString(_key743);
+              Type& _val744 = this->success[_key743];
+              xfer += _val744.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2670,11 +2670,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter743;
-      for (_iter743 = this->success.begin(); _iter743 != this->success.end(); ++_iter743)
+      std::map<std::string, Type> ::const_iterator _iter745;
+      for (_iter745 = this->success.begin(); _iter745 != this->success.end(); ++_iter745)
       {
-        xfer += oprot->writeString(_iter743->first);
-        xfer += _iter743->second.write(oprot);
+        xfer += oprot->writeString(_iter745->first);
+        xfer += _iter745->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -2719,17 +2719,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size744;
-            ::apache::thrift::protocol::TType _ktype745;
-            ::apache::thrift::protocol::TType _vtype746;
-            xfer += iprot->readMapBegin(_ktype745, _vtype746, _size744);
-            uint32_t _i748;
-            for (_i748 = 0; _i748 < _size744; ++_i748)
+            uint32_t _size746;
+            ::apache::thrift::protocol::TType _ktype747;
+            ::apache::thrift::protocol::TType _vtype748;
+            xfer += iprot->readMapBegin(_ktype747, _vtype748, _size746);
+            uint32_t _i750;
+            for (_i750 = 0; _i750 < _size746; ++_i750)
             {
-              std::string _key749;
-              xfer += iprot->readString(_key749);
-              Type& _val750 = (*(this->success))[_key749];
-              xfer += _val750.read(iprot);
+              std::string _key751;
+              xfer += iprot->readString(_key751);
+              Type& _val752 = (*(this->success))[_key751];
+              xfer += _val752.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2883,14 +2883,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size751;
-            ::apache::thrift::protocol::TType _etype754;
-            xfer += iprot->readListBegin(_etype754, _size751);
-            this->success.resize(_size751);
-            uint32_t _i755;
-            for (_i755 = 0; _i755 < _size751; ++_i755)
+            uint32_t _size753;
+            ::apache::thrift::protocol::TType _etype756;
+            xfer += iprot->readListBegin(_etype756, _size753);
+            this->success.resize(_size753);
+            uint32_t _i757;
+            for (_i757 = 0; _i757 < _size753; ++_i757)
             {
-              xfer += this->success[_i755].read(iprot);
+              xfer += this->success[_i757].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2945,10 +2945,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter756;
-      for (_iter756 = this->success.begin(); _iter756 != this->success.end(); ++_iter756)
+      std::vector<FieldSchema> ::const_iterator _iter758;
+      for (_iter758 = this->success.begin(); _iter758 != this->success.end(); ++_iter758)
       {
-        xfer += (*_iter756).write(oprot);
+        xfer += (*_iter758).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3001,14 +3001,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size757;
-            ::apache::thrift::protocol::TType _etype760;
-            xfer += iprot->readListBegin(_etype760, _size757);
-            (*(this->success)).resize(_size757);
-            uint32_t _i761;
-            for (_i761 = 0; _i761 < _size757; ++_i761)
+            uint32_t _size759;
+            ::apache::thrift::protocol::TType _etype762;
+            xfer += iprot->readListBegin(_etype762, _size759);
+            (*(this->success)).resize(_size759);
+            uint32_t _i763;
+            for (_i763 = 0; _i763 < _size759; ++_i763)
             {
-              xfer += (*(this->success))[_i761].read(iprot);
+              xfer += (*(this->success))[_i763].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3194,14 +3194,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size762;
-            ::apache::thrift::protocol::TType _etype765;
-            xfer += iprot->readListBegin(_etype765, _size762);
-            this->success.resize(_size762);
-            uint32_t _i766;
-            for (_i766 = 0; _i766 < _size762; ++_i766)
+            uint32_t _size764;
+            ::apache::thrift::protocol::TType _etype767;
+            xfer += iprot->readListBegin(_etype767, _size764);
+            this->success.resize(_size764);
+            uint32_t _i768;
+            for (_i768 = 0; _i768 < _size764; ++_i768)
             {
-              xfer += this->success[_i766].read(iprot);
+              xfer += this->success[_i768].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3256,10 +3256,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter767;
-      for (_iter767 = this->success.begin(); _iter767 != this->success.end(); ++_iter767)
+      std::vector<FieldSchema> ::const_iterator _iter769;
+      for (_iter769 = this->success.begin(); _iter769 != this->success.end(); ++_iter769)
       {
-        xfer += (*_iter767).write(oprot);
+        xfer += (*_iter769).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3312,14 +3312,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size768;
-            ::apache::thrift::protocol::TType _etype771;
-            xfer += iprot->readListBegin(_etype771, _size768);
-            (*(this->success)).resize(_size768);
-            uint32_t _i772;
-            for (_i772 = 0; _i772 < _size768; ++_i772)
+            uint32_t _size770;
+            ::apache::thrift::protocol::TType _etype773;
+            xfer += iprot->readListBegin(_etype773, _size770);
+            (*(this->success)).resize(_size770);
+            uint32_t _i774;
+            for (_i774 = 0; _i774 < _size770; ++_i774)
             {
-              xfer += (*(this->success))[_i772].read(iprot);
+              xfer += (*(this->success))[_i774].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3489,14 +3489,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size773;
-            ::apache::thrift::protocol::TType _etype776;
-            xfer += iprot->readListBegin(_etype776, _size773);
-            this->success.resize(_size773);
-            uint32_t _i777;
-            for (_i777 = 0; _i777 < _size773; ++_i777)
+            uint32_t _size775;
+            ::apache::thrift::protocol::TType _etype778;
+            xfer += iprot->readListBegin(_etype778, _size775);
+            this->success.resize(_size775);
+            uint32_t _i779;
+            for (_i779 = 0; _i779 < _size775; ++_i779)
             {
-              xfer += this->success[_i777].read(iprot);
+              xfer += this->success[_i779].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3551,10 +3551,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter778;
-      for (_iter778 = this->success.begin(); _iter778 != this->success.end(); ++_iter778)
+      std::vector<FieldSchema> ::const_iterator _iter780;
+      for (_iter780 = this->success.begin(); _iter780 != this->success.end(); ++_iter780)
       {
-        xfer += (*_iter778).write(oprot);
+        xfer += (*_iter780).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3607,14 +3607,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size779;
-            ::apache::thrift::protocol::TType _etype782;
-            xfer += iprot->readListBegin(_etype782, _size779);
-            (*(this->success)).resize(_size779);
-            uint32_t _i783;
-            for (_i783 = 0; _i783 < _size779; ++_i783)
+            uint32_t _size781;
+            ::apache::thrift::protocol::TType _etype784;
+            xfer += iprot->readListBegin(_etype784, _size781);
+            (*(this->success)).resize(_size781);
+            uint32_t _i785;
+            for (_i785 = 0; _i785 < _size781; ++_i785)
             {
-              xfer += (*(this->success))[_i783].read(iprot);
+              xfer += (*(this->success))[_i785].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3800,14 +3800,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size784;
-            ::apache::thrift::protocol::TType _etype787;
-            xfer += iprot->readListBegin(_etype787, _size784);
-            this->success.resize(_size784);
-            uint32_t _i788;
-            for (_i788 = 0; _i788 < _size784; ++_i788)
+            uint32_t _size786;
+            ::apache::thrift::protocol::TType _etype789;
+            xfer += iprot->readListBegin(_etype789, _size786);
+            this->success.resize(_size786);
+            uint32_t _i790;
+            for (_i790 = 0; _i790 < _size786; ++_i790)
             {
-              xfer += this->success[_i788].read(iprot);
+              xfer += this->success[_i790].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3862,10 +3862,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter789;
-      for (_iter789 = this->success.begin(); _iter789 != this->success.end(); ++_iter789)
+      std::vector<FieldSchema> ::const_iterator _iter791;
+      for (_iter791 = this->success.begin(); _iter791 != this->success.end(); ++_iter791)
       {
-        xfer += (*_iter789).write(oprot);
+        xfer += (*_iter791).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3918,14 +3918,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size790;
-            ::apache::thrift::protocol::TType _etype793;
-            xfer += iprot->readListBegin(_etype793, _size790);
-            (*(this->success)).resize(_size790);
-            uint32_t _i794;
-            for (_i794 = 0; _i794 < _size790; ++_i794)
+            uint32_t _size792;
+            ::apache::thrift::protocol::TType _etype795;
+            xfer += iprot->readListBegin(_etype795, _size792);
+            (*(this->success)).resize(_size792);
+            uint32_t _i796;
+            for (_i796 = 0; _i796 < _size792; ++_i796)
             {
-              xfer += (*(this->success))[_i794].read(iprot);
+              xfer += (*(this->success))[_i796].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5099,14 +5099,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size795;
-            ::apache::thrift::protocol::TType _etype798;
-            xfer += iprot->readListBegin(_etype798, _size795);
-            this->success.resize(_size795);
-            uint32_t _i799;
-            for (_i799 = 0; _i799 < _size795; ++_i799)
+            uint32_t _size797;
+            ::apache::thrift::protocol::TType _etype800;
+            xfer += iprot->readListBegin(_etype800, _size797);
+            this->success.resize(_size797);
+            uint32_t _i801;
+            for (_i801 = 0; _i801 < _size797; ++_i801)
             {
-              xfer += iprot->readString(this->success[_i799]);
+              xfer += iprot->readString(this->success[_i801]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5145,10 +5145,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter800;
-      for (_iter800 = this->success.begin(); _iter800 != this->success.end(); ++_iter800)
+      std::vector<std::string> ::const_iterator _iter802;
+      for (_iter802 = this->success.begin(); _iter802 != this->success.end(); ++_iter802)
       {
-        xfer += oprot->writeString((*_iter800));
+        xfer += oprot->writeString((*_iter802));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5193,14 +5193,313 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size801;
-            ::apache::thrift::protocol::TType _etype804;
-            xfer += iprot->readListBegin(_etype804, _size801);
-            (*(this->success)).resize(_size801);
-            uint32_t _i805;
-            for (_i805 = 0; _i805 < _size801; ++_i805)
+            uint32_t _size803;
+            ::apache::thrift::protocol::TType _etype806;
+            xfer += iprot->readListBegin(_etype806, _size803);
+            (*(this->success)).resize(_size803);
+            uint32_t _i807;
+            for (_i807 = 0; _i807 < _size803; ++_i807)
             {
-              xfer += iprot->readString((*(this->success))[_i805]);
+              xfer += iprot->readString((*(this->success))[_i807]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_table_meta_args::~ThriftHiveMetastore_get_table_meta_args() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->db_patterns);
+          this->__isset.db_patterns = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->tbl_patterns);
+          this->__isset.tbl_patterns = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->tbl_types.clear();
+            uint32_t _size808;
+            ::apache::thrift::protocol::TType _etype811;
+            xfer += iprot->readListBegin(_etype811, _size808);
+            this->tbl_types.resize(_size808);
+            uint32_t _i812;
+            for (_i812 = 0; _i812 < _size808; ++_i812)
+            {
+              xfer += iprot->readString(this->tbl_types[_i812]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.tbl_types = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_meta_args");
+
+  xfer += oprot->writeFieldBegin("db_patterns", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->db_patterns);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tbl_patterns", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->tbl_patterns);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_types.size()));
+    std::vector<std::string> ::const_iterator _iter813;
+    for (_iter813 = this->tbl_types.begin(); _iter813 != this->tbl_types.end(); ++_iter813)
+    {
+      xfer += oprot->writeString((*_iter813));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_table_meta_pargs::~ThriftHiveMetastore_get_table_meta_pargs() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_meta_pargs");
+
+  xfer += oprot->writeFieldBegin("db_patterns", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->db_patterns)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tbl_patterns", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString((*(this->tbl_patterns)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_types)).size()));
+    std::vector<std::string> ::const_iterator _iter814;
+    for (_iter814 = (*(this->tbl_types)).begin(); _iter814 != (*(this->tbl_types)).end(); ++_iter814)
+    {
+      xfer += oprot->writeString((*_iter814));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_table_meta_result::~ThriftHiveMetastore_get_table_meta_result() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 0:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->success.clear();
+            uint32_t _size815;
+            ::apache::thrift::protocol::TType _etype818;
+            xfer += iprot->readListBegin(_etype818, _size815);
+            this->success.resize(_size815);
+            uint32_t _i819;
+            for (_i819 = 0; _i819 < _size815; ++_i819)
+            {
+              xfer += this->success[_i819].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_meta_result");
+
+  if (this->__isset.success) {
+    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
+      std::vector<TableMeta> ::const_iterator _iter820;
+      for (_iter820 = this->success.begin(); _iter820 != this->success.end(); ++_iter820)
+      {
+        xfer += (*_iter820).write(oprot);
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o1) {
+    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
+    xfer += this->o1.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_table_meta_presult::~ThriftHiveMetastore_get_table_meta_presult() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 0:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            (*(this->success)).clear();
+            uint32_t _size821;
+            ::apache::thrift::protocol::TType _etype824;
+            xfer += iprot->readListBegin(_etype824, _size821);
+            (*(this->success)).resize(_size821);
+            uint32_t _i825;
+            for (_i825 = 0; _i825 < _size821; ++_i825)
+            {
+              xfer += (*(this->success))[_i825].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5338,14 +5637,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size806;
-            ::apache::thrift::protocol::TType _etype809;
-            xfer += iprot->readListBegin(_etype809, _size806);
-            this->success.resize(_size806);
-            uint32_t _i810;
-            for (_i810 = 0; _i810 < _size806; ++_i810)
+            uint32_t _size826;
+            ::apache::thrift::protocol::TType _etype829;
+            xfer += iprot->readListBegin(_etype829, _size826);
+            this->success.resize(_size826);
+            uint32_t _i830;
+            for (_i830 = 0; _i830 < _size826; ++_i830)
             {
-              xfer += iprot->readString(this->success[_i810]);
+              xfer += iprot->readString(this->success[_i830]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5384,10 +5683,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter811;
-      for (_iter811 = this->success.begin(); _iter811 != this->success.end(); ++_iter811)
+      std::vector<std::string> ::const_iterator _iter831;
+      for (_iter831 = this->success.begin(); _iter831 != this->success.end(); ++_iter831)
       {
-        xfer += oprot->writeString((*_iter811));
+        xfer += oprot->writeString((*_iter831));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5432,14 +5731,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size812;
-            ::apache::thrift::protocol::TType _etype815;
-            xfer += iprot->readListBegin(_etype815, _size812);
-            (*(this->success)).resize(_size812);
-            uint32_t _i816;
-            for (_i816 = 0; _i816 < _size812; ++_i816)
+            uint32_t _size832;
+            ::apache::thrift::protocol::TType _etype835;
+            xfer += iprot->readListBegin(_etype835, _size832);
+            (*(this->success)).resize(_size832);
+            uint32_t _i836;
+            for (_i836 = 0; _i836 < _size832; ++_i836)
             {
-              xfer += iprot->readString((*(this->success))[_i816]);
+              xfer += iprot->readString((*(this->success))[_i836]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5749,14 +6048,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size817;
-            ::apache::thrift::protocol::TType _etype820;
-            xfer += iprot->readListBegin(_etype820, _size817);
-            this->tbl_names.resize(_size817);
-            uint32_t _i821;
-            for (_i821 = 0; _i821 < _size817; ++_i821)
+            uint32_t _size837;
+            ::apache::thrift::protocol::TType _etype840;
+            xfer += iprot->readListBegin(_etype840, _size837);
+            this->tbl_names.resize(_size837);
+            uint32_t _i841;
+            for (_i841 = 0; _i841 < _size837; ++_i841)
             {
-              xfer += iprot->readString(this->tbl_names[_i821]);
+              xfer += iprot->readString(this->tbl_names[_i841]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5789,10 +6088,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter822;
-    for (_iter822 = this->tbl_names.begin(); _iter822 != this->tbl_names.end(); ++_iter822)
+    std::vector<std::string> ::const_iterator _iter842;
+    for (_iter842 = this->tbl_names.begin(); _iter842 != this->tbl_names.end(); ++_iter842)
     {
-      xfer += oprot->writeString((*_iter822));
+      xfer += oprot->writeString((*_iter842));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5820,10 +6119,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter823;
-    for (_iter823 = (*(this->tbl_names)).begin(); _iter823 != (*(this->tbl_names)).end(); ++_iter823)
+    std::vector<std::string> ::const_iterator _iter843;
+    for (_iter843 = (*(this->tbl_names)).begin(); _iter843 != (*(this->tbl_names)).end(); ++_iter843)
     {
-      xfer += oprot->writeString((*_iter823));
+      xfer += oprot->writeString((*_iter843));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5864,14 +6163,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size824;
-            ::apache::thrift::protocol::TType _etype827;
-            xfer += iprot->readListBegin(_etype827, _size824);
-            this->success.resize(_size824);
-            uint32_t _i828;
-            for (_i828 = 0; _i828 < _size824; ++_i828)
+            uint32_t _size844;
+            ::apache::thrift::protocol::TType _etype847;
+            xfer += iprot->readListBegin(_etype847, _size844);
+            this->success.resize(_size844);
+            uint32_t _i848;
+            for (_i848 = 0; _i848 < _size844; ++_i848)
             {
-              xfer += this->success[_i828].read(iprot);
+              xfer += this->success[_i848].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5926,10 +6225,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Table> ::const_iterator _iter829;
-      for (_iter829 = this->success.begin(); _iter829 != this->success.end(); ++_iter829)
+      std::vector<Table> ::const_iterator _iter849;
+      for (_iter849 = this->success.begin(); _iter849 != this->success.end(); ++_iter849)
       {
-        xfer += (*_iter829).write(oprot);
+        xfer += (*_iter849).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5982,14 +6281,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size830;
-            ::apache::thrift::protocol::TType _etype833;
-            xfer += iprot->readListBegin(_etype833, _size830);
-            (*(this->success)).resize(_size830);
-            uint32_t _i834;
-            for (_i834 = 0; _i834 < _size830; ++_i834)
+            uint32_t _size850;
+            ::apache::thrift::protocol::TType _etype853;
+            xfer += iprot->readListBegin(_etype853, _size850);
+            (*(this->success)).resize(_size850);
+            uint32_t _i854;
+            for (_i854 = 0; _i854 < _size850; ++_i854)
             {
-              xfer += (*(this->success))[_i834].read(iprot);
+              xfer += (*(this->success))[_i854].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6175,14 +6474,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size835;
-            ::apache::thrift::protocol::TType _etype838;
-            xfer += iprot->readListBegin(_etype838, _size835);
-            this->success.resize(_size835);
-            uint32_t _i839;
-            for (_i839 = 0; _i839 < _size835; ++_i839)
+            uint32_t _size855;
+            ::apache::thrift::protocol::TType _etype858;
+            xfer += iprot->readListBegin(_etype858, _size855);
+            this->success.resize(_size855);
+            uint32_t _i859;
+            for (_i859 = 0; _i859 < _size855; ++_i859)
             {
-              xfer += iprot->readString(this->success[_i839]);
+              xfer += iprot->readString(this->success[_i859]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6237,10 +6536,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter840;
-      for (_iter840 = this->success.begin(); _iter840 != this->success.end(); ++_iter840)
+      std::vector<std::string> ::const_iterator _iter860;
+      for (_iter860 = this->success.begin(); _iter860 != this->success.end(); ++_iter860)
       {
-        xfer += oprot->writeString((*_iter840));
+        xfer += oprot->writeString((*_iter860));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6293,14 +6592,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size841;
-            ::apache::thrift::protocol::TType _etype844;
-            xfer += iprot->readListBegin(_etype844, _size841);
-            (*(this->success)).resize(_size841);
-            uint32_t _i845;
-            for (_i845 = 0; _i845 < _size841; ++_i845)
+            uint32_t _size861;
+            ::apache::thrift::protocol::TType _etype864;
+            xfer += iprot->readListBegin(_etype864, _size861);
+            (*(this->success)).resize(_size861);
+            uint32_t _i865;
+            for (_i865 = 0; _i865 < _size861; ++_i865)
             {
-              xfer += iprot->readString((*(this->success))[_i845]);
+              xfer += iprot->readString((*(this->success))[_i865]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7634,14 +7933,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size846;
-            ::apache::thrift::protocol::TType _etype849;
-            xfer += iprot->readListBegin(_etype849, _size846);
-            this->new_parts.resize(_size846);
-            uint32_t _i850;
-            for (_i850 = 0; _i850 < _size846; ++_i850)
+            uint32_t _size866;
+            ::apache::thrift::protocol::TType _etype869;
+            xfer += iprot->readListBegin(_etype869, _size866);
+            this->new_parts.resize(_size866);
+            uint32_t _i870;
+            for (_i870 = 0; _i870 < _size866; ++_i870)
             {
-              xfer += this->new_parts[_i850].read(iprot);
+              xfer += this->new_parts[_i870].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7670,10 +7969,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter851;
-    for (_iter851 = this->new_parts.begin(); _iter851 != this->new_parts.end(); ++_iter851)
+    std::vector<Partition> ::const_iterator _iter871;
+    for (_iter871 = this->new_parts.begin(); _iter871 != this->new_parts.end(); ++_iter871)
     {
-      xfer += (*_iter851).write(oprot);
+      xfer += (*_iter871).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7697,10 +7996,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter852;
-    for (_iter852 = (*(this->new_parts)).begin(); _iter852 != (*(this->new_parts)).end(); ++_iter852)
+    std::vector<Partition> ::const_iterator _iter872;
+    for (_iter872 = (*(this->new_parts)).begin(); _iter872 != (*(this->new_parts)).end(); ++_iter872)
     {
-      xfer += (*_iter852).write(oprot);
+      xfer += (*_iter872).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7909,14 +8208,14 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size853;
-            ::apache::thrift::protocol::TType _etype856;
-            xfer += iprot->readListBegin(_etype856, _size853);
-            this->new_parts.resize(_size853);
-            uint32_t _i857;
-            for (_i857 = 0; _i857 < _size853; ++_i857)
+            uint32_t _size873;
+            ::apache::thrift::protocol::TType _etype876;
+            xfer += iprot->readListBegin(_etype876, _size873);
+            this->new_parts.resize(_size873);
+            uint32_t _i877;
+            for (_i877 = 0; _i877 < _size873; ++_i877)
             {
-              xfer += this->new_parts[_i857].read(iprot);
+              xfer += this->new_parts[_i877].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7945,10 +8244,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<PartitionSpec> ::const_iterator _iter858;
-    for (_iter858 = this->new_parts.begin(); _iter858 != this->new_parts.end(); ++_iter858)
+    std::vector<PartitionSpec> ::const_iterator _iter878;
+    for (_iter878 = this->new_parts.begin(); _iter878 != this->new_parts.end(); ++_iter878)
     {
-      xfer += (*_iter858).write(oprot);
+      xfer += (*_iter878).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7972,10 +8271,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift:
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<PartitionSpec> ::const_iterator _iter859;
-    for (_iter859 = (*(this->new_parts)).begin(); _iter859 != (*(this->new_parts)).end(); ++_iter859)
+    std::vector<PartitionSpec> ::const_iterator _iter879;
+    for (_iter879 = (*(this->new_parts)).begin(); _iter879 != (*(this->new_parts)).end(); ++_iter879)
     {
-      xfer += (*_iter859).write(oprot);
+      xfer += (*_iter879).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8200,14 +8499,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size860;
-            ::apache::thrift::protocol::TType _etype863;
-            xfer += iprot->readListBegin(_etype863, _size860);
-            this->part_vals.resize(_size860);
-            uint32_t _i864;
-            for (_i864 = 0; _i864 < _size860; ++_i864)
+            uint32_t _size880;
+            ::apache::thrift::protocol::TType _etype883;
+            xfer += iprot->readListBegin(_etype883, _size880);
+            this->part_vals.resize(_size880);
+            uint32_t _i884;
+            for (_i884 = 0; _i884 < _size880; ++_i884)
             {
-              xfer += iprot->readString(this->part_vals[_i864]);
+              xfer += iprot->readString(this->part_vals[_i884]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8244,10 +8543,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter865;
-    for (_iter865 = this->part_vals.begin(); _iter865 != this->part_vals.end(); ++_iter865)
+    std::vector<std::string> ::const_iterator _iter885;
+    for (_iter885 = this->part_vals.begin(); _iter885 != this->part_vals.end(); ++_iter885)
     {
-      xfer += oprot->writeString((*_iter865));
+      xfer += oprot->writeString((*_iter885));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8279,10 +8578,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter866;
-    for (_iter866 = (*(this->part_vals)).begin(); _iter866 != (*(this->part_vals)).end(); ++_iter866)
+    std::vector<std::string> ::const_iterator _iter886;
+    for (_iter886 = (*(this->part_vals)).begin(); _iter886 != (*(this->part_vals)).end(); ++_iter886)
     {
-      xfer += oprot->writeString((*_iter866));
+      xfer += oprot->writeString((*_iter886));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8754,14 +9053,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size867;
-            ::apache::thrift::protocol::TType _etype870;
-            xfer += iprot->readListBegin(_etype870, _size867);
-            this->part_vals.resize(_size867);
-            uint32_t _i871;
-            for (_i871 = 0; _i871 < _size867; ++_i871)
+            uint32_t _size887;
+            ::apache::thrift::protocol::TType _etype890;
+            xfer += iprot->readListBegin(_etype890, _size887);
+            this->part_vals.resize(_size887);
+            uint32_t _i891;
+            for (_i891 = 0; _i891 < _size887; ++_i891)
             {
-              xfer += iprot->readString(this->part_vals[_i871]);
+              xfer += iprot->readString(this->part_vals[_i891]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8806,10 +9105,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter872;
-    for (_iter872 = this->part_vals.begin(); _iter872 != this->part_vals.end(); ++_iter872)
+    std::vector<std::string> ::const_iterator _iter892;
+    for (_iter892 = this->part_vals.begin(); _iter892 != this->part_vals.end(); ++_iter892)
     {
-      xfer += oprot->writeString((*_iter872));
+      xfer += oprot->writeString((*_iter892));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8845,10 +9144,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::wr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter873;
-    for (_iter873 = (*(this->part_vals)).begin(); _iter873 != (*(this->part_vals)).end(); ++_iter873)
+    std::vector<std::string> ::const_iterator _iter893;
+    for (_iter893 = (*(this->part_vals)).begin(); _iter893 != (*(this->part_vals)).end(); ++_iter893)
     {
-      xfer += oprot->writeString((*_iter873));
+      xfer += oprot->writeString((*_iter893));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9651,14 +9950,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size874;
-            ::apache::thrift::protocol::TType _etype877;
-            xfer += iprot->readListBegin(_etype877, _size874);
-            this->part_vals.resize(_size874);
-            uint32_t _i878;
-            for (_i878 = 0; _i878 < _size874; ++_i878)
+            uint32_t _size894;
+            ::apache::thrift::protocol::TType _etype897;
+            xfer += iprot->readListBegin(_etype897, _size894);
+            this->part_vals.resize(_size894);
+            uint32_t _i898;
+            for (_i898 = 0; _i898 < _size894; ++_i898)
             {
-              xfer += iprot->readString(this->part_vals[_i878]);
+              xfer += iprot->readString(this->part_vals[_i898]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9703,10 +10002,10 @@ uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter879;
-    for (_iter879 = this->part_vals.begin(); _iter879 != this->part_vals.end(); ++_iter879)
+    std::vector<std::string> ::const_iterator _iter899;
+    for (_iter899 = this->part_vals.begin(); _iter899 != this->part_vals.end(); ++_iter899)
     {
-      xfer += oprot->writeString((*_iter879));
+      xfer += oprot->writeString((*_iter899));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9742,10 +10041,10 @@ uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter880;
-    for (_iter880 = (*(this->part_vals)).begin(); _iter880 != (*(this->part_vals)).end(); ++_iter880)
+    std::vector<std::string> ::const_iterator _iter900;
+    for (_iter900 = (*(this->part_vals)).begin(); _iter900 != (*(this->part_vals)).end(); ++_iter900)
     {
-      xfer += oprot->writeString((*_iter880));
+      xfer += oprot->writeString((*_iter900));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9954,14 +10253,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size881;
-            ::apache::thrift::protocol::TType _etype884;
-            xfer += iprot->readListBegin(_etype884, _size881);
-            this->part_vals.resize(_size881);
-            uint32_t _i885;
-            for (_i885 = 0; _i885 < _size881; ++_i885)
+            uint32_t _size901;
+            ::apache::thrift::protocol::TType _etype904;
+            xfer += iprot->readListBegin(_etype904, _size901);
+            this->part_vals.resize(_size901);
+            uint32_t _i905;
+            for (_i905 = 0; _i905 < _size901; ++_i905)
             {
-              xfer += iprot->readString(this->part_vals[_i885]);
+              xfer += iprot->readString(this->part_vals[_i905]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10014,10 +10313,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter886;
-    for (_iter886 = this->part_vals.begin(); _iter886 != this->part_vals.end(); ++_iter886)
+    std::vector<std::string> ::const_iterator _iter906;
+    for (_iter906 = this->part_vals.begin(); _iter906 != this->part_vals.end(); ++_iter906)
     {
-      xfer += oprot->writeString((*_iter886));
+      xfer += oprot->writeString((*_iter906));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10057,10 +10356,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::writ
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter887;
-    for (_iter887 = (*(this->part_vals)).begin(); _iter887 != (*(this->part_vals)).end(); ++_iter887)
+    std::vector<std::string> ::const_iterator _iter907;
+    for (_iter907 = (*(this->part_vals)).begin(); _iter907 != (*(this->part_vals)).end(); ++_iter907)
     {
-      xfer += oprot->writeString((*_iter887));
+      xfer += oprot->writeString((*_iter907));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11066,14 +11365,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size888;
-            ::apache::thrift::protocol::TType _etype891;
-            xfer += iprot->readListBegin(_etype891, _size888);
-            this->part_vals.resize(_size888);
-            uint32_t _i892;
-            for (_i892 = 0; _i892 < _size888; ++_i892)
+            uint32_t _size908;
+            ::apache::thrift::protocol::TType _etype911;
+            xfer += iprot->readListBegin(_etype911, _size908);
+            this->part_vals.resize(_size908);
+            uint32_t _i912;
+            for (_i912 = 0; _i912 < _size908; ++_i912)
             {
-              xfer += iprot->readString(this->part_vals[_i892]);
+              xfer += iprot->readString(this->part_vals[_i912]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11110,10 +11409,10 @@ uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protoco
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter893;
-    for (_iter893 = this->part_vals.begin(); _iter893 != this->part_vals.end(); ++_iter893)
+    std::vector<std::string> ::const_iterator _iter913;
+    for (_iter913 = this->part_vals.begin(); _iter913 != this->part_vals.end(); ++_iter913)
     {
-      xfer += oprot->writeString((*_iter893));
+      xfer += oprot->writeString((*_iter913));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11145,10 +11444,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter894;
-    for (_iter894 = (*(this->part_vals)).begin(); _iter894 != (*(this->part_vals)).end(); ++_iter894)
+    std::vector<std::string> ::const_iterator _iter914;
+    for (_iter914 = (*(this->part_vals)).begin(); _iter914 != (*(this->part_vals)).end(); ++_iter914)
     {
-      xfer += oprot->writeString((*_iter894));
+      xfer += oprot->writeString((*_iter914));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11337,17 +11636,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size895;
-            ::apache::thrift::protocol::TType _ktype896;
-            ::apache::thrift::protocol::TType _vtype897;
-            xfer += iprot->readMapBegin(_ktype896, _vtype897, _size895);
-            uint32_t _i899;
-            for (_i899 = 0; _i899 < _size895; ++_i899)
+            uint32_t _size915;
+            ::apache::thrift::protocol::TType _ktype916;
+            ::apache::thrift::protocol::TType _vtype917;
+            xfer += iprot->readMapBegin(_ktype916, _vtype917, _size915);
+            uint32_t _i919;
+            for (_i919 = 0; _i919 < _size915; ++_i919)
             {
-              std::string _key900;
-              xfer += iprot->readString(_key900);
-              std::string& _val901 = this->partitionSpecs[_key900];
-              xfer += iprot->readString(_val901);
+              std::string _key920;
+              xfer += iprot->readString(_key920);
+              std::string& _val921 = this->partitionSpecs[_key920];
+              xfer += iprot->readString(_val921);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11408,11 +11707,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter902;
-    for (_iter902 = this->partitionSpecs.begin(); _iter902 != this->partitionSpecs.end(); ++_iter902)
+    std::map<std::string, std::string> ::const_iterator _iter922;
+    for (_iter922 = this->partitionSpecs.begin(); _iter922 != this->partitionSpecs.end(); ++_iter922)
     {
-      xfer += oprot->writeString(_iter902->first);
-      xfer += oprot->writeString(_iter902->second);
+      xfer += oprot->writeString(_iter922->first);
+      xfer += oprot->writeString(_iter922->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11452,11 +11751,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter903;
-    for (_iter903 = (*(this->partitionSpecs)).begin(); _iter903 != (*(this->partitionSpecs)).end(); ++_iter903)
+    std::map<std::string, std::string> ::const_iterator _iter923;
+    for (_iter923 = (*(this->partitionSpecs)).begin(); _iter923 != (*(this->partitionSpecs)).end(); ++_iter923)
     {
-      xfer += oprot->writeString(_iter903->first);
-      xfer += oprot->writeString(_iter903->second);
+      xfer += oprot->writeString(_iter923->first);
+      xfer += oprot->writeString(_iter923->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11701,17 +12000,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size904;
-            ::apache::thrift::protocol::TType _ktype905;
-            ::apache::thrift::protocol::TType _vtype906;
-            xfer += iprot->readMapBegin(_ktype905, _vtype906, _size904);
-            uint32_t _i908;
-            for (_i908 = 0; _i908 < _size904; ++_i908)
+            uint32_t _size924;
+            ::apache::thrift::protocol::TType _ktype925;
+            ::apache::thrift::protocol::TType _vtype926;
+            xfer += iprot->readMapBegin(_ktype925, _vtype926, _size924);
+            uint32_t _i928;
+            for (_i928 = 0; _i928 < _size924; ++_i928)
             {
-              std::string _key909;
-              xfer += iprot->readString(_key909);
-              std::string& _val910 = this->partitionSpecs[_key909];
-              xfer += iprot->readString(_val910);
+              std::string _key929;
+              xfer += iprot->readString(_key929);
+              std::string& _val930 = this->partitionSpecs[_key929];
+              xfer += iprot->readString(_val930);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11772,11 +12071,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter911;
-    for (_iter911 = this->partitionSpecs.begin(); _iter911 != this->partitionSpecs.end(); ++_iter911)
+    std::map<std::string, std::string> ::const_iterator _iter931;
+    for (_iter931 = this->partitionSpecs.begin(); _iter931 != this->partitionSpecs.end(); ++_iter931)
     {
-      xfer += oprot->writeString(_iter911->first);
-      xfer += oprot->writeString(_iter911->second);
+      xfer += oprot->writeString(_iter931->first);
+      xfer += oprot->writeString(_iter931->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11816,11 +12115,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_pargs::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter912;
-    for (_iter912 = (*(this->partitionSpecs)).begin(); _iter912 != (*(this->partitionSpecs)).end(); ++_iter912)
+    std::map<std::string, std::string> ::const_iterator _iter932;
+    for (_iter932 = (*(this->partitionSpecs)).begin(); _iter932 != (*(this->partitionSpecs)).end(); ++_iter932)
     {
-      xfer += oprot->writeString(_iter912->first);
-      xfer += oprot->writeString(_iter912->second);
+      xfer += oprot->writeString(_iter932->first);
+      xfer += oprot->writeString(_iter932->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11877,14 +12176,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size913;
-            ::apache::thrift::protocol::TType _etype916;
-            xfer += iprot->readListBegin(_etype916, _size913);
-            this->success.resize(_size913);
-            uint32_t _i917;
-            for (_i917 = 0; _i917 < _size913; ++_i917)
+            uint32_t _size933;
+            ::apache::thrift::protocol::TType _etype936;
+            xfer += iprot->readListBegin(_etype936, _size933);
+            this->success.resize(_size933);
+            uint32_t _i937;
+            for (_i937 = 0; _i937 < _size933; ++_i937)
             {
-              xfer += this->success[_i917].read(iprot);
+              xfer += this->success[_i937].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11947,10 +12246,10 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter918;
-      for (_iter918 = this->success.begin(); _iter918 != this->success.end(); ++_iter918)
+      std::vector<Partition> ::const_iterator _iter938;
+      for (_iter938 = this->success.begin(); _iter938 != this->success.end(); ++_iter938)
       {
-        xfer += (*_iter918).write(oprot);
+        xfer += (*_iter938).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12007,14 +12306,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size919;
-            ::apache::thrift::protocol::TType _etype922;
-            xfer += iprot->readListBegin(_etype922, _size919);
-            (*(this->success)).resize(_size919);
-            uint32_t _i923;
-            for (_i923 = 0; _i923 < _size919; ++_i923)
+            uint32_t _size939;
+            ::apache::thrift::protocol::TType _etype942;
+            xfer += iprot->readListBegin(_etype942, _size939);
+            (*(this->success)).resize(_size939);
+            uint32_t _i943;
+            for (_i943 = 0; _i943 < _size939; ++_i943)
             {
-              xfer += (*(this->success))[_i923].read(iprot);
+              xfer += (*(this->success))[_i943].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12113,14 +12412,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size924;
-            ::apache::thrift::protocol::TType _etype927;
-            xfer += iprot->readListBegin(_etype927, _size924);
-            this->part_vals.resize(_size924);
-            uint32_t _i928;
-            for (_i928 = 0; _i928 < _size924; ++_i928)
+            uint32_t _size944;
+            ::apache::thrift::protocol::TType _etype947;
+            xfer += iprot->readListBegin(_etype947, _size944);
+            this->part_vals.resize(_size944);
+            uint32_t _i948;
+            for (_i948 = 0; _i948 < _size944; ++_i948)
             {
-              xfer += iprot->readString(this->part_vals[_i928]);
+              xfer += iprot->readString(this->part_vals[_i948]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12141,14 +12440,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size929;
-            ::apache::thrift::protocol::TType _etype932;
-            xfer += iprot->readListBegin(_etype932, _size929);
-            this->group_names.resize(_size929);
-            uint32_t _i933;
-            for (_i933 = 0; _i933 < _size929; ++_i933)
+            uint32_t _size949;
+            ::apache::thrift::protocol::TType _etype952;
+            xfer += iprot->readListBegin(_etype952, _size949);
+            this->group_names.resize(_size949);
+            uint32_t _i953;
+            for (_i953 = 0; _i953 < _size949; ++_i953)
             {
-              xfer += iprot->readString(this->group_names[_i933]);
+              xfer += iprot->readString(this->group_names[_i953]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12185,10 +12484,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter934;
-    for (_iter934 = this->part_vals.begin(); _iter934 != this->part_vals.end(); ++_iter934)
+    std::vector<std::string> ::const_iterator _iter954;
+    for (_iter954 = this->part_vals.begin(); _iter954 != this->part_vals.end(); ++_iter954)
     {
-      xfer += oprot->writeString((*_iter934));
+      xfer += oprot->writeString((*_iter954));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12201,10 +12500,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter935;
-    for (_iter935 = this->group_names.begin(); _iter935 != this->group_names.end(); ++_iter935)
+    std::vector<std::string> ::const_iterator _iter955;
+    for (_iter955 = this->group_names.begin(); _iter955 != this->group_names.end(); ++_iter955)
     {
-      xfer += oprot->writeString((*_iter935));
+      xfer += oprot->writeString((*_iter955));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12236,10 +12535,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter936;
-    for (_iter936 = (*(this->part_vals)).begin(); _iter936 != (*(this->part_vals)).end(); ++_iter936)
+    std::vector<std::string> ::const_iterator _iter956;
+    for (_iter956 = (*(this->part_vals)).begin(); _iter956 != (*(this->part_vals)).end(); ++_iter956)
     {
-      xfer += oprot->writeString((*_iter936));
+      xfer += oprot->writeString((*_iter956));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12252,10 +12551,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter937;
-    for (_iter937 = (*(this->group_names)).begin(); _iter937 != (*(this->group_names)).end(); ++_iter937)
+    std::vector<std::string> ::const_iterator _iter957;
+    for (_iter957 = (*(this->group_names)).begin(); _iter957 != (*(this->group_names)).end(); ++_iter957)
     {
-      xfer += oprot->writeString((*_iter937));
+      xfer += oprot->writeString((*_iter957));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12814,14 +13113,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size938;
-            ::apache::thrift::protocol::TType _etype941;
-            xfer += iprot->readListBegin(_etype941, _size938);
-            this->success.resize(_size938);
-            uint32_t _i942;
-            for (_i942 = 0; _i942 < _size938; ++_i942)
+            uint32_t _size958;
+            ::apache::thrift::protocol::TType _etype961;
+            xfer += iprot->readListBegin(_etype961, _size958);
+            this->success.resize(_size958);
+            uint32_t _i962;
+            for (_i962 = 0; _i962 < _size958; ++_i962)
             {
-              xfer += this->success[_i942].read(iprot);
+              xfer += this->success[_i962].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12868,10 +13167,10 @@ uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter943;
-      for (_iter943 = this->success.begin(); _iter943 != this->success.end(); ++_iter943)
+      std::vector<Partition> ::const_iterator _iter963;
+      for (_iter963 = this->success.begin(); _iter963 != this->success.end(); ++_iter963)
       {
-        xfer += (*_iter943).write(oprot);
+        xfer += (*_iter963).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12920,14 +13219,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size944;
-            ::apache::thrift::protocol::TType _etype947;
-            xfer += iprot->readListBegin(_etype947, _size944);
-            (*(this->success)).resize(_size944);
-            uint32_t _i948;
-            for (_i948 = 0; _i948 < _size944; ++_i948)
+            uint32_t _size964;
+            ::apache::thrift::protocol::TType _etype967;
+            xfer += iprot->readListBegin(_etype967, _size964);
+            (*(this->success)).resize(_size964);
+            uint32_t _i968;
+            for (_i968 = 0; _i968 < _size964; ++_i968)
             {
-              xfer += (*(this->success))[_i948].read(iprot);
+              xfer += (*(this->success))[_i968].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13026,14 +13325,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size949;
-            ::apache::thrift::protocol::TType _etype952;
-            xfer += iprot->readListBegin(_etype952, _size949);
-            this->group_names.resize(_size949);
-            uint32_t _i953;
-            for (_i953 = 0; _i953 < _size949; ++_i953)
+            uint32_t _size969;
+            ::apache::thrift::protocol::TType _etype972;
+            xfer += iprot->readListBegin(_etype972, _size969);
+            this->group_names.resize(_size969);
+            uint32_t _i973;
+            for (_i973 = 0; _i973 < _size969; ++_i973)
             {
-              xfer += iprot->readString(this->group_names[_i953]);
+              xfer += iprot->readString(this->group_names[_i973]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13078,10 +13377,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter954;
-    for (_iter954 = this->group_names.begin(); _iter954 != this->group_names.end(); ++_iter954)
+    std::vector<std::string> ::const_iterator _iter974;
+    for (_iter974 = this->group_names.begin(); _iter974 != this->group_names.end(); ++_iter974)
     {
-      xfer += oprot->writeString((*_iter954));
+      xfer += oprot->writeString((*_iter974));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13121,10 +13420,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter955;
-    for (_iter955 = (*(this->group_names)).begin(); _iter955 != (*(this->group_names)).end(); ++_iter955)
+    std::vector<std::string> ::const_iterator _iter975;
+    for (_iter975 = (*(this->group_names)).begin(); _iter975 != (*(this->group_names)).end(); ++_iter975)
     {
-      xfer += oprot->writeString((*_iter955));
+      xfer += oprot->writeString((*_iter975));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13165,14 +13464,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size956;
-            ::apache::thrift::protocol::TType _etype959;
-            xfer += iprot->readListBegin(_etype959, _size956);
-            this->success.resize(_size956);
-            uint32_t _i960;
-            for (_i960 = 0; _i960 < _size956; ++_i960)
+            uint32_t _size976;
+            ::apache::thrift::protocol::TType _etype979;
+            xfer += iprot->readListBegin(_etype979, _size976);
+            this->success.resize(_size976);
+            uint32_t _i980;
+            for (_i980 = 0; _i980 < _size976; ++_i980)
             {
-              xfer += this->success[_i960].read(iprot);
+              xfer += this->success[_i980].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13219,10 +13518,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter961;
-      for (_iter961 = this->success.begin(); _iter961 != this->success.end(); ++_iter961)
+      std::vector<Partition> ::const_iterator _iter981;
+      for (_iter981 = this->success.begin(); _iter981 != this->success.end(); ++_iter981)
       {
-        xfer += (*_iter961).write(oprot);
+        xfer += (*_iter981).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13271,14 +13570,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size962;
-            ::apache::thrift::protocol::TType _etype965;
-            xfer += iprot->readListBegin(_etype965, _size962);
-            (*(this->success)).resize(_size962);
-            uint32_t _i966;
-            for (_i966 = 0; _i966 < _size962; ++_i966)
+            uint32_t _size982;
+            ::apache::thrift::protocol::TType _etype985;
+            xfer += iprot->readListBegin(_etype985, _size982);
+            (*(this->success)).resize(_size982);
+            uint32_t _i986;
+            for (_i986 = 0; _i986 < _size982; ++_i986)
             {
-              xfer += (*(this->success))[_i966].read(iprot);
+              xfer += (*(this->success))[_i986].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13456,14 +13755,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size967;
-            ::apache::thrift::protocol::TType _etype970;
-            xfer += iprot->readListBegin(_etype970, _size967);
-            this->success.resize(_size967);
-            uint32_t _i971;
-            for (_i971 = 0; _i971 < _size967; ++_i971)
+            uint32_t _size987;
+            ::apache::thrift::protocol::TType _etype990;
+            xfer += iprot->readListBegin(_etype990, _size987);
+            this->success.resize(_size987);
+            uint32_t _i991;
+            for (_i991 = 0; _i991 < _size987; ++_i991)
             {
-              xfer += this->success[_i971].read(iprot);
+              xfer += this->success[_i991].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13510,10 +13809,10 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::write(::apache::thrift
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<PartitionSpec> ::const_iterator _iter972;
-      for (_iter972 = this->success.begin(); _iter972 != this->success.end(); ++_iter972)
+      std::vector<PartitionSpec> ::const_iterator _iter992;
+      for (_iter992 = this->success.begin(); _iter992 != this->success.end(); ++_iter992)
       {
-        xfer += (*_iter972).write(oprot);
+        xfer += (*_iter992).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13562,14 +13861,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size973;
-            ::apache::thrift::protocol::TType _etype976;
-            xfer += iprot->readListBegin(_etype976, _size973);
-            (*(this->success)).resize(_size973);
-            uint32_t _i977;
-            for (_i977 = 0; _i977 < _size973; ++_i977)
+            uint32_t _size993;
+            ::apache::thrift::protocol::TType _etype996;
+            xfer += iprot->readListBegin(_etype996, _size993);
+            (*(this->success)).resize(_size993);
+            uint32_t _i997;
+            for (_i997 = 0; _i997 < _size993; ++_i997)
             {
-              xfer += (*(this->success))[_i977].read(iprot);
+              xfer += (*(this->success))[_i997].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13747,14 +14046,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size978;
-            ::apache::thrift::protocol::TType _etype981;
-            xfer += iprot->readListBegin(_etype981, _size978);
-            this->success.resize(_size978);
-            uint32_t _i982;
-            for (_i982 = 0; _i982 < _size978; ++_i982)
+            uint32_t _size998;
+            ::apache::thrift::protocol::TType _etype1001;
+            xfer += iprot->readListBegin(_etype1001, _size998);
+            this->success.resize(_size998);
+            uint32_t _i1002;
+            for (_i1002 = 0; _i1002 < _size998; ++_i1002)
             {
-              xfer += iprot->readString(this->success[_i982]);
+              xfer += iprot->readString(this->success[_i1002]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13793,10 +14092,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter983;
-      for (_iter983 = this->success.begin(); _iter983 != this->success.end(); ++_iter983)
+      std::vector<std::string> ::const_iterator _iter1003;
+      for (_iter1003 = this->success.begin(); _iter1003 != this->success.end(); ++_iter1003)
       {
-        xfer += oprot->writeString((*_iter983));
+        xfer += oprot->writeString((*_iter1003));
       }
       xfer += oprot->writeListEnd();
     }
@@ -13841,14 +14140,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size984;
-            ::apache::thrift::protocol::TType _etype987;
-            xfer += iprot->readListBegin(_etype987, _size984);
-            (*(this->success)).resize(_size984);
-            uint32_t _i988;
-            for (_i988 = 0; _i988 < _size984; ++_i988)
+            uint32_t _size1004;
+            ::apache::thrift::protocol::TType _etype1007;
+            xfer += iprot->readListBegin(_etype1007, _size1004);
+            (*(this->success)).resize(_size1004);
+            uint32_t

<TRUNCATED>

[3/7] hive git commit: HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)

Posted by ai...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index a6862be..34c2205 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -181,6 +181,14 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function get_tables($db_name, $pattern);
   /**
+   * @param string $db_patterns
+   * @param string $tbl_patterns
+   * @param string[] $tbl_types
+   * @return \metastore\TableMeta[]
+   * @throws \metastore\MetaException
+   */
+  public function get_table_meta($db_patterns, $tbl_patterns, array $tbl_types);
+  /**
    * @param string $db_name
    * @return string[]
    * @throws \metastore\MetaException
@@ -2254,6 +2262,62 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("get_tables failed: unknown result");
   }
 
+  public function get_table_meta($db_patterns, $tbl_patterns, array $tbl_types)
+  {
+    $this->send_get_table_meta($db_patterns, $tbl_patterns, $tbl_types);
+    return $this->recv_get_table_meta();
+  }
+
+  public function send_get_table_meta($db_patterns, $tbl_patterns, array $tbl_types)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_table_meta_args();
+    $args->db_patterns = $db_patterns;
+    $args->tbl_patterns = $tbl_patterns;
+    $args->tbl_types = $tbl_types;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_table_meta', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_table_meta', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_table_meta()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_table_meta_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_get_table_meta_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    throw new \Exception("get_table_meta failed: unknown result");
+  }
+
   public function get_all_tables($db_name)
   {
     $this->send_get_all_tables($db_name);
@@ -13287,6 +13351,281 @@ class ThriftHiveMetastore_get_tables_result {
 
 }
 
+class ThriftHiveMetastore_get_table_meta_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $db_patterns = null;
+  /**
+   * @var string
+   */
+  public $tbl_patterns = null;
+  /**
+   * @var string[]
+   */
+  public $tbl_types = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'db_patterns',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'tbl_patterns',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'tbl_types',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['db_patterns'])) {
+        $this->db_patterns = $vals['db_patterns'];
+      }
+      if (isset($vals['tbl_patterns'])) {
+        $this->tbl_patterns = $vals['tbl_patterns'];
+      }
+      if (isset($vals['tbl_types'])) {
+        $this->tbl_types = $vals['tbl_types'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_table_meta_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->db_patterns);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tbl_patterns);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::LST) {
+            $this->tbl_types = array();
+            $_size583 = 0;
+            $_etype586 = 0;
+            $xfer += $input->readListBegin($_etype586, $_size583);
+            for ($_i587 = 0; $_i587 < $_size583; ++$_i587)
+            {
+              $elem588 = null;
+              $xfer += $input->readString($elem588);
+              $this->tbl_types []= $elem588;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_meta_args');
+    if ($this->db_patterns !== null) {
+      $xfer += $output->writeFieldBegin('db_patterns', TType::STRING, 1);
+      $xfer += $output->writeString($this->db_patterns);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tbl_patterns !== null) {
+      $xfer += $output->writeFieldBegin('tbl_patterns', TType::STRING, 2);
+      $xfer += $output->writeString($this->tbl_patterns);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tbl_types !== null) {
+      if (!is_array($this->tbl_types)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('tbl_types', TType::LST, 3);
+      {
+        $output->writeListBegin(TType::STRING, count($this->tbl_types));
+        {
+          foreach ($this->tbl_types as $iter589)
+          {
+            $xfer += $output->writeString($iter589);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_table_meta_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\TableMeta[]
+   */
+  public $success = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o1 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\TableMeta',
+            ),
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_table_meta_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::LST) {
+            $this->success = array();
+            $_size590 = 0;
+            $_etype593 = 0;
+            $xfer += $input->readListBegin($_etype593, $_size590);
+            for ($_i594 = 0; $_i594 < $_size590; ++$_i594)
+            {
+              $elem595 = null;
+              $elem595 = new \metastore\TableMeta();
+              $xfer += $elem595->read($input);
+              $this->success []= $elem595;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\MetaException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_meta_result');
+    if ($this->success !== null) {
+      if (!is_array($this->success)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('success', TType::LST, 0);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->success));
+        {
+          foreach ($this->success as $iter596)
+          {
+            $xfer += $iter596->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class ThriftHiveMetastore_get_all_tables_args {
   static $_TSPEC;
 
@@ -13424,14 +13763,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size583 = 0;
-            $_etype586 = 0;
-            $xfer += $input->readListBegin($_etype586, $_size583);
-            for ($_i587 = 0; $_i587 < $_size583; ++$_i587)
+            $_size597 = 0;
+            $_etype600 = 0;
+            $xfer += $input->readListBegin($_etype600, $_size597);
+            for ($_i601 = 0; $_i601 < $_size597; ++$_i601)
             {
-              $elem588 = null;
-              $xfer += $input->readString($elem588);
-              $this->success []= $elem588;
+              $elem602 = null;
+              $xfer += $input->readString($elem602);
+              $this->success []= $elem602;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13467,9 +13806,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter589)
+          foreach ($this->success as $iter603)
           {
-            $xfer += $output->writeString($iter589);
+            $xfer += $output->writeString($iter603);
           }
         }
         $output->writeListEnd();
@@ -13784,14 +14123,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size590 = 0;
-            $_etype593 = 0;
-            $xfer += $input->readListBegin($_etype593, $_size590);
-            for ($_i594 = 0; $_i594 < $_size590; ++$_i594)
+            $_size604 = 0;
+            $_etype607 = 0;
+            $xfer += $input->readListBegin($_etype607, $_size604);
+            for ($_i608 = 0; $_i608 < $_size604; ++$_i608)
             {
-              $elem595 = null;
-              $xfer += $input->readString($elem595);
-              $this->tbl_names []= $elem595;
+              $elem609 = null;
+              $xfer += $input->readString($elem609);
+              $this->tbl_names []= $elem609;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13824,9 +14163,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter596)
+          foreach ($this->tbl_names as $iter610)
           {
-            $xfer += $output->writeString($iter596);
+            $xfer += $output->writeString($iter610);
           }
         }
         $output->writeListEnd();
@@ -13927,15 +14266,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size597 = 0;
-            $_etype600 = 0;
-            $xfer += $input->readListBegin($_etype600, $_size597);
-            for ($_i601 = 0; $_i601 < $_size597; ++$_i601)
+            $_size611 = 0;
+            $_etype614 = 0;
+            $xfer += $input->readListBegin($_etype614, $_size611);
+            for ($_i615 = 0; $_i615 < $_size611; ++$_i615)
             {
-              $elem602 = null;
-              $elem602 = new \metastore\Table();
-              $xfer += $elem602->read($input);
-              $this->success []= $elem602;
+              $elem616 = null;
+              $elem616 = new \metastore\Table();
+              $xfer += $elem616->read($input);
+              $this->success []= $elem616;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13987,9 +14326,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter603)
+          foreach ($this->success as $iter617)
           {
-            $xfer += $iter603->write($output);
+            $xfer += $iter617->write($output);
           }
         }
         $output->writeListEnd();
@@ -14225,14 +14564,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size604 = 0;
-            $_etype607 = 0;
-            $xfer += $input->readListBegin($_etype607, $_size604);
-            for ($_i608 = 0; $_i608 < $_size604; ++$_i608)
+            $_size618 = 0;
+            $_etype621 = 0;
+            $xfer += $input->readListBegin($_etype621, $_size618);
+            for ($_i622 = 0; $_i622 < $_size618; ++$_i622)
             {
-              $elem609 = null;
-              $xfer += $input->readString($elem609);
-              $this->success []= $elem609;
+              $elem623 = null;
+              $xfer += $input->readString($elem623);
+              $this->success []= $elem623;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14284,9 +14623,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter610)
+          foreach ($this->success as $iter624)
           {
-            $xfer += $output->writeString($iter610);
+            $xfer += $output->writeString($iter624);
           }
         }
         $output->writeListEnd();
@@ -15599,15 +15938,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size611 = 0;
-            $_etype614 = 0;
-            $xfer += $input->readListBegin($_etype614, $_size611);
-            for ($_i615 = 0; $_i615 < $_size611; ++$_i615)
+            $_size625 = 0;
+            $_etype628 = 0;
+            $xfer += $input->readListBegin($_etype628, $_size625);
+            for ($_i629 = 0; $_i629 < $_size625; ++$_i629)
             {
-              $elem616 = null;
-              $elem616 = new \metastore\Partition();
-              $xfer += $elem616->read($input);
-              $this->new_parts []= $elem616;
+              $elem630 = null;
+              $elem630 = new \metastore\Partition();
+              $xfer += $elem630->read($input);
+              $this->new_parts []= $elem630;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15635,9 +15974,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter617)
+          foreach ($this->new_parts as $iter631)
           {
-            $xfer += $iter617->write($output);
+            $xfer += $iter631->write($output);
           }
         }
         $output->writeListEnd();
@@ -15852,15 +16191,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size618 = 0;
-            $_etype621 = 0;
-            $xfer += $input->readListBegin($_etype621, $_size618);
-            for ($_i622 = 0; $_i622 < $_size618; ++$_i622)
+            $_size632 = 0;
+            $_etype635 = 0;
+            $xfer += $input->readListBegin($_etype635, $_size632);
+            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
             {
-              $elem623 = null;
-              $elem623 = new \metastore\PartitionSpec();
-              $xfer += $elem623->read($input);
-              $this->new_parts []= $elem623;
+              $elem637 = null;
+              $elem637 = new \metastore\PartitionSpec();
+              $xfer += $elem637->read($input);
+              $this->new_parts []= $elem637;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15888,9 +16227,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter624)
+          foreach ($this->new_parts as $iter638)
           {
-            $xfer += $iter624->write($output);
+            $xfer += $iter638->write($output);
           }
         }
         $output->writeListEnd();
@@ -16140,14 +16479,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size625 = 0;
-            $_etype628 = 0;
-            $xfer += $input->readListBegin($_etype628, $_size625);
-            for ($_i629 = 0; $_i629 < $_size625; ++$_i629)
+            $_size639 = 0;
+            $_etype642 = 0;
+            $xfer += $input->readListBegin($_etype642, $_size639);
+            for ($_i643 = 0; $_i643 < $_size639; ++$_i643)
             {
-              $elem630 = null;
-              $xfer += $input->readString($elem630);
-              $this->part_vals []= $elem630;
+              $elem644 = null;
+              $xfer += $input->readString($elem644);
+              $this->part_vals []= $elem644;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16185,9 +16524,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter631)
+          foreach ($this->part_vals as $iter645)
           {
-            $xfer += $output->writeString($iter631);
+            $xfer += $output->writeString($iter645);
           }
         }
         $output->writeListEnd();
@@ -16689,14 +17028,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size632 = 0;
-            $_etype635 = 0;
-            $xfer += $input->readListBegin($_etype635, $_size632);
-            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
+            $_size646 = 0;
+            $_etype649 = 0;
+            $xfer += $input->readListBegin($_etype649, $_size646);
+            for ($_i650 = 0; $_i650 < $_size646; ++$_i650)
             {
-              $elem637 = null;
-              $xfer += $input->readString($elem637);
-              $this->part_vals []= $elem637;
+              $elem651 = null;
+              $xfer += $input->readString($elem651);
+              $this->part_vals []= $elem651;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16742,9 +17081,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter638)
+          foreach ($this->part_vals as $iter652)
           {
-            $xfer += $output->writeString($iter638);
+            $xfer += $output->writeString($iter652);
           }
         }
         $output->writeListEnd();
@@ -17598,14 +17937,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size639 = 0;
-            $_etype642 = 0;
-            $xfer += $input->readListBegin($_etype642, $_size639);
-            for ($_i643 = 0; $_i643 < $_size639; ++$_i643)
+            $_size653 = 0;
+            $_etype656 = 0;
+            $xfer += $input->readListBegin($_etype656, $_size653);
+            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
             {
-              $elem644 = null;
-              $xfer += $input->readString($elem644);
-              $this->part_vals []= $elem644;
+              $elem658 = null;
+              $xfer += $input->readString($elem658);
+              $this->part_vals []= $elem658;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17650,9 +17989,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter645)
+          foreach ($this->part_vals as $iter659)
           {
-            $xfer += $output->writeString($iter645);
+            $xfer += $output->writeString($iter659);
           }
         }
         $output->writeListEnd();
@@ -17905,14 +18244,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size646 = 0;
-            $_etype649 = 0;
-            $xfer += $input->readListBegin($_etype649, $_size646);
-            for ($_i650 = 0; $_i650 < $_size646; ++$_i650)
+            $_size660 = 0;
+            $_etype663 = 0;
+            $xfer += $input->readListBegin($_etype663, $_size660);
+            for ($_i664 = 0; $_i664 < $_size660; ++$_i664)
             {
-              $elem651 = null;
-              $xfer += $input->readString($elem651);
-              $this->part_vals []= $elem651;
+              $elem665 = null;
+              $xfer += $input->readString($elem665);
+              $this->part_vals []= $elem665;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17965,9 +18304,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter652)
+          foreach ($this->part_vals as $iter666)
           {
-            $xfer += $output->writeString($iter652);
+            $xfer += $output->writeString($iter666);
           }
         }
         $output->writeListEnd();
@@ -18981,14 +19320,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size653 = 0;
-            $_etype656 = 0;
-            $xfer += $input->readListBegin($_etype656, $_size653);
-            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
+            $_size667 = 0;
+            $_etype670 = 0;
+            $xfer += $input->readListBegin($_etype670, $_size667);
+            for ($_i671 = 0; $_i671 < $_size667; ++$_i671)
             {
-              $elem658 = null;
-              $xfer += $input->readString($elem658);
-              $this->part_vals []= $elem658;
+              $elem672 = null;
+              $xfer += $input->readString($elem672);
+              $this->part_vals []= $elem672;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19026,9 +19365,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter659)
+          foreach ($this->part_vals as $iter673)
           {
-            $xfer += $output->writeString($iter659);
+            $xfer += $output->writeString($iter673);
           }
         }
         $output->writeListEnd();
@@ -19270,17 +19609,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size660 = 0;
-            $_ktype661 = 0;
-            $_vtype662 = 0;
-            $xfer += $input->readMapBegin($_ktype661, $_vtype662, $_size660);
-            for ($_i664 = 0; $_i664 < $_size660; ++$_i664)
+            $_size674 = 0;
+            $_ktype675 = 0;
+            $_vtype676 = 0;
+            $xfer += $input->readMapBegin($_ktype675, $_vtype676, $_size674);
+            for ($_i678 = 0; $_i678 < $_size674; ++$_i678)
             {
-              $key665 = '';
-              $val666 = '';
-              $xfer += $input->readString($key665);
-              $xfer += $input->readString($val666);
-              $this->partitionSpecs[$key665] = $val666;
+              $key679 = '';
+              $val680 = '';
+              $xfer += $input->readString($key679);
+              $xfer += $input->readString($val680);
+              $this->partitionSpecs[$key679] = $val680;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19336,10 +19675,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter667 => $viter668)
+          foreach ($this->partitionSpecs as $kiter681 => $viter682)
           {
-            $xfer += $output->writeString($kiter667);
-            $xfer += $output->writeString($viter668);
+            $xfer += $output->writeString($kiter681);
+            $xfer += $output->writeString($viter682);
           }
         }
         $output->writeMapEnd();
@@ -19651,17 +19990,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size669 = 0;
-            $_ktype670 = 0;
-            $_vtype671 = 0;
-            $xfer += $input->readMapBegin($_ktype670, $_vtype671, $_size669);
-            for ($_i673 = 0; $_i673 < $_size669; ++$_i673)
+            $_size683 = 0;
+            $_ktype684 = 0;
+            $_vtype685 = 0;
+            $xfer += $input->readMapBegin($_ktype684, $_vtype685, $_size683);
+            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
             {
-              $key674 = '';
-              $val675 = '';
-              $xfer += $input->readString($key674);
-              $xfer += $input->readString($val675);
-              $this->partitionSpecs[$key674] = $val675;
+              $key688 = '';
+              $val689 = '';
+              $xfer += $input->readString($key688);
+              $xfer += $input->readString($val689);
+              $this->partitionSpecs[$key688] = $val689;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19717,10 +20056,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter676 => $viter677)
+          foreach ($this->partitionSpecs as $kiter690 => $viter691)
           {
-            $xfer += $output->writeString($kiter676);
-            $xfer += $output->writeString($viter677);
+            $xfer += $output->writeString($kiter690);
+            $xfer += $output->writeString($viter691);
           }
         }
         $output->writeMapEnd();
@@ -19853,15 +20192,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size678 = 0;
-            $_etype681 = 0;
-            $xfer += $input->readListBegin($_etype681, $_size678);
-            for ($_i682 = 0; $_i682 < $_size678; ++$_i682)
+            $_size692 = 0;
+            $_etype695 = 0;
+            $xfer += $input->readListBegin($_etype695, $_size692);
+            for ($_i696 = 0; $_i696 < $_size692; ++$_i696)
             {
-              $elem683 = null;
-              $elem683 = new \metastore\Partition();
-              $xfer += $elem683->read($input);
-              $this->success []= $elem683;
+              $elem697 = null;
+              $elem697 = new \metastore\Partition();
+              $xfer += $elem697->read($input);
+              $this->success []= $elem697;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19921,9 +20260,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter684)
+          foreach ($this->success as $iter698)
           {
-            $xfer += $iter684->write($output);
+            $xfer += $iter698->write($output);
           }
         }
         $output->writeListEnd();
@@ -20069,14 +20408,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size685 = 0;
-            $_etype688 = 0;
-            $xfer += $input->readListBegin($_etype688, $_size685);
-            for ($_i689 = 0; $_i689 < $_size685; ++$_i689)
+            $_size699 = 0;
+            $_etype702 = 0;
+            $xfer += $input->readListBegin($_etype702, $_size699);
+            for ($_i703 = 0; $_i703 < $_size699; ++$_i703)
             {
-              $elem690 = null;
-              $xfer += $input->readString($elem690);
-              $this->part_vals []= $elem690;
+              $elem704 = null;
+              $xfer += $input->readString($elem704);
+              $this->part_vals []= $elem704;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20093,14 +20432,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size691 = 0;
-            $_etype694 = 0;
-            $xfer += $input->readListBegin($_etype694, $_size691);
-            for ($_i695 = 0; $_i695 < $_size691; ++$_i695)
+            $_size705 = 0;
+            $_etype708 = 0;
+            $xfer += $input->readListBegin($_etype708, $_size705);
+            for ($_i709 = 0; $_i709 < $_size705; ++$_i709)
             {
-              $elem696 = null;
-              $xfer += $input->readString($elem696);
-              $this->group_names []= $elem696;
+              $elem710 = null;
+              $xfer += $input->readString($elem710);
+              $this->group_names []= $elem710;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20138,9 +20477,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter697)
+          foreach ($this->part_vals as $iter711)
           {
-            $xfer += $output->writeString($iter697);
+            $xfer += $output->writeString($iter711);
           }
         }
         $output->writeListEnd();
@@ -20160,9 +20499,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter698)
+          foreach ($this->group_names as $iter712)
           {
-            $xfer += $output->writeString($iter698);
+            $xfer += $output->writeString($iter712);
           }
         }
         $output->writeListEnd();
@@ -20753,15 +21092,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size699 = 0;
-            $_etype702 = 0;
-            $xfer += $input->readListBegin($_etype702, $_size699);
-            for ($_i703 = 0; $_i703 < $_size699; ++$_i703)
+            $_size713 = 0;
+            $_etype716 = 0;
+            $xfer += $input->readListBegin($_etype716, $_size713);
+            for ($_i717 = 0; $_i717 < $_size713; ++$_i717)
             {
-              $elem704 = null;
-              $elem704 = new \metastore\Partition();
-              $xfer += $elem704->read($input);
-              $this->success []= $elem704;
+              $elem718 = null;
+              $elem718 = new \metastore\Partition();
+              $xfer += $elem718->read($input);
+              $this->success []= $elem718;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20805,9 +21144,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter705)
+          foreach ($this->success as $iter719)
           {
-            $xfer += $iter705->write($output);
+            $xfer += $iter719->write($output);
           }
         }
         $output->writeListEnd();
@@ -20953,14 +21292,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size706 = 0;
-            $_etype709 = 0;
-            $xfer += $input->readListBegin($_etype709, $_size706);
-            for ($_i710 = 0; $_i710 < $_size706; ++$_i710)
+            $_size720 = 0;
+            $_etype723 = 0;
+            $xfer += $input->readListBegin($_etype723, $_size720);
+            for ($_i724 = 0; $_i724 < $_size720; ++$_i724)
             {
-              $elem711 = null;
-              $xfer += $input->readString($elem711);
-              $this->group_names []= $elem711;
+              $elem725 = null;
+              $xfer += $input->readString($elem725);
+              $this->group_names []= $elem725;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21008,9 +21347,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter712)
+          foreach ($this->group_names as $iter726)
           {
-            $xfer += $output->writeString($iter712);
+            $xfer += $output->writeString($iter726);
           }
         }
         $output->writeListEnd();
@@ -21099,15 +21438,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size713 = 0;
-            $_etype716 = 0;
-            $xfer += $input->readListBegin($_etype716, $_size713);
-            for ($_i717 = 0; $_i717 < $_size713; ++$_i717)
+            $_size727 = 0;
+            $_etype730 = 0;
+            $xfer += $input->readListBegin($_etype730, $_size727);
+            for ($_i731 = 0; $_i731 < $_size727; ++$_i731)
             {
-              $elem718 = null;
-              $elem718 = new \metastore\Partition();
-              $xfer += $elem718->read($input);
-              $this->success []= $elem718;
+              $elem732 = null;
+              $elem732 = new \metastore\Partition();
+              $xfer += $elem732->read($input);
+              $this->success []= $elem732;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21151,9 +21490,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter719)
+          foreach ($this->success as $iter733)
           {
-            $xfer += $iter719->write($output);
+            $xfer += $iter733->write($output);
           }
         }
         $output->writeListEnd();
@@ -21373,15 +21712,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size720 = 0;
-            $_etype723 = 0;
-            $xfer += $input->readListBegin($_etype723, $_size720);
-            for ($_i724 = 0; $_i724 < $_size720; ++$_i724)
+            $_size734 = 0;
+            $_etype737 = 0;
+            $xfer += $input->readListBegin($_etype737, $_size734);
+            for ($_i738 = 0; $_i738 < $_size734; ++$_i738)
             {
-              $elem725 = null;
-              $elem725 = new \metastore\PartitionSpec();
-              $xfer += $elem725->read($input);
-              $this->success []= $elem725;
+              $elem739 = null;
+              $elem739 = new \metastore\PartitionSpec();
+              $xfer += $elem739->read($input);
+              $this->success []= $elem739;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21425,9 +21764,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter726)
+          foreach ($this->success as $iter740)
           {
-            $xfer += $iter726->write($output);
+            $xfer += $iter740->write($output);
           }
         }
         $output->writeListEnd();
@@ -21634,14 +21973,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size727 = 0;
-            $_etype730 = 0;
-            $xfer += $input->readListBegin($_etype730, $_size727);
-            for ($_i731 = 0; $_i731 < $_size727; ++$_i731)
+            $_size741 = 0;
+            $_etype744 = 0;
+            $xfer += $input->readListBegin($_etype744, $_size741);
+            for ($_i745 = 0; $_i745 < $_size741; ++$_i745)
             {
-              $elem732 = null;
-              $xfer += $input->readString($elem732);
-              $this->success []= $elem732;
+              $elem746 = null;
+              $xfer += $input->readString($elem746);
+              $this->success []= $elem746;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21677,9 +22016,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter733)
+          foreach ($this->success as $iter747)
           {
-            $xfer += $output->writeString($iter733);
+            $xfer += $output->writeString($iter747);
           }
         }
         $output->writeListEnd();
@@ -21795,14 +22134,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size734 = 0;
-            $_etype737 = 0;
-            $xfer += $input->readListBegin($_etype737, $_size734);
-            for ($_i738 = 0; $_i738 < $_size734; ++$_i738)
+            $_size748 = 0;
+            $_etype751 = 0;
+            $xfer += $input->readListBegin($_etype751, $_size748);
+            for ($_i752 = 0; $_i752 < $_size748; ++$_i752)
             {
-              $elem739 = null;
-              $xfer += $input->readString($elem739);
-              $this->part_vals []= $elem739;
+              $elem753 = null;
+              $xfer += $input->readString($elem753);
+              $this->part_vals []= $elem753;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21847,9 +22186,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter740)
+          foreach ($this->part_vals as $iter754)
           {
-            $xfer += $output->writeString($iter740);
+            $xfer += $output->writeString($iter754);
           }
         }
         $output->writeListEnd();
@@ -21943,15 +22282,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size741 = 0;
-            $_etype744 = 0;
-            $xfer += $input->readListBegin($_etype744, $_size741);
-            for ($_i745 = 0; $_i745 < $_size741; ++$_i745)
+            $_size755 = 0;
+            $_etype758 = 0;
+            $xfer += $input->readListBegin($_etype758, $_size755);
+            for ($_i759 = 0; $_i759 < $_size755; ++$_i759)
             {
-              $elem746 = null;
-              $elem746 = new \metastore\Partition();
-              $xfer += $elem746->read($input);
-              $this->success []= $elem746;
+              $elem760 = null;
+              $elem760 = new \metastore\Partition();
+              $xfer += $elem760->read($input);
+              $this->success []= $elem760;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21995,9 +22334,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter747)
+          foreach ($this->success as $iter761)
           {
-            $xfer += $iter747->write($output);
+            $xfer += $iter761->write($output);
           }
         }
         $output->writeListEnd();
@@ -22144,14 +22483,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size748 = 0;
-            $_etype751 = 0;
-            $xfer += $input->readListBegin($_etype751, $_size748);
-            for ($_i752 = 0; $_i752 < $_size748; ++$_i752)
+            $_size762 = 0;
+            $_etype765 = 0;
+            $xfer += $input->readListBegin($_etype765, $_size762);
+            for ($_i766 = 0; $_i766 < $_size762; ++$_i766)
             {
-              $elem753 = null;
-              $xfer += $input->readString($elem753);
-              $this->part_vals []= $elem753;
+              $elem767 = null;
+              $xfer += $input->readString($elem767);
+              $this->part_vals []= $elem767;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22175,14 +22514,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size754 = 0;
-            $_etype757 = 0;
-            $xfer += $input->readListBegin($_etype757, $_size754);
-            for ($_i758 = 0; $_i758 < $_size754; ++$_i758)
+            $_size768 = 0;
+            $_etype771 = 0;
+            $xfer += $input->readListBegin($_etype771, $_size768);
+            for ($_i772 = 0; $_i772 < $_size768; ++$_i772)
             {
-              $elem759 = null;
-              $xfer += $input->readString($elem759);
-              $this->group_names []= $elem759;
+              $elem773 = null;
+              $xfer += $input->readString($elem773);
+              $this->group_names []= $elem773;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22220,9 +22559,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter760)
+          foreach ($this->part_vals as $iter774)
           {
-            $xfer += $output->writeString($iter760);
+            $xfer += $output->writeString($iter774);
           }
         }
         $output->writeListEnd();
@@ -22247,9 +22586,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter761)
+          foreach ($this->group_names as $iter775)
           {
-            $xfer += $output->writeString($iter761);
+            $xfer += $output->writeString($iter775);
           }
         }
         $output->writeListEnd();
@@ -22338,15 +22677,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size762 = 0;
-            $_etype765 = 0;
-            $xfer += $input->readListBegin($_etype765, $_size762);
-            for ($_i766 = 0; $_i766 < $_size762; ++$_i766)
+            $_size776 = 0;
+            $_etype779 = 0;
+            $xfer += $input->readListBegin($_etype779, $_size776);
+            for ($_i780 = 0; $_i780 < $_size776; ++$_i780)
             {
-              $elem767 = null;
-              $elem767 = new \metastore\Partition();
-              $xfer += $elem767->read($input);
-              $this->success []= $elem767;
+              $elem781 = null;
+              $elem781 = new \metastore\Partition();
+              $xfer += $elem781->read($input);
+              $this->success []= $elem781;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22390,9 +22729,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter768)
+          foreach ($this->success as $iter782)
           {
-            $xfer += $iter768->write($output);
+            $xfer += $iter782->write($output);
           }
         }
         $output->writeListEnd();
@@ -22513,14 +22852,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size769 = 0;
-            $_etype772 = 0;
-            $xfer += $input->readListBegin($_etype772, $_size769);
-            for ($_i773 = 0; $_i773 < $_size769; ++$_i773)
+            $_size783 = 0;
+            $_etype786 = 0;
+            $xfer += $input->readListBegin($_etype786, $_size783);
+            for ($_i787 = 0; $_i787 < $_size783; ++$_i787)
             {
-              $elem774 = null;
-              $xfer += $input->readString($elem774);
-              $this->part_vals []= $elem774;
+              $elem788 = null;
+              $xfer += $input->readString($elem788);
+              $this->part_vals []= $elem788;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22565,9 +22904,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter775)
+          foreach ($this->part_vals as $iter789)
           {
-            $xfer += $output->writeString($iter775);
+            $xfer += $output->writeString($iter789);
           }
         }
         $output->writeListEnd();
@@ -22660,14 +22999,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size776 = 0;
-            $_etype779 = 0;
-            $xfer += $input->readListBegin($_etype779, $_size776);
-            for ($_i780 = 0; $_i780 < $_size776; ++$_i780)
+            $_size790 = 0;
+            $_etype793 = 0;
+            $xfer += $input->readListBegin($_etype793, $_size790);
+            for ($_i794 = 0; $_i794 < $_size790; ++$_i794)
             {
-              $elem781 = null;
-              $xfer += $input->readString($elem781);
-              $this->success []= $elem781;
+              $elem795 = null;
+              $xfer += $input->readString($elem795);
+              $this->success []= $elem795;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22711,9 +23050,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter782)
+          foreach ($this->success as $iter796)
           {
-            $xfer += $output->writeString($iter782);
+            $xfer += $output->writeString($iter796);
           }
         }
         $output->writeListEnd();
@@ -22956,15 +23295,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size783 = 0;
-            $_etype786 = 0;
-            $xfer += $input->readListBegin($_etype786, $_size783);
-            for ($_i787 = 0; $_i787 < $_size783; ++$_i787)
+            $_size797 = 0;
+            $_etype800 = 0;
+            $xfer += $input->readListBegin($_etype800, $_size797);
+            for ($_i801 = 0; $_i801 < $_size797; ++$_i801)
             {
-              $elem788 = null;
-              $elem788 = new \metastore\Partition();
-              $xfer += $elem788->read($input);
-              $this->success []= $elem788;
+              $elem802 = null;
+              $elem802 = new \metastore\Partition();
+              $xfer += $elem802->read($input);
+              $this->success []= $elem802;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23008,9 +23347,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter789)
+          foreach ($this->success as $iter803)
           {
-            $xfer += $iter789->write($output);
+            $xfer += $iter803->write($output);
           }
         }
         $output->writeListEnd();
@@ -23253,15 +23592,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size790 = 0;
-            $_etype793 = 0;
-            $xfer += $input->readListBegin($_etype793, $_size790);
-            for ($_i794 = 0; $_i794 < $_size790; ++$_i794)
+            $_size804 = 0;
+            $_etype807 = 0;
+            $xfer += $input->readListBegin($_etype807, $_size804);
+            for ($_i808 = 0; $_i808 < $_size804; ++$_i808)
             {
-              $elem795 = null;
-              $elem795 = new \metastore\PartitionSpec();
-              $xfer += $elem795->read($input);
-              $this->success []= $elem795;
+              $elem809 = null;
+              $elem809 = new \metastore\PartitionSpec();
+              $xfer += $elem809->read($input);
+              $this->success []= $elem809;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23305,9 +23644,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter796)
+          foreach ($this->success as $iter810)
           {
-            $xfer += $iter796->write($output);
+            $xfer += $iter810->write($output);
           }
         }
         $output->writeListEnd();
@@ -23627,14 +23966,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size797 = 0;
-            $_etype800 = 0;
-            $xfer += $input->readListBegin($_etype800, $_size797);
-            for ($_i801 = 0; $_i801 < $_size797; ++$_i801)
+            $_size811 = 0;
+            $_etype814 = 0;
+            $xfer += $input->readListBegin($_etype814, $_size811);
+            for ($_i815 = 0; $_i815 < $_size811; ++$_i815)
             {
-              $elem802 = null;
-              $xfer += $input->readString($elem802);
-              $this->names []= $elem802;
+              $elem816 = null;
+              $xfer += $input->readString($elem816);
+              $this->names []= $elem816;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23672,9 +24011,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter803)
+          foreach ($this->names as $iter817)
           {
-            $xfer += $output->writeString($iter803);
+            $xfer += $output->writeString($iter817);
           }
         }
         $output->writeListEnd();
@@ -23763,15 +24102,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size804 = 0;
-            $_etype807 = 0;
-            $xfer += $input->readListBegin($_etype807, $_size804);
-            for ($_i808 = 0; $_i808 < $_size804; ++$_i808)
+            $_size818 = 0;
+            $_etype821 = 0;
+            $xfer += $input->readListBegin($_etype821, $_size818);
+            for ($_i822 = 0; $_i822 < $_size818; ++$_i822)
             {
-              $elem809 = null;
-              $elem809 = new \metastore\Partition();
-              $xfer += $elem809->read($input);
-              $this->success []= $elem809;
+              $elem823 = null;
+              $elem823 = new \metastore\Partition();
+              $xfer += $elem823->read($input);
+              $this->success []= $elem823;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23815,9 +24154,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter810)
+          foreach ($this->success as $iter824)
           {
-            $xfer += $iter810->write($output);
+            $xfer += $iter824->write($output);
           }
         }
         $output->writeListEnd();
@@ -24156,15 +24495,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size811 = 0;
-            $_etype814 = 0;
-            $xfer += $input->readListBegin($_etype814, $_size811);
-            for ($_i815 = 0; $_i815 < $_size811; ++$_i815)
+            $_size825 = 0;
+            $_etype828 = 0;
+            $xfer += $input->readListBegin($_etype828, $_size825);
+            for ($_i829 = 0; $_i829 < $_size825; ++$_i829)
             {
-              $elem816 = null;
-              $elem816 = new \metastore\Partition();
-              $xfer += $elem816->read($input);
-              $this->new_parts []= $elem816;
+              $elem830 = null;
+              $elem830 = new \metastore\Partition();
+              $xfer += $elem830->read($input);
+              $this->new_parts []= $elem830;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24202,9 +24541,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter817)
+          foreach ($this->new_parts as $iter831)
           {
-            $xfer += $iter817->write($output);
+            $xfer += $iter831->write($output);
           }
         }
         $output->writeListEnd();
@@ -24674,14 +25013,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size818 = 0;
-            $_etype821 = 0;
-            $xfer += $input->readListBegin($_etype821, $_size818);
-            for ($_i822 = 0; $_i822 < $_size818; ++$_i822)
+            $_size832 = 0;
+            $_etype835 = 0;
+            $xfer += $input->readListBegin($_etype835, $_size832);
+            for ($_i836 = 0; $_i836 < $_size832; ++$_i836)
             {
-              $elem823 = null;
-              $xfer += $input->readString($elem823);
-              $this->part_vals []= $elem823;
+              $elem837 = null;
+              $xfer += $input->readString($elem837);
+              $this->part_vals []= $elem837;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24727,9 +25066,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter824)
+          foreach ($this->part_vals as $iter838)
           {
-            $xfer += $output->writeString($iter824);
+            $xfer += $output->writeString($iter838);
           }
         }
         $output->writeListEnd();
@@ -24914,14 +25253,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size825 = 0;
-            $_etype828 = 0;
-            $xfer += $input->readListBegin($_etype828, $_size825);
-            for ($_i829 = 0; $_i829 < $_size825; ++$_i829)
+            $_size839 = 0;
+            $_etype842 = 0;
+            $xfer += $input->readListBegin($_etype842, $_size839);
+            for ($_i843 = 0; $_i843 < $_size839; ++$_i843)
             {
-              $elem830 = null;
-              $xfer += $input->readString($elem830);
-              $this->part_vals []= $elem830;
+              $elem844 = null;
+              $xfer += $input->readString($elem844);
+              $this->part_vals []= $elem844;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24956,9 +25295,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter831)
+          foreach ($this->part_vals as $iter845)
           {
-            $xfer += $output->writeString($iter831);
+            $xfer += $output->writeString($iter845);
           }
         }
         $output->writeListEnd();
@@ -25412,14 +25751,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size832 = 0;
-            $_etype835 = 0;
-            $xfer += $input->readListBegin($_etype835, $_size832);
-            for ($_i836 = 0; $_i836 < $_size832; ++$_i836)
+            $_size846 = 0;
+            $_etype849 = 0;
+            $xfer += $input->readListBegin($_etype849, $_size846);
+            for ($_i850 = 0; $_i850 < $_size846; ++$_i850)
             {
-              $elem837 = null;
-              $xfer += $input->readString($elem837);
-              $this->success []= $elem837;
+              $elem851 = null;
+              $xfer += $input->readString($elem851);
+              $this->success []= $elem851;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25455,9 +25794,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter838)
+          foreach ($this->success as $iter852)
           {
-            $xfer += $output->writeString($iter838);
+            $xfer += $output->writeString($iter852);
           }
         }
         $output->writeListEnd();
@@ -25617,17 +25956,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size839 = 0;
-            $_ktype840 = 0;
-            $_vtype841 = 0;
-            $xfer += $input->readMapBegin($_ktype840, $_vtype841, $_size839);
-            for ($_i843 = 0; $_i843 < $_size839; ++$_i843)
+            $_size853 = 0;
+            $_ktype854 = 0;
+            $_vtype855 = 0;
+            $xfer += $input->readMapBegin($_ktype854, $_vtype855, $_size853);
+            for ($_i857 = 0; $_i857 < $_size853; ++$_i857)
             {
-              $key844 = '';
-              $val845 = '';
-              $xfer += $input->readString($key844);
-              $xfer += $input->readString($val845);
-              $this->success[$key844] = $val845;
+              $key858 = '';
+              $val859 = '';
+              $xfer += $input->readString($key858);
+              $xfer += $input->readString($val859);
+              $this->success[$key858] = $val859;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25663,10 +26002,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter846 => $viter847)
+          foreach ($this->success as $kiter860 => $viter861)
           {
-            $xfer += $output->writeString($kiter846);
-            $xfer += $output->writeString($viter847);
+            $xfer += $output->writeString($kiter860);
+            $xfer += $output->writeString($viter861);
           }
         }
         $output->writeMapEnd();
@@ -25786,17 +26125,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size848 = 0;
-            $_ktype849 = 0;
-            $_vtype850 = 0;
-            $xfer += $input->readMapBegin($_ktype849, $_vtype850, $_size848);
-            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
+            $_size862 = 0;
+            $_ktype863 = 0;
+            $_vtype864 = 0;
+            $xfer += $input->readMapBegin($_ktype863, $_vtype864, $_size862);
+            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
             {
-              $key853 = '';
-              $val854 = '';
-              $xfer += $input->readString($key853);
-              $xfer += $input->readString($val854);
-              $this->part_vals[$key853] = $val854;
+              $key867 = '';
+              $val868 = '';
+              $xfer += $input->readString($key867);
+              $xfer += $input->readString($val868);
+              $this->part_vals[$key867] = $val868;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25841,10 +26180,10 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter855 => $viter856)
+          foreach ($this->part_vals as $kiter869 => $viter870)
           {
-            $xfer += $output->writeString($kiter855);
-            $xfer += $output->writeString($viter856);
+            $xfer += $output->writeString($kiter869);
+            $xfer += $output->writeString($viter870);
           }
         }
         $output->writeMapEnd();
@@ -26166,17 +26505,17 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size857 = 0;
-            $_ktype858 = 0;
-            $_vtype859 = 0;
-            $xfer += $input->readMapBegin($_ktype858, $_vtype859, $_size857);
-            for ($_i861 = 0; $_i861 < $_size857; ++$_i861)
+            $_size871 = 0;
+            $_ktype872 = 0;
+            $_vtype873 = 0;
+            $xfer += $input->readMapBegin($_ktype872, $_vtype873, $_size871);
+            for ($_i875 = 0; $_i875 < $_size871; ++$_i875)
             {
-              $key862 = '';
-              $val863 = '';
-              $xfer += $input->readString($key862);
-              $xfer += $input->readString($val863);
-              $this->part_vals[$key862] = $val863;
+              $key876 = '';
+              $val877 = '';
+              $xfer += $input->readString($key876);
+              $xfer += $input->readString($val877);
+              $this->part_vals[$key876] = $val877;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -26221,10 +26560,10 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter864 => $viter865)
+          foreach ($this->part_vals as $kiter878 => $viter879)
           {
-            $xfer += $output->writeString($kiter864);
-            $xfer += $output->writeString($viter865);
+            $xfer += $output->writeString($kiter878);
+            $xfer += $output->writeString($viter879);
           }
         }
         $output->writeMapEnd();
@@ -27698,15 +28037,15 @@ class ThriftHiveMetastore_get_indexes_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size866 = 0;
-            $_etype869 = 0;
-            $xfer += $input->readListBegin($_etype869, $_size866);
-            for ($_i870 = 0; $_i870 < $_size866; ++$_i870)
+            $_size880 = 0;
+            $_etype883 = 0;
+            $xfer += $input->readListBegin($_etype883, $_size880);
+            for ($_i884 = 0; $_i884 < $_size880; ++$_i884)
             {
-              $elem871 = null;
-              $elem871 = new \metastore\Index();
-              $xfer += $elem871->read($input);
-              $this->success []= $elem871;
+              $elem885 = null;
+              $elem885 = new \metastore\Index();
+              $xfer += $elem885->read($input);
+              $this->success []= $elem885;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27750,9 +28089,9 @@ class ThriftHiveMetastore_get_indexes_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter872)
+          foreach ($this->success as $iter886)
           {
-            $xfer += $iter872->write($output);
+            $xfer += $iter886->write($output);
           }
         }
         $output->writeListEnd();
@@ -27959,14 +28298,14 @@ class ThriftHiveMetastore_get_index_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size873 = 0;
-            $_etype876 = 0;
-            $xfer += $input->readListBegin($_etype876, $_size873);
-            for ($_i877 = 0; $_i877 < $_size873; ++$_i877)
+            $_size887 = 0;
+            $_etype890 = 0;
+            $xfer += $input->readListBegin($_etype890, $_size887);
+            for ($_i891 = 0; $_i891 < $_size887; ++$_i891)
             {
-              $elem878 = null;
-              $xfer += $input->readString($elem878);
-              $this->success []= $elem878;
+              $elem892 = null;
+              $xfer += $input->readString($elem892);
+              $this->success []= $elem892;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28002,9 +28341,9 @@ class ThriftHiveMetastore_get_index_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter879)
+          foreach ($this->success as $iter893)
           {
-            $xfer += $output->writeString($iter879);
+            $xfer += $output->writeString($iter893);
           }
         }
         $output->writeListEnd();
@@ -31478,14 +31817,14 @@ class ThriftHiveMetastore_get_functions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size880 = 0;
-            $_etype883 = 0;
-            $xfer += $input->readListBegin($_etype883, $_size880);
-            for ($_i884 = 0; $_i884 < $_size880; ++$_i884)
+            $_size894 = 0;
+            $_etype897 = 0;
+            $xfer += $input->readListBegin($_etype897, $_size894);
+            for ($_i898 = 0; $_i898 < $_size894; ++$_i898)
             {
-              $elem885 = null;
-              $xfer += $input->readString($elem885);
-              $this->success []= $elem885;
+              $elem899 = null;
+              $xfer += $input->readString($elem899);
+              $this->success []= $elem899;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31521,9 +31860,9 @@ class ThriftHiveMetastore_get_functions_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter886)
+          foreach ($this->success as $iter900)
           {
-            $xfer += $output->writeString($iter886);
+            $xfer += $output->writeString($iter900);
           }
         }
         $output->writeListEnd();
@@ -32392,14 +32731,14 @@ class ThriftHiveMetastore_get_role_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size887 = 0;
-            $_etype890 = 0;
-            $xfer += $input->readListBegin($_etype890, $_size887);
-            for ($_i891 = 0; $_i891 < $_size887; ++$_i891)
+            $_size901 = 0;
+            $_etype904 = 0;
+            $xfer += $input->readListBegin($_etype904, $_size901);
+            for ($_i905 = 0; $_i905 < $_size901; ++$_i905)
             {
-              $elem892 = null;
-              $xfer += $input->readString($elem892);
-              $this->success []= $elem892;
+              $elem906 = null;
+              $xfer += $input->readString($elem906);
+              $this->success []= $elem906;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32435,9 +32774,9 @@ class ThriftHiveMetastore_get_role_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter893)
+          foreach ($this->success as $iter907)
           {
-            $xfer += $output->writeString($iter893);
+            $xfer += $output->writeString($iter907);
           }
         }
         $output->writeListEnd();
@@ -33128,15 +33467,15 @@ class ThriftHiveMetastore_list_roles_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size894 = 0;
-            $_etype897 = 0;
-            $xfer += $input->readListBegin($_etype897, $_size894);
-            for ($_i898 = 0; $_i898 < $_size894; ++$_i898)
+            $_size908 = 0;
+            $_etype911 = 0;
+            $xfer += $input->readListBegin($_etype911, $_size908);
+            for ($_i912 = 0; $_i912 < $_size908; ++$_i912)
             {
-              $elem899 = null;
-              $elem899 = new \metastore\Role();
-              $xfer += $elem899->read($input);
-              $this->success []= $elem899;
+              $elem913 = null;
+              $elem913 = new \metastore\Role();
+              $xfer += $elem913->read($input);
+              $this->success []= $elem913;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33172,9 +33511,9 @@ class ThriftHiveMetastore_list_roles_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter900)
+          foreach ($this->success as $iter914)
           {
-            $xfer += $iter900->write($output);
+            $xfer += $iter914->write($output);
           }
         }
         $output->writeListEnd();
@@ -33836,14 +34175,14 @@ class ThriftHiveMetastore_get_privilege_set_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size901 = 0;
-            $_etype904 = 0;
-            $xfer += $input->readListBegin($_etype904, $_size901);
-            for ($_i905 = 0; $_i905 < $_size901; ++$_i905)
+            $_size915 = 0;
+            $_etype918 = 0;
+            $xfer += $input->readListBegin($_etype918, $_size915);
+            for ($_i919 = 0; $_i919 < $_size915; ++$_i919)
             {
-              $elem906 = null;
-              $xfer += $input->readString($elem906);
-              $this->group_names []= $elem906;
+              $elem920 = null;
+              $xfer += $input->readString($elem920);
+              $this->group_names []= $elem920;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33884,9 +34223,9 @@ class ThriftHiveMetastore_get_privilege_set_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter907)
+          foreach ($this->group_names as $iter921)
           {
-            $xfer += $output->writeString($iter907);
+            $xfer += $output->writeString($iter921);
           }
         }
         $output->writeListEnd();
@@ -34194,15 +34533,15 @@ class ThriftHiveMetastore_list_privileges_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size908 = 0;
-            $_etype911 = 0;
-            $xfer += $input->readListBegin($_etype911, $_size908);
-            for ($_i912 = 0; $_i912 < $_size908; ++$_i912)
+            $_size922 = 0;
+            $_etype925 = 0;
+            $xfer += $input->readListBegin($_etype925, $_size922);
+            for ($_i926 = 0; $_i926 < $_size922; ++$_i926)
             {
-              $elem913 = null;
-              $elem913 = new \metastore\HiveObjectPrivilege();
-              $xfer += $elem913->read($input);
-              $this->success []= $elem913;
+              $elem927 = null;
+              $elem927 = new \metastore\HiveObjectPrivilege();
+              $xfer += $elem927->read($input);
+              $this->success []= $elem927;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34238,9 +34577,9 @@ class ThriftHiveMetastore_list_privileges_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter914)
+          foreach ($this->success as $iter928)
           {
-            $xfer += $iter914->write($output);
+            $xfer += $iter928->write($output);
           }
         }
         $output->writeListEnd();
@@ -34872,14 +35211,14 @@ class ThriftHiveMetastore_set_ugi_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size915 = 0;
-            $_etype918 = 0;
-            $xfer += $input->readListBegin($_etype918, $_size915);
-            for ($_i919 = 0; $_i919 < $_size915; ++$_i919)
+            $_size929 = 0;
+            $_etype932 = 0;
+            $xfer += $input->readListBegin($_etype932, $_size929);
+            for ($_i933 = 0; $_i933 < $_size929; ++$_i933)
             {
-              $elem920 = null;
-              $xfer += $input->readString($elem920);
-              $this->group_names []= $elem920;
+              $elem934 = null;
+              $xfer += $input->readString($elem934);
+              $this->group_names []= $elem934;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34912,9 +35251,9 @@ class ThriftHiveMetastore_set_ugi_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter921)
+          foreach ($this->group_names as $iter935)
           {
-            $xfer += $output->writeString($iter921);
+            $xfer += $output->writeString($iter935);
           }
         }
         $output->writeListEnd();
@@ -34990,14 +35329,14 @@ class ThriftHiveMetastore_set_ugi_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size922 = 0;
-            $_etype925 = 0;
-            $xfer += $input->readListBegin($_etype925, $_size922);
-            for ($_i926 = 0; $_i926 < $_size922; ++$_i926)
+            $_size936 = 0;
+            $_etype939 = 0;
+            $xfer += $input->readListBegin($_etype939, $_size936);
+            for ($_i940 = 0; $_i940 < $_size936; ++$_i940)
             {
-              $elem927 = null;
-              $xfer += $input->readString($elem927);
-              $this->success []= $elem927;
+              $elem941 = null;
+              $xfer += $input->readString($elem941);
+              $this->success []= $elem941;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35033,9 +35372,9 @@ class ThriftHiveMetastore_set_ugi_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter928)
+          foreach ($this->success as $iter942)
           {
-            $xfer += $output->writeString($iter928);
+            $xfer += $output->writeString($iter942);
           }
         }
         $output->writeListEnd();

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/Types.php b/metastore/src/gen/thrift/gen-php/metastore/Types.php
index e63213d..e43a13d 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -14957,6 +14957,150 @@ class GetAllFunctionsResponse {
 
 }
 
+class TableMeta {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $dbName = null;
+  /**
+   * @var string
+   */
+  public $tableName = null;
+  /**
+   * @var string
+   */
+  public $tableType = null;
+  /**
+   * @var string
+   */
+  public $comments = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'dbName',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'tableName',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'tableType',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'comments',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['dbName'])) {
+        $this->dbName = $vals['dbName'];
+      }
+      if (isset($vals['tableName'])) {
+        $this->tableName = $vals['tableName'];
+      }
+      if (isset($vals['tableType'])) {
+        $this->tableType = $vals['tableType'];
+      }
+      if (isset($vals['comments'])) {
+        $this->comments = $vals['comments'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'TableMeta';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dbName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tableName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tableType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->comments);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('TableMeta');
+    if ($this->dbName !== null) {
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1);
+      $xfer += $output->writeString($this->dbName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tableName !== null) {
+      $xfer += $output->writeFieldBegin('tableName', TType::STRING, 2);
+      $xfer += $output->writeString($this->tableName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tableType !== null) {
+      $xfer += $output->writeFieldBegin('tableType', TType::STRING, 3);
+      $xfer += $output->writeString($this->tableType);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->comments !== null) {
+      $xfer += $output->writeFieldBegin('comments', TType::STRING, 4);
+      $xfer += $output->writeString($this->comments);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class MetaException extends TException {
   static $_TSPEC;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index 65ba10e..22d794f 100755
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -45,6 +45,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  void drop_table(string dbname, string name, bool deleteData)')
   print('  void drop_table_with_environment_context(string dbname, string name, bool deleteData, EnvironmentContext environment_context)')
   print('   get_tables(string db_name, string pattern)')
+  print('   get_table_meta(string db_patterns, string tbl_patterns,  tbl_types)')
   print('   get_all_tables(string db_name)')
   print('  Table get_table(string dbname, string tbl_name)')
   print('   get_table_objects_by_name(string dbname,  tbl_names)')
@@ -349,6 +350,12 @@ elif cmd == 'get_tables':
     sys.exit(1)
   pp.pprint(client.get_tables(args[0],args[1],))
 
+elif cmd == 'get_table_meta':
+  if len(args) != 3:
+    print('get_table_meta requires 3 args')
+    sys.exit(1)
+  pp.pprint(client.get_table_meta(args[0],args[1],eval(args[2]),))
+
 elif cmd == 'get_all_tables':
   if len(args) != 1:
     print('get_all_tables requires 1 args')


[4/7] hive git commit: HIVE-7575 GetTables thrift call is very slow (Navis via Aihua Xu, reviewed by Szehon Ho, Aihua Xu)

Posted by ai...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b678ed85/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 3bc7e10..4690093 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -84,6 +84,8 @@ public class ThriftHiveMetastore {
 
     public List<String> get_tables(String db_name, String pattern) throws MetaException, org.apache.thrift.TException;
 
+    public List<TableMeta> get_table_meta(String db_patterns, String tbl_patterns, List<String> tbl_types) throws MetaException, org.apache.thrift.TException;
+
     public List<String> get_all_tables(String db_name) throws MetaException, org.apache.thrift.TException;
 
     public Table get_table(String dbname, String tbl_name) throws MetaException, NoSuchObjectException, org.apache.thrift.TException;
@@ -348,6 +350,8 @@ public class ThriftHiveMetastore {
 
     public void get_tables(String db_name, String pattern, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void get_table_meta(String db_patterns, String tbl_patterns, List<String> tbl_types, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void get_all_tables(String db_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void get_table(String dbname, String tbl_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -1204,6 +1208,34 @@ public class ThriftHiveMetastore {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_tables failed: unknown result");
     }
 
+    public List<TableMeta> get_table_meta(String db_patterns, String tbl_patterns, List<String> tbl_types) throws MetaException, org.apache.thrift.TException
+    {
+      send_get_table_meta(db_patterns, tbl_patterns, tbl_types);
+      return recv_get_table_meta();
+    }
+
+    public void send_get_table_meta(String db_patterns, String tbl_patterns, List<String> tbl_types) throws org.apache.thrift.TException
+    {
+      get_table_meta_args args = new get_table_meta_args();
+      args.setDb_patterns(db_patterns);
+      args.setTbl_patterns(tbl_patterns);
+      args.setTbl_types(tbl_types);
+      sendBase("get_table_meta", args);
+    }
+
+    public List<TableMeta> recv_get_table_meta() throws MetaException, org.apache.thrift.TException
+    {
+      get_table_meta_result result = new get_table_meta_result();
+      receiveBase(result, "get_table_meta");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_table_meta failed: unknown result");
+    }
+
     public List<String> get_all_tables(String db_name) throws MetaException, org.apache.thrift.TException
     {
       send_get_all_tables(db_name);
@@ -5131,6 +5163,44 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void get_table_meta(String db_patterns, String tbl_patterns, List<String> tbl_types, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_table_meta_call method_call = new get_table_meta_call(db_patterns, tbl_patterns, tbl_types, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class get_table_meta_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String db_patterns;
+      private String tbl_patterns;
+      private List<String> tbl_types;
+      public get_table_meta_call(String db_patterns, String tbl_patterns, List<String> tbl_types, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.db_patterns = db_patterns;
+        this.tbl_patterns = tbl_patterns;
+        this.tbl_types = tbl_types;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_table_meta", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_table_meta_args args = new get_table_meta_args();
+        args.setDb_patterns(db_patterns);
+        args.setTbl_patterns(tbl_patterns);
+        args.setTbl_types(tbl_types);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public List<TableMeta> getResult() throws MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_table_meta();
+      }
+    }
+
     public void get_all_tables(String db_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       get_all_tables_call method_call = new get_all_tables_call(db_name, resultHandler, this, ___protocolFactory, ___transport);
@@ -9061,6 +9131,7 @@ public class ThriftHiveMetastore {
       processMap.put("drop_table", new drop_table());
       processMap.put("drop_table_with_environment_context", new drop_table_with_environment_context());
       processMap.put("get_tables", new get_tables());
+      processMap.put("get_table_meta", new get_table_meta());
       processMap.put("get_all_tables", new get_all_tables());
       processMap.put("get_table", new get_table());
       processMap.put("get_table_objects_by_name", new get_table_objects_by_name());
@@ -9731,6 +9802,30 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_table_meta<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_table_meta_args> {
+      public get_table_meta() {
+        super("get_table_meta");
+      }
+
+      public get_table_meta_args getEmptyArgsInstance() {
+        return new get_table_meta_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_table_meta_result getResult(I iface, get_table_meta_args args) throws org.apache.thrift.TException {
+        get_table_meta_result result = new get_table_meta_result();
+        try {
+          result.success = iface.get_table_meta(args.db_patterns, args.tbl_patterns, args.tbl_types);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        }
+        return result;
+      }
+    }
+
     public static class get_all_tables<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_all_tables_args> {
       public get_all_tables() {
         super("get_all_tables");
@@ -12558,6 +12653,7 @@ public class ThriftHiveMetastore {
       processMap.put("drop_table", new drop_table());
       processMap.put("drop_table_with_environment_context", new drop_table_with_environment_context());
       processMap.put("get_tables", new get_tables());
+      processMap.put("get_table_meta", new get_table_meta());
       processMap.put("get_all_tables", new get_all_tables());
       processMap.put("get_table", new get_table());
       processMap.put("get_table_objects_by_name", new get_table_objects_by_name());
@@ -13991,6 +14087,63 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_table_meta<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_meta_args, List<TableMeta>> {
+      public get_table_meta() {
+        super("get_table_meta");
+      }
+
+      public get_table_meta_args getEmptyArgsInstance() {
+        return new get_table_meta_args();
+      }
+
+      public AsyncMethodCallback<List<TableMeta>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<List<TableMeta>>() { 
+          public void onComplete(List<TableMeta> o) {
+            get_table_meta_result result = new get_table_meta_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            get_table_meta_result result = new get_table_meta_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, get_table_meta_args args, org.apache.thrift.async.AsyncMethodCallback<List<TableMeta>> resultHandler) throws TException {
+        iface.get_table_meta(args.db_patterns, args.tbl_patterns, args.tbl_types,resultHandler);
+      }
+    }
+
     public static class get_all_tables<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_all_tables_args, List<String>> {
       public get_all_tables() {
         super("get_all_tables");
@@ -40884,15 +41037,960 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class drop_table_with_environment_context_resultStandardSchemeFactory implements SchemeFactory {
-      public drop_table_with_environment_context_resultStandardScheme getScheme() {
-        return new drop_table_with_environment_context_resultStandardScheme();
+    private static class drop_table_with_environment_context_resultStandardSchemeFactory implements SchemeFactory {
+      public drop_table_with_environment_context_resultStandardScheme getScheme() {
+        return new drop_table_with_environment_context_resultStandardScheme();
+      }
+    }
+
+    private static class drop_table_with_environment_context_resultStandardScheme extends StandardScheme<drop_table_with_environment_context_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // O1
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.o1 = new NoSuchObjectException();
+                struct.o1.read(iprot);
+                struct.setO1IsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // O3
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.o3 = new MetaException();
+                struct.o3.read(iprot);
+                struct.setO3IsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.o1 != null) {
+          oprot.writeFieldBegin(O1_FIELD_DESC);
+          struct.o1.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.o3 != null) {
+          oprot.writeFieldBegin(O3_FIELD_DESC);
+          struct.o3.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class drop_table_with_environment_context_resultTupleSchemeFactory implements SchemeFactory {
+      public drop_table_with_environment_context_resultTupleScheme getScheme() {
+        return new drop_table_with_environment_context_resultTupleScheme();
+      }
+    }
+
+    private static class drop_table_with_environment_context_resultTupleScheme extends TupleScheme<drop_table_with_environment_context_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetO1()) {
+          optionals.set(0);
+        }
+        if (struct.isSetO3()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetO1()) {
+          struct.o1.write(oprot);
+        }
+        if (struct.isSetO3()) {
+          struct.o3.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.o1 = new NoSuchObjectException();
+          struct.o1.read(iprot);
+          struct.setO1IsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.o3 = new MetaException();
+          struct.o3.read(iprot);
+          struct.setO3IsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class get_tables_args implements org.apache.thrift.TBase<get_tables_args, get_tables_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_args");
+
+    private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("pattern", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_tables_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_tables_argsTupleSchemeFactory());
+    }
+
+    private String db_name; // required
+    private String pattern; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      DB_NAME((short)1, "db_name"),
+      PATTERN((short)2, "pattern");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // DB_NAME
+            return DB_NAME;
+          case 2: // PATTERN
+            return PATTERN;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.PATTERN, new org.apache.thrift.meta_data.FieldMetaData("pattern", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_tables_args.class, metaDataMap);
+    }
+
+    public get_tables_args() {
+    }
+
+    public get_tables_args(
+      String db_name,
+      String pattern)
+    {
+      this();
+      this.db_name = db_name;
+      this.pattern = pattern;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_tables_args(get_tables_args other) {
+      if (other.isSetDb_name()) {
+        this.db_name = other.db_name;
+      }
+      if (other.isSetPattern()) {
+        this.pattern = other.pattern;
+      }
+    }
+
+    public get_tables_args deepCopy() {
+      return new get_tables_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.db_name = null;
+      this.pattern = null;
+    }
+
+    public String getDb_name() {
+      return this.db_name;
+    }
+
+    public void setDb_name(String db_name) {
+      this.db_name = db_name;
+    }
+
+    public void unsetDb_name() {
+      this.db_name = null;
+    }
+
+    /** Returns true if field db_name is set (has been assigned a value) and false otherwise */
+    public boolean isSetDb_name() {
+      return this.db_name != null;
+    }
+
+    public void setDb_nameIsSet(boolean value) {
+      if (!value) {
+        this.db_name = null;
+      }
+    }
+
+    public String getPattern() {
+      return this.pattern;
+    }
+
+    public void setPattern(String pattern) {
+      this.pattern = pattern;
+    }
+
+    public void unsetPattern() {
+      this.pattern = null;
+    }
+
+    /** Returns true if field pattern is set (has been assigned a value) and false otherwise */
+    public boolean isSetPattern() {
+      return this.pattern != null;
+    }
+
+    public void setPatternIsSet(boolean value) {
+      if (!value) {
+        this.pattern = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case DB_NAME:
+        if (value == null) {
+          unsetDb_name();
+        } else {
+          setDb_name((String)value);
+        }
+        break;
+
+      case PATTERN:
+        if (value == null) {
+          unsetPattern();
+        } else {
+          setPattern((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case DB_NAME:
+        return getDb_name();
+
+      case PATTERN:
+        return getPattern();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case DB_NAME:
+        return isSetDb_name();
+      case PATTERN:
+        return isSetPattern();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_tables_args)
+        return this.equals((get_tables_args)that);
+      return false;
+    }
+
+    public boolean equals(get_tables_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_db_name = true && this.isSetDb_name();
+      boolean that_present_db_name = true && that.isSetDb_name();
+      if (this_present_db_name || that_present_db_name) {
+        if (!(this_present_db_name && that_present_db_name))
+          return false;
+        if (!this.db_name.equals(that.db_name))
+          return false;
+      }
+
+      boolean this_present_pattern = true && this.isSetPattern();
+      boolean that_present_pattern = true && that.isSetPattern();
+      if (this_present_pattern || that_present_pattern) {
+        if (!(this_present_pattern && that_present_pattern))
+          return false;
+        if (!this.pattern.equals(that.pattern))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_db_name = true && (isSetDb_name());
+      list.add(present_db_name);
+      if (present_db_name)
+        list.add(db_name);
+
+      boolean present_pattern = true && (isSetPattern());
+      list.add(present_pattern);
+      if (present_pattern)
+        list.add(pattern);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(get_tables_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(other.isSetDb_name());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetDb_name()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_name, other.db_name);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetPattern()).compareTo(other.isSetPattern());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetPattern()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pattern, other.pattern);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("get_tables_args(");
+      boolean first = true;
+
+      sb.append("db_name:");
+      if (this.db_name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.db_name);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("pattern:");
+      if (this.pattern == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.pattern);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class get_tables_argsStandardSchemeFactory implements SchemeFactory {
+      public get_tables_argsStandardScheme getScheme() {
+        return new get_tables_argsStandardScheme();
+      }
+    }
+
+    private static class get_tables_argsStandardScheme extends StandardScheme<get_tables_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // DB_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.db_name = iprot.readString();
+                struct.setDb_nameIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // PATTERN
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.pattern = iprot.readString();
+                struct.setPatternIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.db_name != null) {
+          oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
+          oprot.writeString(struct.db_name);
+          oprot.writeFieldEnd();
+        }
+        if (struct.pattern != null) {
+          oprot.writeFieldBegin(PATTERN_FIELD_DESC);
+          oprot.writeString(struct.pattern);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class get_tables_argsTupleSchemeFactory implements SchemeFactory {
+      public get_tables_argsTupleScheme getScheme() {
+        return new get_tables_argsTupleScheme();
+      }
+    }
+
+    private static class get_tables_argsTupleScheme extends TupleScheme<get_tables_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetDb_name()) {
+          optionals.set(0);
+        }
+        if (struct.isSetPattern()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetDb_name()) {
+          oprot.writeString(struct.db_name);
+        }
+        if (struct.isSetPattern()) {
+          oprot.writeString(struct.pattern);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.db_name = iprot.readString();
+          struct.setDb_nameIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.pattern = iprot.readString();
+          struct.setPatternIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class get_tables_result implements org.apache.thrift.TBase<get_tables_result, get_tables_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
+    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_tables_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_tables_resultTupleSchemeFactory());
+    }
+
+    private List<String> success; // required
+    private MetaException o1; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      O1((short)1, "o1");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // O1
+            return O1;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+      tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_tables_result.class, metaDataMap);
+    }
+
+    public get_tables_result() {
+    }
+
+    public get_tables_result(
+      List<String> success,
+      MetaException o1)
+    {
+      this();
+      this.success = success;
+      this.o1 = o1;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_tables_result(get_tables_result other) {
+      if (other.isSetSuccess()) {
+        List<String> __this__success = new ArrayList<String>(other.success);
+        this.success = __this__success;
+      }
+      if (other.isSetO1()) {
+        this.o1 = new MetaException(other.o1);
+      }
+    }
+
+    public get_tables_result deepCopy() {
+      return new get_tables_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.o1 = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    public java.util.Iterator<String> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(String elem) {
+      if (this.success == null) {
+        this.success = new ArrayList<String>();
+      }
+      this.success.add(elem);
+    }
+
+    public List<String> getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(List<String> success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public MetaException getO1() {
+      return this.o1;
+    }
+
+    public void setO1(MetaException o1) {
+      this.o1 = o1;
+    }
+
+    public void unsetO1() {
+      this.o1 = null;
+    }
+
+    /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
+    public boolean isSetO1() {
+      return this.o1 != null;
+    }
+
+    public void setO1IsSet(boolean value) {
+      if (!value) {
+        this.o1 = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((List<String>)value);
+        }
+        break;
+
+      case O1:
+        if (value == null) {
+          unsetO1();
+        } else {
+          setO1((MetaException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      case O1:
+        return getO1();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      case O1:
+        return isSetO1();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_tables_result)
+        return this.equals((get_tables_result)that);
+      return false;
+    }
+
+    public boolean equals(get_tables_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_o1 = true && this.isSetO1();
+      boolean that_present_o1 = true && that.isSetO1();
+      if (this_present_o1 || that_present_o1) {
+        if (!(this_present_o1 && that_present_o1))
+          return false;
+        if (!this.o1.equals(that.o1))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (isSetSuccess());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      boolean present_o1 = true && (isSetO1());
+      list.add(present_o1);
+      if (present_o1)
+        list.add(o1);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(get_tables_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetO1()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, other.o1);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("get_tables_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("o1:");
+      if (this.o1 == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.o1);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class get_tables_resultStandardSchemeFactory implements SchemeFactory {
+      public get_tables_resultStandardScheme getScheme() {
+        return new get_tables_resultStandardScheme();
       }
     }
 
-    private static class drop_table_with_environment_context_resultStandardScheme extends StandardScheme<drop_table_with_environment_context_result> {
+    private static class get_tables_resultStandardScheme extends StandardScheme<get_tables_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -40902,20 +42000,29 @@ public class ThriftHiveMetastore {
             break;
           }
           switch (schemeField.id) {
-            case 1: // O1
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.o1 = new NoSuchObjectException();
-                struct.o1.read(iprot);
-                struct.setO1IsSet(true);
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list650.size);
+                  String _elem651;
+                  for (int _i652 = 0; _i652 < _list650.size; ++_i652)
+                  {
+                    _elem651 = iprot.readString();
+                    struct.success.add(_elem651);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setSuccessIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 2: // O3
+            case 1: // O1
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.o3 = new MetaException();
-                struct.o3.read(iprot);
-                struct.setO3IsSet(true);
+                struct.o1 = new MetaException();
+                struct.o1.read(iprot);
+                struct.setO1IsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -40929,91 +42036,115 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
+            for (String _iter653 : struct.success)
+            {
+              oprot.writeString(_iter653);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
         if (struct.o1 != null) {
           oprot.writeFieldBegin(O1_FIELD_DESC);
           struct.o1.write(oprot);
           oprot.writeFieldEnd();
         }
-        if (struct.o3 != null) {
-          oprot.writeFieldBegin(O3_FIELD_DESC);
-          struct.o3.write(oprot);
-          oprot.writeFieldEnd();
-        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class drop_table_with_environment_context_resultTupleSchemeFactory implements SchemeFactory {
-      public drop_table_with_environment_context_resultTupleScheme getScheme() {
-        return new drop_table_with_environment_context_resultTupleScheme();
+    private static class get_tables_resultTupleSchemeFactory implements SchemeFactory {
+      public get_tables_resultTupleScheme getScheme() {
+        return new get_tables_resultTupleScheme();
       }
     }
 
-    private static class drop_table_with_environment_context_resultTupleScheme extends TupleScheme<drop_table_with_environment_context_result> {
+    private static class get_tables_resultTupleScheme extends TupleScheme<get_tables_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
-        if (struct.isSetO1()) {
+        if (struct.isSetSuccess()) {
           optionals.set(0);
         }
-        if (struct.isSetO3()) {
+        if (struct.isSetO1()) {
           optionals.set(1);
         }
         oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (String _iter654 : struct.success)
+            {
+              oprot.writeString(_iter654);
+            }
+          }
+        }
         if (struct.isSetO1()) {
           struct.o1.write(oprot);
         }
-        if (struct.isSetO3()) {
-          struct.o3.write(oprot);
-        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, drop_table_with_environment_context_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
-          struct.o1 = new NoSuchObjectException();
-          struct.o1.read(iprot);
-          struct.setO1IsSet(true);
+          {
+            org.apache.thrift.protocol.TList _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list655.size);
+            String _elem656;
+            for (int _i657 = 0; _i657 < _list655.size; ++_i657)
+            {
+              _elem656 = iprot.readString();
+              struct.success.add(_elem656);
+            }
+          }
+          struct.setSuccessIsSet(true);
         }
         if (incoming.get(1)) {
-          struct.o3 = new MetaException();
-          struct.o3.read(iprot);
-          struct.setO3IsSet(true);
+          struct.o1 = new MetaException();
+          struct.o1.read(iprot);
+          struct.setO1IsSet(true);
         }
       }
     }
 
   }
 
-  public static class get_tables_args implements org.apache.thrift.TBase<get_tables_args, get_tables_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_args");
+  public static class get_table_meta_args implements org.apache.thrift.TBase<get_table_meta_args, get_table_meta_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_table_meta_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_table_meta_args");
 
-    private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1);
-    private static final org.apache.thrift.protocol.TField PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("pattern", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField DB_PATTERNS_FIELD_DESC = new org.apache.thrift.protocol.TField("db_patterns", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField TBL_PATTERNS_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_patterns", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField TBL_TYPES_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_types", org.apache.thrift.protocol.TType.LIST, (short)3);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_tables_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_tables_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new get_table_meta_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_table_meta_argsTupleSchemeFactory());
     }
 
-    private String db_name; // required
-    private String pattern; // required
+    private String db_patterns; // required
+    private String tbl_patterns; // required
+    private List<String> tbl_types; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      DB_NAME((short)1, "db_name"),
-      PATTERN((short)2, "pattern");
+      DB_PATTERNS((short)1, "db_patterns"),
+      TBL_PATTERNS((short)2, "tbl_patterns"),
+      TBL_TYPES((short)3, "tbl_types");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -41028,10 +42159,12 @@ public class ThriftHiveMetastore {
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // DB_NAME
-            return DB_NAME;
-          case 2: // PATTERN
-            return PATTERN;
+          case 1: // DB_PATTERNS
+            return DB_PATTERNS;
+          case 2: // TBL_PATTERNS
+            return TBL_PATTERNS;
+          case 3: // TBL_TYPES
+            return TBL_TYPES;
           default:
             return null;
         }
@@ -41075,109 +42208,165 @@ public class ThriftHiveMetastore {
     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.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+      tmpMap.put(_Fields.DB_PATTERNS, new org.apache.thrift.meta_data.FieldMetaData("db_patterns", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.PATTERN, new org.apache.thrift.meta_data.FieldMetaData("pattern", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+      tmpMap.put(_Fields.TBL_PATTERNS, new org.apache.thrift.meta_data.FieldMetaData("tbl_patterns", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.TBL_TYPES, new org.apache.thrift.meta_data.FieldMetaData("tbl_types", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_tables_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_table_meta_args.class, metaDataMap);
     }
 
-    public get_tables_args() {
+    public get_table_meta_args() {
     }
 
-    public get_tables_args(
-      String db_name,
-      String pattern)
+    public get_table_meta_args(
+      String db_patterns,
+      String tbl_patterns,
+      List<String> tbl_types)
     {
       this();
-      this.db_name = db_name;
-      this.pattern = pattern;
+      this.db_patterns = db_patterns;
+      this.tbl_patterns = tbl_patterns;
+      this.tbl_types = tbl_types;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_tables_args(get_tables_args other) {
-      if (other.isSetDb_name()) {
-        this.db_name = other.db_name;
+    public get_table_meta_args(get_table_meta_args other) {
+      if (other.isSetDb_patterns()) {
+        this.db_patterns = other.db_patterns;
       }
-      if (other.isSetPattern()) {
-        this.pattern = other.pattern;
+      if (other.isSetTbl_patterns()) {
+        this.tbl_patterns = other.tbl_patterns;
+      }
+      if (other.isSetTbl_types()) {
+        List<String> __this__tbl_types = new ArrayList<String>(other.tbl_types);
+        this.tbl_types = __this__tbl_types;
       }
     }
 
-    public get_tables_args deepCopy() {
-      return new get_tables_args(this);
+    public get_table_meta_args deepCopy() {
+      return new get_table_meta_args(this);
     }
 
     @Override
     public void clear() {
-      this.db_name = null;
-      this.pattern = null;
+      this.db_patterns = null;
+      this.tbl_patterns = null;
+      this.tbl_types = null;
     }
 
-    public String getDb_name() {
-      return this.db_name;
+    public String getDb_patterns() {
+      return this.db_patterns;
     }
 
-    public void setDb_name(String db_name) {
-      this.db_name = db_name;
+    public void setDb_patterns(String db_patterns) {
+      this.db_patterns = db_patterns;
     }
 
-    public void unsetDb_name() {
-      this.db_name = null;
+    public void unsetDb_patterns() {
+      this.db_patterns = null;
     }
 
-    /** Returns true if field db_name is set (has been assigned a value) and false otherwise */
-    public boolean isSetDb_name() {
-      return this.db_name != null;
+    /** Returns true if field db_patterns is set (has been assigned a value) and false otherwise */
+    public boolean isSetDb_patterns() {
+      return this.db_patterns != null;
     }
 
-    public void setDb_nameIsSet(boolean value) {
+    public void setDb_patternsIsSet(boolean value) {
       if (!value) {
-        this.db_name = null;
+        this.db_patterns = null;
       }
     }
 
-    public String getPattern() {
-      return this.pattern;
+    public String getTbl_patterns() {
+      return this.tbl_patterns;
     }
 
-    public void setPattern(String pattern) {
-      this.pattern = pattern;
+    public void setTbl_patterns(String tbl_patterns) {
+      this.tbl_patterns = tbl_patterns;
     }
 
-    public void unsetPattern() {
-      this.pattern = null;
+    public void unsetTbl_patterns() {
+      this.tbl_patterns = null;
     }
 
-    /** Returns true if field pattern is set (has been assigned a value) and false otherwise */
-    public boolean isSetPattern() {
-      return this.pattern != null;
+    /** Returns true if field tbl_patterns is set (has been assigned a value) and false otherwise */
+    public boolean isSetTbl_patterns() {
+      return this.tbl_patterns != null;
     }
 
-    public void setPatternIsSet(boolean value) {
+    public void setTbl_patternsIsSet(boolean value) {
       if (!value) {
-        this.pattern = null;
+        this.tbl_patterns = null;
+      }
+    }
+
+    public int getTbl_typesSize() {
+      return (this.tbl_types == null) ? 0 : this.tbl_types.size();
+    }
+
+    public java.util.Iterator<String> getTbl_typesIterator() {
+      return (this.tbl_types == null) ? null : this.tbl_types.iterator();
+    }
+
+    public void addToTbl_types(String elem) {
+      if (this.tbl_types == null) {
+        this.tbl_types = new ArrayList<String>();
+      }
+      this.tbl_types.add(elem);
+    }
+
+    public List<String> getTbl_types() {
+      return this.tbl_types;
+    }
+
+    public void setTbl_types(List<String> tbl_types) {
+      this.tbl_types = tbl_types;
+    }
+
+    public void unsetTbl_types() {
+      this.tbl_types = null;
+    }
+
+    /** Returns true if field tbl_types is set (has been assigned a value) and false otherwise */
+    public boolean isSetTbl_types() {
+      return this.tbl_types != null;
+    }
+
+    public void setTbl_typesIsSet(boolean value) {
+      if (!value) {
+        this.tbl_types = null;
       }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
-      case DB_NAME:
+      case DB_PATTERNS:
         if (value == null) {
-          unsetDb_name();
+          unsetDb_patterns();
         } else {
-          setDb_name((String)value);
+          setDb_patterns((String)value);
         }
         break;
 
-      case PATTERN:
+      case TBL_PATTERNS:
         if (value == null) {
-          unsetPattern();
+          unsetTbl_patterns();
         } else {
-          setPattern((String)value);
+          setTbl_patterns((String)value);
+        }
+        break;
+
+      case TBL_TYPES:
+        if (value == null) {
+          unsetTbl_types();
+        } else {
+          setTbl_types((List<String>)value);
         }
         break;
 
@@ -41186,11 +42375,14 @@ public class ThriftHiveMetastore {
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
-      case DB_NAME:
-        return getDb_name();
+      case DB_PATTERNS:
+        return getDb_patterns();
 
-      case PATTERN:
-        return getPattern();
+      case TBL_PATTERNS:
+        return getTbl_patterns();
+
+      case TBL_TYPES:
+        return getTbl_types();
 
       }
       throw new IllegalStateException();
@@ -41203,10 +42395,12 @@ public class ThriftHiveMetastore {
       }
 
       switch (field) {
-      case DB_NAME:
-        return isSetDb_name();
-      case PATTERN:
-        return isSetPattern();
+      case DB_PATTERNS:
+        return isSetDb_patterns();
+      case TBL_PATTERNS:
+        return isSetTbl_patterns();
+      case TBL_TYPES:
+        return isSetTbl_types();
       }
       throw new IllegalStateException();
     }
@@ -41215,30 +42409,39 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_tables_args)
-        return this.equals((get_tables_args)that);
+      if (that instanceof get_table_meta_args)
+        return this.equals((get_table_meta_args)that);
       return false;
     }
 
-    public boolean equals(get_tables_args that) {
+    public boolean equals(get_table_meta_args that) {
       if (that == null)
         return false;
 
-      boolean this_present_db_name = true && this.isSetDb_name();
-      boolean that_present_db_name = true && that.isSetDb_name();
-      if (this_present_db_name || that_present_db_name) {
-        if (!(this_present_db_name && that_present_db_name))
+      boolean this_present_db_patterns = true && this.isSetDb_patterns();
+      boolean that_present_db_patterns = true && that.isSetDb_patterns();
+      if (this_present_db_patterns || that_present_db_patterns) {
+        if (!(this_present_db_patterns && that_present_db_patterns))
           return false;
-        if (!this.db_name.equals(that.db_name))
+        if (!this.db_patterns.equals(that.db_patterns))
           return false;
       }
 
-      boolean this_present_pattern = true && this.isSetPattern();
-      boolean that_present_pattern = true && that.isSetPattern();
-      if (this_present_pattern || that_present_pattern) {
-        if (!(this_present_pattern && that_present_pattern))
+      boolean this_present_tbl_patterns = true && this.isSetTbl_patterns();
+      boolean that_present_tbl_patterns = true && that.isSetTbl_patterns();
+      if (this_present_tbl_patterns || that_present_tbl_patterns) {
+        if (!(this_present_tbl_patterns && that_present_tbl_patterns))
           return false;
-        if (!this.pattern.equals(that.pattern))
+        if (!this.tbl_patterns.equals(that.tbl_patterns))
+          return false;
+      }
+
+      boolean this_present_tbl_types = true && this.isSetTbl_types();
+      boolean that_present_tbl_types = true && that.isSetTbl_types();
+      if (this_present_tbl_types || that_present_tbl_types) {
+        if (!(this_present_tbl_types && that_present_tbl_types))
+          return false;
+        if (!this.tbl_types.equals(that.tbl_types))
           return false;
       }
 
@@ -41249,43 +42452,58 @@ public class ThriftHiveMetastore {
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
-      boolean present_db_name = true && (isSetDb_name());
-      list.add(present_db_name);
-      if (present_db_name)
-        list.add(db_name);
+      boolean present_db_patterns = true && (isSetDb_patterns());
+      list.add(present_db_patterns);
+      if (present_db_patterns)
+        list.add(db_patterns);
 
-      boolean present_pattern = true && (isSetPattern());
-      list.add(present_pattern);
-      if (present_pattern)
-        list.add(pattern);
+      boolean present_tbl_patterns = true && (isSetTbl_patterns());
+      list.add(present_tbl_patterns);
+      if (present_tbl_patterns)
+        list.add(tbl_patterns);
+
+      boolean present_tbl_types = true && (isSetTbl_types());
+      list.add(present_tbl_types);
+      if (present_tbl_types)
+        list.add(tbl_types);
 
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(get_tables_args other) {
+    public int compareTo(get_table_meta_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(other.isSetDb_name());
+      lastComparison = Boolean.valueOf(isSetDb_patterns()).compareTo(other.isSetDb_patterns());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetDb_name()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_name, other.db_name);
+      if (isSetDb_patterns()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_patterns, other.db_patterns);
         if (lastComparison != 0) {
           return lastComparison;
         }
       }
-      lastComparison = Boolean.valueOf(isSetPattern()).compareTo(other.isSetPattern());
+      lastComparison = Boolean.valueOf(isSetTbl_patterns()).compareTo(other.isSetTbl_patterns());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetPattern()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pattern, other.pattern);
+      if (isSetTbl_patterns()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tbl_patterns, other.tbl_patterns);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetTbl_types()).compareTo(other.isSetTbl_types());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTbl_types()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tbl_types, other.tbl_types);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -41307,22 +42525,30 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_tables_args(");
+      StringBuilder sb = new StringBuilder("get_table_meta_args(");
       boolean first = true;
 
-      sb.append("db_name:");
-      if (this.db_name == null) {
+      sb.append("db_patterns:");
+      if (this.db_patterns == null) {
         sb.append("null");
       } else {
-        sb.append(this.db_name);
+        sb.append(this.db_patterns);
       }
       first = false;
       if (!first) sb.append(", ");
-      sb.append("pattern:");
-      if (this.pattern == null) {
+      sb.append("tbl_patterns:");
+      if (this.tbl_patterns == null) {
         sb.append("null");
       } else {
-        sb.append(this.pattern);
+        sb.append(this.tbl_patterns);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("tbl_types:");
+      if (this.tbl_types == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tbl_types);
       }
       first = false;
       sb.append(")");
@@ -41350,15 +42576,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class get_tables_argsStandardSchemeFactory implements SchemeFactory {
-      public get_tables_argsStandardScheme getScheme() {
-        return new get_tables_argsStandardScheme();
+    private static class get_table_meta_argsStandardSchemeFactory implements SchemeFactory {
+      public get_table_meta_argsStandardScheme getScheme() {
+        return new get_table_meta_argsStandardScheme();
       }
     }
 
-    private static class get_tables_argsStandardScheme extends StandardScheme<get_tables_args> {
+    private static class get_table_meta_argsStandardScheme extends StandardScheme<get_table_meta_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_meta_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -41368,18 +42594,36 @@ public class ThriftHiveMetastore {
             break;
           }
           switch (schemeField.id) {
-            case 1: // DB_NAME
+            case 1: // DB_PATTERNS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.db_name = iprot.readString();
-                struct.setDb_nameIsSet(true);
+                struct.db_patterns = iprot.readString();
+                struct.setDb_patternsIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 2: // PATTERN
+            case 2: // TBL_PATTERNS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.pattern = iprot.readString();
-                struct.setPatternIsSet(true);
+                struct.tbl_patterns = iprot.readString();
+                struct.setTbl_patternsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // TBL_TYPES
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list658.size);
+                  String _elem659;
+                  for (int _i660 = 0; _i660 < _list658.size; ++_i660)
+                  {
+                    _elem659 = iprot.readString();
+                    struct.tbl_types.add(_elem659);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setTbl_typesIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -41393,18 +42637,30 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_meta_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.db_name != null) {
-          oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
-          oprot.writeString(struct.db_name);
+        if (struct.db_patterns != null) {
+          oprot.writeFieldBegin(DB_PATTERNS_FIELD_DESC);
+          oprot.writeString(struct.db_patterns);
           oprot.writeFieldEnd();
         }
-        if (struct.pattern != null) {
-          oprot.writeFieldBegin(PATTERN_FIELD_DESC);
-          oprot.writeString(struct.pattern);
+        if (struct.tbl_patterns != null) {
+          oprot.writeFieldBegin(TBL_PATTERNS_FIELD_DESC);
+          oprot.writeString(struct.tbl_patterns);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tbl_types != null) {
+          oprot.writeFieldBegin(TBL_TYPES_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_types.size()));
+            for (String _iter661 : struct.tbl_types)
+            {
+              oprot.writeString(_iter661);
+            }
+            oprot.writeListEnd();
+          }
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -41413,63 +42669,88 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class get_tables_argsTupleSchemeFactory implements SchemeFactory {
-      public get_tables_argsTupleScheme getScheme() {
-        return new get_tables_argsTupleScheme();
+    private static class get_table_meta_argsTupleSchemeFactory implements SchemeFactory {
+      public get_table_meta_argsTupleScheme getScheme() {
+        return new get_table_meta_argsTupleScheme();
       }
     }
 
-    private static class get_tables_argsTupleScheme extends TupleScheme<get_tables_args> {
+    private static class get_table_meta_argsTupleScheme extends TupleScheme<get_table_meta_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_table_meta_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
-        if (struct.isSetDb_name()) {
+        if (struct.isSetDb_patterns()) {
           optionals.set(0);
         }
-        if (struct.isSetPattern()) {
+        if (struct.isSetTbl_patterns()) {
           optionals.set(1);
         }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetDb_name()) {
-          oprot.writeString(struct.db_name);
+        if (struct.isSetTbl_types()) {
+          optionals.set(2);
         }
-        if (struct.isSetPattern()) {
-          oprot.writeString(struct.pattern);
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetDb_patterns()) {
+          oprot.writeString(struct.db_patterns);
+        }
+        if (struct.isSetTbl_patterns()) {
+          oprot.writeString(struct.tbl_patterns);
+        }
+        if (struct.isSetTbl_types()) {
+          {
+            oprot.writeI32(struct.tbl_types.size());
+            for (String _iter662 : struct.tbl_types)
+            {
+              oprot.writeString(_iter662);
+            }
+          }
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_table_meta_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(2);
+        BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
-          struct.db_name = iprot.readString();
-          struct.setDb_nameIsSet(true);
+          struct.db_patterns = iprot.readString();
+          struct.setDb_patternsIsSet(true);
         }
         if (incoming.get(1)) {
-          struct.pattern = iprot.readString();
-          struct.setPatternIsSet(true);
+          struct.tbl_patterns = iprot.readString();
+          struct.setTbl_patternsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          {
+            org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list663.size);
+            String _elem664;
+            for (int _i665 = 0; _i665 < _list663.size; ++_i665)
+            {
+              _elem664 = iprot.readString();
+              struct.tbl_types.add(_elem664);
+            }
+          }
+          struct.setTbl_typesIsSet(true);
         }
       }
     }
 
   }
 
-  public static class get_tables_result implements org.apache.thrift.TBase<get_tables_result, get_tables_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_result");
+  public static class get_table_meta_result implements org.apache.thrift.TBase<get_table_meta_result, get_table_meta_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_table_meta_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_table_meta_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
     private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_tables_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_tables_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new get_table_meta_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_table_meta_resultTupleSchemeFactory());
     }
 
-    private List<String> success; // required
+    private List<TableMeta> success; // required
     private MetaException o1; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -41539,18 +42820,18 @@ public class ThriftHiveMetastore {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TableMeta.class))));
       tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_tables_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_table_meta_result.class, metaDataMap);
     }
 
-    public get_tables_result() {
+    public get_table_meta_result() {
     }
 
-    public get_tables_result(
-      List<String> success,
+    public get_table_meta_result(
+      List<TableMeta> success,
       MetaException o1)
     {
       this();
@@ -41561,9 +42842,12 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_tables_result(get_tables_result other) {
+    public get_table_meta_result(get_table_meta_result other) {
       if (other.isSetSuccess()) {
-        List<String> __this__success = new ArrayList<String>(other.success);
+        List<TableMeta> __this__success = new ArrayList<TableMeta>(other.success.size());
+        for (TableMeta other_element : other.success) {
+          __this__success.add(new TableMeta(other_element));
+        }
         this.success = __this__success;
       }
       if (other.isSetO1()) {
@@ -41571,8 +42855,8 @@ public class ThriftHiveMetastore {
       }
     }
 
-    public get_tables_result deepCopy() {
-      return new get_tables_result(this);
+    public get_table_meta_result deepCopy() {
+      return new get_table_meta_result(this);
     }
 
     @Override
@@ -41585,22 +42869,22 @@ public class ThriftHiveMetastore {
       return (this.success == null) ? 0 : this.success.size();
     }
 
-    public java.util.Iterator<String> getSuccessIterator() {
+    public java.util.Iterator<TableMeta> getSuccessIterator() {
       return (this.success == null) ? null : this.success.iterator();
     }
 
-    public void addToSuccess(String elem) {
+    public void addToSuccess(TableMeta elem) {
       if (this.success == null) {
-        this.success = new ArrayList<String>();
+        this.success = new ArrayList<TableMeta>();
       }
       this.success.add(elem);
     }
 
-    public List<String> getSuccess() {
+    public List<TableMeta> getSuccess() {
       return this.success;
     }
 
-    public void setSuccess(List<String> success) {
+    public void setSuccess(List<TableMeta> success) {
       this.success = success;
     }
 
@@ -41648,7 +42932,7 @@ public class ThriftHiveMetastore {
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((List<String>)value);
+          setSuccess((List<TableMeta>)value);
         }
         break;
 
@@ -41694,12 +42978,12 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_tables_result)
-        return this.equals((get_tables_result)that);
+      if (that instanceof get_table_meta_result)
+        return this.equals((get_table_meta_result)that);
       return false;
     }
 
-    public boolean equals(get_tables_result that) {
+    public boolean equals(get_table_meta_result that) {
       if (that == null)
         return false;
 
@@ -41742,7 +43026,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(get_tables_result other) {
+    public int compareTo(get_table_meta_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -41786,7 +43070,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_tables_result(");
+      StringBuilder sb = new StringBuilder("get_table_meta_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -41829,15 +43113,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class get_tables_resultStandardSchemeFactory implements SchemeFactory {
-      public get_tables_resultStandardScheme getScheme() {
-        return new get_tables_resultStandardScheme();
+    private static class get_table_meta_resultStandardSchemeFactory implements SchemeFactory {
+      public get_table_meta_resultStandardScheme getScheme() {
+        return new get_table_meta_resultStandardScheme();
       }
     }
 
-    private static class get_tables_resultStandardScheme extends StandardScheme<get_tables_result> {
+    private static class get_table_meta_resultStandardScheme extends StandardScheme<get_table_meta_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_meta_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -41850,13 +43134,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list650.size);
-                  String _elem651;
-                  for (int _i652 = 0; _i652 < _list650.size; ++_i652)
+                  org.apache.thrift.protocol.TList _list666 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list666.size);
+                  TableMeta _elem667;
+                  for (int _i668 = 0; _i668 < _list666.size; ++_i668)
                   {
-                    _elem651 = iprot.readString();
-                    struct.success.add(_elem651);
+                    _elem667 = new TableMeta();
+                    _elem667.read(iprot);
+                    struct.success.add(_elem667);
                   }
                   iprot.readListEnd();
                 }
@@ -41883,17 +43168,17 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_meta_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
         if (struct.success != null) {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
-            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter653 : struct.success)
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
+            for (TableMeta _iter669 : struct.success)
             {
-              oprot.writeString(_iter653);
+              _iter669.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -41910,16 +43195,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class get_tables_resultTupleSchemeFactory implements SchemeFactory {
-      public get_tables_resultTupleScheme getScheme() {
-        return new get_tables_resultTupleScheme();
+    private static class get_table_meta_resultTupleSchemeFactory implements SchemeFactory {
+      public get_table_meta_resultTupleScheme getScheme() {
+        return new get_table_meta_resultTupleScheme();
       }
     }
 
-    private static class get_tables_resultTupleScheme extends TupleScheme<get_tables_result> {
+    private static class get_table_meta_resultTupleScheme extends TupleScheme<get_table_meta_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_table_meta_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSuccess()) {
@@ -41932,9 +43217,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter654 : struct.success)
+            for (TableMeta _iter670 : struct.success)
             {
-              oprot.writeString(_iter654);
+              _iter670.write(oprot);
             }
           }
         }
@@ -41944,18 +43229,19 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_table_meta_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list655.size);
-            String _elem656;
-            for (int _i657 = 0; _i657 < _list655.size; ++_i657)
+            org.apache.thrift.protocol.TList _list671 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list671.size);
+            TableMeta _elem672;
+            for (int _i673 = 0; _i673 < _list671.size; ++_i673)
             {
-              _elem656 = iprot.readString();
-              struct.success.add(_elem656);
+              _elem672 = new TableMeta();
+              _elem672.read(iprot);
+              struct.success.add(_elem672);
             }
           }
           struct.setSuccessIsSet(true);
@@ -42721,13 +44007,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list658.size);
-                  String _elem659;
-                  for (int _i660 = 0; _i660 < _list658.size; ++_i660)
+                  org.apache.thrift.protocol.TList _list674 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list674.size);
+                  String _elem675;
+                  for (int _i676 = 0; _i676 < _list674.size; ++_i676)
                   {
-                    _elem659 = iprot.readString();
-                    struct.success.add(_elem659);
+                    _elem675 = iprot.readString();
+                    struct.success.add(_elem675);
                   }
                   i

<TRUNCATED>