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 2016/10/04 14:10:54 UTC

[1/7] hive git commit: HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)

Repository: hive
Updated Branches:
  refs/heads/master e19f0e35e -> 21a0142f3


http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index de6adb5..691c3a8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -1313,7 +1313,7 @@ public class Hive {
    * @throws HiveException
    */
   public List<String> getAllTables() throws HiveException {
-    return getAllTables(SessionState.get().getCurrentDatabase());
+    return getTablesByType(SessionState.get().getCurrentDatabase(), null, null);
   }
 
   /**
@@ -1323,7 +1323,7 @@ public class Hive {
    * @throws HiveException
    */
   public List<String> getAllTables(String dbName) throws HiveException {
-    return getTablesByPattern(dbName, ".*");
+    return getTablesByType(dbName, ".*", null);
   }
 
   /**
@@ -1336,8 +1336,8 @@ public class Hive {
    * @throws HiveException
    */
   public List<String> getTablesByPattern(String tablePattern) throws HiveException {
-    return getTablesByPattern(SessionState.get().getCurrentDatabase(),
-        tablePattern);
+    return getTablesByType(SessionState.get().getCurrentDatabase(),
+        tablePattern, null);
   }
 
   /**
@@ -1349,11 +1349,7 @@ public class Hive {
    * @throws HiveException
    */
   public List<String> getTablesByPattern(String dbName, String tablePattern) throws HiveException {
-    try {
-      return getMSC().getTables(dbName, tablePattern);
-    } catch (Exception e) {
-      throw new HiveException(e);
-    }
+    return getTablesByType(dbName, tablePattern, null);
   }
 
   /**
@@ -1369,8 +1365,38 @@ public class Hive {
    */
   public List<String> getTablesForDb(String database, String tablePattern)
       throws HiveException {
+    return getTablesByType(database, tablePattern, null);
+  }
+
+  /**
+   * Returns all existing tables of a type (VIRTUAL_VIEW|EXTERNAL_TABLE|MANAGED_TABLE) from the specified
+   * database which match the given pattern. The matching occurs as per Java regular expressions.
+   * @param dbName Database name to find the tables in. if null, uses the current database in this session.
+   * @param pattern A pattern to match for the table names.If null, returns all names from this DB.
+   * @param type The type of tables to return. VIRTUAL_VIEWS for views. If null, returns all tables and views.
+   * @return list of table names that match the pattern.
+   * @throws HiveException
+   */
+  public List<String> getTablesByType(String dbName, String pattern, TableType type)
+      throws HiveException {
+    List<String> retList = new ArrayList<String>();
+    if (dbName == null)
+      dbName = SessionState.get().getCurrentDatabase();
+
     try {
-      return getMSC().getTables(database, tablePattern);
+      if (type != null) {
+        if (pattern != null) {
+          return getMSC().getTables(dbName, pattern, type);
+        } else {
+          return getMSC().getTables(dbName, ".*", type);
+        }
+      } else {
+        if (pattern != null) {
+          return getMSC().getTables(dbName, pattern);
+        } else {
+          return getMSC().getTables(dbName, ".*");
+        }
+      }
     } catch (Exception e) {
       throw new HiveException(e);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index 8b0db4a..a264c4d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -394,6 +394,10 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       ctx.setResFile(ctx.getLocalTmpPath());
       analyzeShowConf(ast);
       break;
+    case HiveParser.TOK_SHOWVIEWS:
+      ctx.setResFile(ctx.getLocalTmpPath());
+      analyzeShowViews(ast);
+      break;
     case HiveParser.TOK_DESCFUNCTION:
       ctx.setResFile(ctx.getLocalTmpPath());
       analyzeDescFunction(ast);
@@ -2402,6 +2406,45 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     setFetchTask(createFetchTask(showConfDesc.getSchema()));
   }
 
+  private void analyzeShowViews(ASTNode ast) throws SemanticException {
+    ShowTablesDesc showViewsDesc;
+    String dbName = SessionState.get().getCurrentDatabase();
+    String viewNames = null;
+
+    if (ast.getChildCount() > 3) {
+      throw new SemanticException(ErrorMsg.GENERIC_ERROR.getMsg());
+    }
+
+    switch (ast.getChildCount()) {
+    case 1: // Uses a pattern
+      viewNames = unescapeSQLString(ast.getChild(0).getText());
+      showViewsDesc = new ShowTablesDesc(ctx.getResFile(), dbName, viewNames, TableType.VIRTUAL_VIEW);
+      break;
+    case 2: // Specifies a DB
+      assert (ast.getChild(0).getType() == HiveParser.TOK_FROM);
+      dbName = unescapeIdentifier(ast.getChild(1).getText());
+      validateDatabase(dbName);
+      showViewsDesc = new ShowTablesDesc(ctx.getResFile(), dbName);
+      showViewsDesc.setType(TableType.VIRTUAL_VIEW);
+      break;
+    case 3: // Uses a pattern and specifies a DB
+      assert (ast.getChild(0).getType() == HiveParser.TOK_FROM);
+      dbName = unescapeIdentifier(ast.getChild(1).getText());
+      viewNames = unescapeSQLString(ast.getChild(2).getText());
+      validateDatabase(dbName);
+      showViewsDesc = new ShowTablesDesc(ctx.getResFile(), dbName, viewNames, TableType.VIRTUAL_VIEW);
+      break;
+    default: // No pattern or DB
+      showViewsDesc = new ShowTablesDesc(ctx.getResFile(), dbName);
+      showViewsDesc.setType(TableType.VIRTUAL_VIEW);
+      break;
+    }
+
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
+        showViewsDesc), conf));
+    setFetchTask(createFetchTask(showViewsDesc.getSchema()));
+  }
+
   /**
    * Add the task according to the parsed command tree. This is used for the CLI
    * command "LOCK TABLE ..;".

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index b623187..58e76c8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -216,6 +216,7 @@ KW_MINUS: 'MINUS';
 KW_FETCH: 'FETCH';
 KW_INTERSECT: 'INTERSECT';
 KW_VIEW: 'VIEW';
+KW_VIEWS: 'VIEWS';
 KW_IN: 'IN';
 KW_DATABASE: 'DATABASE';
 KW_DATABASES: 'DATABASES';

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 62bbcc6..bf78545 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -368,6 +368,7 @@ TOK_ANONYMOUS;
 TOK_COL_NAME;
 TOK_URI_TYPE;
 TOK_SERVER_TYPE;
+TOK_SHOWVIEWS;
 TOK_START_TRANSACTION;
 TOK_ISOLATION_LEVEL;
 TOK_ISOLATION_SNAPSHOT;
@@ -1443,6 +1444,7 @@ showStatement
 @after { popMsg(state); }
     : KW_SHOW (KW_DATABASES|KW_SCHEMAS) (KW_LIKE showStmtIdentifier)? -> ^(TOK_SHOWDATABASES showStmtIdentifier?)
     | KW_SHOW KW_TABLES ((KW_FROM|KW_IN) db_name=identifier)? (KW_LIKE showStmtIdentifier|showStmtIdentifier)?  -> ^(TOK_SHOWTABLES (TOK_FROM $db_name)? showStmtIdentifier?)
+    | KW_SHOW KW_VIEWS ((KW_FROM|KW_IN) db_name=identifier)? (KW_LIKE showStmtIdentifier|showStmtIdentifier)?  -> ^(TOK_SHOWVIEWS (TOK_FROM $db_name)? showStmtIdentifier?)
     | KW_SHOW KW_COLUMNS (KW_FROM|KW_IN) tableName ((KW_FROM|KW_IN) db_name=identifier)?
     -> ^(TOK_SHOWCOLUMNS tableName $db_name?)
     | KW_SHOW KW_FUNCTIONS (KW_LIKE showFunctionIdentifier|showFunctionIdentifier)?  -> ^(TOK_SHOWFUNCTIONS KW_LIKE? showFunctionIdentifier?)

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
index 18439f9..4f0ead0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
@@ -78,6 +78,7 @@ public final class SemanticAnalyzerFactory {
     commandType.put(HiveParser.TOK_SHOWLOCKS, HiveOperation.SHOWLOCKS);
     commandType.put(HiveParser.TOK_SHOWDBLOCKS, HiveOperation.SHOWLOCKS);
     commandType.put(HiveParser.TOK_SHOWCONF, HiveOperation.SHOWCONF);
+    commandType.put(HiveParser.TOK_SHOWVIEWS, HiveOperation.SHOWVIEWS);
     commandType.put(HiveParser.TOK_CREATEFUNCTION, HiveOperation.CREATEFUNCTION);
     commandType.put(HiveParser.TOK_DROPFUNCTION, HiveOperation.DROPFUNCTION);
     commandType.put(HiveParser.TOK_RELOADFUNCTION, HiveOperation.RELOADFUNCTION);
@@ -256,6 +257,7 @@ public final class SemanticAnalyzerFactory {
       case HiveParser.TOK_SHOW_TRANSACTIONS:
       case HiveParser.TOK_ABORT_TRANSACTIONS:
       case HiveParser.TOK_SHOWCONF:
+      case HiveParser.TOK_SHOWVIEWS:
       case HiveParser.TOK_CREATEINDEX:
       case HiveParser.TOK_DROPINDEX:
       case HiveParser.TOK_ALTERTABLE_CLUSTER_SORT:

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
index 8b54609..e069acd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
@@ -524,7 +524,7 @@ public class DDLWork implements Serializable {
     this.cacheMetadataDesc = cacheMetadataDesc;
   }
 
-    /**
+  /**
    * @return Create Database descriptor
    */
   public CreateDatabaseDesc getCreateDatabaseDesc() {

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
index 6fd4df4..d333f91 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
@@ -72,6 +72,7 @@ public enum HiveOperation {
   SHOWPARTITIONS("SHOWPARTITIONS", null, null),
   SHOWLOCKS("SHOWLOCKS", null, null),
   SHOWCONF("SHOWCONF", null, null),
+  SHOWVIEWS("SHOWVIEWS", null, null),
   CREATEFUNCTION("CREATEFUNCTION", null, null),
   DROPFUNCTION("DROPFUNCTION", null, null),
   RELOADFUNCTION("RELOADFUNCTION", null, null),

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTablesDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTablesDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTablesDesc.java
index 9e9d639..c9193b7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTablesDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTablesDesc.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.plan;
 import java.io.Serializable;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
 
 
@@ -34,6 +35,7 @@ public class ShowTablesDesc extends DDLDesc implements Serializable {
   String pattern;
   String dbName;
   String resFile;
+  TableType type;
   /**
    * table name for the result of show tables.
    */
@@ -82,6 +84,17 @@ public class ShowTablesDesc extends DDLDesc implements Serializable {
   }
 
   /**
+   * @param type
+   *          type of the tables to show
+   */
+  public ShowTablesDesc(Path resFile, String dbName, String pattern, TableType type) {
+    this.resFile = resFile.toString();
+    this.dbName = dbName;
+    this.pattern = pattern;
+    this.type    = type;
+  }
+
+  /**
    * @return the pattern
    */
   @Explain(displayName = "pattern")
@@ -98,6 +111,22 @@ public class ShowTablesDesc extends DDLDesc implements Serializable {
   }
 
   /**
+   * @return the table type to be fetched
+   */
+  @Explain(displayName = "type")
+  public TableType getType() {
+    return type;
+  }
+
+  /**
+   * @param type
+   *          the table type to set
+   */
+  public void setType(TableType type) {
+    this.type = type;
+  }
+
+  /**
    * @return the resFile
    */
   @Explain(displayName = "result file", explainLevels = { Level.EXTENDED })

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
index 2da4249..7da44e8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
@@ -76,6 +76,7 @@ public enum HiveOperationType {
   SHOWPARTITIONS,
   SHOWLOCKS,
   SHOWCONF,
+  SHOWVIEWS,
   CREATEFUNCTION,
   DROPFUNCTION,
   RELOADFUNCTION,

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
index e5389bd..e7bbd54 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
@@ -360,6 +360,8 @@ public class Operation2Privilege {
 (null, null));
     op2Priv.put(HiveOperationType.SHOWCONF, PrivRequirement.newIOPrivRequirement
 (null, null));
+    op2Priv.put(HiveOperationType.SHOWVIEWS, PrivRequirement.newIOPrivRequirement
+(null, null));
 
     op2Priv.put(HiveOperationType.LOCKTABLE, PrivRequirement.newIOPrivRequirement
 (null, null));

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/test/queries/clientpositive/show_views.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/show_views.q b/ql/src/test/queries/clientpositive/show_views.q
new file mode 100644
index 0000000..61aad43
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/show_views.q
@@ -0,0 +1,56 @@
+CREATE DATABASE test1;
+CREATE DATABASE test2;
+
+USE test1;
+CREATE TABLE shtb_test1(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE;
+CREATE VIEW shtb_test1_view1 AS SELECT * FROM shtb_test1 where KEY > 1000 and KEY < 2000;
+CREATE VIEW shtb_test1_view2 AS SELECT * FROM shtb_test1 where KEY > 100 and KEY < 200;
+CREATE VIEW shtb_full_view2 AS SELECT * FROM shtb_test1;
+USE test2;
+CREATE TABLE shtb_test1(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE;
+CREATE TABLE shtb_test2(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE;
+CREATE VIEW shtb_test1_view1 AS SELECT * FROM shtb_test1 where KEY > 1000 and KEY < 2000;
+CREATE VIEW shtb_test2_view2 AS SELECT * FROM shtb_test2 where KEY > 100 and KEY < 200;
+
+USE test1;
+SHOW VIEWS;
+SHOW VIEWS 'test_*';
+SHOW VIEWS '*view2';
+SHOW VIEWS LIKE 'test_view1|test_view2';
+
+USE test2;
+SHOW VIEWS 'shtb_*';
+
+-- SHOW VIEWS basic syntax tests
+USE default;
+SHOW VIEWS FROM test1;
+SHOW VIEWS FROM test2;
+SHOW VIEWS IN test1;
+SHOW VIEWS IN default;
+SHOW VIEWS IN test1 "shtb_test_*";
+SHOW VIEWS IN test2 LIKE "nomatch";
+
+-- SHOW VIEWS from a database with a name that requires escaping
+CREATE DATABASE `database`;
+USE `database`;
+CREATE TABLE foo(a INT);
+CREATE VIEW fooview AS SELECT * FROM foo;
+USE default;
+SHOW VIEWS FROM `database` LIKE "fooview";
+
+DROP VIEW fooview;
+DROP TABLE foo;
+
+USE test1;
+DROP VIEW shtb_test1_view1;
+DROP VIEW shtb_test1_view2;
+DROP VIEW shtb_full_view2;
+DROP TABLE shtb_test1;
+DROP DATABASE test1;
+
+USE test2;
+DROP VIEW shtb_test1_view1;
+DROP VIEW shtb_test2_view2;
+DROP TABLE shtb_test1;
+DROP TABLE shtb_test2;
+DROP DATABASE test2;

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/test/results/clientpositive/show_views.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/show_views.q.out b/ql/src/test/results/clientpositive/show_views.q.out
new file mode 100644
index 0000000..61b5f1d
--- /dev/null
+++ b/ql/src/test/results/clientpositive/show_views.q.out
@@ -0,0 +1,320 @@
+PREHOOK: query: CREATE DATABASE test1
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:test1
+POSTHOOK: query: CREATE DATABASE test1
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:test1
+PREHOOK: query: CREATE DATABASE test2
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:test2
+POSTHOOK: query: CREATE DATABASE test2
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:test2
+PREHOOK: query: USE test1
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:test1
+POSTHOOK: query: USE test1
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:test1
+PREHOOK: query: CREATE TABLE shtb_test1(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:test1
+PREHOOK: Output: test1@shtb_test1
+POSTHOOK: query: CREATE TABLE shtb_test1(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:test1
+POSTHOOK: Output: test1@shtb_test1
+PREHOOK: query: CREATE VIEW shtb_test1_view1 AS SELECT * FROM shtb_test1 where KEY > 1000 and KEY < 2000
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: test1@shtb_test1
+PREHOOK: Output: database:test1
+PREHOOK: Output: test1@shtb_test1_view1
+POSTHOOK: query: CREATE VIEW shtb_test1_view1 AS SELECT * FROM shtb_test1 where KEY > 1000 and KEY < 2000
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: test1@shtb_test1
+POSTHOOK: Output: database:test1
+POSTHOOK: Output: test1@shtb_test1_view1
+PREHOOK: query: CREATE VIEW shtb_test1_view2 AS SELECT * FROM shtb_test1 where KEY > 100 and KEY < 200
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: test1@shtb_test1
+PREHOOK: Output: database:test1
+PREHOOK: Output: test1@shtb_test1_view2
+POSTHOOK: query: CREATE VIEW shtb_test1_view2 AS SELECT * FROM shtb_test1 where KEY > 100 and KEY < 200
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: test1@shtb_test1
+POSTHOOK: Output: database:test1
+POSTHOOK: Output: test1@shtb_test1_view2
+PREHOOK: query: CREATE VIEW shtb_full_view2 AS SELECT * FROM shtb_test1
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: test1@shtb_test1
+PREHOOK: Output: database:test1
+PREHOOK: Output: test1@shtb_full_view2
+POSTHOOK: query: CREATE VIEW shtb_full_view2 AS SELECT * FROM shtb_test1
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: test1@shtb_test1
+POSTHOOK: Output: database:test1
+POSTHOOK: Output: test1@shtb_full_view2
+PREHOOK: query: USE test2
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:test2
+POSTHOOK: query: USE test2
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:test2
+PREHOOK: query: CREATE TABLE shtb_test1(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:test2
+PREHOOK: Output: test2@shtb_test1
+POSTHOOK: query: CREATE TABLE shtb_test1(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:test2
+POSTHOOK: Output: test2@shtb_test1
+PREHOOK: query: CREATE TABLE shtb_test2(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:test2
+PREHOOK: Output: test2@shtb_test2
+POSTHOOK: query: CREATE TABLE shtb_test2(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:test2
+POSTHOOK: Output: test2@shtb_test2
+PREHOOK: query: CREATE VIEW shtb_test1_view1 AS SELECT * FROM shtb_test1 where KEY > 1000 and KEY < 2000
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: test2@shtb_test1
+PREHOOK: Output: database:test2
+PREHOOK: Output: test2@shtb_test1_view1
+POSTHOOK: query: CREATE VIEW shtb_test1_view1 AS SELECT * FROM shtb_test1 where KEY > 1000 and KEY < 2000
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: test2@shtb_test1
+POSTHOOK: Output: database:test2
+POSTHOOK: Output: test2@shtb_test1_view1
+PREHOOK: query: CREATE VIEW shtb_test2_view2 AS SELECT * FROM shtb_test2 where KEY > 100 and KEY < 200
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: test2@shtb_test2
+PREHOOK: Output: database:test2
+PREHOOK: Output: test2@shtb_test2_view2
+POSTHOOK: query: CREATE VIEW shtb_test2_view2 AS SELECT * FROM shtb_test2 where KEY > 100 and KEY < 200
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: test2@shtb_test2
+POSTHOOK: Output: database:test2
+POSTHOOK: Output: test2@shtb_test2_view2
+PREHOOK: query: USE test1
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:test1
+POSTHOOK: query: USE test1
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:test1
+PREHOOK: query: SHOW VIEWS
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS
+POSTHOOK: type: SHOWVIEWS
+shtb_full_view2
+shtb_test1_view1
+shtb_test1_view2
+PREHOOK: query: SHOW VIEWS 'test_*'
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS 'test_*'
+POSTHOOK: type: SHOWVIEWS
+PREHOOK: query: SHOW VIEWS '*view2'
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS '*view2'
+POSTHOOK: type: SHOWVIEWS
+shtb_full_view2
+shtb_test1_view2
+PREHOOK: query: SHOW VIEWS LIKE 'test_view1|test_view2'
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS LIKE 'test_view1|test_view2'
+POSTHOOK: type: SHOWVIEWS
+PREHOOK: query: USE test2
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:test2
+POSTHOOK: query: USE test2
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:test2
+PREHOOK: query: SHOW VIEWS 'shtb_*'
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS 'shtb_*'
+POSTHOOK: type: SHOWVIEWS
+shtb_test1_view1
+shtb_test2_view2
+PREHOOK: query: -- SHOW VIEWS basic syntax tests
+USE default
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:default
+POSTHOOK: query: -- SHOW VIEWS basic syntax tests
+USE default
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:default
+PREHOOK: query: SHOW VIEWS FROM test1
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS FROM test1
+POSTHOOK: type: SHOWVIEWS
+shtb_full_view2
+shtb_test1_view1
+shtb_test1_view2
+PREHOOK: query: SHOW VIEWS FROM test2
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS FROM test2
+POSTHOOK: type: SHOWVIEWS
+shtb_test1_view1
+shtb_test2_view2
+PREHOOK: query: SHOW VIEWS IN test1
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS IN test1
+POSTHOOK: type: SHOWVIEWS
+shtb_full_view2
+shtb_test1_view1
+shtb_test1_view2
+PREHOOK: query: SHOW VIEWS IN default
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS IN default
+POSTHOOK: type: SHOWVIEWS
+PREHOOK: query: SHOW VIEWS IN test1 "shtb_test_*"
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS IN test1 "shtb_test_*"
+POSTHOOK: type: SHOWVIEWS
+PREHOOK: query: SHOW VIEWS IN test2 LIKE "nomatch"
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS IN test2 LIKE "nomatch"
+POSTHOOK: type: SHOWVIEWS
+PREHOOK: query: -- SHOW VIEWS from a database with a name that requires escaping
+CREATE DATABASE `database`
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:database
+POSTHOOK: query: -- SHOW VIEWS from a database with a name that requires escaping
+CREATE DATABASE `database`
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:database
+PREHOOK: query: USE `database`
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:database
+POSTHOOK: query: USE `database`
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:database
+PREHOOK: query: CREATE TABLE foo(a INT)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:database
+PREHOOK: Output: database@foo
+POSTHOOK: query: CREATE TABLE foo(a INT)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:database
+POSTHOOK: Output: database@foo
+PREHOOK: query: CREATE VIEW fooview AS SELECT * FROM foo
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: database@foo
+PREHOOK: Output: database:database
+PREHOOK: Output: database@fooview
+POSTHOOK: query: CREATE VIEW fooview AS SELECT * FROM foo
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: database@foo
+POSTHOOK: Output: database:database
+POSTHOOK: Output: database@fooview
+PREHOOK: query: USE default
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:default
+POSTHOOK: query: USE default
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:default
+PREHOOK: query: SHOW VIEWS FROM `database` LIKE "fooview"
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS FROM `database` LIKE "fooview"
+POSTHOOK: type: SHOWVIEWS
+fooview
+PREHOOK: query: DROP VIEW fooview
+PREHOOK: type: DROPVIEW
+POSTHOOK: query: DROP VIEW fooview
+POSTHOOK: type: DROPVIEW
+PREHOOK: query: DROP TABLE foo
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE foo
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: USE test1
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:test1
+POSTHOOK: query: USE test1
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:test1
+PREHOOK: query: DROP VIEW shtb_test1_view1
+PREHOOK: type: DROPVIEW
+PREHOOK: Input: test1@shtb_test1_view1
+PREHOOK: Output: test1@shtb_test1_view1
+POSTHOOK: query: DROP VIEW shtb_test1_view1
+POSTHOOK: type: DROPVIEW
+POSTHOOK: Input: test1@shtb_test1_view1
+POSTHOOK: Output: test1@shtb_test1_view1
+PREHOOK: query: DROP VIEW shtb_test1_view2
+PREHOOK: type: DROPVIEW
+PREHOOK: Input: test1@shtb_test1_view2
+PREHOOK: Output: test1@shtb_test1_view2
+POSTHOOK: query: DROP VIEW shtb_test1_view2
+POSTHOOK: type: DROPVIEW
+POSTHOOK: Input: test1@shtb_test1_view2
+POSTHOOK: Output: test1@shtb_test1_view2
+PREHOOK: query: DROP VIEW shtb_full_view2
+PREHOOK: type: DROPVIEW
+PREHOOK: Input: test1@shtb_full_view2
+PREHOOK: Output: test1@shtb_full_view2
+POSTHOOK: query: DROP VIEW shtb_full_view2
+POSTHOOK: type: DROPVIEW
+POSTHOOK: Input: test1@shtb_full_view2
+POSTHOOK: Output: test1@shtb_full_view2
+PREHOOK: query: DROP TABLE shtb_test1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: test1@shtb_test1
+PREHOOK: Output: test1@shtb_test1
+POSTHOOK: query: DROP TABLE shtb_test1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: test1@shtb_test1
+POSTHOOK: Output: test1@shtb_test1
+PREHOOK: query: DROP DATABASE test1
+PREHOOK: type: DROPDATABASE
+PREHOOK: Input: database:test1
+PREHOOK: Output: database:test1
+POSTHOOK: query: DROP DATABASE test1
+POSTHOOK: type: DROPDATABASE
+POSTHOOK: Input: database:test1
+POSTHOOK: Output: database:test1
+PREHOOK: query: USE test2
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:test2
+POSTHOOK: query: USE test2
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:test2
+PREHOOK: query: DROP VIEW shtb_test1_view1
+PREHOOK: type: DROPVIEW
+PREHOOK: Input: test2@shtb_test1_view1
+PREHOOK: Output: test2@shtb_test1_view1
+POSTHOOK: query: DROP VIEW shtb_test1_view1
+POSTHOOK: type: DROPVIEW
+POSTHOOK: Input: test2@shtb_test1_view1
+POSTHOOK: Output: test2@shtb_test1_view1
+PREHOOK: query: DROP VIEW shtb_test2_view2
+PREHOOK: type: DROPVIEW
+PREHOOK: Input: test2@shtb_test2_view2
+PREHOOK: Output: test2@shtb_test2_view2
+POSTHOOK: query: DROP VIEW shtb_test2_view2
+POSTHOOK: type: DROPVIEW
+POSTHOOK: Input: test2@shtb_test2_view2
+POSTHOOK: Output: test2@shtb_test2_view2
+PREHOOK: query: DROP TABLE shtb_test1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: test2@shtb_test1
+PREHOOK: Output: test2@shtb_test1
+POSTHOOK: query: DROP TABLE shtb_test1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: test2@shtb_test1
+POSTHOOK: Output: test2@shtb_test1
+PREHOOK: query: DROP TABLE shtb_test2
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: test2@shtb_test2
+PREHOOK: Output: test2@shtb_test2
+POSTHOOK: query: DROP TABLE shtb_test2
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: test2@shtb_test2
+POSTHOOK: Output: test2@shtb_test2
+PREHOOK: query: DROP DATABASE test2
+PREHOOK: type: DROPDATABASE
+PREHOOK: Input: database:test2
+PREHOOK: Output: database:test2
+POSTHOOK: query: DROP DATABASE test2
+POSTHOOK: type: DROPDATABASE
+POSTHOOK: Input: database:test2
+POSTHOOK: Output: database:test2


[7/7] hive git commit: HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)

Posted by ai...@apache.org.
HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)


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

Branch: refs/heads/master
Commit: 21a0142f333fba231f2648db53a48dc41384ad72
Parents: e19f0e3
Author: Aihua Xu <ai...@apache.org>
Authored: Tue Oct 4 09:53:12 2016 -0400
Committer: Aihua Xu <ai...@apache.org>
Committed: Tue Oct 4 09:53:12 2016 -0400

----------------------------------------------------------------------
 metastore/if/hive_metastore.thrift              |    1 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2163 +++++++-----
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  148 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../hive/metastore/api/ThriftHiveMetastore.java | 3184 ++++++++++++------
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1341 +++++---
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       |  938 ++++--
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   65 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   23 +
 .../hive/metastore/HiveMetaStoreClient.java     |   12 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |   15 +
 .../hadoop/hive/metastore/ObjectStore.java      |    8 +
 .../apache/hadoop/hive/metastore/RawStore.java  |    4 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |    6 +
 .../DummyRawStoreControlledCommit.java          |    6 +
 .../DummyRawStoreForJdoConnection.java          |    6 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   45 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   46 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   43 +
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |    1 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |    2 +
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |    2 +
 .../org/apache/hadoop/hive/ql/plan/DDLWork.java |    2 +-
 .../hadoop/hive/ql/plan/HiveOperation.java      |    1 +
 .../hadoop/hive/ql/plan/ShowTablesDesc.java     |   29 +
 .../authorization/plugin/HiveOperationType.java |    1 +
 .../plugin/sqlstd/Operation2Privilege.java      |    2 +
 ql/src/test/queries/clientpositive/show_views.q |   56 +
 .../results/clientpositive/show_views.q.out     |  320 ++
 30 files changed, 5774 insertions(+), 2708 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index 872c0f3..c5ba309 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -1030,6 +1030,7 @@ 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<string> get_tables_by_type(1: string db_name, 2: string pattern, 3: string tableType) 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)


[3/7] hive git commit: HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)

Posted by ai...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 2d82c92..24b3ba1 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -209,6 +209,14 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function get_tables($db_name, $pattern);
   /**
+   * @param string $db_name
+   * @param string $pattern
+   * @param string $tableType
+   * @return string[]
+   * @throws \metastore\MetaException
+   */
+  public function get_tables_by_type($db_name, $pattern, $tableType);
+  /**
    * @param string $db_patterns
    * @param string $tbl_patterns
    * @param string[] $tbl_types
@@ -2598,6 +2606,62 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("get_tables failed: unknown result");
   }
 
+  public function get_tables_by_type($db_name, $pattern, $tableType)
+  {
+    $this->send_get_tables_by_type($db_name, $pattern, $tableType);
+    return $this->recv_get_tables_by_type();
+  }
+
+  public function send_get_tables_by_type($db_name, $pattern, $tableType)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_tables_by_type_args();
+    $args->db_name = $db_name;
+    $args->pattern = $pattern;
+    $args->tableType = $tableType;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_tables_by_type', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_tables_by_type', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_tables_by_type()
+  {
+    $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_tables_by_type_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_tables_by_type_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_tables_by_type 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);
@@ -15313,6 +15377,253 @@ class ThriftHiveMetastore_get_tables_result {
 
 }
 
+class ThriftHiveMetastore_get_tables_by_type_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $db_name = null;
+  /**
+   * @var string
+   */
+  public $pattern = null;
+  /**
+   * @var string
+   */
+  public $tableType = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'db_name',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'pattern',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'tableType',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['db_name'])) {
+        $this->db_name = $vals['db_name'];
+      }
+      if (isset($vals['pattern'])) {
+        $this->pattern = $vals['pattern'];
+      }
+      if (isset($vals['tableType'])) {
+        $this->tableType = $vals['tableType'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_tables_by_type_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_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->pattern);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tableType);
+          } 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_tables_by_type_args');
+    if ($this->db_name !== null) {
+      $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1);
+      $xfer += $output->writeString($this->db_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->pattern !== null) {
+      $xfer += $output->writeFieldBegin('pattern', TType::STRING, 2);
+      $xfer += $output->writeString($this->pattern);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tableType !== null) {
+      $xfer += $output->writeFieldBegin('tableType', TType::STRING, 3);
+      $xfer += $output->writeString($this->tableType);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_tables_by_type_result {
+  static $_TSPEC;
+
+  /**
+   * @var string[]
+   */
+  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::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        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_tables_by_type_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();
+            $_size641 = 0;
+            $_etype644 = 0;
+            $xfer += $input->readListBegin($_etype644, $_size641);
+            for ($_i645 = 0; $_i645 < $_size641; ++$_i645)
+            {
+              $elem646 = null;
+              $xfer += $input->readString($elem646);
+              $this->success []= $elem646;
+            }
+            $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_tables_by_type_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::STRING, count($this->success));
+        {
+          foreach ($this->success as $iter647)
+          {
+            $xfer += $output->writeString($iter647);
+          }
+        }
+        $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_table_meta_args {
   static $_TSPEC;
 
@@ -15399,14 +15710,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size641 = 0;
-            $_etype644 = 0;
-            $xfer += $input->readListBegin($_etype644, $_size641);
-            for ($_i645 = 0; $_i645 < $_size641; ++$_i645)
+            $_size648 = 0;
+            $_etype651 = 0;
+            $xfer += $input->readListBegin($_etype651, $_size648);
+            for ($_i652 = 0; $_i652 < $_size648; ++$_i652)
             {
-              $elem646 = null;
-              $xfer += $input->readString($elem646);
-              $this->tbl_types []= $elem646;
+              $elem653 = null;
+              $xfer += $input->readString($elem653);
+              $this->tbl_types []= $elem653;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15444,9 +15755,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter647)
+          foreach ($this->tbl_types as $iter654)
           {
-            $xfer += $output->writeString($iter647);
+            $xfer += $output->writeString($iter654);
           }
         }
         $output->writeListEnd();
@@ -15523,15 +15834,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size648 = 0;
-            $_etype651 = 0;
-            $xfer += $input->readListBegin($_etype651, $_size648);
-            for ($_i652 = 0; $_i652 < $_size648; ++$_i652)
+            $_size655 = 0;
+            $_etype658 = 0;
+            $xfer += $input->readListBegin($_etype658, $_size655);
+            for ($_i659 = 0; $_i659 < $_size655; ++$_i659)
             {
-              $elem653 = null;
-              $elem653 = new \metastore\TableMeta();
-              $xfer += $elem653->read($input);
-              $this->success []= $elem653;
+              $elem660 = null;
+              $elem660 = new \metastore\TableMeta();
+              $xfer += $elem660->read($input);
+              $this->success []= $elem660;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15567,9 +15878,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter654)
+          foreach ($this->success as $iter661)
           {
-            $xfer += $iter654->write($output);
+            $xfer += $iter661->write($output);
           }
         }
         $output->writeListEnd();
@@ -15725,14 +16036,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size655 = 0;
-            $_etype658 = 0;
-            $xfer += $input->readListBegin($_etype658, $_size655);
-            for ($_i659 = 0; $_i659 < $_size655; ++$_i659)
+            $_size662 = 0;
+            $_etype665 = 0;
+            $xfer += $input->readListBegin($_etype665, $_size662);
+            for ($_i666 = 0; $_i666 < $_size662; ++$_i666)
             {
-              $elem660 = null;
-              $xfer += $input->readString($elem660);
-              $this->success []= $elem660;
+              $elem667 = null;
+              $xfer += $input->readString($elem667);
+              $this->success []= $elem667;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15768,9 +16079,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter661)
+          foreach ($this->success as $iter668)
           {
-            $xfer += $output->writeString($iter661);
+            $xfer += $output->writeString($iter668);
           }
         }
         $output->writeListEnd();
@@ -16085,14 +16396,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size662 = 0;
-            $_etype665 = 0;
-            $xfer += $input->readListBegin($_etype665, $_size662);
-            for ($_i666 = 0; $_i666 < $_size662; ++$_i666)
+            $_size669 = 0;
+            $_etype672 = 0;
+            $xfer += $input->readListBegin($_etype672, $_size669);
+            for ($_i673 = 0; $_i673 < $_size669; ++$_i673)
             {
-              $elem667 = null;
-              $xfer += $input->readString($elem667);
-              $this->tbl_names []= $elem667;
+              $elem674 = null;
+              $xfer += $input->readString($elem674);
+              $this->tbl_names []= $elem674;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16125,9 +16436,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter668)
+          foreach ($this->tbl_names as $iter675)
           {
-            $xfer += $output->writeString($iter668);
+            $xfer += $output->writeString($iter675);
           }
         }
         $output->writeListEnd();
@@ -16228,15 +16539,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size669 = 0;
-            $_etype672 = 0;
-            $xfer += $input->readListBegin($_etype672, $_size669);
-            for ($_i673 = 0; $_i673 < $_size669; ++$_i673)
+            $_size676 = 0;
+            $_etype679 = 0;
+            $xfer += $input->readListBegin($_etype679, $_size676);
+            for ($_i680 = 0; $_i680 < $_size676; ++$_i680)
             {
-              $elem674 = null;
-              $elem674 = new \metastore\Table();
-              $xfer += $elem674->read($input);
-              $this->success []= $elem674;
+              $elem681 = null;
+              $elem681 = new \metastore\Table();
+              $xfer += $elem681->read($input);
+              $this->success []= $elem681;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16288,9 +16599,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter675)
+          foreach ($this->success as $iter682)
           {
-            $xfer += $iter675->write($output);
+            $xfer += $iter682->write($output);
           }
         }
         $output->writeListEnd();
@@ -16526,14 +16837,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size676 = 0;
-            $_etype679 = 0;
-            $xfer += $input->readListBegin($_etype679, $_size676);
-            for ($_i680 = 0; $_i680 < $_size676; ++$_i680)
+            $_size683 = 0;
+            $_etype686 = 0;
+            $xfer += $input->readListBegin($_etype686, $_size683);
+            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
             {
-              $elem681 = null;
-              $xfer += $input->readString($elem681);
-              $this->success []= $elem681;
+              $elem688 = null;
+              $xfer += $input->readString($elem688);
+              $this->success []= $elem688;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16585,9 +16896,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter682)
+          foreach ($this->success as $iter689)
           {
-            $xfer += $output->writeString($iter682);
+            $xfer += $output->writeString($iter689);
           }
         }
         $output->writeListEnd();
@@ -17900,15 +18211,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size683 = 0;
-            $_etype686 = 0;
-            $xfer += $input->readListBegin($_etype686, $_size683);
-            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
+            $_size690 = 0;
+            $_etype693 = 0;
+            $xfer += $input->readListBegin($_etype693, $_size690);
+            for ($_i694 = 0; $_i694 < $_size690; ++$_i694)
             {
-              $elem688 = null;
-              $elem688 = new \metastore\Partition();
-              $xfer += $elem688->read($input);
-              $this->new_parts []= $elem688;
+              $elem695 = null;
+              $elem695 = new \metastore\Partition();
+              $xfer += $elem695->read($input);
+              $this->new_parts []= $elem695;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17936,9 +18247,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter689)
+          foreach ($this->new_parts as $iter696)
           {
-            $xfer += $iter689->write($output);
+            $xfer += $iter696->write($output);
           }
         }
         $output->writeListEnd();
@@ -18153,15 +18464,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size690 = 0;
-            $_etype693 = 0;
-            $xfer += $input->readListBegin($_etype693, $_size690);
-            for ($_i694 = 0; $_i694 < $_size690; ++$_i694)
+            $_size697 = 0;
+            $_etype700 = 0;
+            $xfer += $input->readListBegin($_etype700, $_size697);
+            for ($_i701 = 0; $_i701 < $_size697; ++$_i701)
             {
-              $elem695 = null;
-              $elem695 = new \metastore\PartitionSpec();
-              $xfer += $elem695->read($input);
-              $this->new_parts []= $elem695;
+              $elem702 = null;
+              $elem702 = new \metastore\PartitionSpec();
+              $xfer += $elem702->read($input);
+              $this->new_parts []= $elem702;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18189,9 +18500,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter696)
+          foreach ($this->new_parts as $iter703)
           {
-            $xfer += $iter696->write($output);
+            $xfer += $iter703->write($output);
           }
         }
         $output->writeListEnd();
@@ -18441,14 +18752,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size697 = 0;
-            $_etype700 = 0;
-            $xfer += $input->readListBegin($_etype700, $_size697);
-            for ($_i701 = 0; $_i701 < $_size697; ++$_i701)
+            $_size704 = 0;
+            $_etype707 = 0;
+            $xfer += $input->readListBegin($_etype707, $_size704);
+            for ($_i708 = 0; $_i708 < $_size704; ++$_i708)
             {
-              $elem702 = null;
-              $xfer += $input->readString($elem702);
-              $this->part_vals []= $elem702;
+              $elem709 = null;
+              $xfer += $input->readString($elem709);
+              $this->part_vals []= $elem709;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18486,9 +18797,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter703)
+          foreach ($this->part_vals as $iter710)
           {
-            $xfer += $output->writeString($iter703);
+            $xfer += $output->writeString($iter710);
           }
         }
         $output->writeListEnd();
@@ -18990,14 +19301,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size704 = 0;
-            $_etype707 = 0;
-            $xfer += $input->readListBegin($_etype707, $_size704);
-            for ($_i708 = 0; $_i708 < $_size704; ++$_i708)
+            $_size711 = 0;
+            $_etype714 = 0;
+            $xfer += $input->readListBegin($_etype714, $_size711);
+            for ($_i715 = 0; $_i715 < $_size711; ++$_i715)
             {
-              $elem709 = null;
-              $xfer += $input->readString($elem709);
-              $this->part_vals []= $elem709;
+              $elem716 = null;
+              $xfer += $input->readString($elem716);
+              $this->part_vals []= $elem716;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19043,9 +19354,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter710)
+          foreach ($this->part_vals as $iter717)
           {
-            $xfer += $output->writeString($iter710);
+            $xfer += $output->writeString($iter717);
           }
         }
         $output->writeListEnd();
@@ -19899,14 +20210,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size711 = 0;
-            $_etype714 = 0;
-            $xfer += $input->readListBegin($_etype714, $_size711);
-            for ($_i715 = 0; $_i715 < $_size711; ++$_i715)
+            $_size718 = 0;
+            $_etype721 = 0;
+            $xfer += $input->readListBegin($_etype721, $_size718);
+            for ($_i722 = 0; $_i722 < $_size718; ++$_i722)
             {
-              $elem716 = null;
-              $xfer += $input->readString($elem716);
-              $this->part_vals []= $elem716;
+              $elem723 = null;
+              $xfer += $input->readString($elem723);
+              $this->part_vals []= $elem723;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19951,9 +20262,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter717)
+          foreach ($this->part_vals as $iter724)
           {
-            $xfer += $output->writeString($iter717);
+            $xfer += $output->writeString($iter724);
           }
         }
         $output->writeListEnd();
@@ -20206,14 +20517,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size718 = 0;
-            $_etype721 = 0;
-            $xfer += $input->readListBegin($_etype721, $_size718);
-            for ($_i722 = 0; $_i722 < $_size718; ++$_i722)
+            $_size725 = 0;
+            $_etype728 = 0;
+            $xfer += $input->readListBegin($_etype728, $_size725);
+            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
             {
-              $elem723 = null;
-              $xfer += $input->readString($elem723);
-              $this->part_vals []= $elem723;
+              $elem730 = null;
+              $xfer += $input->readString($elem730);
+              $this->part_vals []= $elem730;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20266,9 +20577,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter724)
+          foreach ($this->part_vals as $iter731)
           {
-            $xfer += $output->writeString($iter724);
+            $xfer += $output->writeString($iter731);
           }
         }
         $output->writeListEnd();
@@ -21282,14 +21593,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size725 = 0;
-            $_etype728 = 0;
-            $xfer += $input->readListBegin($_etype728, $_size725);
-            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
+            $_size732 = 0;
+            $_etype735 = 0;
+            $xfer += $input->readListBegin($_etype735, $_size732);
+            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
             {
-              $elem730 = null;
-              $xfer += $input->readString($elem730);
-              $this->part_vals []= $elem730;
+              $elem737 = null;
+              $xfer += $input->readString($elem737);
+              $this->part_vals []= $elem737;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21327,9 +21638,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter731)
+          foreach ($this->part_vals as $iter738)
           {
-            $xfer += $output->writeString($iter731);
+            $xfer += $output->writeString($iter738);
           }
         }
         $output->writeListEnd();
@@ -21571,17 +21882,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size732 = 0;
-            $_ktype733 = 0;
-            $_vtype734 = 0;
-            $xfer += $input->readMapBegin($_ktype733, $_vtype734, $_size732);
-            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
+            $_size739 = 0;
+            $_ktype740 = 0;
+            $_vtype741 = 0;
+            $xfer += $input->readMapBegin($_ktype740, $_vtype741, $_size739);
+            for ($_i743 = 0; $_i743 < $_size739; ++$_i743)
             {
-              $key737 = '';
-              $val738 = '';
-              $xfer += $input->readString($key737);
-              $xfer += $input->readString($val738);
-              $this->partitionSpecs[$key737] = $val738;
+              $key744 = '';
+              $val745 = '';
+              $xfer += $input->readString($key744);
+              $xfer += $input->readString($val745);
+              $this->partitionSpecs[$key744] = $val745;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -21637,10 +21948,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter739 => $viter740)
+          foreach ($this->partitionSpecs as $kiter746 => $viter747)
           {
-            $xfer += $output->writeString($kiter739);
-            $xfer += $output->writeString($viter740);
+            $xfer += $output->writeString($kiter746);
+            $xfer += $output->writeString($viter747);
           }
         }
         $output->writeMapEnd();
@@ -21952,17 +22263,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size741 = 0;
-            $_ktype742 = 0;
-            $_vtype743 = 0;
-            $xfer += $input->readMapBegin($_ktype742, $_vtype743, $_size741);
-            for ($_i745 = 0; $_i745 < $_size741; ++$_i745)
+            $_size748 = 0;
+            $_ktype749 = 0;
+            $_vtype750 = 0;
+            $xfer += $input->readMapBegin($_ktype749, $_vtype750, $_size748);
+            for ($_i752 = 0; $_i752 < $_size748; ++$_i752)
             {
-              $key746 = '';
-              $val747 = '';
-              $xfer += $input->readString($key746);
-              $xfer += $input->readString($val747);
-              $this->partitionSpecs[$key746] = $val747;
+              $key753 = '';
+              $val754 = '';
+              $xfer += $input->readString($key753);
+              $xfer += $input->readString($val754);
+              $this->partitionSpecs[$key753] = $val754;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -22018,10 +22329,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter748 => $viter749)
+          foreach ($this->partitionSpecs as $kiter755 => $viter756)
           {
-            $xfer += $output->writeString($kiter748);
-            $xfer += $output->writeString($viter749);
+            $xfer += $output->writeString($kiter755);
+            $xfer += $output->writeString($viter756);
           }
         }
         $output->writeMapEnd();
@@ -22154,15 +22465,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size750 = 0;
-            $_etype753 = 0;
-            $xfer += $input->readListBegin($_etype753, $_size750);
-            for ($_i754 = 0; $_i754 < $_size750; ++$_i754)
+            $_size757 = 0;
+            $_etype760 = 0;
+            $xfer += $input->readListBegin($_etype760, $_size757);
+            for ($_i761 = 0; $_i761 < $_size757; ++$_i761)
             {
-              $elem755 = null;
-              $elem755 = new \metastore\Partition();
-              $xfer += $elem755->read($input);
-              $this->success []= $elem755;
+              $elem762 = null;
+              $elem762 = new \metastore\Partition();
+              $xfer += $elem762->read($input);
+              $this->success []= $elem762;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22222,9 +22533,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter756)
+          foreach ($this->success as $iter763)
           {
-            $xfer += $iter756->write($output);
+            $xfer += $iter763->write($output);
           }
         }
         $output->writeListEnd();
@@ -22370,14 +22681,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size757 = 0;
-            $_etype760 = 0;
-            $xfer += $input->readListBegin($_etype760, $_size757);
-            for ($_i761 = 0; $_i761 < $_size757; ++$_i761)
+            $_size764 = 0;
+            $_etype767 = 0;
+            $xfer += $input->readListBegin($_etype767, $_size764);
+            for ($_i768 = 0; $_i768 < $_size764; ++$_i768)
             {
-              $elem762 = null;
-              $xfer += $input->readString($elem762);
-              $this->part_vals []= $elem762;
+              $elem769 = null;
+              $xfer += $input->readString($elem769);
+              $this->part_vals []= $elem769;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22394,14 +22705,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size763 = 0;
-            $_etype766 = 0;
-            $xfer += $input->readListBegin($_etype766, $_size763);
-            for ($_i767 = 0; $_i767 < $_size763; ++$_i767)
+            $_size770 = 0;
+            $_etype773 = 0;
+            $xfer += $input->readListBegin($_etype773, $_size770);
+            for ($_i774 = 0; $_i774 < $_size770; ++$_i774)
             {
-              $elem768 = null;
-              $xfer += $input->readString($elem768);
-              $this->group_names []= $elem768;
+              $elem775 = null;
+              $xfer += $input->readString($elem775);
+              $this->group_names []= $elem775;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22439,9 +22750,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter769)
+          foreach ($this->part_vals as $iter776)
           {
-            $xfer += $output->writeString($iter769);
+            $xfer += $output->writeString($iter776);
           }
         }
         $output->writeListEnd();
@@ -22461,9 +22772,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter770)
+          foreach ($this->group_names as $iter777)
           {
-            $xfer += $output->writeString($iter770);
+            $xfer += $output->writeString($iter777);
           }
         }
         $output->writeListEnd();
@@ -23054,15 +23365,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size771 = 0;
-            $_etype774 = 0;
-            $xfer += $input->readListBegin($_etype774, $_size771);
-            for ($_i775 = 0; $_i775 < $_size771; ++$_i775)
+            $_size778 = 0;
+            $_etype781 = 0;
+            $xfer += $input->readListBegin($_etype781, $_size778);
+            for ($_i782 = 0; $_i782 < $_size778; ++$_i782)
             {
-              $elem776 = null;
-              $elem776 = new \metastore\Partition();
-              $xfer += $elem776->read($input);
-              $this->success []= $elem776;
+              $elem783 = null;
+              $elem783 = new \metastore\Partition();
+              $xfer += $elem783->read($input);
+              $this->success []= $elem783;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23106,9 +23417,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter777)
+          foreach ($this->success as $iter784)
           {
-            $xfer += $iter777->write($output);
+            $xfer += $iter784->write($output);
           }
         }
         $output->writeListEnd();
@@ -23254,14 +23565,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size778 = 0;
-            $_etype781 = 0;
-            $xfer += $input->readListBegin($_etype781, $_size778);
-            for ($_i782 = 0; $_i782 < $_size778; ++$_i782)
+            $_size785 = 0;
+            $_etype788 = 0;
+            $xfer += $input->readListBegin($_etype788, $_size785);
+            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
             {
-              $elem783 = null;
-              $xfer += $input->readString($elem783);
-              $this->group_names []= $elem783;
+              $elem790 = null;
+              $xfer += $input->readString($elem790);
+              $this->group_names []= $elem790;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23309,9 +23620,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter784)
+          foreach ($this->group_names as $iter791)
           {
-            $xfer += $output->writeString($iter784);
+            $xfer += $output->writeString($iter791);
           }
         }
         $output->writeListEnd();
@@ -23400,15 +23711,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size785 = 0;
-            $_etype788 = 0;
-            $xfer += $input->readListBegin($_etype788, $_size785);
-            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
+            $_size792 = 0;
+            $_etype795 = 0;
+            $xfer += $input->readListBegin($_etype795, $_size792);
+            for ($_i796 = 0; $_i796 < $_size792; ++$_i796)
             {
-              $elem790 = null;
-              $elem790 = new \metastore\Partition();
-              $xfer += $elem790->read($input);
-              $this->success []= $elem790;
+              $elem797 = null;
+              $elem797 = new \metastore\Partition();
+              $xfer += $elem797->read($input);
+              $this->success []= $elem797;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23452,9 +23763,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter791)
+          foreach ($this->success as $iter798)
           {
-            $xfer += $iter791->write($output);
+            $xfer += $iter798->write($output);
           }
         }
         $output->writeListEnd();
@@ -23674,15 +23985,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size792 = 0;
-            $_etype795 = 0;
-            $xfer += $input->readListBegin($_etype795, $_size792);
-            for ($_i796 = 0; $_i796 < $_size792; ++$_i796)
+            $_size799 = 0;
+            $_etype802 = 0;
+            $xfer += $input->readListBegin($_etype802, $_size799);
+            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
             {
-              $elem797 = null;
-              $elem797 = new \metastore\PartitionSpec();
-              $xfer += $elem797->read($input);
-              $this->success []= $elem797;
+              $elem804 = null;
+              $elem804 = new \metastore\PartitionSpec();
+              $xfer += $elem804->read($input);
+              $this->success []= $elem804;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23726,9 +24037,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter798)
+          foreach ($this->success as $iter805)
           {
-            $xfer += $iter798->write($output);
+            $xfer += $iter805->write($output);
           }
         }
         $output->writeListEnd();
@@ -23935,14 +24246,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size799 = 0;
-            $_etype802 = 0;
-            $xfer += $input->readListBegin($_etype802, $_size799);
-            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
+            $_size806 = 0;
+            $_etype809 = 0;
+            $xfer += $input->readListBegin($_etype809, $_size806);
+            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
             {
-              $elem804 = null;
-              $xfer += $input->readString($elem804);
-              $this->success []= $elem804;
+              $elem811 = null;
+              $xfer += $input->readString($elem811);
+              $this->success []= $elem811;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23978,9 +24289,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter805)
+          foreach ($this->success as $iter812)
           {
-            $xfer += $output->writeString($iter805);
+            $xfer += $output->writeString($iter812);
           }
         }
         $output->writeListEnd();
@@ -24096,14 +24407,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size806 = 0;
-            $_etype809 = 0;
-            $xfer += $input->readListBegin($_etype809, $_size806);
-            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
+            $_size813 = 0;
+            $_etype816 = 0;
+            $xfer += $input->readListBegin($_etype816, $_size813);
+            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
             {
-              $elem811 = null;
-              $xfer += $input->readString($elem811);
-              $this->part_vals []= $elem811;
+              $elem818 = null;
+              $xfer += $input->readString($elem818);
+              $this->part_vals []= $elem818;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24148,9 +24459,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter812)
+          foreach ($this->part_vals as $iter819)
           {
-            $xfer += $output->writeString($iter812);
+            $xfer += $output->writeString($iter819);
           }
         }
         $output->writeListEnd();
@@ -24244,15 +24555,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size813 = 0;
-            $_etype816 = 0;
-            $xfer += $input->readListBegin($_etype816, $_size813);
-            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
+            $_size820 = 0;
+            $_etype823 = 0;
+            $xfer += $input->readListBegin($_etype823, $_size820);
+            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
             {
-              $elem818 = null;
-              $elem818 = new \metastore\Partition();
-              $xfer += $elem818->read($input);
-              $this->success []= $elem818;
+              $elem825 = null;
+              $elem825 = new \metastore\Partition();
+              $xfer += $elem825->read($input);
+              $this->success []= $elem825;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24296,9 +24607,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter819)
+          foreach ($this->success as $iter826)
           {
-            $xfer += $iter819->write($output);
+            $xfer += $iter826->write($output);
           }
         }
         $output->writeListEnd();
@@ -24445,14 +24756,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size820 = 0;
-            $_etype823 = 0;
-            $xfer += $input->readListBegin($_etype823, $_size820);
-            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
+            $_size827 = 0;
+            $_etype830 = 0;
+            $xfer += $input->readListBegin($_etype830, $_size827);
+            for ($_i831 = 0; $_i831 < $_size827; ++$_i831)
             {
-              $elem825 = null;
-              $xfer += $input->readString($elem825);
-              $this->part_vals []= $elem825;
+              $elem832 = null;
+              $xfer += $input->readString($elem832);
+              $this->part_vals []= $elem832;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24476,14 +24787,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size826 = 0;
-            $_etype829 = 0;
-            $xfer += $input->readListBegin($_etype829, $_size826);
-            for ($_i830 = 0; $_i830 < $_size826; ++$_i830)
+            $_size833 = 0;
+            $_etype836 = 0;
+            $xfer += $input->readListBegin($_etype836, $_size833);
+            for ($_i837 = 0; $_i837 < $_size833; ++$_i837)
             {
-              $elem831 = null;
-              $xfer += $input->readString($elem831);
-              $this->group_names []= $elem831;
+              $elem838 = null;
+              $xfer += $input->readString($elem838);
+              $this->group_names []= $elem838;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24521,9 +24832,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter832)
+          foreach ($this->part_vals as $iter839)
           {
-            $xfer += $output->writeString($iter832);
+            $xfer += $output->writeString($iter839);
           }
         }
         $output->writeListEnd();
@@ -24548,9 +24859,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter833)
+          foreach ($this->group_names as $iter840)
           {
-            $xfer += $output->writeString($iter833);
+            $xfer += $output->writeString($iter840);
           }
         }
         $output->writeListEnd();
@@ -24639,15 +24950,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size834 = 0;
-            $_etype837 = 0;
-            $xfer += $input->readListBegin($_etype837, $_size834);
-            for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
+            $_size841 = 0;
+            $_etype844 = 0;
+            $xfer += $input->readListBegin($_etype844, $_size841);
+            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
             {
-              $elem839 = null;
-              $elem839 = new \metastore\Partition();
-              $xfer += $elem839->read($input);
-              $this->success []= $elem839;
+              $elem846 = null;
+              $elem846 = new \metastore\Partition();
+              $xfer += $elem846->read($input);
+              $this->success []= $elem846;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24691,9 +25002,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter840)
+          foreach ($this->success as $iter847)
           {
-            $xfer += $iter840->write($output);
+            $xfer += $iter847->write($output);
           }
         }
         $output->writeListEnd();
@@ -24814,14 +25125,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size841 = 0;
-            $_etype844 = 0;
-            $xfer += $input->readListBegin($_etype844, $_size841);
-            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
+            $_size848 = 0;
+            $_etype851 = 0;
+            $xfer += $input->readListBegin($_etype851, $_size848);
+            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
             {
-              $elem846 = null;
-              $xfer += $input->readString($elem846);
-              $this->part_vals []= $elem846;
+              $elem853 = null;
+              $xfer += $input->readString($elem853);
+              $this->part_vals []= $elem853;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24866,9 +25177,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter847)
+          foreach ($this->part_vals as $iter854)
           {
-            $xfer += $output->writeString($iter847);
+            $xfer += $output->writeString($iter854);
           }
         }
         $output->writeListEnd();
@@ -24961,14 +25272,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size848 = 0;
-            $_etype851 = 0;
-            $xfer += $input->readListBegin($_etype851, $_size848);
-            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
+            $_size855 = 0;
+            $_etype858 = 0;
+            $xfer += $input->readListBegin($_etype858, $_size855);
+            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
             {
-              $elem853 = null;
-              $xfer += $input->readString($elem853);
-              $this->success []= $elem853;
+              $elem860 = null;
+              $xfer += $input->readString($elem860);
+              $this->success []= $elem860;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25012,9 +25323,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter854)
+          foreach ($this->success as $iter861)
           {
-            $xfer += $output->writeString($iter854);
+            $xfer += $output->writeString($iter861);
           }
         }
         $output->writeListEnd();
@@ -25257,15 +25568,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size855 = 0;
-            $_etype858 = 0;
-            $xfer += $input->readListBegin($_etype858, $_size855);
-            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
+            $_size862 = 0;
+            $_etype865 = 0;
+            $xfer += $input->readListBegin($_etype865, $_size862);
+            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
             {
-              $elem860 = null;
-              $elem860 = new \metastore\Partition();
-              $xfer += $elem860->read($input);
-              $this->success []= $elem860;
+              $elem867 = null;
+              $elem867 = new \metastore\Partition();
+              $xfer += $elem867->read($input);
+              $this->success []= $elem867;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25309,9 +25620,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter861)
+          foreach ($this->success as $iter868)
           {
-            $xfer += $iter861->write($output);
+            $xfer += $iter868->write($output);
           }
         }
         $output->writeListEnd();
@@ -25554,15 +25865,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size862 = 0;
-            $_etype865 = 0;
-            $xfer += $input->readListBegin($_etype865, $_size862);
-            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
+            $_size869 = 0;
+            $_etype872 = 0;
+            $xfer += $input->readListBegin($_etype872, $_size869);
+            for ($_i873 = 0; $_i873 < $_size869; ++$_i873)
             {
-              $elem867 = null;
-              $elem867 = new \metastore\PartitionSpec();
-              $xfer += $elem867->read($input);
-              $this->success []= $elem867;
+              $elem874 = null;
+              $elem874 = new \metastore\PartitionSpec();
+              $xfer += $elem874->read($input);
+              $this->success []= $elem874;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25606,9 +25917,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter868)
+          foreach ($this->success as $iter875)
           {
-            $xfer += $iter868->write($output);
+            $xfer += $iter875->write($output);
           }
         }
         $output->writeListEnd();
@@ -26174,14 +26485,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size869 = 0;
-            $_etype872 = 0;
-            $xfer += $input->readListBegin($_etype872, $_size869);
-            for ($_i873 = 0; $_i873 < $_size869; ++$_i873)
+            $_size876 = 0;
+            $_etype879 = 0;
+            $xfer += $input->readListBegin($_etype879, $_size876);
+            for ($_i880 = 0; $_i880 < $_size876; ++$_i880)
             {
-              $elem874 = null;
-              $xfer += $input->readString($elem874);
-              $this->names []= $elem874;
+              $elem881 = null;
+              $xfer += $input->readString($elem881);
+              $this->names []= $elem881;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26219,9 +26530,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter875)
+          foreach ($this->names as $iter882)
           {
-            $xfer += $output->writeString($iter875);
+            $xfer += $output->writeString($iter882);
           }
         }
         $output->writeListEnd();
@@ -26310,15 +26621,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size876 = 0;
-            $_etype879 = 0;
-            $xfer += $input->readListBegin($_etype879, $_size876);
-            for ($_i880 = 0; $_i880 < $_size876; ++$_i880)
+            $_size883 = 0;
+            $_etype886 = 0;
+            $xfer += $input->readListBegin($_etype886, $_size883);
+            for ($_i887 = 0; $_i887 < $_size883; ++$_i887)
             {
-              $elem881 = null;
-              $elem881 = new \metastore\Partition();
-              $xfer += $elem881->read($input);
-              $this->success []= $elem881;
+              $elem888 = null;
+              $elem888 = new \metastore\Partition();
+              $xfer += $elem888->read($input);
+              $this->success []= $elem888;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26362,9 +26673,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter882)
+          foreach ($this->success as $iter889)
           {
-            $xfer += $iter882->write($output);
+            $xfer += $iter889->write($output);
           }
         }
         $output->writeListEnd();
@@ -26703,15 +27014,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size883 = 0;
-            $_etype886 = 0;
-            $xfer += $input->readListBegin($_etype886, $_size883);
-            for ($_i887 = 0; $_i887 < $_size883; ++$_i887)
+            $_size890 = 0;
+            $_etype893 = 0;
+            $xfer += $input->readListBegin($_etype893, $_size890);
+            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
             {
-              $elem888 = null;
-              $elem888 = new \metastore\Partition();
-              $xfer += $elem888->read($input);
-              $this->new_parts []= $elem888;
+              $elem895 = null;
+              $elem895 = new \metastore\Partition();
+              $xfer += $elem895->read($input);
+              $this->new_parts []= $elem895;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26749,9 +27060,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter889)
+          foreach ($this->new_parts as $iter896)
           {
-            $xfer += $iter889->write($output);
+            $xfer += $iter896->write($output);
           }
         }
         $output->writeListEnd();
@@ -26966,15 +27277,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size890 = 0;
-            $_etype893 = 0;
-            $xfer += $input->readListBegin($_etype893, $_size890);
-            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
+            $_size897 = 0;
+            $_etype900 = 0;
+            $xfer += $input->readListBegin($_etype900, $_size897);
+            for ($_i901 = 0; $_i901 < $_size897; ++$_i901)
             {
-              $elem895 = null;
-              $elem895 = new \metastore\Partition();
-              $xfer += $elem895->read($input);
-              $this->new_parts []= $elem895;
+              $elem902 = null;
+              $elem902 = new \metastore\Partition();
+              $xfer += $elem902->read($input);
+              $this->new_parts []= $elem902;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27020,9 +27331,9 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter896)
+          foreach ($this->new_parts as $iter903)
           {
-            $xfer += $iter896->write($output);
+            $xfer += $iter903->write($output);
           }
         }
         $output->writeListEnd();
@@ -27500,14 +27811,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size897 = 0;
-            $_etype900 = 0;
-            $xfer += $input->readListBegin($_etype900, $_size897);
-            for ($_i901 = 0; $_i901 < $_size897; ++$_i901)
+            $_size904 = 0;
+            $_etype907 = 0;
+            $xfer += $input->readListBegin($_etype907, $_size904);
+            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
             {
-              $elem902 = null;
-              $xfer += $input->readString($elem902);
-              $this->part_vals []= $elem902;
+              $elem909 = null;
+              $xfer += $input->readString($elem909);
+              $this->part_vals []= $elem909;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27553,9 +27864,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter903)
+          foreach ($this->part_vals as $iter910)
           {
-            $xfer += $output->writeString($iter903);
+            $xfer += $output->writeString($iter910);
           }
         }
         $output->writeListEnd();
@@ -27740,14 +28051,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size904 = 0;
-            $_etype907 = 0;
-            $xfer += $input->readListBegin($_etype907, $_size904);
-            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
+            $_size911 = 0;
+            $_etype914 = 0;
+            $xfer += $input->readListBegin($_etype914, $_size911);
+            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
             {
-              $elem909 = null;
-              $xfer += $input->readString($elem909);
-              $this->part_vals []= $elem909;
+              $elem916 = null;
+              $xfer += $input->readString($elem916);
+              $this->part_vals []= $elem916;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27782,9 +28093,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter910)
+          foreach ($this->part_vals as $iter917)
           {
-            $xfer += $output->writeString($iter910);
+            $xfer += $output->writeString($iter917);
           }
         }
         $output->writeListEnd();
@@ -28238,14 +28549,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size911 = 0;
-            $_etype914 = 0;
-            $xfer += $input->readListBegin($_etype914, $_size911);
-            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
+            $_size918 = 0;
+            $_etype921 = 0;
+            $xfer += $input->readListBegin($_etype921, $_size918);
+            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
             {
-              $elem916 = null;
-              $xfer += $input->readString($elem916);
-              $this->success []= $elem916;
+              $elem923 = null;
+              $xfer += $input->readString($elem923);
+              $this->success []= $elem923;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28281,9 +28592,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter917)
+          foreach ($this->success as $iter924)
           {
-            $xfer += $output->writeString($iter917);
+            $xfer += $output->writeString($iter924);
           }
         }
         $output->writeListEnd();
@@ -28443,17 +28754,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size918 = 0;
-            $_ktype919 = 0;
-            $_vtype920 = 0;
-            $xfer += $input->readMapBegin($_ktype919, $_vtype920, $_size918);
-            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
+            $_size925 = 0;
+            $_ktype926 = 0;
+            $_vtype927 = 0;
+            $xfer += $input->readMapBegin($_ktype926, $_vtype927, $_size925);
+            for ($_i929 = 0; $_i929 < $_size925; ++$_i929)
             {
-              $key923 = '';
-              $val924 = '';
-              $xfer += $input->readString($key923);
-              $xfer += $input->readString($val924);
-              $this->success[$key923] = $val924;
+              $key930 = '';
+              $val931 = '';
+              $xfer += $input->readString($key930);
+              $xfer += $input->readString($val931);
+              $this->success[$key930] = $val931;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -28489,10 +28800,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter925 => $viter926)
+          foreach ($this->success as $kiter932 => $viter933)
           {
-            $xfer += $output->writeString($kiter925);
-            $xfer += $output->writeString($viter926);
+            $xfer += $output->writeString($kiter932);
+            $xfer += $output->writeString($viter933);
           }
         }
         $output->writeMapEnd();
@@ -28612,17 +28923,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size927 = 0;
-            $_ktype928 = 0;
-            $_vtype929 = 0;
-            $xfer += $input->readMapBegin($_ktype928, $_vtype929, $_size927);
-            for ($_i931 = 0; $_i931 < $_size927; ++$_i931)
+            $_size934 = 0;
+            $_ktype935 = 0;
+            $_vtype936 = 0;
+            $xfer += $input->readMapBegin($_ktype935, $_vtype936, $_size934);
+            for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
             {
-              $key932 = '';
-              $val933 = '';
-              $xfer += $input->readString($key932);
-              $xfer += $input->readString($val933);
-              $this->part_vals[$key932] = $val933;
+              $key939 = '';
+              $val940 = '';
+              $xfer += $input->readString($key939);
+              $xfer += $input->readString($val940);
+              $this->part_vals[$key939] = $val940;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -28667,10 +28978,10 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter934 => $viter935)
+          foreach ($this->part_vals as $kiter941 => $viter942)
           {
-            $xfer += $output->writeString($kiter934);
-            $xfer += $output->writeString($viter935);
+            $xfer += $output->writeString($kiter941);
+            $xfer += $output->writeString($viter942);
           }
         }
         $output->writeMapEnd();
@@ -28992,17 +29303,17 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size936 = 0;
-            $_ktype937 = 0;
-            $_vtype938 = 0;
-            $xfer += $input->readMapBegin($_ktype937, $_vtype938, $_size936);
-            for ($_i940 = 0; $_i940 < $_size936; ++$_i940)
+            $_size943 = 0;
+            $_ktype944 = 0;
+            $_vtype945 = 0;
+            $xfer += $input->readMapBegin($_ktype944, $_vtype945, $_size943);
+            for ($_i947 = 0; $_i947 < $_size943; ++$_i947)
             {
-              $key941 = '';
-              $val942 = '';
-              $xfer += $input->readString($key941);
-              $xfer += $input->readString($val942);
-              $this->part_vals[$key941] = $val942;
+              $key948 = '';
+              $val949 = '';
+              $xfer += $input->readString($key948);
+              $xfer += $input->readString($val949);
+              $this->part_vals[$key948] = $val949;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -29047,10 +29358,10 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter943 => $viter944)
+          foreach ($this->part_vals as $kiter950 => $viter951)
           {
-            $xfer += $output->writeString($kiter943);
-            $xfer += $output->writeString($viter944);
+            $xfer += $output->writeString($kiter950);
+            $xfer += $output->writeString($viter951);
           }
         }
         $output->writeMapEnd();
@@ -30524,15 +30835,15 @@ class ThriftHiveMetastore_get_indexes_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size945 = 0;
-            $_etype948 = 0;
-            $xfer += $input->readListBegin($_etype948, $_size945);
-            for ($_i949 = 0; $_i949 < $_size945; ++$_i949)
+            $_size952 = 0;
+            $_etype955 = 0;
+            $xfer += $input->readListBegin($_etype955, $_size952);
+            for ($_i956 = 0; $_i956 < $_size952; ++$_i956)
             {
-              $elem950 = null;
-              $elem950 = new \metastore\Index();
-              $xfer += $elem950->read($input);
-              $this->success []= $elem950;
+              $elem957 = null;
+              $elem957 = new \metastore\Index();
+              $xfer += $elem957->read($input);
+              $this->success []= $elem957;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30576,9 +30887,9 @@ class ThriftHiveMetastore_get_indexes_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter951)
+          foreach ($this->success as $iter958)
           {
-            $xfer += $iter951->write($output);
+            $xfer += $iter958->write($output);
           }
         }
         $output->writeListEnd();
@@ -30785,14 +31096,14 @@ class ThriftHiveMetastore_get_index_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size952 = 0;
-            $_etype955 = 0;
-            $xfer += $input->readListBegin($_etype955, $_size952);
-            for ($_i956 = 0; $_i956 < $_size952; ++$_i956)
+            $_size959 = 0;
+            $_etype962 = 0;
+            $xfer += $input->readListBegin($_etype962, $_size959);
+            for ($_i963 = 0; $_i963 < $_size959; ++$_i963)
             {
-              $elem957 = null;
-              $xfer += $input->readString($elem957);
-              $this->success []= $elem957;
+              $elem964 = null;
+              $xfer += $input->readString($elem964);
+              $this->success []= $elem964;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30828,9 +31139,9 @@ class ThriftHiveMetastore_get_index_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter958)
+          foreach ($this->success as $iter965)
           {
-            $xfer += $output->writeString($iter958);
+            $xfer += $output->writeString($iter965);
           }
         }
         $output->writeListEnd();
@@ -34724,14 +35035,14 @@ class ThriftHiveMetastore_get_functions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size959 = 0;
-            $_etype962 = 0;
-            $xfer += $input->readListBegin($_etype962, $_size959);
-            for ($_i963 = 0; $_i963 < $_size959; ++$_i963)
+            $_size966 = 0;
+            $_etype969 = 0;
+            $xfer += $input->readListBegin($_etype969, $_size966);
+            for ($_i970 = 0; $_i970 < $_size966; ++$_i970)
             {
-              $elem964 = null;
-              $xfer += $input->readString($elem964);
-              $this->success []= $elem964;
+              $elem971 = null;
+              $xfer += $input->readString($elem971);
+              $this->success []= $elem971;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34767,9 +35078,9 @@ class ThriftHiveMetastore_get_functions_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter965)
+          foreach ($this->success as $iter972)
           {
-            $xfer += $output->writeString($iter965);
+            $xfer += $output->writeString($iter972);
           }
         }
         $output->writeListEnd();
@@ -35638,14 +35949,14 @@ class ThriftHiveMetastore_get_role_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size966 = 0;
-            $_etype969 = 0;
-            $xfer += $input->readListBegin($_etype969, $_size966);
-            for ($_i970 = 0; $_i970 < $_size966; ++$_i970)
+            $_size973 = 0;
+            $_etype976 = 0;
+            $xfer += $input->readListBegin($_etype976, $_size973);
+            for ($_i977 = 0; $_i977 < $_size973; ++$_i977)
             {
-              $elem971 = null;
-              $xfer += $input->readString($elem971);
-              $this->success []= $elem971;
+              $elem978 = null;
+              $xfer += $input->readString($elem978);
+              $this->success []= $elem978;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35681,9 +35992,9 @@ class ThriftHiveMetastore_get_role_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter972)
+          foreach ($this->success as $iter979)
           {
-            $xfer += $output->writeString($iter972);
+            $xfer += $output->writeString($iter979);
           }
         }
         $output->writeListEnd();
@@ -36374,15 +36685,15 @@ class ThriftHiveMetastore_list_roles_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size973 = 0;
-            $_etype976 = 0;
-            $xfer += $input->readListBegin($_etype976, $_size973);
-            for ($_i977 = 0; $_i977 < $_size973; ++$_i977)
+            $_size980 = 0;
+            $_etype983 = 0;
+            $xfer += $input->readListBegin($_etype983, $_size980);
+            for ($_i984 = 0; $_i984 < $_size980; ++$_i984)
             {
-              $elem978 = null;
-              $elem978 = new \metastore\Role();
-              $xfer += $elem978->read($input);
-              $this->success []= $elem978;
+              $elem985 = null;
+              $elem985 = new \metastore\Role();
+              $xfer += $elem985->read($input);
+              $this->success []= $elem985;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36418,9 +36729,9 @@ class ThriftHiveMetastore_list_roles_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter979)
+          foreach ($this->success as $iter986)
           {
-            $xfer += $iter979->write($output);
+            $xfer += $iter986->write($output);
           }
         }
         $output->writeListEnd();
@@ -37082,14 +37393,14 @@ class ThriftHiveMetastore_get_privilege_set_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size980 = 0;
-            $_etype983 = 0;
-            $xfer += $input->readListBegin($_etype983, $_size980);
-            for ($_i984 = 0; $_i984 < $_size980; ++$_i984)
+            $_size987 = 0;
+            $_etype990 = 0;
+            $xfer += $input->readListBegin($_etype990, $_size987);
+            for ($_i991 = 0; $_i991 < $_size987; ++$_i991)
             {
-              $elem985 = null;
-              $xfer += $input->readString($elem985);
-              $this->group_names []= $elem985;
+              $elem992 = null;
+              $xfer += $input->readString($elem992);
+              $this->group_names []= $elem992;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -37130,9 +37441,9 @@ class ThriftHiveMetastore_get_privilege_set_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter986)
+          foreach ($this->group_names as $iter993)
           {
-            $xfer += $output->writeString($iter986);
+            $xfer += $output->writeString($iter993);
           }
         }
         $output->writeListEnd();
@@ -37440,15 +37751,15 @@ class ThriftHiveMetastore_list_privileges_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size987 = 0;
-            $_etype990 = 0;
-            $xfer += $input->readListBegin($_etype990, $_size987);
-            for ($_i991 = 0; $_i991 < $_size987; ++$_i991)
+            $_size994 = 0;
+            $_etype997 = 0;
+            $xfer += $input->readListBegin($_etype997, $_size994);
+            for ($_i998 = 0; $_i998 < $_size994; ++$_i998)
             {
-              $elem992 = null;
-              $elem992 = new \metastore\HiveObjectPrivilege();
-              $xfer += $elem992->read($input);
-              $this->success []= $elem992;
+              $elem999 = null;
+              $elem999 = new \metastore\HiveObjectPrivilege();
+              $xfer += $elem999->read($input);
+              $this->success []= $elem999;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -37484,9 +37795,9 @@ class ThriftHiveMetastore_list_privileges_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter993)
+          foreach ($this->success as $iter1000)
           {
-            $xfer += $iter993->write($output);
+            $xfer += $iter1000->write($output);
           }
         }
         $output->writeListEnd();
@@ -38118,14 +38429,14 @@ class ThriftHiveMetastore_set_ugi_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size994 = 0;
-            $_etype997 = 0;
-            $xfer += $input->readListBegin($_etype997, $_size994);
-            for ($_i998 = 0; $_i998 < $_size994; ++$_i998)
+            $_size1001 = 0;
+            $_etype1004 = 0;
+            $xfer += $input->readListBegin($_etype1004, $_size1001);
+            for ($_i1005 = 0; $_i1005 < $_size1001; ++$_i1005)
             {
-              $elem999 = null;
-              $xfer += $input->readString($elem999);
-              $this->group_names []= $elem999;
+              $elem1006 = null;
+              $xfer += $input->readString($elem1006);
+              $this->group_names []= $elem1006;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -38158,9 +38469,9 @@ class ThriftHiveMetastore_set_ugi_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1000)
+          foreach ($this->group_names as $iter1007)
           {
-            $xfer += $output->writeString($iter1000);
+            $xfer += $output->writeString($iter1007);
           }
         }
         $output->writeListEnd();
@@ -38236,14 +38547,14 @@ class ThriftHiveMetastore_set_ugi_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1001 = 0;
-            $_etype1004 = 0;
-            $xfer += $input->readListBegin($_etype1004, $_size1001);
-            for ($_i1005 = 0; $_i1005 < $_size1001; ++$_i1005)
+            $_size1008 = 0;
+            $_etype1011 = 0;
+            $xfer += $input->readListBegin($_etype1011, $_size1008);
+            for ($_i1012 = 0; $_i1012 < $_size1008; ++$_i1012)
             {
-              $elem1006 = null;
-              $xfer += $input->readString($elem1006);
-              $this->success []= $elem1006;
+              $elem1013 = null;
+              $xfer += $input->readString($elem1013);
+              $this->success []= $elem1013;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -38279,9 +38590,9 @@ class ThriftHiveMetastore_set_ugi_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1007)
+          foreach ($this->success as $iter1014)
           {
-            $xfer += $output->writeString($iter1007);
+            $xfer += $output->writeString($iter1014);
           }
         }
         $output->writeListEnd();
@@ -39398,14 +39709,14 @@ class ThriftHiveMetastore_get_all_token_identifiers_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1008 = 0;
-            $_etype1011 = 0;
-            $xfer += $input->readListBegin($_etype1011, $_size1008);
-            for ($_i1012 = 0; $_i1012 < $_size1008; ++$_i1012)
+            $_size1015 = 0;
+            $_etype1018 = 0;
+            $xfer += $input->readListBegin($_etype1018, $_size1015);
+            for ($_i1019 = 0; $_i1019 < $_size1015; ++$_i1019)
             {
-              $elem1013 = null;
-              $xfer += $input->readString($elem1013);
-              $this->success []= $elem1013;
+              $elem1020 = null;
+              $xfer += $input->readString($elem1020);
+              $this->success []= $elem1020;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -39433,9 +39744,9 @@ class ThriftHiveMetastore_get_all_token_identifiers_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1014)
+          foreach ($this->success as $iter1021)
           {
-            $xfer += $output->writeString($iter1014);
+            $xfer += $output->writeString($iter1021);
           }
         }
         $output->writeListEnd();
@@ -40074,14 +40385,14 @@ class ThriftHiveMetastore_get_master_keys_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1015 = 0;
-            $_etype1018 = 0;
-            $xfer += $input->readListBegin($_etype1018, $_size1015);
-            for ($_i1019 = 0; $_i1019 < $_size1015; ++$_i1019)
+            $_size1022 = 0;
+            $_etype1025 = 0;
+            $xfer += $input->readListBegin($_etype1025, $_size1022);
+            for ($_i1026 = 0; $_i1026 < $_size1022; ++$_i1026)
             {
-              $elem1020 = null;
-              $xfer += $input->readString($elem1020);
-              $this->success []= $elem1020;
+              $elem1027 = null;
+              $xfer += $input->readString($elem1027);
+              $this->success []= $elem1027;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -40109,9 +40420,9 @@ class ThriftHiveMetastore_get_master_keys_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1021)
+          foreach ($this->success as $iter1028)
           {
-            $xfer += $output->writeString($iter1021);
+            $xfer += $output->writeString($iter1028);
           }
         }
         $output->writeListEnd();

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 c58ccf2..8f505f1 100755
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -49,6 +49,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_tables_by_type(string db_name, string pattern, string tableType)')
   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)')
@@ -392,6 +393,12 @@ elif cmd == 'get_tables':
     sys.exit(1)
   pp.pprint(client.get_tables(args[0],args[1],))
 
+elif cmd == 'get_tables_by_type':
+  if len(args) != 3:
+    print('get_tables_by_type requires 3 args')
+    sys.exit(1)
+  pp.pprint(client.get_tables_by_type(args[0],args[1],args[2],))
+
 elif cmd == 'get_table_meta':
   if len(args) != 3:
     print('get_table_meta requires 3 args')


[2/7] hive git commit: HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)

Posted by ai...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 57a748a..10778f2 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -213,6 +213,15 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def get_tables_by_type(self, db_name, pattern, tableType):
+    """
+    Parameters:
+     - db_name
+     - pattern
+     - tableType
+    """
+    pass
+
   def get_table_meta(self, db_patterns, tbl_patterns, tbl_types):
     """
     Parameters:
@@ -2106,6 +2115,43 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o1
     raise TApplicationException(TApplicationException.MISSING_RESULT, "get_tables failed: unknown result")
 
+  def get_tables_by_type(self, db_name, pattern, tableType):
+    """
+    Parameters:
+     - db_name
+     - pattern
+     - tableType
+    """
+    self.send_get_tables_by_type(db_name, pattern, tableType)
+    return self.recv_get_tables_by_type()
+
+  def send_get_tables_by_type(self, db_name, pattern, tableType):
+    self._oprot.writeMessageBegin('get_tables_by_type', TMessageType.CALL, self._seqid)
+    args = get_tables_by_type_args()
+    args.db_name = db_name
+    args.pattern = pattern
+    args.tableType = tableType
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_tables_by_type(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_tables_by_type_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_tables_by_type failed: unknown result")
+
   def get_table_meta(self, db_patterns, tbl_patterns, tbl_types):
     """
     Parameters:
@@ -6625,6 +6671,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_tables_by_type"] = Processor.process_get_tables_by_type
     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
@@ -7411,6 +7458,28 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_get_tables_by_type(self, seqid, iprot, oprot):
+    args = get_tables_by_type_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_tables_by_type_result()
+    try:
+      result.success = self._handler.get_tables_by_type(args.db_name, args.pattern, args.tableType)
+      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_tables_by_type", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_get_table_meta(self, seqid, iprot, oprot):
     args = get_table_meta_args()
     args.read(iprot)
@@ -14626,6 +14695,183 @@ class get_tables_result:
   def __ne__(self, other):
     return not (self == other)
 
+class get_tables_by_type_args:
+  """
+  Attributes:
+   - db_name
+   - pattern
+   - tableType
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'db_name', None, None, ), # 1
+    (2, TType.STRING, 'pattern', None, None, ), # 2
+    (3, TType.STRING, 'tableType', None, None, ), # 3
+  )
+
+  def __init__(self, db_name=None, pattern=None, tableType=None,):
+    self.db_name = db_name
+    self.pattern = pattern
+    self.tableType = tableType
+
+  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_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.pattern = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.tableType = 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('get_tables_by_type_args')
+    if self.db_name is not None:
+      oprot.writeFieldBegin('db_name', TType.STRING, 1)
+      oprot.writeString(self.db_name)
+      oprot.writeFieldEnd()
+    if self.pattern is not None:
+      oprot.writeFieldBegin('pattern', TType.STRING, 2)
+      oprot.writeString(self.pattern)
+      oprot.writeFieldEnd()
+    if self.tableType is not None:
+      oprot.writeFieldBegin('tableType', TType.STRING, 3)
+      oprot.writeString(self.tableType)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.db_name)
+    value = (value * 31) ^ hash(self.pattern)
+    value = (value * 31) ^ hash(self.tableType)
+    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_tables_by_type_result:
+  """
+  Attributes:
+   - success
+   - o1
+  """
+
+  thrift_spec = (
+    (0, TType.LIST, 'success', (TType.STRING,None), 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 = []
+          (_etype641, _size638) = iprot.readListBegin()
+          for _i642 in xrange(_size638):
+            _elem643 = iprot.readString()
+            self.success.append(_elem643)
+          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_tables_by_type_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.LIST, 0)
+      oprot.writeListBegin(TType.STRING, len(self.success))
+      for iter644 in self.success:
+        oprot.writeString(iter644)
+      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_table_meta_args:
   """
   Attributes:
@@ -14668,10 +14914,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype641, _size638) = iprot.readListBegin()
-          for _i642 in xrange(_size638):
-            _elem643 = iprot.readString()
-            self.tbl_types.append(_elem643)
+          (_etype648, _size645) = iprot.readListBegin()
+          for _i649 in xrange(_size645):
+            _elem650 = iprot.readString()
+            self.tbl_types.append(_elem650)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14696,8 +14942,8 @@ class get_table_meta_args:
     if self.tbl_types is not None:
       oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.tbl_types))
-      for iter644 in self.tbl_types:
-        oprot.writeString(iter644)
+      for iter651 in self.tbl_types:
+        oprot.writeString(iter651)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14753,11 +14999,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype648, _size645) = iprot.readListBegin()
-          for _i649 in xrange(_size645):
-            _elem650 = TableMeta()
-            _elem650.read(iprot)
-            self.success.append(_elem650)
+          (_etype655, _size652) = iprot.readListBegin()
+          for _i656 in xrange(_size652):
+            _elem657 = TableMeta()
+            _elem657.read(iprot)
+            self.success.append(_elem657)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14780,8 +15026,8 @@ class get_table_meta_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter651 in self.success:
-        iter651.write(oprot)
+      for iter658 in self.success:
+        iter658.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14905,10 +15151,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype655, _size652) = iprot.readListBegin()
-          for _i656 in xrange(_size652):
-            _elem657 = iprot.readString()
-            self.success.append(_elem657)
+          (_etype662, _size659) = iprot.readListBegin()
+          for _i663 in xrange(_size659):
+            _elem664 = iprot.readString()
+            self.success.append(_elem664)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14931,8 +15177,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 iter658 in self.success:
-        oprot.writeString(iter658)
+      for iter665 in self.success:
+        oprot.writeString(iter665)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15168,10 +15414,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype662, _size659) = iprot.readListBegin()
-          for _i663 in xrange(_size659):
-            _elem664 = iprot.readString()
-            self.tbl_names.append(_elem664)
+          (_etype669, _size666) = iprot.readListBegin()
+          for _i670 in xrange(_size666):
+            _elem671 = iprot.readString()
+            self.tbl_names.append(_elem671)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15192,8 +15438,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 iter665 in self.tbl_names:
-        oprot.writeString(iter665)
+      for iter672 in self.tbl_names:
+        oprot.writeString(iter672)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15254,11 +15500,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype669, _size666) = iprot.readListBegin()
-          for _i670 in xrange(_size666):
-            _elem671 = Table()
-            _elem671.read(iprot)
-            self.success.append(_elem671)
+          (_etype676, _size673) = iprot.readListBegin()
+          for _i677 in xrange(_size673):
+            _elem678 = Table()
+            _elem678.read(iprot)
+            self.success.append(_elem678)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15293,8 +15539,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 iter672 in self.success:
-        iter672.write(oprot)
+      for iter679 in self.success:
+        iter679.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15460,10 +15706,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype676, _size673) = iprot.readListBegin()
-          for _i677 in xrange(_size673):
-            _elem678 = iprot.readString()
-            self.success.append(_elem678)
+          (_etype683, _size680) = iprot.readListBegin()
+          for _i684 in xrange(_size680):
+            _elem685 = iprot.readString()
+            self.success.append(_elem685)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15498,8 +15744,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 iter679 in self.success:
-        oprot.writeString(iter679)
+      for iter686 in self.success:
+        oprot.writeString(iter686)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16469,11 +16715,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype683, _size680) = iprot.readListBegin()
-          for _i684 in xrange(_size680):
-            _elem685 = Partition()
-            _elem685.read(iprot)
-            self.new_parts.append(_elem685)
+          (_etype690, _size687) = iprot.readListBegin()
+          for _i691 in xrange(_size687):
+            _elem692 = Partition()
+            _elem692.read(iprot)
+            self.new_parts.append(_elem692)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16490,8 +16736,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 iter686 in self.new_parts:
-        iter686.write(oprot)
+      for iter693 in self.new_parts:
+        iter693.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16649,11 +16895,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype690, _size687) = iprot.readListBegin()
-          for _i691 in xrange(_size687):
-            _elem692 = PartitionSpec()
-            _elem692.read(iprot)
-            self.new_parts.append(_elem692)
+          (_etype697, _size694) = iprot.readListBegin()
+          for _i698 in xrange(_size694):
+            _elem699 = PartitionSpec()
+            _elem699.read(iprot)
+            self.new_parts.append(_elem699)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16670,8 +16916,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 iter693 in self.new_parts:
-        iter693.write(oprot)
+      for iter700 in self.new_parts:
+        iter700.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16845,10 +17091,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype697, _size694) = iprot.readListBegin()
-          for _i698 in xrange(_size694):
-            _elem699 = iprot.readString()
-            self.part_vals.append(_elem699)
+          (_etype704, _size701) = iprot.readListBegin()
+          for _i705 in xrange(_size701):
+            _elem706 = iprot.readString()
+            self.part_vals.append(_elem706)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16873,8 +17119,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 iter700 in self.part_vals:
-        oprot.writeString(iter700)
+      for iter707 in self.part_vals:
+        oprot.writeString(iter707)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17227,10 +17473,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype704, _size701) = iprot.readListBegin()
-          for _i705 in xrange(_size701):
-            _elem706 = iprot.readString()
-            self.part_vals.append(_elem706)
+          (_etype711, _size708) = iprot.readListBegin()
+          for _i712 in xrange(_size708):
+            _elem713 = iprot.readString()
+            self.part_vals.append(_elem713)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17261,8 +17507,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 iter707 in self.part_vals:
-        oprot.writeString(iter707)
+      for iter714 in self.part_vals:
+        oprot.writeString(iter714)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -17857,10 +18103,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype711, _size708) = iprot.readListBegin()
-          for _i712 in xrange(_size708):
-            _elem713 = iprot.readString()
-            self.part_vals.append(_elem713)
+          (_etype718, _size715) = iprot.readListBegin()
+          for _i719 in xrange(_size715):
+            _elem720 = iprot.readString()
+            self.part_vals.append(_elem720)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17890,8 +18136,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 iter714 in self.part_vals:
-        oprot.writeString(iter714)
+      for iter721 in self.part_vals:
+        oprot.writeString(iter721)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -18064,10 +18310,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype718, _size715) = iprot.readListBegin()
-          for _i719 in xrange(_size715):
-            _elem720 = iprot.readString()
-            self.part_vals.append(_elem720)
+          (_etype725, _size722) = iprot.readListBegin()
+          for _i726 in xrange(_size722):
+            _elem727 = iprot.readString()
+            self.part_vals.append(_elem727)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18103,8 +18349,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 iter721 in self.part_vals:
-        oprot.writeString(iter721)
+      for iter728 in self.part_vals:
+        oprot.writeString(iter728)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -18841,10 +19087,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype725, _size722) = iprot.readListBegin()
-          for _i726 in xrange(_size722):
-            _elem727 = iprot.readString()
-            self.part_vals.append(_elem727)
+          (_etype732, _size729) = iprot.readListBegin()
+          for _i733 in xrange(_size729):
+            _elem734 = iprot.readString()
+            self.part_vals.append(_elem734)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18869,8 +19115,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 iter728 in self.part_vals:
-        oprot.writeString(iter728)
+      for iter735 in self.part_vals:
+        oprot.writeString(iter735)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19029,11 +19275,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype730, _vtype731, _size729 ) = iprot.readMapBegin()
-          for _i733 in xrange(_size729):
-            _key734 = iprot.readString()
-            _val735 = iprot.readString()
-            self.partitionSpecs[_key734] = _val735
+          (_ktype737, _vtype738, _size736 ) = iprot.readMapBegin()
+          for _i740 in xrange(_size736):
+            _key741 = iprot.readString()
+            _val742 = iprot.readString()
+            self.partitionSpecs[_key741] = _val742
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -19070,9 +19316,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 kiter736,viter737 in self.partitionSpecs.items():
-        oprot.writeString(kiter736)
-        oprot.writeString(viter737)
+      for kiter743,viter744 in self.partitionSpecs.items():
+        oprot.writeString(kiter743)
+        oprot.writeString(viter744)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -19277,11 +19523,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype739, _vtype740, _size738 ) = iprot.readMapBegin()
-          for _i742 in xrange(_size738):
-            _key743 = iprot.readString()
-            _val744 = iprot.readString()
-            self.partitionSpecs[_key743] = _val744
+          (_ktype746, _vtype747, _size745 ) = iprot.readMapBegin()
+          for _i749 in xrange(_size745):
+            _key750 = iprot.readString()
+            _val751 = iprot.readString()
+            self.partitionSpecs[_key750] = _val751
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -19318,9 +19564,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 kiter745,viter746 in self.partitionSpecs.items():
-        oprot.writeString(kiter745)
-        oprot.writeString(viter746)
+      for kiter752,viter753 in self.partitionSpecs.items():
+        oprot.writeString(kiter752)
+        oprot.writeString(viter753)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -19403,11 +19649,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype750, _size747) = iprot.readListBegin()
-          for _i751 in xrange(_size747):
-            _elem752 = Partition()
-            _elem752.read(iprot)
-            self.success.append(_elem752)
+          (_etype757, _size754) = iprot.readListBegin()
+          for _i758 in xrange(_size754):
+            _elem759 = Partition()
+            _elem759.read(iprot)
+            self.success.append(_elem759)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19448,8 +19694,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 iter753 in self.success:
-        iter753.write(oprot)
+      for iter760 in self.success:
+        iter760.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19543,10 +19789,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype757, _size754) = iprot.readListBegin()
-          for _i758 in xrange(_size754):
-            _elem759 = iprot.readString()
-            self.part_vals.append(_elem759)
+          (_etype764, _size761) = iprot.readListBegin()
+          for _i765 in xrange(_size761):
+            _elem766 = iprot.readString()
+            self.part_vals.append(_elem766)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19558,10 +19804,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype763, _size760) = iprot.readListBegin()
-          for _i764 in xrange(_size760):
-            _elem765 = iprot.readString()
-            self.group_names.append(_elem765)
+          (_etype770, _size767) = iprot.readListBegin()
+          for _i771 in xrange(_size767):
+            _elem772 = iprot.readString()
+            self.group_names.append(_elem772)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19586,8 +19832,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 iter766 in self.part_vals:
-        oprot.writeString(iter766)
+      for iter773 in self.part_vals:
+        oprot.writeString(iter773)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -19597,8 +19843,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 iter767 in self.group_names:
-        oprot.writeString(iter767)
+      for iter774 in self.group_names:
+        oprot.writeString(iter774)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20027,11 +20273,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype771, _size768) = iprot.readListBegin()
-          for _i772 in xrange(_size768):
-            _elem773 = Partition()
-            _elem773.read(iprot)
-            self.success.append(_elem773)
+          (_etype778, _size775) = iprot.readListBegin()
+          for _i779 in xrange(_size775):
+            _elem780 = Partition()
+            _elem780.read(iprot)
+            self.success.append(_elem780)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20060,8 +20306,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 iter774 in self.success:
-        iter774.write(oprot)
+      for iter781 in self.success:
+        iter781.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20155,10 +20401,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype778, _size775) = iprot.readListBegin()
-          for _i779 in xrange(_size775):
-            _elem780 = iprot.readString()
-            self.group_names.append(_elem780)
+          (_etype785, _size782) = iprot.readListBegin()
+          for _i786 in xrange(_size782):
+            _elem787 = iprot.readString()
+            self.group_names.append(_elem787)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20191,8 +20437,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 iter781 in self.group_names:
-        oprot.writeString(iter781)
+      for iter788 in self.group_names:
+        oprot.writeString(iter788)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20253,11 +20499,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype785, _size782) = iprot.readListBegin()
-          for _i786 in xrange(_size782):
-            _elem787 = Partition()
-            _elem787.read(iprot)
-            self.success.append(_elem787)
+          (_etype792, _size789) = iprot.readListBegin()
+          for _i793 in xrange(_size789):
+            _elem794 = Partition()
+            _elem794.read(iprot)
+            self.success.append(_elem794)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20286,8 +20532,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 iter788 in self.success:
-        iter788.write(oprot)
+      for iter795 in self.success:
+        iter795.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20445,11 +20691,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype792, _size789) = iprot.readListBegin()
-          for _i793 in xrange(_size789):
-            _elem794 = PartitionSpec()
-            _elem794.read(iprot)
-            self.success.append(_elem794)
+          (_etype799, _size796) = iprot.readListBegin()
+          for _i800 in xrange(_size796):
+            _elem801 = PartitionSpec()
+            _elem801.read(iprot)
+            self.success.append(_elem801)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20478,8 +20724,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 iter795 in self.success:
-        iter795.write(oprot)
+      for iter802 in self.success:
+        iter802.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20634,10 +20880,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype799, _size796) = iprot.readListBegin()
-          for _i800 in xrange(_size796):
-            _elem801 = iprot.readString()
-            self.success.append(_elem801)
+          (_etype806, _size803) = iprot.readListBegin()
+          for _i807 in xrange(_size803):
+            _elem808 = iprot.readString()
+            self.success.append(_elem808)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20660,8 +20906,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 iter802 in self.success:
-        oprot.writeString(iter802)
+      for iter809 in self.success:
+        oprot.writeString(iter809)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -20737,10 +20983,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype806, _size803) = iprot.readListBegin()
-          for _i807 in xrange(_size803):
-            _elem808 = iprot.readString()
-            self.part_vals.append(_elem808)
+          (_etype813, _size810) = iprot.readListBegin()
+          for _i814 in xrange(_size810):
+            _elem815 = iprot.readString()
+            self.part_vals.append(_elem815)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20770,8 +21016,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 iter809 in self.part_vals:
-        oprot.writeString(iter809)
+      for iter816 in self.part_vals:
+        oprot.writeString(iter816)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -20835,11 +21081,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype813, _size810) = iprot.readListBegin()
-          for _i814 in xrange(_size810):
-            _elem815 = Partition()
-            _elem815.read(iprot)
-            self.success.append(_elem815)
+          (_etype820, _size817) = iprot.readListBegin()
+          for _i821 in xrange(_size817):
+            _elem822 = Partition()
+            _elem822.read(iprot)
+            self.success.append(_elem822)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20868,8 +21114,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 iter816 in self.success:
-        iter816.write(oprot)
+      for iter823 in self.success:
+        iter823.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20956,10 +21202,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype820, _size817) = iprot.readListBegin()
-          for _i821 in xrange(_size817):
-            _elem822 = iprot.readString()
-            self.part_vals.append(_elem822)
+          (_etype827, _size824) = iprot.readListBegin()
+          for _i828 in xrange(_size824):
+            _elem829 = iprot.readString()
+            self.part_vals.append(_elem829)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20976,10 +21222,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype826, _size823) = iprot.readListBegin()
-          for _i827 in xrange(_size823):
-            _elem828 = iprot.readString()
-            self.group_names.append(_elem828)
+          (_etype833, _size830) = iprot.readListBegin()
+          for _i834 in xrange(_size830):
+            _elem835 = iprot.readString()
+            self.group_names.append(_elem835)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21004,8 +21250,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 iter829 in self.part_vals:
-        oprot.writeString(iter829)
+      for iter836 in self.part_vals:
+        oprot.writeString(iter836)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -21019,8 +21265,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 iter830 in self.group_names:
-        oprot.writeString(iter830)
+      for iter837 in self.group_names:
+        oprot.writeString(iter837)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21082,11 +21328,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype834, _size831) = iprot.readListBegin()
-          for _i835 in xrange(_size831):
-            _elem836 = Partition()
-            _elem836.read(iprot)
-            self.success.append(_elem836)
+          (_etype841, _size838) = iprot.readListBegin()
+          for _i842 in xrange(_size838):
+            _elem843 = Partition()
+            _elem843.read(iprot)
+            self.success.append(_elem843)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21115,8 +21361,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 iter837 in self.success:
-        iter837.write(oprot)
+      for iter844 in self.success:
+        iter844.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21197,10 +21443,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype841, _size838) = iprot.readListBegin()
-          for _i842 in xrange(_size838):
-            _elem843 = iprot.readString()
-            self.part_vals.append(_elem843)
+          (_etype848, _size845) = iprot.readListBegin()
+          for _i849 in xrange(_size845):
+            _elem850 = iprot.readString()
+            self.part_vals.append(_elem850)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21230,8 +21476,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 iter844 in self.part_vals:
-        oprot.writeString(iter844)
+      for iter851 in self.part_vals:
+        oprot.writeString(iter851)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -21295,10 +21541,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype848, _size845) = iprot.readListBegin()
-          for _i849 in xrange(_size845):
-            _elem850 = iprot.readString()
-            self.success.append(_elem850)
+          (_etype855, _size852) = iprot.readListBegin()
+          for _i856 in xrange(_size852):
+            _elem857 = iprot.readString()
+            self.success.append(_elem857)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21327,8 +21573,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 iter851 in self.success:
-        oprot.writeString(iter851)
+      for iter858 in self.success:
+        oprot.writeString(iter858)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21499,11 +21745,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype855, _size852) = iprot.readListBegin()
-          for _i856 in xrange(_size852):
-            _elem857 = Partition()
-            _elem857.read(iprot)
-            self.success.append(_elem857)
+          (_etype862, _size859) = iprot.readListBegin()
+          for _i863 in xrange(_size859):
+            _elem864 = Partition()
+            _elem864.read(iprot)
+            self.success.append(_elem864)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21532,8 +21778,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 iter858 in self.success:
-        iter858.write(oprot)
+      for iter865 in self.success:
+        iter865.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21704,11 +21950,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype862, _size859) = iprot.readListBegin()
-          for _i863 in xrange(_size859):
-            _elem864 = PartitionSpec()
-            _elem864.read(iprot)
-            self.success.append(_elem864)
+          (_etype869, _size866) = iprot.readListBegin()
+          for _i870 in xrange(_size866):
+            _elem871 = PartitionSpec()
+            _elem871.read(iprot)
+            self.success.append(_elem871)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21737,8 +21983,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 iter865 in self.success:
-        iter865.write(oprot)
+      for iter872 in self.success:
+        iter872.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22158,10 +22404,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype869, _size866) = iprot.readListBegin()
-          for _i870 in xrange(_size866):
-            _elem871 = iprot.readString()
-            self.names.append(_elem871)
+          (_etype876, _size873) = iprot.readListBegin()
+          for _i877 in xrange(_size873):
+            _elem878 = iprot.readString()
+            self.names.append(_elem878)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22186,8 +22432,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 iter872 in self.names:
-        oprot.writeString(iter872)
+      for iter879 in self.names:
+        oprot.writeString(iter879)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22246,11 +22492,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype876, _size873) = iprot.readListBegin()
-          for _i877 in xrange(_size873):
-            _elem878 = Partition()
-            _elem878.read(iprot)
-            self.success.append(_elem878)
+          (_etype883, _size880) = iprot.readListBegin()
+          for _i884 in xrange(_size880):
+            _elem885 = Partition()
+            _elem885.read(iprot)
+            self.success.append(_elem885)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22279,8 +22525,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 iter879 in self.success:
-        iter879.write(oprot)
+      for iter886 in self.success:
+        iter886.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22530,11 +22776,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype883, _size880) = iprot.readListBegin()
-          for _i884 in xrange(_size880):
-            _elem885 = Partition()
-            _elem885.read(iprot)
-            self.new_parts.append(_elem885)
+          (_etype890, _size887) = iprot.readListBegin()
+          for _i891 in xrange(_size887):
+            _elem892 = Partition()
+            _elem892.read(iprot)
+            self.new_parts.append(_elem892)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22559,8 +22805,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 iter886 in self.new_parts:
-        iter886.write(oprot)
+      for iter893 in self.new_parts:
+        iter893.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22713,11 +22959,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype890, _size887) = iprot.readListBegin()
-          for _i891 in xrange(_size887):
-            _elem892 = Partition()
-            _elem892.read(iprot)
-            self.new_parts.append(_elem892)
+          (_etype897, _size894) = iprot.readListBegin()
+          for _i898 in xrange(_size894):
+            _elem899 = Partition()
+            _elem899.read(iprot)
+            self.new_parts.append(_elem899)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22748,8 +22994,8 @@ class alter_partitions_with_environment_context_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter893 in self.new_parts:
-        iter893.write(oprot)
+      for iter900 in self.new_parts:
+        iter900.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -23093,10 +23339,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype897, _size894) = iprot.readListBegin()
-          for _i898 in xrange(_size894):
-            _elem899 = iprot.readString()
-            self.part_vals.append(_elem899)
+          (_etype904, _size901) = iprot.readListBegin()
+          for _i905 in xrange(_size901):
+            _elem906 = iprot.readString()
+            self.part_vals.append(_elem906)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23127,8 +23373,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 iter900 in self.part_vals:
-        oprot.writeString(iter900)
+      for iter907 in self.part_vals:
+        oprot.writeString(iter907)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -23270,10 +23516,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype904, _size901) = iprot.readListBegin()
-          for _i905 in xrange(_size901):
-            _elem906 = iprot.readString()
-            self.part_vals.append(_elem906)
+          (_etype911, _size908) = iprot.readListBegin()
+          for _i912 in xrange(_size908):
+            _elem913 = iprot.readString()
+            self.part_vals.append(_elem913)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23295,8 +23541,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 iter907 in self.part_vals:
-        oprot.writeString(iter907)
+      for iter914 in self.part_vals:
+        oprot.writeString(iter914)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -23654,10 +23900,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype911, _size908) = iprot.readListBegin()
-          for _i912 in xrange(_size908):
-            _elem913 = iprot.readString()
-            self.success.append(_elem913)
+          (_etype918, _size915) = iprot.readListBegin()
+          for _i919 in xrange(_size915):
+            _elem920 = iprot.readString()
+            self.success.append(_elem920)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23680,8 +23926,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 iter914 in self.success:
-        oprot.writeString(iter914)
+      for iter921 in self.success:
+        oprot.writeString(iter921)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23805,11 +24051,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype916, _vtype917, _size915 ) = iprot.readMapBegin()
-          for _i919 in xrange(_size915):
-            _key920 = iprot.readString()
-            _val921 = iprot.readString()
-            self.success[_key920] = _val921
+          (_ktype923, _vtype924, _size922 ) = iprot.readMapBegin()
+          for _i926 in xrange(_size922):
+            _key927 = iprot.readString()
+            _val928 = iprot.readString()
+            self.success[_key927] = _val928
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -23832,9 +24078,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 kiter922,viter923 in self.success.items():
-        oprot.writeString(kiter922)
-        oprot.writeString(viter923)
+      for kiter929,viter930 in self.success.items():
+        oprot.writeString(kiter929)
+        oprot.writeString(viter930)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23910,11 +24156,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype925, _vtype926, _size924 ) = iprot.readMapBegin()
-          for _i928 in xrange(_size924):
-            _key929 = iprot.readString()
-            _val930 = iprot.readString()
-            self.part_vals[_key929] = _val930
+          (_ktype932, _vtype933, _size931 ) = iprot.readMapBegin()
+          for _i935 in xrange(_size931):
+            _key936 = iprot.readString()
+            _val937 = iprot.readString()
+            self.part_vals[_key936] = _val937
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -23944,9 +24190,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 kiter931,viter932 in self.part_vals.items():
-        oprot.writeString(kiter931)
-        oprot.writeString(viter932)
+      for kiter938,viter939 in self.part_vals.items():
+        oprot.writeString(kiter938)
+        oprot.writeString(viter939)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -24160,11 +24406,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype934, _vtype935, _size933 ) = iprot.readMapBegin()
-          for _i937 in xrange(_size933):
-            _key938 = iprot.readString()
-            _val939 = iprot.readString()
-            self.part_vals[_key938] = _val939
+          (_ktype941, _vtype942, _size940 ) = iprot.readMapBegin()
+          for _i944 in xrange(_size940):
+            _key945 = iprot.readString()
+            _val946 = iprot.readString()
+            self.part_vals[_key945] = _val946
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -24194,9 +24440,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 kiter940,viter941 in self.part_vals.items():
-        oprot.writeString(kiter940)
-        oprot.writeString(viter941)
+      for kiter947,viter948 in self.part_vals.items():
+        oprot.writeString(kiter947)
+        oprot.writeString(viter948)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -25251,11 +25497,11 @@ class get_indexes_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype945, _size942) = iprot.readListBegin()
-          for _i946 in xrange(_size942):
-            _elem947 = Index()
-            _elem947.read(iprot)
-            self.success.append(_elem947)
+          (_etype952, _size949) = iprot.readListBegin()
+          for _i953 in xrange(_size949):
+            _elem954 = Index()
+            _elem954.read(iprot)
+            self.success.append(_elem954)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25284,8 +25530,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 iter948 in self.success:
-        iter948.write(oprot)
+      for iter955 in self.success:
+        iter955.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25440,10 +25686,10 @@ class get_index_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype952, _size949) = iprot.readListBegin()
-          for _i953 in xrange(_size949):
-            _elem954 = iprot.readString()
-            self.success.append(_elem954)
+          (_etype959, _size956) = iprot.readListBegin()
+          for _i960 in xrange(_size956):
+            _elem961 = iprot.readString()
+            self.success.append(_elem961)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25466,8 +25712,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 iter955 in self.success:
-        oprot.writeString(iter955)
+      for iter962 in self.success:
+        oprot.writeString(iter962)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -28333,10 +28579,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype959, _size956) = iprot.readListBegin()
-          for _i960 in xrange(_size956):
-            _elem961 = iprot.readString()
-            self.success.append(_elem961)
+          (_etype966, _size963) = iprot.readListBegin()
+          for _i967 in xrange(_size963):
+            _elem968 = iprot.readString()
+            self.success.append(_elem968)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28359,8 +28605,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 iter962 in self.success:
-        oprot.writeString(iter962)
+      for iter969 in self.success:
+        oprot.writeString(iter969)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29048,10 +29294,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype966, _size963) = iprot.readListBegin()
-          for _i967 in xrange(_size963):
-            _elem968 = iprot.readString()
-            self.success.append(_elem968)
+          (_etype973, _size970) = iprot.readListBegin()
+          for _i974 in xrange(_size970):
+            _elem975 = iprot.readString()
+            self.success.append(_elem975)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29074,8 +29320,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 iter969 in self.success:
-        oprot.writeString(iter969)
+      for iter976 in self.success:
+        oprot.writeString(iter976)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29589,11 +29835,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype973, _size970) = iprot.readListBegin()
-          for _i974 in xrange(_size970):
-            _elem975 = Role()
-            _elem975.read(iprot)
-            self.success.append(_elem975)
+          (_etype980, _size977) = iprot.readListBegin()
+          for _i981 in xrange(_size977):
+            _elem982 = Role()
+            _elem982.read(iprot)
+            self.success.append(_elem982)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29616,8 +29862,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 iter976 in self.success:
-        iter976.write(oprot)
+      for iter983 in self.success:
+        iter983.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30126,10 +30372,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype980, _size977) = iprot.readListBegin()
-          for _i981 in xrange(_size977):
-            _elem982 = iprot.readString()
-            self.group_names.append(_elem982)
+          (_etype987, _size984) = iprot.readListBegin()
+          for _i988 in xrange(_size984):
+            _elem989 = iprot.readString()
+            self.group_names.append(_elem989)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30154,8 +30400,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 iter983 in self.group_names:
-        oprot.writeString(iter983)
+      for iter990 in self.group_names:
+        oprot.writeString(iter990)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -30382,11 +30628,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype987, _size984) = iprot.readListBegin()
-          for _i988 in xrange(_size984):
-            _elem989 = HiveObjectPrivilege()
-            _elem989.read(iprot)
-            self.success.append(_elem989)
+          (_etype994, _size991) = iprot.readListBegin()
+          for _i995 in xrange(_size991):
+            _elem996 = HiveObjectPrivilege()
+            _elem996.read(iprot)
+            self.success.append(_elem996)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30409,8 +30655,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 iter990 in self.success:
-        iter990.write(oprot)
+      for iter997 in self.success:
+        iter997.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30908,10 +31154,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype994, _size991) = iprot.readListBegin()
-          for _i995 in xrange(_size991):
-            _elem996 = iprot.readString()
-            self.group_names.append(_elem996)
+          (_etype1001, _size998) = iprot.readListBegin()
+          for _i1002 in xrange(_size998):
+            _elem1003 = iprot.readString()
+            self.group_names.append(_elem1003)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30932,8 +31178,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 iter997 in self.group_names:
-        oprot.writeString(iter997)
+      for iter1004 in self.group_names:
+        oprot.writeString(iter1004)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -30988,10 +31234,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1001, _size998) = iprot.readListBegin()
-          for _i1002 in xrange(_size998):
-            _elem1003 = iprot.readString()
-            self.success.append(_elem1003)
+          (_etype1008, _size1005) = iprot.readListBegin()
+          for _i1009 in xrange(_size1005):
+            _elem1010 = iprot.readString()
+            self.success.append(_elem1010)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31014,8 +31260,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 iter1004 in self.success:
-        oprot.writeString(iter1004)
+      for iter1011 in self.success:
+        oprot.writeString(iter1011)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -31947,10 +32193,10 @@ class get_all_token_identifiers_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1008, _size1005) = iprot.readListBegin()
-          for _i1009 in xrange(_size1005):
-            _elem1010 = iprot.readString()
-            self.success.append(_elem1010)
+          (_etype1015, _size1012) = iprot.readListBegin()
+          for _i1016 in xrange(_size1012):
+            _elem1017 = iprot.readString()
+            self.success.append(_elem1017)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31967,8 +32213,8 @@ class get_all_token_identifiers_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1011 in self.success:
-        oprot.writeString(iter1011)
+      for iter1018 in self.success:
+        oprot.writeString(iter1018)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -32495,10 +32741,10 @@ class get_master_keys_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1015, _size1012) = iprot.readListBegin()
-          for _i1016 in xrange(_size1012):
-            _elem1017 = iprot.readString()
-            self.success.append(_elem1017)
+          (_etype1022, _size1019) = iprot.readListBegin()
+          for _i1023 in xrange(_size1019):
+            _elem1024 = iprot.readString()
+            self.success.append(_elem1024)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -32515,8 +32761,8 @@ class get_master_keys_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1018 in self.success:
-        oprot.writeString(iter1018)
+      for iter1025 in self.success:
+        oprot.writeString(iter1025)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 51f65c6..76a3339 100644
--- a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -432,6 +432,22 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_tables failed: unknown result')
     end
 
+    def get_tables_by_type(db_name, pattern, tableType)
+      send_get_tables_by_type(db_name, pattern, tableType)
+      return recv_get_tables_by_type()
+    end
+
+    def send_get_tables_by_type(db_name, pattern, tableType)
+      send_message('get_tables_by_type', Get_tables_by_type_args, :db_name => db_name, :pattern => pattern, :tableType => tableType)
+    end
+
+    def recv_get_tables_by_type()
+      result = receive_message(Get_tables_by_type_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_tables_by_type 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()
@@ -2828,6 +2844,17 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'get_tables', seqid)
     end
 
+    def process_get_tables_by_type(seqid, iprot, oprot)
+      args = read_args(iprot, Get_tables_by_type_args)
+      result = Get_tables_by_type_result.new()
+      begin
+        result.success = @handler.get_tables_by_type(args.db_name, args.pattern, args.tableType)
+      rescue ::MetaException => o1
+        result.o1 = o1
+      end
+      write_result(result, oprot, 'get_tables_by_type', seqid)
+    end
+
     def process_get_table_meta(seqid, iprot, oprot)
       args = read_args(iprot, Get_table_meta_args)
       result = Get_table_meta_result.new()
@@ -5261,6 +5288,44 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Get_tables_by_type_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    DB_NAME = 1
+    PATTERN = 2
+    TABLETYPE = 3
+
+    FIELDS = {
+      DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'},
+      PATTERN => {:type => ::Thrift::Types::STRING, :name => 'pattern'},
+      TABLETYPE => {:type => ::Thrift::Types::STRING, :name => 'tableType'}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_tables_by_type_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::LIST, :name => 'success', :element => {:type => ::Thrift::Types::STRING}},
+      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_table_meta_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     DB_PATTERNS = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 c4d03eb..530d2f4 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
 import org.apache.hadoop.hive.common.metrics.common.MetricsVariable;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.*;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
@@ -3744,6 +3745,28 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     @Override
+    public List<String> get_tables_by_type(final String dbname, final String pattern, final String tableType)
+        throws MetaException {
+      startFunction("get_tables_by_type", ": db=" + dbname + " pat=" + pattern + ",type=" + tableType);
+
+      List<String> ret = null;
+      Exception ex = null;
+      try {
+        ret = getMS().getTables(dbname, pattern, TableType.valueOf(tableType));
+      } catch (Exception e) {
+        ex = e;
+        if (e instanceof MetaException) {
+          throw (MetaException) e;
+        } else {
+          throw newMetaException(e);
+        }
+      } finally {
+        endFunction("get_tables_by_type", ret != null, ex);
+      }
+      return ret;
+    }
+
+    @Override
     public List<String> get_all_tables(final String dbname) throws MetaException {
       startFunction("get_all_tables", ": db=" + dbname);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 909d8eb..c32486f 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.common.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.conf.HiveConfUtil;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
 import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest;
 import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
@@ -1383,6 +1384,17 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     return null;
   }
 
+  /** {@inheritDoc} */
+  @Override
+  public List<String> getTables(String dbname, String tablePattern, TableType tableType) throws MetaException {
+    try {
+      return filterHook.filterTableNames(dbname, client.get_tables_by_type(dbname, tablePattern, tableType.toString()));
+    } catch (Exception e) {
+      MetaStoreUtils.logAndThrowMetaException(e);
+    }
+    return null;
+  }
+
   @Override
   public List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)
       throws MetaException {

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 8dc4e28..b770559 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience.Public;
 import org.apache.hadoop.hive.common.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.annotation.NoReconnect;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
@@ -172,6 +173,20 @@ public interface IMetaStoreClient {
       throws MetaException, TException, UnknownDBException;
 
   /**
+   * Get the names of all tables in the specified database that satisfy the supplied
+   * table name pattern and table type (MANAGED_TABLE || EXTERNAL_TABLE || VIRTUAL_VIEW)
+   * @param dbName Name of the database to fetch tables in.
+   * @param tablePattern pattern to match for table names.
+   * @param tableType Type of the table in the HMS store. VIRTUAL_VIEW is for views.
+   * @return List of table names.
+   * @throws MetaException
+   * @throws TException
+   * @throws UnknownDBException
+   */
+  List<String> getTables(String dbName, String tablePattern, TableType tableType)
+      throws MetaException, TException, UnknownDBException;
+
+  /**
    * For quick GetTablesOperation
    */
   List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 74bde3d..90ea641 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -1197,6 +1197,11 @@ public class ObjectStore implements RawStore, Configurable {
 
   @Override
   public List<String> getTables(String dbName, String pattern) throws MetaException {
+    return getTables(dbName, pattern, null);
+  }
+
+  @Override
+  public List<String> getTables(String dbName, String pattern, TableType tableType) throws MetaException {
     boolean commited = false;
     Query query = null;
     List<String> tbls = null;
@@ -1219,6 +1224,9 @@ public class ObjectStore implements RawStore, Configurable {
         first = false;
       }
       queryStr = queryStr + ")";
+      if (tableType != null) {
+        queryStr = queryStr + " && tableType.matches(\"" + tableType.toString() + "\")";
+      }
       query = pm.newQuery(queryStr);
       query.declareParameters("java.lang.String dbName");
       query.setResult("tableName");

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 bbd47b8..a3dd4e5 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -28,6 +28,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
@@ -155,6 +156,9 @@ public interface RawStore extends Configurable {
   public List<String> getTables(String dbName, String pattern)
       throws MetaException;
 
+  public List<String> getTables(String dbName, String pattern, TableType tableType)
+      throws MetaException;
+
   public List<TableMeta> getTableMeta(
       String dbNames, String tableNames, List<String> tableTypes) throws MetaException;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 c65c7a4..f9fad4c 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
@@ -33,6 +33,7 @@ import org.apache.hadoop.hive.metastore.HiveMetaStore;
 import org.apache.hadoop.hive.metastore.PartFilterExprUtil;
 import org.apache.hadoop.hive.metastore.PartitionExpressionProxy;
 import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
@@ -530,6 +531,11 @@ public class HBaseStore implements RawStore {
     }
   }
 
+  @Override
+  public List<String> getTables(String dbName, String pattern, TableType tableType) throws MetaException {
+    throw new UnsupportedOperationException();
+  }
+
   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)));

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 1ea72a0..f64b08d 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -26,6 +26,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
@@ -223,6 +224,11 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
+  public List<String> getTables(String dbName, String pattern, TableType tableType) throws MetaException {
+    return objectStore.getTables(dbName, pattern, tableType);
+  }
+
+  @Override
   public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
       throws MetaException {
     return objectStore.getTableMeta(dbNames, tableNames, tableTypes);

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 3e6acc7..2682886 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -27,6 +27,7 @@ import junit.framework.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
@@ -227,6 +228,11 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
+  public List<String> getTables(String dbName, String pattern, TableType tableType) throws MetaException {
+    return Collections.emptyList();
+  }
+
+  @Override
   public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
       throws MetaException {
     return Collections.emptyList();

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index ec31cf4..cea8ce8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -427,7 +427,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
       ShowTablesDesc showTbls = work.getShowTblsDesc();
       if (showTbls != null) {
-        return showTables(db, showTbls);
+        return showTablesOrViews(db, showTbls);
       }
 
       ShowColumnsDesc showCols = work.getShowColumnsDesc();
@@ -2380,37 +2380,44 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
   }
 
   /**
-   * Write a list of the tables in the database to a file.
+   * Write a list of the tables/views in the database to a file.
    *
    * @param db
-   *          The database in question.
-   * @param showTbls
-   *          These are the tables we're interested in.
+   *          The database in context.
+   * @param showDesc
+   *        A ShowTablesDesc for tables or views we're interested in.
    * @return Returns 0 when execution succeeds and above 0 if it fails.
    * @throws HiveException
    *           Throws this exception if an unexpected error occurs.
    */
-  private int showTables(Hive db, ShowTablesDesc showTbls) throws HiveException {
-    // get the tables for the desired pattern - populate the output stream
-    List<String> tbls = null;
-    String dbName = showTbls.getDbName();
+  private int showTablesOrViews(Hive db, ShowTablesDesc showDesc) throws HiveException {
+    // get the tables/views for the desired pattern - populate the output stream
+    List<String> tablesOrViews = null;
+
+    String dbName      = showDesc.getDbName();
+    String pattern     = showDesc.getPattern(); // if null, all tables/views are returned
+    String resultsFile = showDesc.getResFile();
+    TableType type     = showDesc.getType(); // will be null for tables, VIRTUAL_VIEW for views
 
     if (!db.databaseExists(dbName)) {
       throw new HiveException(ErrorMsg.DATABASE_NOT_EXISTS, dbName);
     }
-    if (showTbls.getPattern() != null) {
-      LOG.info("pattern: " + showTbls.getPattern());
-      tbls = db.getTablesByPattern(dbName, showTbls.getPattern());
-      LOG.info("results : " + tbls.size());
-    } else {
-      tbls = db.getAllTables(dbName);
-    }
+
+    LOG.debug("pattern: " + pattern);
+    tablesOrViews = db.getTablesByType(dbName, pattern, type);
+    LOG.debug("results : " + tablesOrViews.size());
 
     // write the results in the file
-    DataOutputStream outStream = getOutputStream(showTbls.getResFile());
+    DataOutputStream outStream = null;
     try {
-      SortedSet<String> sortedTbls = new TreeSet<String>(tbls);
-      formatter.showTables(outStream, sortedTbls);
+      Path resFile = new Path(resultsFile);
+      FileSystem fs = resFile.getFileSystem(conf);
+      outStream = fs.create(resFile);
+
+      SortedSet<String> sortedSet = new TreeSet<String>(tablesOrViews);
+      formatter.showTables(outStream, sortedSet);
+      outStream.close();
+      outStream = null;
     } catch (Exception e) {
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR, "in database" + dbName);
     } finally {


[4/7] hive git commit: HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)

Posted by ai...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 cb5dec9..d827d6c 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
@@ -92,6 +92,8 @@ public class ThriftHiveMetastore {
 
     public List<String> get_tables(String db_name, String pattern) throws MetaException, org.apache.thrift.TException;
 
+    public List<String> get_tables_by_type(String db_name, String pattern, String tableType) 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;
@@ -394,6 +396,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_tables_by_type(String db_name, String pattern, String tableType, 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;
@@ -1392,6 +1396,34 @@ public class ThriftHiveMetastore {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_tables failed: unknown result");
     }
 
+    public List<String> get_tables_by_type(String db_name, String pattern, String tableType) throws MetaException, org.apache.thrift.TException
+    {
+      send_get_tables_by_type(db_name, pattern, tableType);
+      return recv_get_tables_by_type();
+    }
+
+    public void send_get_tables_by_type(String db_name, String pattern, String tableType) throws org.apache.thrift.TException
+    {
+      get_tables_by_type_args args = new get_tables_by_type_args();
+      args.setDb_name(db_name);
+      args.setPattern(pattern);
+      args.setTableType(tableType);
+      sendBase("get_tables_by_type", args);
+    }
+
+    public List<String> recv_get_tables_by_type() throws MetaException, org.apache.thrift.TException
+    {
+      get_tables_by_type_result result = new get_tables_by_type_result();
+      receiveBase(result, "get_tables_by_type");
+      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_tables_by_type 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);
@@ -5835,6 +5867,44 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void get_tables_by_type(String db_name, String pattern, String tableType, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_tables_by_type_call method_call = new get_tables_by_type_call(db_name, pattern, tableType, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class get_tables_by_type_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String db_name;
+      private String pattern;
+      private String tableType;
+      public get_tables_by_type_call(String db_name, String pattern, String tableType, 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_name = db_name;
+        this.pattern = pattern;
+        this.tableType = tableType;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_tables_by_type", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_tables_by_type_args args = new get_tables_by_type_args();
+        args.setDb_name(db_name);
+        args.setPattern(pattern);
+        args.setTableType(tableType);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public List<String> 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_tables_by_type();
+      }
+    }
+
     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);
@@ -10270,6 +10340,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_tables_by_type", new get_tables_by_type());
       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());
@@ -11063,6 +11134,30 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_tables_by_type<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_tables_by_type_args> {
+      public get_tables_by_type() {
+        super("get_tables_by_type");
+      }
+
+      public get_tables_by_type_args getEmptyArgsInstance() {
+        return new get_tables_by_type_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_tables_by_type_result getResult(I iface, get_tables_by_type_args args) throws org.apache.thrift.TException {
+        get_tables_by_type_result result = new get_tables_by_type_result();
+        try {
+          result.success = iface.get_tables_by_type(args.db_name, args.pattern, args.tableType);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        }
+        return result;
+      }
+    }
+
     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");
@@ -14241,6 +14336,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_tables_by_type", new get_tables_by_type());
       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());
@@ -15943,6 +16039,63 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_tables_by_type<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_tables_by_type_args, List<String>> {
+      public get_tables_by_type() {
+        super("get_tables_by_type");
+      }
+
+      public get_tables_by_type_args getEmptyArgsInstance() {
+        return new get_tables_by_type_args();
+      }
+
+      public AsyncMethodCallback<List<String>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<List<String>>() { 
+          public void onComplete(List<String> o) {
+            get_tables_by_type_result result = new get_tables_by_type_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_tables_by_type_result result = new get_tables_by_type_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_tables_by_type_args args, org.apache.thrift.async.AsyncMethodCallback<List<String>> resultHandler) throws TException {
+        iface.get_tables_by_type(args.db_name, args.pattern, args.tableType,resultHandler);
+      }
+    }
+
     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");
@@ -47851,12 +48004,1036 @@ 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_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 get_tables_resultStandardScheme extends StandardScheme<get_tables_result> {
+
+      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)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list716.size);
+                  String _elem717;
+                  for (int _i718 = 0; _i718 < _list716.size; ++_i718)
+                  {
+                    _elem717 = iprot.readString();
+                    struct.success.add(_elem717);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // O1
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.o1 = new MetaException();
+                struct.o1.read(iprot);
+                struct.setO1IsSet(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_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 _iter719 : struct.success)
+            {
+              oprot.writeString(_iter719);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.o1 != null) {
+          oprot.writeFieldBegin(O1_FIELD_DESC);
+          struct.o1.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class get_tables_resultTupleSchemeFactory implements SchemeFactory {
+      public get_tables_resultTupleScheme getScheme() {
+        return new get_tables_resultTupleScheme();
+      }
+    }
+
+    private static class get_tables_resultTupleScheme extends TupleScheme<get_tables_result> {
+
+      @Override
+      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.isSetSuccess()) {
+          optionals.set(0);
+        }
+        if (struct.isSetO1()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (String _iter720 : struct.success)
+            {
+              oprot.writeString(_iter720);
+            }
+          }
+        }
+        if (struct.isSetO1()) {
+          struct.o1.write(oprot);
+        }
+      }
+
+      @Override
+      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)) {
+          {
+            org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list721.size);
+            String _elem722;
+            for (int _i723 = 0; _i723 < _list721.size; ++_i723)
+            {
+              _elem722 = iprot.readString();
+              struct.success.add(_elem722);
+            }
+          }
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.o1 = new MetaException();
+          struct.o1.read(iprot);
+          struct.setO1IsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class get_tables_by_type_args implements org.apache.thrift.TBase<get_tables_by_type_args, get_tables_by_type_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_by_type_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_by_type_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 TABLE_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tableType", org.apache.thrift.protocol.TType.STRING, (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_by_type_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_tables_by_type_argsTupleSchemeFactory());
+    }
+
+    private String db_name; // required
+    private String pattern; // required
+    private String tableType; // 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"),
+      TABLE_TYPE((short)3, "tableType");
+
+      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;
+          case 3: // TABLE_TYPE
+            return TABLE_TYPE;
+          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)));
+      tmpMap.put(_Fields.TABLE_TYPE, new org.apache.thrift.meta_data.FieldMetaData("tableType", 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_by_type_args.class, metaDataMap);
+    }
+
+    public get_tables_by_type_args() {
+    }
+
+    public get_tables_by_type_args(
+      String db_name,
+      String pattern,
+      String tableType)
+    {
+      this();
+      this.db_name = db_name;
+      this.pattern = pattern;
+      this.tableType = tableType;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_tables_by_type_args(get_tables_by_type_args other) {
+      if (other.isSetDb_name()) {
+        this.db_name = other.db_name;
+      }
+      if (other.isSetPattern()) {
+        this.pattern = other.pattern;
+      }
+      if (other.isSetTableType()) {
+        this.tableType = other.tableType;
+      }
+    }
+
+    public get_tables_by_type_args deepCopy() {
+      return new get_tables_by_type_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.db_name = null;
+      this.pattern = null;
+      this.tableType = 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 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 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;
+
+      case TABLE_TYPE:
+        if (value == null) {
+          unsetTableType();
+        } else {
+          setTableType((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case DB_NAME:
+        return getDb_name();
+
+      case PATTERN:
+        return getPattern();
+
+      case TABLE_TYPE:
+        return getTableType();
+
+      }
+      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();
+      case TABLE_TYPE:
+        return isSetTableType();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_tables_by_type_args)
+        return this.equals((get_tables_by_type_args)that);
       return false;
     }
 
-    public boolean equals(get_tables_args that) {
+    public boolean equals(get_tables_by_type_args that) {
       if (that == null)
         return false;
 
@@ -47878,6 +49055,15 @@ public class ThriftHiveMetastore {
           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;
+      }
+
       return true;
     }
 
@@ -47895,11 +49081,16 @@ public class ThriftHiveMetastore {
       if (present_pattern)
         list.add(pattern);
 
+      boolean present_tableType = true && (isSetTableType());
+      list.add(present_tableType);
+      if (present_tableType)
+        list.add(tableType);
+
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(get_tables_args other) {
+    public int compareTo(get_tables_by_type_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -47926,6 +49117,16 @@ public class ThriftHiveMetastore {
           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;
+        }
+      }
       return 0;
     }
 
@@ -47943,7 +49144,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_tables_args(");
+      StringBuilder sb = new StringBuilder("get_tables_by_type_args(");
       boolean first = true;
 
       sb.append("db_name:");
@@ -47961,6 +49162,14 @@ public class ThriftHiveMetastore {
         sb.append(this.pattern);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("tableType:");
+      if (this.tableType == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tableType);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -47986,15 +49195,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_tables_by_type_argsStandardSchemeFactory implements SchemeFactory {
+      public get_tables_by_type_argsStandardScheme getScheme() {
+        return new get_tables_by_type_argsStandardScheme();
       }
     }
 
-    private static class get_tables_argsStandardScheme extends StandardScheme<get_tables_args> {
+    private static class get_tables_by_type_argsStandardScheme extends StandardScheme<get_tables_by_type_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_tables_by_type_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -48020,6 +49229,14 @@ public class ThriftHiveMetastore {
                 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;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -48029,7 +49246,7 @@ 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_tables_by_type_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -48043,22 +49260,27 @@ public class ThriftHiveMetastore {
           oprot.writeString(struct.pattern);
           oprot.writeFieldEnd();
         }
+        if (struct.tableType != null) {
+          oprot.writeFieldBegin(TABLE_TYPE_FIELD_DESC);
+          oprot.writeString(struct.tableType);
+          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_by_type_argsTupleSchemeFactory implements SchemeFactory {
+      public get_tables_by_type_argsTupleScheme getScheme() {
+        return new get_tables_by_type_argsTupleScheme();
       }
     }
 
-    private static class get_tables_argsTupleScheme extends TupleScheme<get_tables_args> {
+    private static class get_tables_by_type_argsTupleScheme extends TupleScheme<get_tables_by_type_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_tables_by_type_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetDb_name()) {
@@ -48067,19 +49289,25 @@ public class ThriftHiveMetastore {
         if (struct.isSetPattern()) {
           optionals.set(1);
         }
-        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTableType()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
         if (struct.isSetDb_name()) {
           oprot.writeString(struct.db_name);
         }
         if (struct.isSetPattern()) {
           oprot.writeString(struct.pattern);
         }
+        if (struct.isSetTableType()) {
+          oprot.writeString(struct.tableType);
+        }
       }
 
       @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_tables_by_type_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);
@@ -48088,21 +49316,25 @@ public class ThriftHiveMetastore {
           struct.pattern = iprot.readString();
           struct.setPatternIsSet(true);
         }
+        if (incoming.get(2)) {
+          struct.tableType = iprot.readString();
+          struct.setTableTypeIsSet(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_tables_by_type_result implements org.apache.thrift.TBase<get_tables_by_type_result, get_tables_by_type_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_by_type_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_by_type_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_tables_by_type_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_tables_by_type_resultTupleSchemeFactory());
     }
 
     private List<String> success; // required
@@ -48179,13 +49411,13 @@ public class ThriftHiveMetastore {
       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_tables_by_type_result.class, metaDataMap);
     }
 
-    public get_tables_result() {
+    public get_tables_by_type_result() {
     }
 
-    public get_tables_result(
+    public get_tables_by_type_result(
       List<String> success,
       MetaException o1)
     {
@@ -48197,7 +49429,7 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_tables_result(get_tables_result other) {
+    public get_tables_by_type_result(get_tables_by_type_result other) {
       if (other.isSetSuccess()) {
         List<String> __this__success = new ArrayList<String>(other.success);
         this.success = __this__success;
@@ -48207,8 +49439,8 @@ public class ThriftHiveMetastore {
       }
     }
 
-    public get_tables_result deepCopy() {
-      return new get_tables_result(this);
+    public get_tables_by_type_result deepCopy() {
+      return new get_tables_by_type_result(this);
     }
 
     @Override
@@ -48330,12 +49562,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_tables_by_type_result)
+        return this.equals((get_tables_by_type_result)that);
       return false;
     }
 
-    public boolean equals(get_tables_result that) {
+    public boolean equals(get_tables_by_type_result that) {
       if (that == null)
         return false;
 
@@ -48378,7 +49610,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(get_tables_result other) {
+    public int compareTo(get_tables_by_type_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -48422,7 +49654,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_tables_result(");
+      StringBuilder sb = new StringBuilder("get_tables_by_type_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -48465,15 +49697,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_tables_by_type_resultStandardSchemeFactory implements SchemeFactory {
+      public get_tables_by_type_resultStandardScheme getScheme() {
+        return new get_tables_by_type_resultStandardScheme();
       }
     }
 
-    private static class get_tables_resultStandardScheme extends StandardScheme<get_tables_result> {
+    private static class get_tables_by_type_resultStandardScheme extends StandardScheme<get_tables_by_type_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_tables_by_type_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -48486,13 +49718,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list716.size);
-                  String _elem717;
-                  for (int _i718 = 0; _i718 < _list716.size; ++_i718)
+                  org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list724.size);
+                  String _elem725;
+                  for (int _i726 = 0; _i726 < _list724.size; ++_i726)
                   {
-                    _elem717 = iprot.readString();
-                    struct.success.add(_elem717);
+                    _elem725 = iprot.readString();
+                    struct.success.add(_elem725);
                   }
                   iprot.readListEnd();
                 }
@@ -48519,7 +49751,7 @@ 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_tables_by_type_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -48527,9 +49759,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter719 : struct.success)
+            for (String _iter727 : struct.success)
             {
-              oprot.writeString(_iter719);
+              oprot.writeString(_iter727);
             }
             oprot.writeListEnd();
           }
@@ -48546,16 +49778,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_tables_by_type_resultTupleSchemeFactory implements SchemeFactory {
+      public get_tables_by_type_resultTupleScheme getScheme() {
+        return new get_tables_by_type_resultTupleScheme();
       }
     }
 
-    private static class get_tables_resultTupleScheme extends TupleScheme<get_tables_result> {
+    private static class get_tables_by_type_resultTupleScheme extends TupleScheme<get_tables_by_type_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_tables_by_type_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSuccess()) {
@@ -48568,9 +49800,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter720 : struct.success)
+            for (String _iter728 : struct.success)
             {
-              oprot.writeString(_iter720);
+              oprot.writeString(_iter728);
             }
           }
         }
@@ -48580,18 +49812,18 @@ 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_tables_by_type_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 _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list721.size);
-            String _elem722;
-            for (int _i723 = 0; _i723 < _list721.size; ++_i723)
+            org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list729.size);
+            String _elem730;
+            for (int _i731 = 0; _i731 < _list729.size; ++_i731)
             {
-              _elem722 = iprot.readString();
-              struct.success.add(_elem722);
+              _elem730 = iprot.readString();
+              struct.success.add(_elem730);
             }
           }
           struct.setSuccessIsSet(true);
@@ -49096,13 +50328,13 @@ public class ThriftHiveMetastore {
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list724.size);
-                  String _elem725;
-                  for (int _i726 = 0; _i726 < _list724.size; ++_i726)
+                  org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list732.size);
+                  String _elem733;
+                  for (int _i734 = 0; _i734 < _list732.size; ++_i734)
                   {
-                    _elem725 = iprot.readString();
-                    struct.tbl_types.add(_elem725);
+                    _elem733 = iprot.readString();
+                    struct.tbl_types.add(_elem733);
                   }
                   iprot.readListEnd();
                 }
@@ -49138,9 +50370,9 @@ public class ThriftHiveMetastore {
           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 _iter727 : struct.tbl_types)
+            for (String _iter735 : struct.tbl_types)
             {
-              oprot.writeString(_iter727);
+              oprot.writeString(_iter735);
             }
             oprot.writeListEnd();
           }
@@ -49183,9 +50415,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter728 : struct.tbl_types)
+            for (String _iter736 : struct.tbl_types)
             {
-              oprot.writeString(_iter728);
+              oprot.writeString(_iter736);
             }
           }
         }
@@ -49205,13 +50437,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list729.size);
-            String _elem730;
-            for (int _i731 = 0; _i731 < _list729.size; ++_i731)
+            org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list737.size);
+            String _elem738;
+            for (int _i739 = 0; _i739 < _list737.size; ++_i739)
             {
-              _elem730 = iprot.readString();
-              struct.tbl_types.add(_elem730);
+              _elem738 = iprot.readString();
+              struct.tbl_types.add(_elem738);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -49617,14 +50849,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
-                  struct.success = new ArrayList<TableMeta>(_list732.size);
-                  TableMeta _elem733;
-                  for (int _i734 = 0; _i734 < _list732.size; ++_i734)
+                  org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list740.size);
+                  TableMeta _elem741;
+                  for (int _i742 = 0; _i742 < _list740.size; ++_i742)
                   {
-                    _elem733 = new TableMeta();
-                    _elem733.read(iprot);
-                    struct.success.add(_elem733);
+                    _elem741 = new TableMeta();
+                    _elem741.read(iprot);
+                    struct.success.add(_elem741);
                   }
                   iprot.readListEnd();
                 }
@@ -49659,9 +50891,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TableMeta _iter735 : struct.success)
+            for (TableMeta _iter743 : struct.success)
             {
-              _iter735.write(oprot);
+              _iter743.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -49700,9 +50932,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter736 : struct.success)
+            for (TableMeta _iter744 : struct.success)
             {
-              _iter736.write(oprot);
+              _iter744.write(oprot);
             }
           }
         }
@@ -49717,14 +50949,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TableMeta>(_list737.size);
-            TableMeta _elem738;
-            for (int _i739 = 0; _i739 < _list737.size; ++_i739)
+            org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list745.size);
+            TableMeta _elem746;
+            for (int _i747 = 0; _i747 < _list745.size; ++_i747)
             {
-              _elem738 = new TableMeta();
-              _elem738.read(iprot);
-              struct.success.add(_elem738);
+              _elem746 = new TableMeta();
+              _elem746.read(iprot);
+              struct.success.add(_elem746);
             }
           }
           struct.setSuccessIsSet(true);
@@ -50490,13 +51722,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list740.size);
-                  String _elem741;
-                  for (int _i742 = 0; _i742 < _list740.size; ++_i742)
+                  org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list748.size);
+                  String _elem749;
+                  for (int _i750 = 0; _i750 < _list748.size; ++_i750)
                   {
-                    _elem741 = iprot.readString();
-                    struct.success.add(_elem741);
+                    _elem749 = iprot.readString();
+                    struct.success.add(_elem749);
                   }
                   iprot.readListEnd();
                 }
@@ -50531,9 +51763,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter743 : struct.success)
+            for (String _iter751 : struct.success)
             {
-              oprot.writeString(_iter743);
+              oprot.writeString(_iter751);
             }
             oprot.writeListEnd();
           }
@@ -50572,9 +51804,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter744 : struct.success)
+            for (String _iter752 : struct.success)
             {
-              oprot.writeString(_iter744);
+              oprot.writeString(_iter752);
             }
           }
         }
@@ -50589,13 +51821,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list745.size);
-            String _elem746;
-            for (int _i747 = 0; _i747 < _list745.size; ++_i747)
+            org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list753.size);
+            String _elem754;
+            for (int _i755 = 0; _i755 < _list753.size; ++_i755)
             {
-              _elem746 = iprot.readString();
-              struct.success.add(_elem746);
+              _elem754 = iprot.readString();
+              struct.success.add(_elem754);
             }
           }
           struct.setSuccessIsSet(true);
@@ -52048,13 +53280,13 @@ public class ThriftHiveMetastore {
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list748.size);
-                  String _elem749;
-                  for (int _i750 = 0; _i750 < _list748.size; ++_i750)
+                  org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list756.size);
+                  String _elem757;
+                  for (int _i758 = 0; _i758 < _list756.size; ++_i758)
                   {
-                    _elem749 = iprot.readString();
-                    struct.tbl_names.add(_elem749);
+                    _elem757 = iprot.readString();
+                    struct.tbl_names.add(_elem757);
                   }
                   iprot.readListEnd();
                 }
@@ -52085,9 +53317,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter751 : struct.tbl_names)
+            for (String _iter759 : struct.tbl_names)
             {
-              oprot.writeString(_iter751);
+              oprot.writeString(_iter759);
             }
             oprot.writeListEnd();
           }
@@ -52124,9 +53356,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter752 : struct.tbl_names)
+            for (String _iter760 : struct.tbl_names)
             {
-              oprot.writeString(_iter752);
+              oprot.writeString(_iter760);
             }
           }
         }
@@ -52142,13 +53374,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list753.size);
-            String _elem754;
-            for (int _i755 = 0; _i755 < _list753.size; ++_i755)
+            org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list761.size);
+            String _elem762;
+            for (int _i763 = 0; _i763 < _list761.size; ++_i763)
             {
-              _elem754 = iprot.readString();
-              struct.tbl_names.add(_elem754);
+              _elem762 = iprot.readString();
+              struct.tbl_names.add(_elem762);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -52716,14 +53948,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list756.size);
-                  Table _elem757;
-                  for (int _i758 = 0; _i758 < _list756.size; ++_i758)
+                  org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list764.size);
+                  Table _elem765;
+                  for (int _i766 = 0; _i766 < _list764.size; ++_i766)
                   {
-                    _elem757 = new Table();
-                    _elem757.read(iprot);
-                    struct.success.add(_elem757);
+                    _elem765 = new Table();
+                    _elem765.read(iprot);
+                    struct.success.add(_elem765);
                   }
                   iprot.readListEnd();
                 }
@@ -52776,9 +54008,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter759 : struct.success)
+            for (Table _iter767 : struct.success)
             {
-              _iter759.write(oprot);
+              _iter767.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -52833,9 +54065,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter760 : struct.success)
+            for (Table _iter768 : struct.success)
             {
-              _iter760.write(oprot);
+              _iter768.write(oprot);
             }
           }
         }
@@ -52856,14 +54088,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list761.size);
-            Table _elem762;
-            for (int _i763 = 0; _i763 < _list761.size; ++_i763)
+            org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list769.size);
+            Table _elem770;
+            for (int _i771 = 0; _i771 < _list769.size; ++_i771)
             {
-              _elem762 = new Table();
-              _elem762.read(iprot);
-              struct.success.add(_elem762);
+              _elem770 = new Table();
+              _elem770.read(iprot);
+              struct.success.add(_elem770);
             }
           }
           struct.setSuccessIsSet(true);
@@ -54009,13 +55241,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list764.size);
-                  String _elem765;
-                  for (int _i766 = 0; _i766 < _list764.size; ++_i766)
+                  org.apache.thrift.protocol.TList _list772 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list772.size);
+                  String _elem773;
+                  for (int _i774 = 0; _i774 < _list772.size; ++_i774)
                   {
-                    _elem765 = iprot.readString();
-                    struct.success.add(_elem765);
+                    _elem773 = iprot.readString();
+                    struct.success.add(_elem773);
                   }
                   iprot.readListEnd();
                 }
@@ -54068,9 +55300,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter767 : struct.success)
+            for (String _iter775 : struct.success)
             {
-              oprot.writeString(_iter767);
+              oprot.writeString(_iter775);
             }
             oprot.writeListEnd();
           }
@@ -54125,9 +55357,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter768 : struct.success)
+            for (String _iter776 : struct.success)
             {
-              oprot.writeString(_iter768);
+              oprot.writeString(_iter776);
             }
           }
         }
@@ -54148,13 +55380,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list769.size);
-            String _elem770;
-            for (int _i771 = 0; _i771 < _list769.size; ++_i771)
+            org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list777.size);
+            String _elem778;
+            for (int _i779 = 0; _i779 < _list777.size; ++_i779)
             {
-              _elem770 = iprot.readString();
-              struct.success.add(_elem770);
+              _elem778 = iprot.readString();
+              struct.success.add(_elem778);
             }
           }
           struct.setSuccessIsSet(true);
@@ -60013,14 +61245,14 @@ public class ThriftHiveMetastore {
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list772 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list772.size);
-                  Partition _elem773;
-                  for (int _i774 = 0; _i774 < _list772.size; ++_i774)
+                  org.apache.thrift.protocol.TList _list780 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list780.size);
+                  Partition _elem781;
+                  for (int _i782 = 0; _i782 < _list780.size; ++_i782)
                   {
-                    _elem773 = new Partition();
-                    _elem773.read(iprot);
-                    struct.new_parts.add(_elem773);
+                    _elem781 = new Partition();
+                    _elem781.read(iprot);
+                    struct.new_parts.add(_elem781);
                   }
                   iprot.readListEnd();
                 }
@@ -60046,9 +61278,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (Partition _iter775 : struct.new_parts)
+            for (Partition _iter783 : struct.new_parts)
             {
-              _iter775.write(oprot);
+              _iter783.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -60079,9 +61311,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter776 : struct.new_parts)
+            for (Partition _iter784 : struct.new_parts)
             {
-              _iter776.write(oprot);
+              _iter784.write(oprot);
             }
           }
         }
@@ -60093,14 +61325,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list777.size);
-            Partition _elem778;
-            for (int _i779 = 0; _i779 < _list777.size; ++_i779)
+            org.apache.thrift.protocol.TList _list785 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list785.size);
+            Partition _elem786;
+            for (int _i787 = 0; _i787 < _list785.size; ++_i787)
             {
-              _elem778 = new Partition();
-              _elem778.read(iprot);
-              struct.new_parts.add(_elem778);
+              _elem786 = new Partition();
+              _elem786.read(iprot);
+              struct.new_parts.add(_elem786);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -61101,14 +62333,14 @@ public class ThriftHiveMetastore {
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list780 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<PartitionSpec>(_list780.size);
-                  PartitionSpec _elem781;
-                  for (int _i782 = 0; _i782 < _list780.size; ++_i782)
+                  org.apache.thrift.protocol.TList _list788 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<PartitionSpec>(_list788.size);
+                  PartitionSpec _elem789;
+                  for (int _i790 = 0; _i790 < _list788.size; ++_i790)
                   {
-                    _elem781 = new PartitionSpec();
-                    _elem781.read(iprot);
-                    struct.new_parts.add(_elem781);
+                    _elem789 = new PartitionSpec();
+                    _elem789.read(iprot);
+                    struct.new_parts.add(_elem789);
                   }
                   iprot.readListEnd();
                 }
@@ -61134,9 +62366,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (PartitionSpec _iter783 : struct.new_parts)
+            for (PartitionSpec _iter791 : struct.new_parts)
             {
-              _iter783.write(oprot);
+              _iter791.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -61167,9 +62399,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (PartitionSpec _iter784 : struct.new_parts)
+            for (PartitionSpec _iter792 : struct.new_parts)
             {
-              _iter784.write(oprot);
+              _iter792.write(oprot);
             }
           }
         }
@@ -61181,14 +62413,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list785 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<PartitionSpec>(_list785.size);
-            PartitionSpec _elem786;
-            for (int _i787 = 0; _i787 < _list785.size; ++_i787)
+            org.apache.thrift.protocol.TList _list793 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<PartitionSpec>(_list793.size);
+            PartitionSpec _elem794;
+            for (int _i795 = 0; _i795 < _list793.size; ++_i795)
             {
-              _elem786 = new PartitionSpec();
-              _elem786.read(iprot);
-              struct.new_parts.add(_elem786);
+              _elem794 = new PartitionSpec();
+              _elem794.read(iprot);
+              struct.new_parts.add(_elem794);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -62364,13 +63596,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list788 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list788.size);
-                  String _elem789;
-                  for (int _i790 = 0; _i790 < _list788.size; ++_i790)
+                  org.apache.thrift.protocol.TList _list796 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list796.size);
+                  String _elem797;
+                  for (int _i798 = 0; _i798 < _list796.size; ++_i798)
                   {
-                    _elem789 = iprot.readString();
-                    struct.part_vals.add(_elem789);
+                    _elem797 = iprot.readString();
+                    struct.part_vals.add(_elem797);
                   }
                   iprot.readListEnd();
                 }
@@ -62406,9 +63638,9 @@ public class

<TRUNCATED>

[6/7] hive git commit: HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)

Posted by ai...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 5a35a50..b4a05b2 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -6186,6 +6186,277 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
 }
 
 
+ThriftHiveMetastore_get_tables_by_type_args::~ThriftHiveMetastore_get_tables_by_type_args() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_tables_by_type_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_name);
+          this->__isset.db_name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->pattern);
+          this->__isset.pattern = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->tableType);
+          this->__isset.tableType = 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_tables_by_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_args");
+
+  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->db_name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->pattern);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->tableType);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_tables_by_type_pargs::~ThriftHiveMetastore_get_tables_by_type_pargs() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_tables_by_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_pargs");
+
+  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->db_name)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString((*(this->pattern)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString((*(this->tableType)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_tables_by_type_result::~ThriftHiveMetastore_get_tables_by_type_result() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_tables_by_type_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 _size887;
+            ::apache::thrift::protocol::TType _etype890;
+            xfer += iprot->readListBegin(_etype890, _size887);
+            this->success.resize(_size887);
+            uint32_t _i891;
+            for (_i891 = 0; _i891 < _size887; ++_i891)
+            {
+              xfer += iprot->readString(this->success[_i891]);
+            }
+            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_tables_by_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_result");
+
+  if (this->__isset.success) {
+    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 _iter892;
+      for (_iter892 = this->success.begin(); _iter892 != this->success.end(); ++_iter892)
+      {
+        xfer += oprot->writeString((*_iter892));
+      }
+      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_tables_by_type_presult::~ThriftHiveMetastore_get_tables_by_type_presult() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_tables_by_type_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 _size893;
+            ::apache::thrift::protocol::TType _etype896;
+            xfer += iprot->readListBegin(_etype896, _size893);
+            (*(this->success)).resize(_size893);
+            uint32_t _i897;
+            for (_i897 = 0; _i897 < _size893; ++_i897)
+            {
+              xfer += iprot->readString((*(this->success))[_i897]);
+            }
+            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() {
 }
 
@@ -6231,14 +6502,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_types.clear();
-            uint32_t _size887;
-            ::apache::thrift::protocol::TType _etype890;
-            xfer += iprot->readListBegin(_etype890, _size887);
-            this->tbl_types.resize(_size887);
-            uint32_t _i891;
-            for (_i891 = 0; _i891 < _size887; ++_i891)
+            uint32_t _size898;
+            ::apache::thrift::protocol::TType _etype901;
+            xfer += iprot->readListBegin(_etype901, _size898);
+            this->tbl_types.resize(_size898);
+            uint32_t _i902;
+            for (_i902 = 0; _i902 < _size898; ++_i902)
             {
-              xfer += iprot->readString(this->tbl_types[_i891]);
+              xfer += iprot->readString(this->tbl_types[_i902]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6275,10 +6546,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protoc
   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 _iter892;
-    for (_iter892 = this->tbl_types.begin(); _iter892 != this->tbl_types.end(); ++_iter892)
+    std::vector<std::string> ::const_iterator _iter903;
+    for (_iter903 = this->tbl_types.begin(); _iter903 != this->tbl_types.end(); ++_iter903)
     {
-      xfer += oprot->writeString((*_iter892));
+      xfer += oprot->writeString((*_iter903));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6310,10 +6581,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::proto
   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 _iter893;
-    for (_iter893 = (*(this->tbl_types)).begin(); _iter893 != (*(this->tbl_types)).end(); ++_iter893)
+    std::vector<std::string> ::const_iterator _iter904;
+    for (_iter904 = (*(this->tbl_types)).begin(); _iter904 != (*(this->tbl_types)).end(); ++_iter904)
     {
-      xfer += oprot->writeString((*_iter893));
+      xfer += oprot->writeString((*_iter904));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6354,14 +6625,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size894;
-            ::apache::thrift::protocol::TType _etype897;
-            xfer += iprot->readListBegin(_etype897, _size894);
-            this->success.resize(_size894);
-            uint32_t _i898;
-            for (_i898 = 0; _i898 < _size894; ++_i898)
+            uint32_t _size905;
+            ::apache::thrift::protocol::TType _etype908;
+            xfer += iprot->readListBegin(_etype908, _size905);
+            this->success.resize(_size905);
+            uint32_t _i909;
+            for (_i909 = 0; _i909 < _size905; ++_i909)
             {
-              xfer += this->success[_i898].read(iprot);
+              xfer += this->success[_i909].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6400,10 +6671,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_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<TableMeta> ::const_iterator _iter899;
-      for (_iter899 = this->success.begin(); _iter899 != this->success.end(); ++_iter899)
+      std::vector<TableMeta> ::const_iterator _iter910;
+      for (_iter910 = this->success.begin(); _iter910 != this->success.end(); ++_iter910)
       {
-        xfer += (*_iter899).write(oprot);
+        xfer += (*_iter910).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6448,14 +6719,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size900;
-            ::apache::thrift::protocol::TType _etype903;
-            xfer += iprot->readListBegin(_etype903, _size900);
-            (*(this->success)).resize(_size900);
-            uint32_t _i904;
-            for (_i904 = 0; _i904 < _size900; ++_i904)
+            uint32_t _size911;
+            ::apache::thrift::protocol::TType _etype914;
+            xfer += iprot->readListBegin(_etype914, _size911);
+            (*(this->success)).resize(_size911);
+            uint32_t _i915;
+            for (_i915 = 0; _i915 < _size911; ++_i915)
             {
-              xfer += (*(this->success))[_i904].read(iprot);
+              xfer += (*(this->success))[_i915].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6593,14 +6864,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size905;
-            ::apache::thrift::protocol::TType _etype908;
-            xfer += iprot->readListBegin(_etype908, _size905);
-            this->success.resize(_size905);
-            uint32_t _i909;
-            for (_i909 = 0; _i909 < _size905; ++_i909)
+            uint32_t _size916;
+            ::apache::thrift::protocol::TType _etype919;
+            xfer += iprot->readListBegin(_etype919, _size916);
+            this->success.resize(_size916);
+            uint32_t _i920;
+            for (_i920 = 0; _i920 < _size916; ++_i920)
             {
-              xfer += iprot->readString(this->success[_i909]);
+              xfer += iprot->readString(this->success[_i920]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6639,10 +6910,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 _iter910;
-      for (_iter910 = this->success.begin(); _iter910 != this->success.end(); ++_iter910)
+      std::vector<std::string> ::const_iterator _iter921;
+      for (_iter921 = this->success.begin(); _iter921 != this->success.end(); ++_iter921)
       {
-        xfer += oprot->writeString((*_iter910));
+        xfer += oprot->writeString((*_iter921));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6687,14 +6958,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size911;
-            ::apache::thrift::protocol::TType _etype914;
-            xfer += iprot->readListBegin(_etype914, _size911);
-            (*(this->success)).resize(_size911);
-            uint32_t _i915;
-            for (_i915 = 0; _i915 < _size911; ++_i915)
+            uint32_t _size922;
+            ::apache::thrift::protocol::TType _etype925;
+            xfer += iprot->readListBegin(_etype925, _size922);
+            (*(this->success)).resize(_size922);
+            uint32_t _i926;
+            for (_i926 = 0; _i926 < _size922; ++_i926)
             {
-              xfer += iprot->readString((*(this->success))[_i915]);
+              xfer += iprot->readString((*(this->success))[_i926]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7004,14 +7275,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 _size916;
-            ::apache::thrift::protocol::TType _etype919;
-            xfer += iprot->readListBegin(_etype919, _size916);
-            this->tbl_names.resize(_size916);
-            uint32_t _i920;
-            for (_i920 = 0; _i920 < _size916; ++_i920)
+            uint32_t _size927;
+            ::apache::thrift::protocol::TType _etype930;
+            xfer += iprot->readListBegin(_etype930, _size927);
+            this->tbl_names.resize(_size927);
+            uint32_t _i931;
+            for (_i931 = 0; _i931 < _size927; ++_i931)
             {
-              xfer += iprot->readString(this->tbl_names[_i920]);
+              xfer += iprot->readString(this->tbl_names[_i931]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7044,10 +7315,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 _iter921;
-    for (_iter921 = this->tbl_names.begin(); _iter921 != this->tbl_names.end(); ++_iter921)
+    std::vector<std::string> ::const_iterator _iter932;
+    for (_iter932 = this->tbl_names.begin(); _iter932 != this->tbl_names.end(); ++_iter932)
     {
-      xfer += oprot->writeString((*_iter921));
+      xfer += oprot->writeString((*_iter932));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7075,10 +7346,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 _iter922;
-    for (_iter922 = (*(this->tbl_names)).begin(); _iter922 != (*(this->tbl_names)).end(); ++_iter922)
+    std::vector<std::string> ::const_iterator _iter933;
+    for (_iter933 = (*(this->tbl_names)).begin(); _iter933 != (*(this->tbl_names)).end(); ++_iter933)
     {
-      xfer += oprot->writeString((*_iter922));
+      xfer += oprot->writeString((*_iter933));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7119,14 +7390,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 _size923;
-            ::apache::thrift::protocol::TType _etype926;
-            xfer += iprot->readListBegin(_etype926, _size923);
-            this->success.resize(_size923);
-            uint32_t _i927;
-            for (_i927 = 0; _i927 < _size923; ++_i927)
+            uint32_t _size934;
+            ::apache::thrift::protocol::TType _etype937;
+            xfer += iprot->readListBegin(_etype937, _size934);
+            this->success.resize(_size934);
+            uint32_t _i938;
+            for (_i938 = 0; _i938 < _size934; ++_i938)
             {
-              xfer += this->success[_i927].read(iprot);
+              xfer += this->success[_i938].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7181,10 +7452,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 _iter928;
-      for (_iter928 = this->success.begin(); _iter928 != this->success.end(); ++_iter928)
+      std::vector<Table> ::const_iterator _iter939;
+      for (_iter939 = this->success.begin(); _iter939 != this->success.end(); ++_iter939)
       {
-        xfer += (*_iter928).write(oprot);
+        xfer += (*_iter939).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -7237,14 +7508,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 _size929;
-            ::apache::thrift::protocol::TType _etype932;
-            xfer += iprot->readListBegin(_etype932, _size929);
-            (*(this->success)).resize(_size929);
-            uint32_t _i933;
-            for (_i933 = 0; _i933 < _size929; ++_i933)
+            uint32_t _size940;
+            ::apache::thrift::protocol::TType _etype943;
+            xfer += iprot->readListBegin(_etype943, _size940);
+            (*(this->success)).resize(_size940);
+            uint32_t _i944;
+            for (_i944 = 0; _i944 < _size940; ++_i944)
             {
-              xfer += (*(this->success))[_i933].read(iprot);
+              xfer += (*(this->success))[_i944].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7430,14 +7701,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 _size934;
-            ::apache::thrift::protocol::TType _etype937;
-            xfer += iprot->readListBegin(_etype937, _size934);
-            this->success.resize(_size934);
-            uint32_t _i938;
-            for (_i938 = 0; _i938 < _size934; ++_i938)
+            uint32_t _size945;
+            ::apache::thrift::protocol::TType _etype948;
+            xfer += iprot->readListBegin(_etype948, _size945);
+            this->success.resize(_size945);
+            uint32_t _i949;
+            for (_i949 = 0; _i949 < _size945; ++_i949)
             {
-              xfer += iprot->readString(this->success[_i938]);
+              xfer += iprot->readString(this->success[_i949]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7492,10 +7763,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 _iter939;
-      for (_iter939 = this->success.begin(); _iter939 != this->success.end(); ++_iter939)
+      std::vector<std::string> ::const_iterator _iter950;
+      for (_iter950 = this->success.begin(); _iter950 != this->success.end(); ++_iter950)
       {
-        xfer += oprot->writeString((*_iter939));
+        xfer += oprot->writeString((*_iter950));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7548,14 +7819,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 _size940;
-            ::apache::thrift::protocol::TType _etype943;
-            xfer += iprot->readListBegin(_etype943, _size940);
-            (*(this->success)).resize(_size940);
-            uint32_t _i944;
-            for (_i944 = 0; _i944 < _size940; ++_i944)
+            uint32_t _size951;
+            ::apache::thrift::protocol::TType _etype954;
+            xfer += iprot->readListBegin(_etype954, _size951);
+            (*(this->success)).resize(_size951);
+            uint32_t _i955;
+            for (_i955 = 0; _i955 < _size951; ++_i955)
             {
-              xfer += iprot->readString((*(this->success))[_i944]);
+              xfer += iprot->readString((*(this->success))[_i955]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8889,14 +9160,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size945;
-            ::apache::thrift::protocol::TType _etype948;
-            xfer += iprot->readListBegin(_etype948, _size945);
-            this->new_parts.resize(_size945);
-            uint32_t _i949;
-            for (_i949 = 0; _i949 < _size945; ++_i949)
+            uint32_t _size956;
+            ::apache::thrift::protocol::TType _etype959;
+            xfer += iprot->readListBegin(_etype959, _size956);
+            this->new_parts.resize(_size956);
+            uint32_t _i960;
+            for (_i960 = 0; _i960 < _size956; ++_i960)
             {
-              xfer += this->new_parts[_i949].read(iprot);
+              xfer += this->new_parts[_i960].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8925,10 +9196,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 _iter950;
-    for (_iter950 = this->new_parts.begin(); _iter950 != this->new_parts.end(); ++_iter950)
+    std::vector<Partition> ::const_iterator _iter961;
+    for (_iter961 = this->new_parts.begin(); _iter961 != this->new_parts.end(); ++_iter961)
     {
-      xfer += (*_iter950).write(oprot);
+      xfer += (*_iter961).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8952,10 +9223,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 _iter951;
-    for (_iter951 = (*(this->new_parts)).begin(); _iter951 != (*(this->new_parts)).end(); ++_iter951)
+    std::vector<Partition> ::const_iterator _iter962;
+    for (_iter962 = (*(this->new_parts)).begin(); _iter962 != (*(this->new_parts)).end(); ++_iter962)
     {
-      xfer += (*_iter951).write(oprot);
+      xfer += (*_iter962).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9164,14 +9435,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 _size952;
-            ::apache::thrift::protocol::TType _etype955;
-            xfer += iprot->readListBegin(_etype955, _size952);
-            this->new_parts.resize(_size952);
-            uint32_t _i956;
-            for (_i956 = 0; _i956 < _size952; ++_i956)
+            uint32_t _size963;
+            ::apache::thrift::protocol::TType _etype966;
+            xfer += iprot->readListBegin(_etype966, _size963);
+            this->new_parts.resize(_size963);
+            uint32_t _i967;
+            for (_i967 = 0; _i967 < _size963; ++_i967)
             {
-              xfer += this->new_parts[_i956].read(iprot);
+              xfer += this->new_parts[_i967].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9200,10 +9471,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 _iter957;
-    for (_iter957 = this->new_parts.begin(); _iter957 != this->new_parts.end(); ++_iter957)
+    std::vector<PartitionSpec> ::const_iterator _iter968;
+    for (_iter968 = this->new_parts.begin(); _iter968 != this->new_parts.end(); ++_iter968)
     {
-      xfer += (*_iter957).write(oprot);
+      xfer += (*_iter968).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9227,10 +9498,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 _iter958;
-    for (_iter958 = (*(this->new_parts)).begin(); _iter958 != (*(this->new_parts)).end(); ++_iter958)
+    std::vector<PartitionSpec> ::const_iterator _iter969;
+    for (_iter969 = (*(this->new_parts)).begin(); _iter969 != (*(this->new_parts)).end(); ++_iter969)
     {
-      xfer += (*_iter958).write(oprot);
+      xfer += (*_iter969).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9455,14 +9726,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size959;
-            ::apache::thrift::protocol::TType _etype962;
-            xfer += iprot->readListBegin(_etype962, _size959);
-            this->part_vals.resize(_size959);
-            uint32_t _i963;
-            for (_i963 = 0; _i963 < _size959; ++_i963)
+            uint32_t _size970;
+            ::apache::thrift::protocol::TType _etype973;
+            xfer += iprot->readListBegin(_etype973, _size970);
+            this->part_vals.resize(_size970);
+            uint32_t _i974;
+            for (_i974 = 0; _i974 < _size970; ++_i974)
             {
-              xfer += iprot->readString(this->part_vals[_i963]);
+              xfer += iprot->readString(this->part_vals[_i974]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9499,10 +9770,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 _iter964;
-    for (_iter964 = this->part_vals.begin(); _iter964 != this->part_vals.end(); ++_iter964)
+    std::vector<std::string> ::const_iterator _iter975;
+    for (_iter975 = this->part_vals.begin(); _iter975 != this->part_vals.end(); ++_iter975)
     {
-      xfer += oprot->writeString((*_iter964));
+      xfer += oprot->writeString((*_iter975));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9534,10 +9805,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 _iter965;
-    for (_iter965 = (*(this->part_vals)).begin(); _iter965 != (*(this->part_vals)).end(); ++_iter965)
+    std::vector<std::string> ::const_iterator _iter976;
+    for (_iter976 = (*(this->part_vals)).begin(); _iter976 != (*(this->part_vals)).end(); ++_iter976)
     {
-      xfer += oprot->writeString((*_iter965));
+      xfer += oprot->writeString((*_iter976));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10009,14 +10280,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size966;
-            ::apache::thrift::protocol::TType _etype969;
-            xfer += iprot->readListBegin(_etype969, _size966);
-            this->part_vals.resize(_size966);
-            uint32_t _i970;
-            for (_i970 = 0; _i970 < _size966; ++_i970)
+            uint32_t _size977;
+            ::apache::thrift::protocol::TType _etype980;
+            xfer += iprot->readListBegin(_etype980, _size977);
+            this->part_vals.resize(_size977);
+            uint32_t _i981;
+            for (_i981 = 0; _i981 < _size977; ++_i981)
             {
-              xfer += iprot->readString(this->part_vals[_i970]);
+              xfer += iprot->readString(this->part_vals[_i981]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10061,10 +10332,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 _iter971;
-    for (_iter971 = this->part_vals.begin(); _iter971 != this->part_vals.end(); ++_iter971)
+    std::vector<std::string> ::const_iterator _iter982;
+    for (_iter982 = this->part_vals.begin(); _iter982 != this->part_vals.end(); ++_iter982)
     {
-      xfer += oprot->writeString((*_iter971));
+      xfer += oprot->writeString((*_iter982));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10100,10 +10371,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 _iter972;
-    for (_iter972 = (*(this->part_vals)).begin(); _iter972 != (*(this->part_vals)).end(); ++_iter972)
+    std::vector<std::string> ::const_iterator _iter983;
+    for (_iter983 = (*(this->part_vals)).begin(); _iter983 != (*(this->part_vals)).end(); ++_iter983)
     {
-      xfer += oprot->writeString((*_iter972));
+      xfer += oprot->writeString((*_iter983));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10906,14 +11177,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size973;
-            ::apache::thrift::protocol::TType _etype976;
-            xfer += iprot->readListBegin(_etype976, _size973);
-            this->part_vals.resize(_size973);
-            uint32_t _i977;
-            for (_i977 = 0; _i977 < _size973; ++_i977)
+            uint32_t _size984;
+            ::apache::thrift::protocol::TType _etype987;
+            xfer += iprot->readListBegin(_etype987, _size984);
+            this->part_vals.resize(_size984);
+            uint32_t _i988;
+            for (_i988 = 0; _i988 < _size984; ++_i988)
             {
-              xfer += iprot->readString(this->part_vals[_i977]);
+              xfer += iprot->readString(this->part_vals[_i988]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10958,10 +11229,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 _iter978;
-    for (_iter978 = this->part_vals.begin(); _iter978 != this->part_vals.end(); ++_iter978)
+    std::vector<std::string> ::const_iterator _iter989;
+    for (_iter989 = this->part_vals.begin(); _iter989 != this->part_vals.end(); ++_iter989)
     {
-      xfer += oprot->writeString((*_iter978));
+      xfer += oprot->writeString((*_iter989));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10997,10 +11268,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 _iter979;
-    for (_iter979 = (*(this->part_vals)).begin(); _iter979 != (*(this->part_vals)).end(); ++_iter979)
+    std::vector<std::string> ::const_iterator _iter990;
+    for (_iter990 = (*(this->part_vals)).begin(); _iter990 != (*(this->part_vals)).end(); ++_iter990)
     {
-      xfer += oprot->writeString((*_iter979));
+      xfer += oprot->writeString((*_iter990));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11209,14 +11480,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size980;
-            ::apache::thrift::protocol::TType _etype983;
-            xfer += iprot->readListBegin(_etype983, _size980);
-            this->part_vals.resize(_size980);
-            uint32_t _i984;
-            for (_i984 = 0; _i984 < _size980; ++_i984)
+            uint32_t _size991;
+            ::apache::thrift::protocol::TType _etype994;
+            xfer += iprot->readListBegin(_etype994, _size991);
+            this->part_vals.resize(_size991);
+            uint32_t _i995;
+            for (_i995 = 0; _i995 < _size991; ++_i995)
             {
-              xfer += iprot->readString(this->part_vals[_i984]);
+              xfer += iprot->readString(this->part_vals[_i995]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11269,10 +11540,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 _iter985;
-    for (_iter985 = this->part_vals.begin(); _iter985 != this->part_vals.end(); ++_iter985)
+    std::vector<std::string> ::const_iterator _iter996;
+    for (_iter996 = this->part_vals.begin(); _iter996 != this->part_vals.end(); ++_iter996)
     {
-      xfer += oprot->writeString((*_iter985));
+      xfer += oprot->writeString((*_iter996));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11312,10 +11583,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 _iter986;
-    for (_iter986 = (*(this->part_vals)).begin(); _iter986 != (*(this->part_vals)).end(); ++_iter986)
+    std::vector<std::string> ::const_iterator _iter997;
+    for (_iter997 = (*(this->part_vals)).begin(); _iter997 != (*(this->part_vals)).end(); ++_iter997)
     {
-      xfer += oprot->writeString((*_iter986));
+      xfer += oprot->writeString((*_iter997));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12321,14 +12592,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size987;
-            ::apache::thrift::protocol::TType _etype990;
-            xfer += iprot->readListBegin(_etype990, _size987);
-            this->part_vals.resize(_size987);
-            uint32_t _i991;
-            for (_i991 = 0; _i991 < _size987; ++_i991)
+            uint32_t _size998;
+            ::apache::thrift::protocol::TType _etype1001;
+            xfer += iprot->readListBegin(_etype1001, _size998);
+            this->part_vals.resize(_size998);
+            uint32_t _i1002;
+            for (_i1002 = 0; _i1002 < _size998; ++_i1002)
             {
-              xfer += iprot->readString(this->part_vals[_i991]);
+              xfer += iprot->readString(this->part_vals[_i1002]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12365,10 +12636,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 _iter992;
-    for (_iter992 = this->part_vals.begin(); _iter992 != this->part_vals.end(); ++_iter992)
+    std::vector<std::string> ::const_iterator _iter1003;
+    for (_iter1003 = this->part_vals.begin(); _iter1003 != this->part_vals.end(); ++_iter1003)
     {
-      xfer += oprot->writeString((*_iter992));
+      xfer += oprot->writeString((*_iter1003));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12400,10 +12671,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 _iter993;
-    for (_iter993 = (*(this->part_vals)).begin(); _iter993 != (*(this->part_vals)).end(); ++_iter993)
+    std::vector<std::string> ::const_iterator _iter1004;
+    for (_iter1004 = (*(this->part_vals)).begin(); _iter1004 != (*(this->part_vals)).end(); ++_iter1004)
     {
-      xfer += oprot->writeString((*_iter993));
+      xfer += oprot->writeString((*_iter1004));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12592,17 +12863,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size994;
-            ::apache::thrift::protocol::TType _ktype995;
-            ::apache::thrift::protocol::TType _vtype996;
-            xfer += iprot->readMapBegin(_ktype995, _vtype996, _size994);
-            uint32_t _i998;
-            for (_i998 = 0; _i998 < _size994; ++_i998)
+            uint32_t _size1005;
+            ::apache::thrift::protocol::TType _ktype1006;
+            ::apache::thrift::protocol::TType _vtype1007;
+            xfer += iprot->readMapBegin(_ktype1006, _vtype1007, _size1005);
+            uint32_t _i1009;
+            for (_i1009 = 0; _i1009 < _size1005; ++_i1009)
             {
-              std::string _key999;
-              xfer += iprot->readString(_key999);
-              std::string& _val1000 = this->partitionSpecs[_key999];
-              xfer += iprot->readString(_val1000);
+              std::string _key1010;
+              xfer += iprot->readString(_key1010);
+              std::string& _val1011 = this->partitionSpecs[_key1010];
+              xfer += iprot->readString(_val1011);
             }
             xfer += iprot->readMapEnd();
           }
@@ -12663,11 +12934,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 _iter1001;
-    for (_iter1001 = this->partitionSpecs.begin(); _iter1001 != this->partitionSpecs.end(); ++_iter1001)
+    std::map<std::string, std::string> ::const_iterator _iter1012;
+    for (_iter1012 = this->partitionSpecs.begin(); _iter1012 != this->partitionSpecs.end(); ++_iter1012)
     {
-      xfer += oprot->writeString(_iter1001->first);
-      xfer += oprot->writeString(_iter1001->second);
+      xfer += oprot->writeString(_iter1012->first);
+      xfer += oprot->writeString(_iter1012->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12707,11 +12978,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 _iter1002;
-    for (_iter1002 = (*(this->partitionSpecs)).begin(); _iter1002 != (*(this->partitionSpecs)).end(); ++_iter1002)
+    std::map<std::string, std::string> ::const_iterator _iter1013;
+    for (_iter1013 = (*(this->partitionSpecs)).begin(); _iter1013 != (*(this->partitionSpecs)).end(); ++_iter1013)
     {
-      xfer += oprot->writeString(_iter1002->first);
-      xfer += oprot->writeString(_iter1002->second);
+      xfer += oprot->writeString(_iter1013->first);
+      xfer += oprot->writeString(_iter1013->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12956,17 +13227,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size1003;
-            ::apache::thrift::protocol::TType _ktype1004;
-            ::apache::thrift::protocol::TType _vtype1005;
-            xfer += iprot->readMapBegin(_ktype1004, _vtype1005, _size1003);
-            uint32_t _i1007;
-            for (_i1007 = 0; _i1007 < _size1003; ++_i1007)
+            uint32_t _size1014;
+            ::apache::thrift::protocol::TType _ktype1015;
+            ::apache::thrift::protocol::TType _vtype1016;
+            xfer += iprot->readMapBegin(_ktype1015, _vtype1016, _size1014);
+            uint32_t _i1018;
+            for (_i1018 = 0; _i1018 < _size1014; ++_i1018)
             {
-              std::string _key1008;
-              xfer += iprot->readString(_key1008);
-              std::string& _val1009 = this->partitionSpecs[_key1008];
-              xfer += iprot->readString(_val1009);
+              std::string _key1019;
+              xfer += iprot->readString(_key1019);
+              std::string& _val1020 = this->partitionSpecs[_key1019];
+              xfer += iprot->readString(_val1020);
             }
             xfer += iprot->readMapEnd();
           }
@@ -13027,11 +13298,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 _iter1010;
-    for (_iter1010 = this->partitionSpecs.begin(); _iter1010 != this->partitionSpecs.end(); ++_iter1010)
+    std::map<std::string, std::string> ::const_iterator _iter1021;
+    for (_iter1021 = this->partitionSpecs.begin(); _iter1021 != this->partitionSpecs.end(); ++_iter1021)
     {
-      xfer += oprot->writeString(_iter1010->first);
-      xfer += oprot->writeString(_iter1010->second);
+      xfer += oprot->writeString(_iter1021->first);
+      xfer += oprot->writeString(_iter1021->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13071,11 +13342,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 _iter1011;
-    for (_iter1011 = (*(this->partitionSpecs)).begin(); _iter1011 != (*(this->partitionSpecs)).end(); ++_iter1011)
+    std::map<std::string, std::string> ::const_iterator _iter1022;
+    for (_iter1022 = (*(this->partitionSpecs)).begin(); _iter1022 != (*(this->partitionSpecs)).end(); ++_iter1022)
     {
-      xfer += oprot->writeString(_iter1011->first);
-      xfer += oprot->writeString(_iter1011->second);
+      xfer += oprot->writeString(_iter1022->first);
+      xfer += oprot->writeString(_iter1022->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13132,14 +13403,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1012;
-            ::apache::thrift::protocol::TType _etype1015;
-            xfer += iprot->readListBegin(_etype1015, _size1012);
-            this->success.resize(_size1012);
-            uint32_t _i1016;
-            for (_i1016 = 0; _i1016 < _size1012; ++_i1016)
+            uint32_t _size1023;
+            ::apache::thrift::protocol::TType _etype1026;
+            xfer += iprot->readListBegin(_etype1026, _size1023);
+            this->success.resize(_size1023);
+            uint32_t _i1027;
+            for (_i1027 = 0; _i1027 < _size1023; ++_i1027)
             {
-              xfer += this->success[_i1016].read(iprot);
+              xfer += this->success[_i1027].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13202,10 +13473,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 _iter1017;
-      for (_iter1017 = this->success.begin(); _iter1017 != this->success.end(); ++_iter1017)
+      std::vector<Partition> ::const_iterator _iter1028;
+      for (_iter1028 = this->success.begin(); _iter1028 != this->success.end(); ++_iter1028)
       {
-        xfer += (*_iter1017).write(oprot);
+        xfer += (*_iter1028).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13262,14 +13533,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1018;
-            ::apache::thrift::protocol::TType _etype1021;
-            xfer += iprot->readListBegin(_etype1021, _size1018);
-            (*(this->success)).resize(_size1018);
-            uint32_t _i1022;
-            for (_i1022 = 0; _i1022 < _size1018; ++_i1022)
+            uint32_t _size1029;
+            ::apache::thrift::protocol::TType _etype1032;
+            xfer += iprot->readListBegin(_etype1032, _size1029);
+            (*(this->success)).resize(_size1029);
+            uint32_t _i1033;
+            for (_i1033 = 0; _i1033 < _size1029; ++_i1033)
             {
-              xfer += (*(this->success))[_i1022].read(iprot);
+              xfer += (*(this->success))[_i1033].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13368,14 +13639,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 _size1023;
-            ::apache::thrift::protocol::TType _etype1026;
-            xfer += iprot->readListBegin(_etype1026, _size1023);
-            this->part_vals.resize(_size1023);
-            uint32_t _i1027;
-            for (_i1027 = 0; _i1027 < _size1023; ++_i1027)
+            uint32_t _size1034;
+            ::apache::thrift::protocol::TType _etype1037;
+            xfer += iprot->readListBegin(_etype1037, _size1034);
+            this->part_vals.resize(_size1034);
+            uint32_t _i1038;
+            for (_i1038 = 0; _i1038 < _size1034; ++_i1038)
             {
-              xfer += iprot->readString(this->part_vals[_i1027]);
+              xfer += iprot->readString(this->part_vals[_i1038]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13396,14 +13667,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 _size1028;
-            ::apache::thrift::protocol::TType _etype1031;
-            xfer += iprot->readListBegin(_etype1031, _size1028);
-            this->group_names.resize(_size1028);
-            uint32_t _i1032;
-            for (_i1032 = 0; _i1032 < _size1028; ++_i1032)
+            uint32_t _size1039;
+            ::apache::thrift::protocol::TType _etype1042;
+            xfer += iprot->readListBegin(_etype1042, _size1039);
+            this->group_names.resize(_size1039);
+            uint32_t _i1043;
+            for (_i1043 = 0; _i1043 < _size1039; ++_i1043)
             {
-              xfer += iprot->readString(this->group_names[_i1032]);
+              xfer += iprot->readString(this->group_names[_i1043]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13440,10 +13711,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 _iter1033;
-    for (_iter1033 = this->part_vals.begin(); _iter1033 != this->part_vals.end(); ++_iter1033)
+    std::vector<std::string> ::const_iterator _iter1044;
+    for (_iter1044 = this->part_vals.begin(); _iter1044 != this->part_vals.end(); ++_iter1044)
     {
-      xfer += oprot->writeString((*_iter1033));
+      xfer += oprot->writeString((*_iter1044));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13456,10 +13727,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 _iter1034;
-    for (_iter1034 = this->group_names.begin(); _iter1034 != this->group_names.end(); ++_iter1034)
+    std::vector<std::string> ::const_iterator _iter1045;
+    for (_iter1045 = this->group_names.begin(); _iter1045 != this->group_names.end(); ++_iter1045)
     {
-      xfer += oprot->writeString((*_iter1034));
+      xfer += oprot->writeString((*_iter1045));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13491,10 +13762,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 _iter1035;
-    for (_iter1035 = (*(this->part_vals)).begin(); _iter1035 != (*(this->part_vals)).end(); ++_iter1035)
+    std::vector<std::string> ::const_iterator _iter1046;
+    for (_iter1046 = (*(this->part_vals)).begin(); _iter1046 != (*(this->part_vals)).end(); ++_iter1046)
     {
-      xfer += oprot->writeString((*_iter1035));
+      xfer += oprot->writeString((*_iter1046));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13507,10 +13778,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 _iter1036;
-    for (_iter1036 = (*(this->group_names)).begin(); _iter1036 != (*(this->group_names)).end(); ++_iter1036)
+    std::vector<std::string> ::const_iterator _iter1047;
+    for (_iter1047 = (*(this->group_names)).begin(); _iter1047 != (*(this->group_names)).end(); ++_iter1047)
     {
-      xfer += oprot->writeString((*_iter1036));
+      xfer += oprot->writeString((*_iter1047));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14069,14 +14340,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1037;
-            ::apache::thrift::protocol::TType _etype1040;
-            xfer += iprot->readListBegin(_etype1040, _size1037);
-            this->success.resize(_size1037);
-            uint32_t _i1041;
-            for (_i1041 = 0; _i1041 < _size1037; ++_i1041)
+            uint32_t _size1048;
+            ::apache::thrift::protocol::TType _etype1051;
+            xfer += iprot->readListBegin(_etype1051, _size1048);
+            this->success.resize(_size1048);
+            uint32_t _i1052;
+            for (_i1052 = 0; _i1052 < _size1048; ++_i1052)
             {
-              xfer += this->success[_i1041].read(iprot);
+              xfer += this->success[_i1052].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14123,10 +14394,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 _iter1042;
-      for (_iter1042 = this->success.begin(); _iter1042 != this->success.end(); ++_iter1042)
+      std::vector<Partition> ::const_iterator _iter1053;
+      for (_iter1053 = this->success.begin(); _iter1053 != this->success.end(); ++_iter1053)
       {
-        xfer += (*_iter1042).write(oprot);
+        xfer += (*_iter1053).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14175,14 +14446,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1043;
-            ::apache::thrift::protocol::TType _etype1046;
-            xfer += iprot->readListBegin(_etype1046, _size1043);
-            (*(this->success)).resize(_size1043);
-            uint32_t _i1047;
-            for (_i1047 = 0; _i1047 < _size1043; ++_i1047)
+            uint32_t _size1054;
+            ::apache::thrift::protocol::TType _etype1057;
+            xfer += iprot->readListBegin(_etype1057, _size1054);
+            (*(this->success)).resize(_size1054);
+            uint32_t _i1058;
+            for (_i1058 = 0; _i1058 < _size1054; ++_i1058)
             {
-              xfer += (*(this->success))[_i1047].read(iprot);
+              xfer += (*(this->success))[_i1058].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14281,14 +14552,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 _size1048;
-            ::apache::thrift::protocol::TType _etype1051;
-            xfer += iprot->readListBegin(_etype1051, _size1048);
-            this->group_names.resize(_size1048);
-            uint32_t _i1052;
-            for (_i1052 = 0; _i1052 < _size1048; ++_i1052)
+            uint32_t _size1059;
+            ::apache::thrift::protocol::TType _etype1062;
+            xfer += iprot->readListBegin(_etype1062, _size1059);
+            this->group_names.resize(_size1059);
+            uint32_t _i1063;
+            for (_i1063 = 0; _i1063 < _size1059; ++_i1063)
             {
-              xfer += iprot->readString(this->group_names[_i1052]);
+              xfer += iprot->readString(this->group_names[_i1063]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14333,10 +14604,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 _iter1053;
-    for (_iter1053 = this->group_names.begin(); _iter1053 != this->group_names.end(); ++_iter1053)
+    std::vector<std::string> ::const_iterator _iter1064;
+    for (_iter1064 = this->group_names.begin(); _iter1064 != this->group_names.end(); ++_iter1064)
     {
-      xfer += oprot->writeString((*_iter1053));
+      xfer += oprot->writeString((*_iter1064));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14376,10 +14647,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 _iter1054;
-    for (_iter1054 = (*(this->group_names)).begin(); _iter1054 != (*(this->group_names)).end(); ++_iter1054)
+    std::vector<std::string> ::const_iterator _iter1065;
+    for (_iter1065 = (*(this->group_names)).begin(); _iter1065 != (*(this->group_names)).end(); ++_iter1065)
     {
-      xfer += oprot->writeString((*_iter1054));
+      xfer += oprot->writeString((*_iter1065));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14420,14 +14691,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1055;
-            ::apache::thrift::protocol::TType _etype1058;
-            xfer += iprot->readListBegin(_etype1058, _size1055);
-            this->success.resize(_size1055);
-            uint32_t _i1059;
-            for (_i1059 = 0; _i1059 < _size1055; ++_i1059)
+            uint32_t _size1066;
+            ::apache::thrift::protocol::TType _etype1069;
+            xfer += iprot->readListBegin(_etype1069, _size1066);
+            this->success.resize(_size1066);
+            uint32_t _i1070;
+            for (_i1070 = 0; _i1070 < _size1066; ++_i1070)
             {
-              xfer += this->success[_i1059].read(iprot);
+              xfer += this->success[_i1070].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14474,10 +14745,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 _iter1060;
-      for (_iter1060 = this->success.begin(); _iter1060 != this->success.end(); ++_iter1060)
+      std::vector<Partition> ::const_iterator _iter1071;
+      for (_iter1071 = this->success.begin(); _iter1071 != this->success.end(); ++_iter1071)
       {
-        xfer += (*_iter1060).write(oprot);
+        xfer += (*_iter1071).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14526,14 +14797,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1061;
-            ::apache::thrift::protocol::TType _etype1064;
-            xfer += iprot->readListBegin(_etype1064, _size1061);
-            (*(this->success)).resize(_size1061);
-            uint32_t _i1065;
-            for (_i1065 = 0; _i1065 < _size1061; ++_i1065)
+            uint32_t _size1072;
+            ::apache::thrift::protocol::TType _etype1075;
+            xfer += iprot->readListBegin(_etype1075, _size1072);
+            (*(this->success)).resize(_size1072);
+            uint32_t _i1076;
+            for (_i1076 = 0; _i1076 < _size1072; ++_i1076)
             {
-              xfer += (*(this->success))[_i1065].read(iprot);
+              xfer += (*(this->success))[_i1076].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14711,14 +14982,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1066;
-            ::apache::thrift::protocol::TType _etype1069;
-            xfer += iprot->readListBegin(_etype1069, _size1066);
-            this->success.resize(_size1066);
-            uint32_t _i1070;
-            for (_i1070 = 0; _i1070 < _size1066; ++_i1070)
+            uint32_t _size1077;
+            ::apache::thrift::protocol::TType _etype1080;
+            xfer += iprot->readListBegin(_etype1080, _size1077);
+            this->success.resize(_size1077);
+            uint32_t _i1081;
+            for (_i1081 = 0; _i1081 < _size1077; ++_i1081)
             {
-              xfer += this->success[_i1070].read(iprot);
+              xfer += this->success[_i1081].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14765,10 +15036,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 _iter1071;
-      for (_iter1071 = this->success.begin(); _iter1071 != this->success.end(); ++_iter1071)
+      std::vector<PartitionSpec> ::const_iterator _iter1082;
+      for (_iter1082 = this->success.begin(); _iter1082 != this->success.end(); ++_iter1082)
       {
-        xfer += (*_iter1071).write(oprot);
+        xfer += (*_iter1082).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14817,14 +15088,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1072;
-            ::apache::thrift::protocol::TType _etype1075;
-            xfer += iprot->readListBegin(_etype1075, _size1072);
-            (*(this->success)).resize(_size1072);
-            uint32_t _i1076;
-            for (_i1076 = 0; _i1076 < _size1072; ++_i1076)
+            uint32_t _size1083;
+            ::apache::thrift::protocol::TType _etype1086;
+            xfer += iprot->readListBegin(_etype1086, _size1083);
+            (*(this->success)).resize(_size1083);
+            uint32_t _i1087;
+            for (_i1087 = 0; _i1087 < _size1083; ++_i1087)
             {
-              xfer += (*(this->success))[_i1076].read(iprot);
+              xfer += (*(this->success))[_i1087].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15002,14 +15273,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1077;
-            ::apache::thrift::protocol::TType _etype1080;
-            xfer += iprot->readListBegin(_etype1080, _size1077);
-            this->success.resize(_size1077);
-            uint32_t _i1081;
-            for (_i1081 = 0; _i1081 < _size1077; ++_i1081)
+            uint32_t _size1088;
+            ::apache::thrift::protocol::TType _etype1091;
+            xfer += iprot->readListBegin(_etype1091, _size1088);
+            this->success.resize(_size1088);
+            uint32_t _i1092;
+            for (_i1092 = 0; _i1092 < _size1088; ++_i1092)
             {
-              xfer += iprot->readString(this->success[_i1081]);
+              xfer += iprot->readString(this->success[_i1092]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15048,10 +15319,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 _iter1082;
-      for (_iter1082 = this->success.begin(); _iter1082 != this->success.end(); ++_iter1082)
+      std::vector<std::string> ::const_iterator _iter1093;
+      for (_iter1093 = this->success.begin(); _iter1093 != this->success.end(); ++_iter1093)
       {
-        xfer += oprot->writeString((*_iter1082));
+        xfer += oprot->writeString((*_iter1093));
       }
       xfer += oprot->writeListEnd();
     }
@@ -15096,14 +15367,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1083;
-            ::apache::thrift::protocol::TType _etype1086;
-            xfer += iprot->readListBegin(_etype1086, _size1083);
-            (*(this->success)).resize(_size1083);
-            uint32_t _i1087;
-            for (_i1087 = 0; _i1087 < _size1083; ++_i1087)
+            uint32_t _size1094;
+            ::apache::thrift::protocol::TType _etype1097;
+            xfer += iprot->readListBegin(_etype1097, _size1094);
+            (*(this->success)).resize(_size1094);
+            uint32_t _i1098;
+            for (_i1098 = 0; _i1098 < _size1094; ++_i1098)
             {
-              xfer += iprot->readString((*(this->success))[_i1087]);
+              xfer += iprot->readString((*(this->success))[_i1098]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15178,14 +15449,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1088;
-            ::apache::thrift::protocol::TType _etype1091;
-            xfer += iprot->readListBegin(_etype1091, _size1088);
-            this->part_vals.resize(_size1088);
-            uint32_t _i1092;
-            for (_i1092 = 0; _i1092 < _size1088; ++_i1092)
+            uint32_t _size1099;
+            ::apache::thrift::protocol::TType _etype1102;
+            xfer += iprot->readListBegin(_etype1102, _size1099);
+            this->part_vals.resize(_size1099);
+            uint32_t _i1103;
+            for (_i1103 = 0; _i1103 < _size1099; ++_i1103)
             {
-              xfer += iprot->readString(this->part_vals[_i1092]);
+              xfer += iprot->readString(this->part_vals[_i1103]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15230,10 +15501,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::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 _iter1093;
-    for (_iter1093 = this->part_vals.begin(); _iter1093 != this->part_vals.end(); ++_iter1093)
+    std::vector<std::string> ::const_iterator _iter1104;
+    for (_iter1104 = this->part_vals.begin(); _iter1104 != this->part_vals.end(); ++_iter1104)
     {
-      xfer += oprot->writeString((*_iter1093));
+      xfer += oprot->writeString((*_iter1104));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15269,10 +15540,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_pargs::write(::apache::thrift::pr
   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 _iter1094;
-    for (_iter1094 = (*(this->part_vals)).begin(); _iter1094 != (*(this->part_vals)).end(); ++_iter1094)
+    std::vector<std::string> ::const_iterator _iter1105;
+    for (_iter1105 = (*(this->part_vals)).begin(); _iter1105 != (*(this->part_vals)).end(); ++_iter1105)
     {
-      xfer += oprot->writeString((*_iter1094));
+      xfer += oprot->writeString((*_iter1105));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15317,14 +15588,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1095;
-            ::apache::thrift::protocol::TType _etype1098;
-            xfer += iprot->readListBegin(_etype1098, _size1095);
-            this->success.resize(_size1095);
-            uint32_t _i1099;
-            for (_i1099 = 0; _i1099 < _size1095; ++_i1099)
+            uint32_t _size1106;
+            ::apache::thrift::protocol::TType _etype1109;
+            xfer += iprot->readListBegin(_etype1109, _size1106);
+            this->success.resize(_size1106);
+            uint32_t _i1110;
+            for (_i1110 = 0; _i1110 < _size1106; ++_i1110)
             {
-              xfer += this->success[_i1099].read(iprot);
+              xfer += this->success[_i1110].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15371,10 +15642,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::write(::apache::thrift::p
     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 _iter1100;
-      for (_iter1100 = this->success.begin(); _iter1100 != this->success.end(); ++_iter1100)
+      std::vector<Partition> ::const_iterator _iter1111;
+      for (_iter1111 = this->success.begin(); _iter1111 != this->success.end(); ++_iter1111)
       {
-        xfer += (*_iter1100).write(oprot);
+        xfer += (*_iter1111).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -15423,14 +15694,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1101;
-            ::apache::thrift::protocol::TType _etype1104;
-            xfer += iprot->readListBegin(_etype1104, _size1101);
-            (*(this->success)).resize(_size1101);
-            uint32_t _i1105;
-            for (_i1105 = 0; _i1105 < _size1101; ++_i1105)
+            uint32_t _size1112;
+            ::apache::thrift::protocol::TType _etype1115;
+            xfer += iprot->readListBegin(_etype1115, _size1112);
+            (*(this->success)).resize(_size1112);
+            uint32_t _i1116;
+            for (_i1116 = 0; _i1116 < _size1112; ++_i1116)
             {
-              xfer += (*(this->success))[_i1105].read(iprot);
+              xfer += (*(this->success))[_i1116].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15513,14 +15784,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1106;
-            ::apache::thrift::protocol::TType _etype1109;
-            xfer += iprot->readListBegin(_etype1109, _size1106);
-            this->part_vals.resize(_size1106);
-            uint32_t _i1110;
-            for (_i1110 = 0; _i1110 < _size1106; ++_i1110)
+            uint32_t _size1117;
+            ::apache::thrift::protocol::TType _etype1120;
+            xfer += iprot->readListBegin(_etype1120, _size1117);
+            this->part_vals.resize(_size1117);
+            uint32_t _i1121;
+            for (_i1121 = 0; _i1121 < _size1117; ++_i1121)
             {
-              xfer += iprot->readString(this->part_vals[_i1110]);
+              xfer += iprot->readString(this->part_vals[_i1121]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15549,14 +15820,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size1111;
-            ::apache::thrift::protocol::TType _etype1114;
-            xfer += iprot->readListBegin(_etype1114, _size1111);
-            this->group_names.resize(_size1111);
-            uint32_t _i1115;
-            for (_i1115 = 0; _i1115 < _size1111; ++_i1115)
+            uint32_t _size1122;
+            ::apache::thrift::protocol::TType _etype1125;
+            xfer += iprot->readListBegin(_etype1125, _size1122);
+            this->group_names.resize(_size1122);
+            uint32_t _i1126;
+            for (_i1126 = 0; _i1126 < _size1122; ++_i1126)
             {
-              xfer += iprot->readString(this->group_names[_i1115]);
+              xfer += iprot->readString(this->group_names[_i1126]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15593,10 +15864,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t
   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 _iter1116;
-    for (_iter1116 = this->part_vals.begin(); _iter1116 != this->part_vals.end(); ++_iter1116)
+    std::vector<std::string> ::const_iterator _iter1127;
+    for (_iter1127 = this->part_vals.begin(); _iter1127 != this->part_vals.end(); ++_iter1127)
     {
-      xfer += oprot->writeString((*_iter1116));
+      xfer += oprot->writeString((*_iter1127));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15613,10 +15884,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter1117;
-    for (_iter1117 = this->group_names.begin(); _iter1117 != this->group_names.end(); ++_iter1117)
+    std::vector<std::string> ::const_iterator _iter1128;
+    for (_iter1128 = this->group_names.begin(); _iter1128 != this->group_names.end(); ++_iter1128)
     {
-      xfer += oprot->writeString((*_iter1117));
+      xfer += oprot->writeString((*_iter1128));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15648,10 +15919,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache::
   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 _iter1118;
-    for (_iter1118 = (*(this->part_vals)).begin(); _iter1118 != (*(this->part_vals)).end(); ++_iter1118)
+    std::vector<std::string> ::const_iterator _iter1129;
+    for (_iter1129 = (*(this->part_vals)).begin(); _iter1129 != (*(this->part_vals)).end(); ++_iter1129)
     {
-      xfer += oprot->writeString((*_iter1118));
+      xfer += oprot->writeString((*_iter1129));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15668,10 +15939,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache::
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1119;
-    for (_iter1119 = (*(this->group_names)).begin(); _iter1119 != (*(this->group_names)).end(); ++_iter1119)
+    std::vector<std::string> ::const_iterator _iter1130;
+    for (_iter1130 = (*(this->group_names)).begin(); _iter1130 != (*(this->group_names)).end(); ++_iter1130)
     {
-      xfer += oprot->writeString((*_iter1119));
+      xfer += oprot->writeString((*_iter1130));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15712,14 +15983,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1120;
-            ::apache::thrift::protocol::TType _etype1123;
-            xfer += iprot->readListBegin(_etype1123, _size1120);
-            this->success.resize(_size1120);
-            uint32_t _i1124;
-            for (_i1124 = 0; _i1124 < _size1120; ++_i1124)
+            uint32_t _size1131;
+            ::apache::thrift::protocol::TType _etype1134;
+            xfer += iprot->readListBegin(_etype1134, _size1131);
+            this->success.resize(_size1131);
+            uint32_t _i1135;
+            for (_i1135 = 0; _i1135 < _size1131; ++_i1135)
             {
-              xfer += this->success[_i1124].read(iprot);
+              xfer += this->success[_i1135].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15766,10 +16037,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::write(::apache:
     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 _iter1125;
-      for (_iter1125 = this->success.begin(); _iter1125 != this->success.end(); ++_iter1125)
+      std::vector<Partition> ::const_iterator _iter1136;
+      for (_iter1136 = this->success.begin(); _iter1136 != this->success.end(); ++_iter1136)
       {
-        xfer += (*_iter1125).write(oprot);
+        xfer += (*_iter1136).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -15818,14 +16089,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_presult::read(::apache:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1126;
-            ::apache::thrift::protocol::TType _etype1129;
-            xfer += iprot->readListBegin(_etype1129, _size1126);
-            (*(this->success)).resize(_size1126);
-            uint32_t _i1130;
-            for (_i1130 = 0; _i1130 < _size1126; ++_i1130)
+            uint32_t _size1137;
+            ::apache::thrift::protocol::TType _etype1140;
+            xfer += iprot->readListBegin(_etype1140, _size1137);
+            (*(this->success)).resize(_size1137);
+            uint32_t _i1141;
+            for (_i1141 = 0; _i1141 < _size1137; ++_i1141)
             {
-              xfer += (*(this->success))[_i1130].read(iprot);
+              xfer += (*(this->success))[_i1141].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15908,14 +16179,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_args::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1131;
-            ::apache::thrift::protocol::TType _etype1134;
-            xfer += iprot->readListBegin(_etype1134, _size1131);
-            this->part_vals.resize(_size1131);
-            uint32_t _i1135;
-            for (_i1135 = 0; _i1135 < _size1131; ++_i1135)
+            uint32_t _size1142;
+            ::apache::thrift::protocol::TType _etype1145;
+            xfer += iprot->readListBegin(_etype1145, _size1142);
+            this->part_vals.resize(_size1142);
+            uint32_t _i1146;
+            for (_i1146 = 0; _i1146 < _size1142; ++_i1146)
             {
-              xfer += iprot->readString(this->part_vals[_i1135]);
+              xfer += iprot->readString(this->part_vals[_i1146]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15960,10 +16231,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_args::write(::apache::thrift
   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 _iter1136;
-    for (_iter1136 = this->part_vals.begin(); _iter1136 != this->part_vals.end(); ++_iter1136)
+    std::vector<std::string> ::const_iterator _iter1147;
+    for (_iter1147 = this->part_vals.begin(); _iter1147 != this->part_vals.end(); ++_iter1147)
     {
-      xfer += oprot->writeString((*_iter1136));
+      xfer += oprot->writeString((*_iter1147));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15999,10 +16270,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_pargs::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 _iter1137;
-    for (_iter1137 = (*(this->part_vals)).begin(); _iter1137 != (*(this->part_vals)).end(); ++_iter1137)
+    std::vector<std::string> ::const_iterator _iter1148;
+    for (_iter1148 = (*(this->part_vals)).begin(); _iter1148 != (*(this->part_vals)).end(); ++_iter1148)
     {
-      xfer += oprot->writeString((*_iter1137));
+      xfer += oprot->writeString((*_iter1148));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16047,14 +16318,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1138;
-            ::apache::thrift::protocol::TType _etype1141;
-            xfer += iprot->readListBegin(_etype1141, _size1138);
-            this->success.resize(_size1138);
-            uint32_t _i1142;
-            for (_i1142 = 0; _i1142 < _size1138; ++_i1142)
+            uint32_t _size1149;
+            ::apache::thrift::protocol::TType _etype1152;
+            xfer += iprot->readListBegin(_etype1152, _size1149);
+            this->success.resize(_size1149);
+            uint32_t _i1153;
+            for (_i1153 = 0; _i1153 < _size1149; ++_i1153)
             {
-              xfer += iprot->readString(this->success[_i1142]);
+              xfer += iprot->readString(this->success[_i1153]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16101,10 +16372,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::write(::apache::thri
     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 _iter1143;
-      for (_iter1143 = this->success.begin(); _iter1143 != this->success.end(); ++_iter1143)
+      std::vector<std::string> ::const_iterator _ite

<TRUNCATED>

[5/7] hive git commit: HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)

Posted by ai...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 6498eb1..525408b 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -47,6 +47,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_tables_by_type(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType) = 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;
@@ -277,6 +278,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_tables_by_type(std::vector<std::string> & /* _return */, const std::string& /* db_name */, const std::string& /* pattern */, const std::string& /* tableType */) {
+    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;
   }
@@ -3758,6 +3762,132 @@ class ThriftHiveMetastore_get_tables_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_get_tables_by_type_args__isset {
+  _ThriftHiveMetastore_get_tables_by_type_args__isset() : db_name(false), pattern(false), tableType(false) {}
+  bool db_name :1;
+  bool pattern :1;
+  bool tableType :1;
+} _ThriftHiveMetastore_get_tables_by_type_args__isset;
+
+class ThriftHiveMetastore_get_tables_by_type_args {
+ public:
+
+  ThriftHiveMetastore_get_tables_by_type_args(const ThriftHiveMetastore_get_tables_by_type_args&);
+  ThriftHiveMetastore_get_tables_by_type_args& operator=(const ThriftHiveMetastore_get_tables_by_type_args&);
+  ThriftHiveMetastore_get_tables_by_type_args() : db_name(), pattern(), tableType() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_tables_by_type_args() throw();
+  std::string db_name;
+  std::string pattern;
+  std::string tableType;
+
+  _ThriftHiveMetastore_get_tables_by_type_args__isset __isset;
+
+  void __set_db_name(const std::string& val);
+
+  void __set_pattern(const std::string& val);
+
+  void __set_tableType(const std::string& val);
+
+  bool operator == (const ThriftHiveMetastore_get_tables_by_type_args & rhs) const
+  {
+    if (!(db_name == rhs.db_name))
+      return false;
+    if (!(pattern == rhs.pattern))
+      return false;
+    if (!(tableType == rhs.tableType))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_tables_by_type_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_tables_by_type_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_tables_by_type_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_tables_by_type_pargs() throw();
+  const std::string* db_name;
+  const std::string* pattern;
+  const std::string* tableType;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_tables_by_type_result__isset {
+  _ThriftHiveMetastore_get_tables_by_type_result__isset() : success(false), o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_get_tables_by_type_result__isset;
+
+class ThriftHiveMetastore_get_tables_by_type_result {
+ public:
+
+  ThriftHiveMetastore_get_tables_by_type_result(const ThriftHiveMetastore_get_tables_by_type_result&);
+  ThriftHiveMetastore_get_tables_by_type_result& operator=(const ThriftHiveMetastore_get_tables_by_type_result&);
+  ThriftHiveMetastore_get_tables_by_type_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_tables_by_type_result() throw();
+  std::vector<std::string>  success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_get_tables_by_type_result__isset __isset;
+
+  void __set_success(const std::vector<std::string> & val);
+
+  void __set_o1(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_tables_by_type_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_tables_by_type_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_tables_by_type_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_tables_by_type_presult__isset {
+  _ThriftHiveMetastore_get_tables_by_type_presult__isset() : success(false), o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_get_tables_by_type_presult__isset;
+
+class ThriftHiveMetastore_get_tables_by_type_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_tables_by_type_presult() throw();
+  std::vector<std::string> * success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_get_tables_by_type_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 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;
@@ -19217,6 +19347,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_tables_by_type(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType);
+  void send_get_tables_by_type(const std::string& db_name, const std::string& pattern, const std::string& tableType);
+  void recv_get_tables_by_type(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);
@@ -19624,6 +19757,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_tables_by_type(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);
@@ -19777,6 +19911,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_tables_by_type"] = &ThriftHiveMetastoreProcessor::process_get_tables_by_type;
     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;
@@ -20170,6 +20305,16 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     return;
   }
 
+  void get_tables_by_type(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_tables_by_type(_return, db_name, pattern, tableType);
+    }
+    ifaces_[i]->get_tables_by_type(_return, db_name, pattern, tableType);
+    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;
@@ -21452,6 +21597,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_tables_by_type(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType);
+  int32_t send_get_tables_by_type(const std::string& db_name, const std::string& pattern, const std::string& tableType);
+  void recv_get_tables_by_type(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);

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 3330b2f..d66fdbe 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -147,6 +147,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("get_tables\n");
   }
 
+  void get_tables_by_type(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType) {
+    // Your implementation goes here
+    printf("get_tables_by_type\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");