You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by sp...@apache.org on 2018/05/30 13:24:16 UTC

[1/6] hive git commit: HIVE-19374: Parse and process ALTER TABLE SET OWNER command syntax (Sergio Pena, reviewed by Vihang Karajgaonkar)

Repository: hive
Updated Branches:
  refs/heads/branch-2 d988d4aef -> fbee8fef6


HIVE-19374: Parse and process ALTER TABLE SET OWNER command syntax (Sergio Pena, reviewed by Vihang Karajgaonkar)


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

Branch: refs/heads/branch-2
Commit: fbee8fef69cde550f2676c9b8bde68368cfe5f89
Parents: 72f75d3
Author: Sergio Pena <se...@cloudera.com>
Authored: Wed May 9 22:37:09 2018 -0700
Committer: Sergio Pena <se...@cloudera.com>
Committed: Wed May 30 08:21:12 2018 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |  5 +++++
 .../apache/hadoop/hive/ql/metadata/Table.java   | 15 +++++++++++++
 .../formatting/JsonMetaDataFormatter.java       |  1 +
 .../formatting/MetaDataFormatUtils.java         |  1 +
 .../hive/ql/parse/DDLSemanticAnalyzer.java      | 19 ++++++++++++++++-
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |  8 +++++++
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |  2 ++
 .../hadoop/hive/ql/plan/AlterTableDesc.java     | 22 +++++++++++++++++++-
 .../hadoop/hive/ql/plan/HiveOperation.java      |  1 +
 .../authorization/plugin/HiveOperationType.java |  1 +
 .../plugin/sqlstd/Operation2Privilege.java      |  2 ++
 .../beeline/escape_comments.q.out               |  6 ++++--
 12 files changed, 79 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/fbee8fef/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 e9c0625..af97703 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
@@ -4073,6 +4073,11 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       }
 
       tbl.setStoredAsSubDirectories(alterTbl.isStoredAsSubDirectories());
+    } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.OWNER) {
+      if (alterTbl.getOwnerPrincipal() != null) {
+        tbl.setOwner(alterTbl.getOwnerPrincipal().getName());
+        tbl.setOwnerType(alterTbl.getOwnerPrincipal().getType());
+      }
     } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.ALTERSKEWEDLOCATION) {
       // process location one-by-one
       Map<List<String>,String> locMaps = alterTbl.getSkewedLocations();

http://git-wip-us.apache.org/repos/asf/hive/blob/fbee8fef/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
index e0e57c6..5adb612 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -723,6 +724,12 @@ public class Table implements Serializable {
   }
 
   /**
+   * @return The owner type of the table.
+   * @see org.apache.hadoop.hive.metastore.api.Table#getOwnerType()
+   */
+  public PrincipalType getOwnerType() { return tTable.getOwnerType(); }
+
+  /**
    * @return The table parameters.
    * @see org.apache.hadoop.hive.metastore.api.Table#getParameters()
    */
@@ -747,6 +754,14 @@ public class Table implements Serializable {
   }
 
   /**
+   * @param ownerType
+   * @see org.apache.hadoop.hive.metastore.api.Table#setOwnerType(org.apache.hadoop.hive.metastore.api.PrincipalType)
+   */
+  public void setOwnerType(PrincipalType ownerType) {
+    tTable.setOwnerType(ownerType);
+  }
+
+  /**
    * @param retention
    * @see org.apache.hadoop.hive.metastore.api.Table#setRetention(int)
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/fbee8fef/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
index 3315806..b8348fc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
@@ -189,6 +189,7 @@ public class JsonMetaDataFormatter implements MetaDataFormatter {
     MapBuilder builder = MapBuilder.create();
 
     builder.put("tableName", tbl.getTableName());
+    builder.put("ownerType", (tbl.getOwnerType() != null) ? tbl.getOwnerType().name() : "null");
     builder.put("owner", tbl.getOwner());
     builder.put("location", tblLoc);
     builder.put("inputFormat", inputFormattCls);

http://git-wip-us.apache.org/repos/asf/hive/blob/fbee8fef/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
index f73c610..44cd06e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
@@ -432,6 +432,7 @@ public final class MetaDataFormatUtils {
   private static void getTableMetaDataInformation(StringBuilder tableInfo, Table  tbl,
       boolean isOutputPadded) {
     formatOutput("Database:", tbl.getDbName(), tableInfo);
+    formatOutput("OwnerType:", (tbl.getOwnerType() != null) ? tbl.getOwnerType().name() : "null", tableInfo);
     formatOutput("Owner:", tbl.getOwner(), tableInfo);
     formatOutput("CreateTime:", formatDate(tbl.getTTable().getCreateTime()), tableInfo);
     formatOutput("LastAccessTime:", formatDate(tbl.getTTable().getLastAccessTime()), tableInfo);

http://git-wip-us.apache.org/repos/asf/hive/blob/fbee8fef/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 b215095..0a5ca7c 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
@@ -323,7 +323,9 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       } else if(ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_DROPCONSTRAINT) {
         analyzeAlterTableDropConstraint(ast, tableName);
       } else if(ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_ADDCONSTRAINT) {
-          analyzeAlterTableAddConstraint(ast, tableName);
+        analyzeAlterTableAddConstraint(ast, tableName);
+      } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_OWNER) {
+        analyzeAlterTableOwner(ast, tableName);
       }
       break;
     }
@@ -1560,6 +1562,21 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
   }
 
+  private void analyzeAlterTableOwner(ASTNode ast, String tableName) throws SemanticException {
+    PrincipalDesc ownerPrincipal = AuthorizationParseUtils.getPrincipalDesc((ASTNode) ast.getChild(0));
+
+    if (ownerPrincipal.getType() == null) {
+      throw new SemanticException("Owner type can't be null in alter table set owner command");
+    }
+
+    if (ownerPrincipal.getName() == null) {
+      throw new SemanticException("Owner name can't be null in alter table set owner command");
+    }
+
+    AlterTableDesc alterTblDesc  = new AlterTableDesc(tableName, ownerPrincipal);
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterTblDesc), conf));
+  }
+
   private void analyzeAlterTableLocation(ASTNode ast, String tableName,
       HashMap<String, String> partSpec) throws SemanticException {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/fbee8fef/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 024b5d3..0d27899 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
@@ -188,6 +188,7 @@ TOK_ALTERTABLE_DROPCONSTRAINT;
 TOK_ALTERTABLE_ADDCONSTRAINT;
 TOK_ALTERINDEX_REBUILD;
 TOK_ALTERINDEX_PROPERTIES;
+TOK_ALTERTABLE_OWNER;
 TOK_MSCK;
 TOK_SHOWDATABASES;
 TOK_SHOWTABLES;
@@ -1141,6 +1142,7 @@ alterTableStatementSuffix
     | alterStatementSuffixDropConstraint
     | alterStatementSuffixAddConstraint
     | partitionSpec? alterTblPartitionStatementSuffix -> alterTblPartitionStatementSuffix partitionSpec?
+    | alterStatementSuffixSetOwner
     ;
 
 alterTblPartitionStatementSuffix
@@ -1472,6 +1474,12 @@ alterStatementSuffixCompact
     -> ^(TOK_ALTERTABLE_COMPACT $compactType blocking? tableProperties?)
     ;
 
+alterStatementSuffixSetOwner
+@init { pushMsg("alter table set owner", state); }
+@after { popMsg(state); }
+    : KW_SET KW_OWNER principalName
+    -> ^(TOK_ALTERTABLE_OWNER principalName)
+    ;
 
 fileFormat
 @init { pushMsg("file format specification", state); }

http://git-wip-us.apache.org/repos/asf/hive/blob/fbee8fef/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 9686ca3..3f3c861 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
@@ -96,6 +96,7 @@ public final class SemanticAnalyzerFactory {
     commandType.put(HiveParser.TOK_ALTERVIEW_DROPPROPERTIES, HiveOperation.ALTERVIEW_PROPERTIES);
     commandType.put(HiveParser.TOK_ALTERVIEW_ADDPARTS, HiveOperation.ALTERTABLE_ADDPARTS);
     commandType.put(HiveParser.TOK_ALTERVIEW_DROPPARTS, HiveOperation.ALTERTABLE_DROPPARTS);
+    commandType.put(HiveParser.TOK_ALTERTABLE_OWNER, HiveOperation.ALTERTABLE_OWNER);
     commandType.put(HiveParser.TOK_ALTERVIEW_RENAME, HiveOperation.ALTERVIEW_RENAME);
     commandType.put(HiveParser.TOK_ALTERVIEW, HiveOperation.ALTERVIEW_AS);
     commandType.put(HiveParser.TOK_QUERY, HiveOperation.QUERY);
@@ -213,6 +214,7 @@ public final class SemanticAnalyzerFactory {
           case HiveParser.TOK_ALTERTABLE_DROPPROPERTIES:
           case HiveParser.TOK_ALTERTABLE_EXCHANGEPARTITION:
           case HiveParser.TOK_ALTERTABLE_SKEWED:
+          case HiveParser.TOK_ALTERTABLE_OWNER:
           case HiveParser.TOK_ALTERTABLE_DROPCONSTRAINT:
           case HiveParser.TOK_ALTERTABLE_ADDCONSTRAINT:
           queryState.setCommandType(commandType.get(child.getType()));

http://git-wip-us.apache.org/repos/asf/hive/blob/fbee8fef/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
index b83c16d..6508e4b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
@@ -58,7 +58,8 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
     DROPPARTITION("drop partition"), RENAMEPARTITION("rename partition"), ADDSKEWEDBY("add skew column"),
     ALTERSKEWEDLOCATION("alter skew location"), ALTERBUCKETNUM("alter bucket number"),
     ALTERPARTITION("alter partition"), COMPACT("compact"),
-    TRUNCATE("truncate"), MERGEFILES("merge files"), DROPCONSTRAINT("drop constraint"), ADDCONSTRAINT("add constraint");
+    TRUNCATE("truncate"), MERGEFILES("merge files"), DROPCONSTRAINT("drop constraint"), ADDCONSTRAINT("add constraint"),
+    OWNER("set owner");
     ;
 
     private final String name;
@@ -121,6 +122,7 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
   String dropConstraintName;
   List<SQLPrimaryKey> primaryKeyCols;
   List<SQLForeignKey> foreignKeyCols;
+  PrincipalDesc ownerPrincipal;
 
   public AlterTableDesc() {
   }
@@ -281,6 +283,24 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
     op = AlterTableTypes.ADDCONSTRAINT;
   }
 
+  public AlterTableDesc(String tableName, PrincipalDesc ownerPrincipal) {
+    op  = AlterTableTypes.OWNER;
+    this.oldName = tableName;
+    this.ownerPrincipal = ownerPrincipal;
+  }
+
+  /**
+   * @param ownerPrincipal the owner principal of the table
+   */
+  public void setOwnerPrincipal(PrincipalDesc ownerPrincipal) {
+    this.ownerPrincipal = ownerPrincipal;
+  }
+
+  @Explain(displayName="owner")
+  public PrincipalDesc getOwnerPrincipal() {
+    return this.ownerPrincipal;
+  }
+
   @Explain(displayName = "new columns", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
   public List<String> getNewColsString() {
     return Utilities.getFieldSchemaString(getNewCols());

http://git-wip-us.apache.org/repos/asf/hive/blob/fbee8fef/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 57bf0f7..5b5e94d 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
@@ -48,6 +48,7 @@ public enum HiveOperation {
   ALTERTABLE_ARCHIVE("ALTERTABLE_ARCHIVE", new Privilege[]{Privilege.ALTER_DATA}, null),
   ALTERTABLE_UNARCHIVE("ALTERTABLE_UNARCHIVE", new Privilege[]{Privilege.ALTER_DATA}, null),
   ALTERTABLE_PROPERTIES("ALTERTABLE_PROPERTIES", new Privilege[]{Privilege.ALTER_METADATA}, null),
+  ALTERTABLE_OWNER("ALTERTABLE_OWNER", null, null),
   ALTERTABLE_SERIALIZER("ALTERTABLE_SERIALIZER", new Privilege[]{Privilege.ALTER_METADATA}, null),
   ALTERPARTITION_SERIALIZER("ALTERPARTITION_SERIALIZER", new Privilege[]{Privilege.ALTER_METADATA}, null),
   ALTERTABLE_SERDEPROPERTIES("ALTERTABLE_SERDEPROPERTIES", new Privilege[]{Privilege.ALTER_METADATA}, null),

http://git-wip-us.apache.org/repos/asf/hive/blob/fbee8fef/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 330f9ac..7e6342b 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
@@ -50,6 +50,7 @@ public enum HiveOperationType {
   ALTERTABLE_ARCHIVE,
   ALTERTABLE_UNARCHIVE,
   ALTERTABLE_PROPERTIES,
+  ALTERTABLE_OWNER,
   ALTERTABLE_SERIALIZER,
   ALTERTABLE_PARTCOLTYPE,
   ALTERTABLE_DROPCONSTRAINT,

http://git-wip-us.apache.org/repos/asf/hive/blob/fbee8fef/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 e8cdc9a..26a49d1 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
@@ -194,6 +194,8 @@ public class Operation2Privilege {
 (OWNER_PRIV_AR, OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.ALTERTABLE_PROPERTIES, PrivRequirement.newIOPrivRequirement
 (OWNER_PRIV_AR, OWNER_PRIV_AR));
+    op2Priv.put(HiveOperationType.ALTERTABLE_OWNER, PrivRequirement.newIOPrivRequirement
+(OWNER_PRIV_AR, OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.ALTERTABLE_SERIALIZER, PrivRequirement.newIOPrivRequirement
 (OWNER_PRIV_AR, OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.ALTERTABLE_PARTCOLTYPE, PrivRequirement.newIOPrivRequirement

http://git-wip-us.apache.org/repos/asf/hive/blob/fbee8fef/ql/src/test/results/clientpositive/beeline/escape_comments.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/beeline/escape_comments.q.out b/ql/src/test/results/clientpositive/beeline/escape_comments.q.out
index 726213a..e7cfc9a 100644
--- a/ql/src/test/results/clientpositive/beeline/escape_comments.q.out
+++ b/ql/src/test/results/clientpositive/beeline/escape_comments.q.out
@@ -220,6 +220,7 @@ Shutting down query describe formatted escape_comments_tbl1
 '','NULL','NULL'
 '# Detailed Table Information','NULL','NULL'
 'Database:           ','escape_comments_db  ','NULL'
+'OwnerType:          ','USER                ','NULL'
 'Owner:              ','user                ','NULL'
 'CreateTime:         ','!!TIMESTAMP!!','NULL'
 'LastAccessTime:     ','UNKNOWN             ','NULL'
@@ -246,7 +247,7 @@ Shutting down query describe formatted escape_comments_tbl1
 'Sort Columns:       ','[]                  ','NULL'
 'Storage Desc Params:','NULL','NULL'
 '','serialization.format','1                   '
-37 rows selected 
+38 rows selected 
 >>>  describe pretty escape_comments_tbl1;
 Acquired the compile lock.
 Compiling commandqueryId=(!!{queryId}!!): describe pretty escape_comments_tbl1
@@ -348,6 +349,7 @@ Shutting down query describe formatted escape_comments_view1
 '','NULL','NULL'
 '# Detailed Table Information','NULL','NULL'
 'Database:           ','escape_comments_db  ','NULL'
+'OwnerType:          ','USER                ','NULL'
 'Owner:              ','user                ','NULL'
 'CreateTime:         ','!!TIMESTAMP!!','NULL'
 'LastAccessTime:     ','UNKNOWN             ','NULL'
@@ -370,7 +372,7 @@ Shutting down query describe formatted escape_comments_view1
 'View Original Text: ','select col1 from escape_comments_tbl1','NULL'
 'View Expanded Text: ','SELECT `col1` AS `col1` FROM (select `escape_comments_tbl1`.`col1` from `escape_comments_db`.`escape_comments_tbl1`) `escape_comments_db.escape_comments_view1`','NULL'
 'View Rewrite Enabled:','No                  ','NULL'
-28 rows selected 
+29 rows selected 
 >>>  show formatted index on escape_comments_tbl1;
 Acquired the compile lock.
 Compiling commandqueryId=(!!{queryId}!!): show formatted index on escape_comments_tbl1


[2/6] hive git commit: HIVE-19372: Add table ownerType to JDO/SQL and ObjectStore (Sergio Pena, reviewed by Vihang Karajgaonkar)

Posted by sp...@apache.org.
HIVE-19372: Add table ownerType to JDO/SQL and ObjectStore (Sergio Pena, reviewed by Vihang Karajgaonkar)


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

Branch: refs/heads/branch-2
Commit: 72f75d37675cad605e7b8fd1f2255470255395ce
Parents: 117c3b4
Author: Sergio Pena <se...@cloudera.com>
Authored: Thu May 24 17:02:49 2018 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Wed May 30 08:21:12 2018 -0500

----------------------------------------------------------------------
 .../metastore/hbase/TestHBaseSchemaTool.java     |  8 ++++----
 .../upgrade/derby/041-HIVE-19372.derby.sql       |  1 +
 .../upgrade/derby/hive-schema-2.4.0.derby.sql    |  2 +-
 .../derby/upgrade-2.3.0-to-2.4.0.derby.sql       |  2 ++
 .../upgrade/mssql/026-HIVE-19372.mssql.sql       |  1 +
 .../upgrade/mssql/hive-schema-2.4.0.mssql.sql    |  1 +
 .../mssql/upgrade-2.3.0-to-2.4.0.mssql.sql       |  2 ++
 .../upgrade/mysql/041-HIVE-19372.mysql.sql       |  1 +
 .../upgrade/mysql/hive-schema-2.4.0.mysql.sql    |  1 +
 .../mysql/upgrade-2.3.0-to-2.4.0.mysql.sql       |  2 ++
 .../upgrade/oracle/041-HIVE-19372.oracle.sql     |  1 +
 .../upgrade/oracle/hive-schema-2.4.0.oracle.sql  |  1 +
 .../oracle/upgrade-2.3.0-to-2.4.0.oracle.sql     |  1 +
 .../upgrade/postgres/040-HIVE-19372.postgres.sql |  1 +
 .../postgres/hive-schema-2.4.0.postgres.sql      |  1 +
 .../postgres/upgrade-2.3.0-to-2.4.0.postgres.sql |  1 +
 .../hadoop/hive/metastore/ObjectStore.java       | 16 +++++++++++++++-
 .../hadoop/hive/metastore/model/MTable.java      | 19 ++++++++++++++++++-
 metastore/src/model/package.jdo                  |  3 +++
 .../hadoop/hive/metastore/TestObjectStore.java   | 10 ++++++++++
 20 files changed, 68 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/scripts/upgrade/derby/041-HIVE-19372.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/041-HIVE-19372.derby.sql b/metastore/scripts/upgrade/derby/041-HIVE-19372.derby.sql
new file mode 100644
index 0000000..4119607
--- /dev/null
+++ b/metastore/scripts/upgrade/derby/041-HIVE-19372.derby.sql
@@ -0,0 +1 @@
+ALTER TABLE "APP"."TBLS" ADD COLUMN "OWNER_TYPE" VARCHAR(10);
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-2.4.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-2.4.0.derby.sql b/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-2.4.0.derby.sql
index 4bb473f..58dde50 100644
--- a/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-2.4.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-2.4.0.derby.sql
@@ -1,3 +1,5 @@
 -- Upgrade MetaStore schema from 2.3.0 to 2.4.0
 
+RUN '041-HIVE-19372.derby.sql';
+
 UPDATE "APP".VERSION SET SCHEMA_VERSION='2.4.0', VERSION_COMMENT='Hive release version 2.4.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/scripts/upgrade/mssql/026-HIVE-19372.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/026-HIVE-19372.mssql.sql b/metastore/scripts/upgrade/mssql/026-HIVE-19372.mssql.sql
new file mode 100644
index 0000000..4119607
--- /dev/null
+++ b/metastore/scripts/upgrade/mssql/026-HIVE-19372.mssql.sql
@@ -0,0 +1 @@
+ALTER TABLE "APP"."TBLS" ADD COLUMN "OWNER_TYPE" VARCHAR(10);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/scripts/upgrade/mssql/hive-schema-2.4.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/hive-schema-2.4.0.mssql.sql b/metastore/scripts/upgrade/mssql/hive-schema-2.4.0.mssql.sql
index 20f6be0..c6c733e 100644
--- a/metastore/scripts/upgrade/mssql/hive-schema-2.4.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/hive-schema-2.4.0.mssql.sql
@@ -353,6 +353,7 @@ CREATE TABLE TBLS
     DB_ID bigint NULL,
     LAST_ACCESS_TIME int NOT NULL,
     OWNER nvarchar(767) NULL,
+    OWNER_TYPE nvarchar(10) NULL,
     RETENTION int NOT NULL,
     SD_ID bigint NULL,
     TBL_NAME nvarchar(256) NULL,

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-2.4.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-2.4.0.mssql.sql b/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-2.4.0.mssql.sql
index 6877315..b3cceea 100644
--- a/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-2.4.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-2.4.0.mssql.sql
@@ -1,4 +1,6 @@
 SELECT 'Upgrading MetaStore schema from 2.3.0 to 2.4.0' AS MESSAGE;
 
+:r 026-HIVE-19372.mssql.sql
+
 UPDATE VERSION SET SCHEMA_VERSION='2.4.0', VERSION_COMMENT='Hive release version 2.4.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 2.4.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/scripts/upgrade/mysql/041-HIVE-19372.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/041-HIVE-19372.mysql.sql b/metastore/scripts/upgrade/mysql/041-HIVE-19372.mysql.sql
new file mode 100644
index 0000000..da4aff7
--- /dev/null
+++ b/metastore/scripts/upgrade/mysql/041-HIVE-19372.mysql.sql
@@ -0,0 +1 @@
+ALTER TABLE `TBLS` ADD COLUMN `OWNER_TYPE` VARCHAR(10) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/scripts/upgrade/mysql/hive-schema-2.4.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-schema-2.4.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-schema-2.4.0.mysql.sql
index 2f9aff2..7aed5a8 100644
--- a/metastore/scripts/upgrade/mysql/hive-schema-2.4.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/hive-schema-2.4.0.mysql.sql
@@ -581,6 +581,7 @@ CREATE TABLE IF NOT EXISTS `TBLS` (
   `DB_ID` bigint(20) DEFAULT NULL,
   `LAST_ACCESS_TIME` int(11) NOT NULL,
   `OWNER` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `OWNER_TYPE` varchar(10) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `RETENTION` int(11) NOT NULL,
   `SD_ID` bigint(20) DEFAULT NULL,
   `TBL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/scripts/upgrade/mysql/upgrade-2.3.0-to-2.4.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/upgrade-2.3.0-to-2.4.0.mysql.sql b/metastore/scripts/upgrade/mysql/upgrade-2.3.0-to-2.4.0.mysql.sql
index 68db617..d85b30b 100644
--- a/metastore/scripts/upgrade/mysql/upgrade-2.3.0-to-2.4.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/upgrade-2.3.0-to-2.4.0.mysql.sql
@@ -1,5 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.3.0 to 2.4.0' AS ' ';
 
+SOURCE 041-HIVE-19372.mysql.sql;
+
 UPDATE VERSION SET SCHEMA_VERSION='2.4.0', VERSION_COMMENT='Hive release version 2.4.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 2.4.0' AS ' ';
 

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/scripts/upgrade/oracle/041-HIVE-19372.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/041-HIVE-19372.oracle.sql b/metastore/scripts/upgrade/oracle/041-HIVE-19372.oracle.sql
new file mode 100644
index 0000000..98264f3
--- /dev/null
+++ b/metastore/scripts/upgrade/oracle/041-HIVE-19372.oracle.sql
@@ -0,0 +1 @@
+ALTER TABLE TBLS ADD OWNER_TYPE VARCHAR2(10) NULL;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/scripts/upgrade/oracle/hive-schema-2.4.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/hive-schema-2.4.0.oracle.sql b/metastore/scripts/upgrade/oracle/hive-schema-2.4.0.oracle.sql
index c07ca79..94d1221 100644
--- a/metastore/scripts/upgrade/oracle/hive-schema-2.4.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/hive-schema-2.4.0.oracle.sql
@@ -370,6 +370,7 @@ CREATE TABLE TBLS
     DB_ID NUMBER NULL,
     LAST_ACCESS_TIME NUMBER (10) NOT NULL,
     OWNER VARCHAR2(767) NULL,
+    OWNER_TYPE VARCHAR2(10) NULL,
     RETENTION NUMBER (10) NOT NULL,
     SD_ID NUMBER NULL,
     TBL_NAME VARCHAR2(256) NULL,

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/scripts/upgrade/oracle/upgrade-2.3.0-to-2.4.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/upgrade-2.3.0-to-2.4.0.oracle.sql b/metastore/scripts/upgrade/oracle/upgrade-2.3.0-to-2.4.0.oracle.sql
index 45a274a..3b4e273 100644
--- a/metastore/scripts/upgrade/oracle/upgrade-2.3.0-to-2.4.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/upgrade-2.3.0-to-2.4.0.oracle.sql
@@ -1,6 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.3.0 to 2.4.0' AS Status from dual;
 
 @040-HIVE-16399.oracle.sql;
+@041-HIVE-19372.oracle.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='2.4.0', VERSION_COMMENT='Hive release version 2.4.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 2.4.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/scripts/upgrade/postgres/040-HIVE-19372.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/040-HIVE-19372.postgres.sql b/metastore/scripts/upgrade/postgres/040-HIVE-19372.postgres.sql
new file mode 100644
index 0000000..dfa469c
--- /dev/null
+++ b/metastore/scripts/upgrade/postgres/040-HIVE-19372.postgres.sql
@@ -0,0 +1 @@
+ALTER TABLE "TBLS" ADD COLUMN "OWNER_TYPE" character varying(10) DEFAULT NULL::character varying;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/scripts/upgrade/postgres/hive-schema-2.4.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/hive-schema-2.4.0.postgres.sql b/metastore/scripts/upgrade/postgres/hive-schema-2.4.0.postgres.sql
index c54d9e7..ffa29cf 100644
--- a/metastore/scripts/upgrade/postgres/hive-schema-2.4.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/hive-schema-2.4.0.postgres.sql
@@ -367,6 +367,7 @@ CREATE TABLE "TBLS" (
     "DB_ID" bigint,
     "LAST_ACCESS_TIME" bigint NOT NULL,
     "OWNER" character varying(767) DEFAULT NULL::character varying,
+    "OWNER_TYPE" character varying(10) DEFAULT NULL::character varying,
     "RETENTION" bigint NOT NULL,
     "SD_ID" bigint,
     "TBL_NAME" character varying(256) DEFAULT NULL::character varying,

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/scripts/upgrade/postgres/upgrade-2.3.0-to-2.4.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/upgrade-2.3.0-to-2.4.0.postgres.sql b/metastore/scripts/upgrade/postgres/upgrade-2.3.0-to-2.4.0.postgres.sql
index 5e34d96..1b2a52a 100644
--- a/metastore/scripts/upgrade/postgres/upgrade-2.3.0-to-2.4.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/upgrade-2.3.0-to-2.4.0.postgres.sql
@@ -1,6 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.3.0 to 2.4.0';
 
 \i 039-HIVE-16399.postgres.sql;
+\i 040-HIVE-19372.postgres.sql;
 
 UPDATE "VERSION" SET "SCHEMA_VERSION"='2.4.0', "VERSION_COMMENT"='Hive release version 2.4.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 2.4.0';

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/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 d85a001..ba46e63 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -56,6 +56,7 @@ import javax.jdo.Transaction;
 import javax.jdo.datastore.DataStoreCache;
 import javax.jdo.identity.IntIdentity;
 
+import com.google.common.base.Strings;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
@@ -1445,6 +1446,15 @@ public class ObjectStore implements RawStore, Configurable {
             .getRetention(), convertToStorageDescriptor(mtbl.getSd()),
             convertToFieldSchemas(mtbl.getPartitionKeys()), convertMap(mtbl.getParameters()),
             mtbl.getViewOriginalText(), mtbl.getViewExpandedText(), tableType);
+
+    if (Strings.isNullOrEmpty(mtbl.getOwnerType())) {
+      // Before the ownerType exists in an old Hive schema, USER was the default type for owner.
+      // Let's set the default to USER to keep backward compatibility.
+      table.setOwnerType(PrincipalType.USER);
+    } else {
+      table.setOwnerType(PrincipalType.valueOf(mtbl.getOwnerType()));
+    }
+
     table.setRewriteEnabled(mtbl.isRewriteEnabled());
     return table;
   }
@@ -1478,9 +1488,12 @@ public class ObjectStore implements RawStore, Configurable {
       }
     }
 
+    PrincipalType ownerPrincipalType = tbl.getOwnerType();
+    String ownerType = (ownerPrincipalType == null) ? PrincipalType.USER.name() : ownerPrincipalType.name();
+
     // A new table is always created with a new column descriptor
     return new MTable(HiveStringUtils.normalizeIdentifier(tbl.getTableName()), mdb,
-        convertToMStorageDescriptor(tbl.getSd()), tbl.getOwner(), tbl
+        convertToMStorageDescriptor(tbl.getSd()), tbl.getOwner(), ownerType, tbl
         .getCreateTime(), tbl.getLastAccessTime(), tbl.getRetention(),
         convertToMFieldSchemas(tbl.getPartitionKeys()), tbl.getParameters(),
         tbl.getViewOriginalText(), tbl.getViewExpandedText(), tbl.isRewriteEnabled(),
@@ -3463,6 +3476,7 @@ public class ObjectStore implements RawStore, Configurable {
       oldt.setTableName(HiveStringUtils.normalizeIdentifier(newt.getTableName()));
       oldt.setParameters(newt.getParameters());
       oldt.setOwner(newt.getOwner());
+      oldt.setOwnerType(newt.getOwnerType());
       // Fully copy over the contents of the new SD into the old SD,
       // so we don't create an extra SD in the metastore db that has no references.
       copyMSD(newt.getSd(), oldt.getSd());

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/src/model/org/apache/hadoop/hive/metastore/model/MTable.java
----------------------------------------------------------------------
diff --git a/metastore/src/model/org/apache/hadoop/hive/metastore/model/MTable.java b/metastore/src/model/org/apache/hadoop/hive/metastore/model/MTable.java
index 6cc7157..0fd8bf0 100644
--- a/metastore/src/model/org/apache/hadoop/hive/metastore/model/MTable.java
+++ b/metastore/src/model/org/apache/hadoop/hive/metastore/model/MTable.java
@@ -27,6 +27,7 @@ public class MTable {
   private MDatabase database;
   private MStorageDescriptor sd;
   private String owner;
+  private String ownerType;
   private int createTime;
   private int lastAccessTime;
   private int retention;
@@ -44,6 +45,7 @@ public class MTable {
    * @param database
    * @param sd
    * @param owner
+   * @param ownerType
    * @param createTime
    * @param lastAccessTime
    * @param retention
@@ -53,7 +55,7 @@ public class MTable {
    * @param viewExpandedText
    * @param tableType
    */
-  public MTable(String tableName, MDatabase database, MStorageDescriptor sd, String owner,
+  public MTable(String tableName, MDatabase database, MStorageDescriptor sd, String owner, String ownerType,
       int createTime, int lastAccessTime, int retention, List<MFieldSchema> partitionKeys,
       Map<String, String> parameters, String viewOriginalText, String viewExpandedText,
       boolean rewriteEnabled, String tableType) {
@@ -61,6 +63,7 @@ public class MTable {
     this.database = database;
     this.sd = sd;
     this.owner = owner;
+    this.ownerType = ownerType;
     this.createTime = createTime;
     this.setLastAccessTime(lastAccessTime);
     this.retention = retention;
@@ -73,6 +76,20 @@ public class MTable {
   }
 
   /**
+   * @return the owner type
+   */
+  public String getOwnerType() {
+    return ownerType;
+  }
+
+  /**
+   * @param ownerType the owner type to set
+   */
+  public void setOwnerType(String ownerType) {
+    this.ownerType = ownerType;
+  }
+
+  /**
    * @return the tableName
    */
   public String getTableName() {

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/src/model/package.jdo
----------------------------------------------------------------------
diff --git a/metastore/src/model/package.jdo b/metastore/src/model/package.jdo
index bdecd2a..b272456 100644
--- a/metastore/src/model/package.jdo
+++ b/metastore/src/model/package.jdo
@@ -152,6 +152,9 @@
       <field name="owner">
         <column name="OWNER" length="767" jdbc-type="VARCHAR"/>
       </field>
+      <field name="ownerType">
+        <column name="OWNER_TYPE" length="10" jdbc-type="VARCHAR" allows-null="true"/>
+      </field>
       <field name="createTime">
         <column name="CREATE_TIME" jdbc-type="integer"/>
       </field>

http://git-wip-us.apache.org/repos/asf/hive/blob/72f75d37/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
index 4ba1a03..27dda1f 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -234,11 +234,21 @@ public class TestObjectStore {
     Assert.assertEquals(TABLE1, tables.get(0));
 
     Table newTbl1 = new Table("new" + TABLE1, DB1, "owner", 1, 2, 3, sd, null, params, null, null, "MANAGED_TABLE");
+
+    // Change different fields and verify they were altered
+    newTbl1.setOwner("role1");
+    newTbl1.setOwnerType(PrincipalType.ROLE);
+
     objectStore.alterTable(DB1, TABLE1, newTbl1);
     tables = objectStore.getTables(DB1, "new*");
     Assert.assertEquals(1, tables.size());
     Assert.assertEquals("new" + TABLE1, tables.get(0));
 
+    // Verify fields were altered during the alterTable operation
+    Table alteredTable = objectStore.getTable(DB1, "new" + TABLE1);
+    Assert.assertEquals("Owner of table was not altered", newTbl1.getOwner(), alteredTable.getOwner());
+    Assert.assertEquals("Owner type of table was not altered", newTbl1.getOwnerType(), alteredTable.getOwnerType());
+
     objectStore.dropTable(DB1, "new" + TABLE1);
     tables = objectStore.getAllTables(DB1);
     Assert.assertEquals(0, tables.size());


[4/6] hive git commit: HIVE-19371: Add table ownerType to HMS thrift API (Sergio Pena, reviewed by Sahil Takiar, Vihang Karajgaonkar)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index ed847cd..598d887 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -4484,6 +4484,11 @@ void Table::__set_rewriteEnabled(const bool val) {
 __isset.rewriteEnabled = true;
 }
 
+void Table::__set_ownerType(const PrincipalType::type val) {
+  this->ownerType = val;
+__isset.ownerType = true;
+}
+
 uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -4652,6 +4657,16 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 16:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast219;
+          xfer += iprot->readI32(ecast219);
+          this->ownerType = (PrincipalType::type)ecast219;
+          this->__isset.ownerType = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -4700,10 +4715,10 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("partitionKeys", ::apache::thrift::protocol::T_LIST, 8);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitionKeys.size()));
-    std::vector<FieldSchema> ::const_iterator _iter219;
-    for (_iter219 = this->partitionKeys.begin(); _iter219 != this->partitionKeys.end(); ++_iter219)
+    std::vector<FieldSchema> ::const_iterator _iter220;
+    for (_iter220 = this->partitionKeys.begin(); _iter220 != this->partitionKeys.end(); ++_iter220)
     {
-      xfer += (*_iter219).write(oprot);
+      xfer += (*_iter220).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4712,11 +4727,11 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 9);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter220;
-    for (_iter220 = this->parameters.begin(); _iter220 != this->parameters.end(); ++_iter220)
+    std::map<std::string, std::string> ::const_iterator _iter221;
+    for (_iter221 = this->parameters.begin(); _iter221 != this->parameters.end(); ++_iter221)
     {
-      xfer += oprot->writeString(_iter220->first);
-      xfer += oprot->writeString(_iter220->second);
+      xfer += oprot->writeString(_iter221->first);
+      xfer += oprot->writeString(_iter221->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -4749,6 +4764,11 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
     xfer += oprot->writeBool(this->rewriteEnabled);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.ownerType) {
+    xfer += oprot->writeFieldBegin("ownerType", ::apache::thrift::protocol::T_I32, 16);
+    xfer += oprot->writeI32((int32_t)this->ownerType);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -4771,28 +4791,11 @@ void swap(Table &a, Table &b) {
   swap(a.privileges, b.privileges);
   swap(a.temporary, b.temporary);
   swap(a.rewriteEnabled, b.rewriteEnabled);
+  swap(a.ownerType, b.ownerType);
   swap(a.__isset, b.__isset);
 }
 
-Table::Table(const Table& other221) {
-  tableName = other221.tableName;
-  dbName = other221.dbName;
-  owner = other221.owner;
-  createTime = other221.createTime;
-  lastAccessTime = other221.lastAccessTime;
-  retention = other221.retention;
-  sd = other221.sd;
-  partitionKeys = other221.partitionKeys;
-  parameters = other221.parameters;
-  viewOriginalText = other221.viewOriginalText;
-  viewExpandedText = other221.viewExpandedText;
-  tableType = other221.tableType;
-  privileges = other221.privileges;
-  temporary = other221.temporary;
-  rewriteEnabled = other221.rewriteEnabled;
-  __isset = other221.__isset;
-}
-Table& Table::operator=(const Table& other222) {
+Table::Table(const Table& other222) {
   tableName = other222.tableName;
   dbName = other222.dbName;
   owner = other222.owner;
@@ -4808,7 +4811,27 @@ Table& Table::operator=(const Table& other222) {
   privileges = other222.privileges;
   temporary = other222.temporary;
   rewriteEnabled = other222.rewriteEnabled;
+  ownerType = other222.ownerType;
   __isset = other222.__isset;
+}
+Table& Table::operator=(const Table& other223) {
+  tableName = other223.tableName;
+  dbName = other223.dbName;
+  owner = other223.owner;
+  createTime = other223.createTime;
+  lastAccessTime = other223.lastAccessTime;
+  retention = other223.retention;
+  sd = other223.sd;
+  partitionKeys = other223.partitionKeys;
+  parameters = other223.parameters;
+  viewOriginalText = other223.viewOriginalText;
+  viewExpandedText = other223.viewExpandedText;
+  tableType = other223.tableType;
+  privileges = other223.privileges;
+  temporary = other223.temporary;
+  rewriteEnabled = other223.rewriteEnabled;
+  ownerType = other223.ownerType;
+  __isset = other223.__isset;
   return *this;
 }
 void Table::printTo(std::ostream& out) const {
@@ -4829,6 +4852,7 @@ void Table::printTo(std::ostream& out) const {
   out << ", " << "privileges="; (__isset.privileges ? (out << to_string(privileges)) : (out << "<null>"));
   out << ", " << "temporary="; (__isset.temporary ? (out << to_string(temporary)) : (out << "<null>"));
   out << ", " << "rewriteEnabled="; (__isset.rewriteEnabled ? (out << to_string(rewriteEnabled)) : (out << "<null>"));
+  out << ", " << "ownerType="; (__isset.ownerType ? (out << to_string(ownerType)) : (out << "<null>"));
   out << ")";
 }
 
@@ -4895,14 +4919,14 @@ uint32_t Partition::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->values.clear();
-            uint32_t _size223;
-            ::apache::thrift::protocol::TType _etype226;
-            xfer += iprot->readListBegin(_etype226, _size223);
-            this->values.resize(_size223);
-            uint32_t _i227;
-            for (_i227 = 0; _i227 < _size223; ++_i227)
+            uint32_t _size224;
+            ::apache::thrift::protocol::TType _etype227;
+            xfer += iprot->readListBegin(_etype227, _size224);
+            this->values.resize(_size224);
+            uint32_t _i228;
+            for (_i228 = 0; _i228 < _size224; ++_i228)
             {
-              xfer += iprot->readString(this->values[_i227]);
+              xfer += iprot->readString(this->values[_i228]);
             }
             xfer += iprot->readListEnd();
           }
@@ -4955,17 +4979,17 @@ uint32_t Partition::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size228;
-            ::apache::thrift::protocol::TType _ktype229;
-            ::apache::thrift::protocol::TType _vtype230;
-            xfer += iprot->readMapBegin(_ktype229, _vtype230, _size228);
-            uint32_t _i232;
-            for (_i232 = 0; _i232 < _size228; ++_i232)
+            uint32_t _size229;
+            ::apache::thrift::protocol::TType _ktype230;
+            ::apache::thrift::protocol::TType _vtype231;
+            xfer += iprot->readMapBegin(_ktype230, _vtype231, _size229);
+            uint32_t _i233;
+            for (_i233 = 0; _i233 < _size229; ++_i233)
             {
-              std::string _key233;
-              xfer += iprot->readString(_key233);
-              std::string& _val234 = this->parameters[_key233];
-              xfer += iprot->readString(_val234);
+              std::string _key234;
+              xfer += iprot->readString(_key234);
+              std::string& _val235 = this->parameters[_key234];
+              xfer += iprot->readString(_val235);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5002,10 +5026,10 @@ uint32_t Partition::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->values.size()));
-    std::vector<std::string> ::const_iterator _iter235;
-    for (_iter235 = this->values.begin(); _iter235 != this->values.end(); ++_iter235)
+    std::vector<std::string> ::const_iterator _iter236;
+    for (_iter236 = this->values.begin(); _iter236 != this->values.end(); ++_iter236)
     {
-      xfer += oprot->writeString((*_iter235));
+      xfer += oprot->writeString((*_iter236));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5034,11 +5058,11 @@ uint32_t Partition::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 7);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter236;
-    for (_iter236 = this->parameters.begin(); _iter236 != this->parameters.end(); ++_iter236)
+    std::map<std::string, std::string> ::const_iterator _iter237;
+    for (_iter237 = this->parameters.begin(); _iter237 != this->parameters.end(); ++_iter237)
     {
-      xfer += oprot->writeString(_iter236->first);
-      xfer += oprot->writeString(_iter236->second);
+      xfer += oprot->writeString(_iter237->first);
+      xfer += oprot->writeString(_iter237->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -5067,18 +5091,7 @@ void swap(Partition &a, Partition &b) {
   swap(a.__isset, b.__isset);
 }
 
-Partition::Partition(const Partition& other237) {
-  values = other237.values;
-  dbName = other237.dbName;
-  tableName = other237.tableName;
-  createTime = other237.createTime;
-  lastAccessTime = other237.lastAccessTime;
-  sd = other237.sd;
-  parameters = other237.parameters;
-  privileges = other237.privileges;
-  __isset = other237.__isset;
-}
-Partition& Partition::operator=(const Partition& other238) {
+Partition::Partition(const Partition& other238) {
   values = other238.values;
   dbName = other238.dbName;
   tableName = other238.tableName;
@@ -5088,6 +5101,17 @@ Partition& Partition::operator=(const Partition& other238) {
   parameters = other238.parameters;
   privileges = other238.privileges;
   __isset = other238.__isset;
+}
+Partition& Partition::operator=(const Partition& other239) {
+  values = other239.values;
+  dbName = other239.dbName;
+  tableName = other239.tableName;
+  createTime = other239.createTime;
+  lastAccessTime = other239.lastAccessTime;
+  sd = other239.sd;
+  parameters = other239.parameters;
+  privileges = other239.privileges;
+  __isset = other239.__isset;
   return *this;
 }
 void Partition::printTo(std::ostream& out) const {
@@ -5159,14 +5183,14 @@ uint32_t PartitionWithoutSD::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->values.clear();
-            uint32_t _size239;
-            ::apache::thrift::protocol::TType _etype242;
-            xfer += iprot->readListBegin(_etype242, _size239);
-            this->values.resize(_size239);
-            uint32_t _i243;
-            for (_i243 = 0; _i243 < _size239; ++_i243)
+            uint32_t _size240;
+            ::apache::thrift::protocol::TType _etype243;
+            xfer += iprot->readListBegin(_etype243, _size240);
+            this->values.resize(_size240);
+            uint32_t _i244;
+            for (_i244 = 0; _i244 < _size240; ++_i244)
             {
-              xfer += iprot->readString(this->values[_i243]);
+              xfer += iprot->readString(this->values[_i244]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5203,17 +5227,17 @@ uint32_t PartitionWithoutSD::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size244;
-            ::apache::thrift::protocol::TType _ktype245;
-            ::apache::thrift::protocol::TType _vtype246;
-            xfer += iprot->readMapBegin(_ktype245, _vtype246, _size244);
-            uint32_t _i248;
-            for (_i248 = 0; _i248 < _size244; ++_i248)
+            uint32_t _size245;
+            ::apache::thrift::protocol::TType _ktype246;
+            ::apache::thrift::protocol::TType _vtype247;
+            xfer += iprot->readMapBegin(_ktype246, _vtype247, _size245);
+            uint32_t _i249;
+            for (_i249 = 0; _i249 < _size245; ++_i249)
             {
-              std::string _key249;
-              xfer += iprot->readString(_key249);
-              std::string& _val250 = this->parameters[_key249];
-              xfer += iprot->readString(_val250);
+              std::string _key250;
+              xfer += iprot->readString(_key250);
+              std::string& _val251 = this->parameters[_key250];
+              xfer += iprot->readString(_val251);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5250,10 +5274,10 @@ uint32_t PartitionWithoutSD::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->values.size()));
-    std::vector<std::string> ::const_iterator _iter251;
-    for (_iter251 = this->values.begin(); _iter251 != this->values.end(); ++_iter251)
+    std::vector<std::string> ::const_iterator _iter252;
+    for (_iter252 = this->values.begin(); _iter252 != this->values.end(); ++_iter252)
     {
-      xfer += oprot->writeString((*_iter251));
+      xfer += oprot->writeString((*_iter252));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5274,11 +5298,11 @@ uint32_t PartitionWithoutSD::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 5);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter252;
-    for (_iter252 = this->parameters.begin(); _iter252 != this->parameters.end(); ++_iter252)
+    std::map<std::string, std::string> ::const_iterator _iter253;
+    for (_iter253 = this->parameters.begin(); _iter253 != this->parameters.end(); ++_iter253)
     {
-      xfer += oprot->writeString(_iter252->first);
-      xfer += oprot->writeString(_iter252->second);
+      xfer += oprot->writeString(_iter253->first);
+      xfer += oprot->writeString(_iter253->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -5305,16 +5329,7 @@ void swap(PartitionWithoutSD &a, PartitionWithoutSD &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionWithoutSD::PartitionWithoutSD(const PartitionWithoutSD& other253) {
-  values = other253.values;
-  createTime = other253.createTime;
-  lastAccessTime = other253.lastAccessTime;
-  relativePath = other253.relativePath;
-  parameters = other253.parameters;
-  privileges = other253.privileges;
-  __isset = other253.__isset;
-}
-PartitionWithoutSD& PartitionWithoutSD::operator=(const PartitionWithoutSD& other254) {
+PartitionWithoutSD::PartitionWithoutSD(const PartitionWithoutSD& other254) {
   values = other254.values;
   createTime = other254.createTime;
   lastAccessTime = other254.lastAccessTime;
@@ -5322,6 +5337,15 @@ PartitionWithoutSD& PartitionWithoutSD::operator=(const PartitionWithoutSD& othe
   parameters = other254.parameters;
   privileges = other254.privileges;
   __isset = other254.__isset;
+}
+PartitionWithoutSD& PartitionWithoutSD::operator=(const PartitionWithoutSD& other255) {
+  values = other255.values;
+  createTime = other255.createTime;
+  lastAccessTime = other255.lastAccessTime;
+  relativePath = other255.relativePath;
+  parameters = other255.parameters;
+  privileges = other255.privileges;
+  __isset = other255.__isset;
   return *this;
 }
 void PartitionWithoutSD::printTo(std::ostream& out) const {
@@ -5374,14 +5398,14 @@ uint32_t PartitionSpecWithSharedSD::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size255;
-            ::apache::thrift::protocol::TType _etype258;
-            xfer += iprot->readListBegin(_etype258, _size255);
-            this->partitions.resize(_size255);
-            uint32_t _i259;
-            for (_i259 = 0; _i259 < _size255; ++_i259)
+            uint32_t _size256;
+            ::apache::thrift::protocol::TType _etype259;
+            xfer += iprot->readListBegin(_etype259, _size256);
+            this->partitions.resize(_size256);
+            uint32_t _i260;
+            for (_i260 = 0; _i260 < _size256; ++_i260)
             {
-              xfer += this->partitions[_i259].read(iprot);
+              xfer += this->partitions[_i260].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5418,10 +5442,10 @@ uint32_t PartitionSpecWithSharedSD::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitions.size()));
-    std::vector<PartitionWithoutSD> ::const_iterator _iter260;
-    for (_iter260 = this->partitions.begin(); _iter260 != this->partitions.end(); ++_iter260)
+    std::vector<PartitionWithoutSD> ::const_iterator _iter261;
+    for (_iter261 = this->partitions.begin(); _iter261 != this->partitions.end(); ++_iter261)
     {
-      xfer += (*_iter260).write(oprot);
+      xfer += (*_iter261).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5443,15 +5467,15 @@ void swap(PartitionSpecWithSharedSD &a, PartitionSpecWithSharedSD &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionSpecWithSharedSD::PartitionSpecWithSharedSD(const PartitionSpecWithSharedSD& other261) {
-  partitions = other261.partitions;
-  sd = other261.sd;
-  __isset = other261.__isset;
-}
-PartitionSpecWithSharedSD& PartitionSpecWithSharedSD::operator=(const PartitionSpecWithSharedSD& other262) {
+PartitionSpecWithSharedSD::PartitionSpecWithSharedSD(const PartitionSpecWithSharedSD& other262) {
   partitions = other262.partitions;
   sd = other262.sd;
   __isset = other262.__isset;
+}
+PartitionSpecWithSharedSD& PartitionSpecWithSharedSD::operator=(const PartitionSpecWithSharedSD& other263) {
+  partitions = other263.partitions;
+  sd = other263.sd;
+  __isset = other263.__isset;
   return *this;
 }
 void PartitionSpecWithSharedSD::printTo(std::ostream& out) const {
@@ -5496,14 +5520,14 @@ uint32_t PartitionListComposingSpec::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size263;
-            ::apache::thrift::protocol::TType _etype266;
-            xfer += iprot->readListBegin(_etype266, _size263);
-            this->partitions.resize(_size263);
-            uint32_t _i267;
-            for (_i267 = 0; _i267 < _size263; ++_i267)
+            uint32_t _size264;
+            ::apache::thrift::protocol::TType _etype267;
+            xfer += iprot->readListBegin(_etype267, _size264);
+            this->partitions.resize(_size264);
+            uint32_t _i268;
+            for (_i268 = 0; _i268 < _size264; ++_i268)
             {
-              xfer += this->partitions[_i267].read(iprot);
+              xfer += this->partitions[_i268].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5532,10 +5556,10 @@ uint32_t PartitionListComposingSpec::write(::apache::thrift::protocol::TProtocol
   xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitions.size()));
-    std::vector<Partition> ::const_iterator _iter268;
-    for (_iter268 = this->partitions.begin(); _iter268 != this->partitions.end(); ++_iter268)
+    std::vector<Partition> ::const_iterator _iter269;
+    for (_iter269 = this->partitions.begin(); _iter269 != this->partitions.end(); ++_iter269)
     {
-      xfer += (*_iter268).write(oprot);
+      xfer += (*_iter269).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5552,13 +5576,13 @@ void swap(PartitionListComposingSpec &a, PartitionListComposingSpec &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionListComposingSpec::PartitionListComposingSpec(const PartitionListComposingSpec& other269) {
-  partitions = other269.partitions;
-  __isset = other269.__isset;
-}
-PartitionListComposingSpec& PartitionListComposingSpec::operator=(const PartitionListComposingSpec& other270) {
+PartitionListComposingSpec::PartitionListComposingSpec(const PartitionListComposingSpec& other270) {
   partitions = other270.partitions;
   __isset = other270.__isset;
+}
+PartitionListComposingSpec& PartitionListComposingSpec::operator=(const PartitionListComposingSpec& other271) {
+  partitions = other271.partitions;
+  __isset = other271.__isset;
   return *this;
 }
 void PartitionListComposingSpec::printTo(std::ostream& out) const {
@@ -5710,21 +5734,21 @@ void swap(PartitionSpec &a, PartitionSpec &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionSpec::PartitionSpec(const PartitionSpec& other271) {
-  dbName = other271.dbName;
-  tableName = other271.tableName;
-  rootPath = other271.rootPath;
-  sharedSDPartitionSpec = other271.sharedSDPartitionSpec;
-  partitionList = other271.partitionList;
-  __isset = other271.__isset;
-}
-PartitionSpec& PartitionSpec::operator=(const PartitionSpec& other272) {
+PartitionSpec::PartitionSpec(const PartitionSpec& other272) {
   dbName = other272.dbName;
   tableName = other272.tableName;
   rootPath = other272.rootPath;
   sharedSDPartitionSpec = other272.sharedSDPartitionSpec;
   partitionList = other272.partitionList;
   __isset = other272.__isset;
+}
+PartitionSpec& PartitionSpec::operator=(const PartitionSpec& other273) {
+  dbName = other273.dbName;
+  tableName = other273.tableName;
+  rootPath = other273.rootPath;
+  sharedSDPartitionSpec = other273.sharedSDPartitionSpec;
+  partitionList = other273.partitionList;
+  __isset = other273.__isset;
   return *this;
 }
 void PartitionSpec::printTo(std::ostream& out) const {
@@ -5872,17 +5896,17 @@ uint32_t Index::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size273;
-            ::apache::thrift::protocol::TType _ktype274;
-            ::apache::thrift::protocol::TType _vtype275;
-            xfer += iprot->readMapBegin(_ktype274, _vtype275, _size273);
-            uint32_t _i277;
-            for (_i277 = 0; _i277 < _size273; ++_i277)
+            uint32_t _size274;
+            ::apache::thrift::protocol::TType _ktype275;
+            ::apache::thrift::protocol::TType _vtype276;
+            xfer += iprot->readMapBegin(_ktype275, _vtype276, _size274);
+            uint32_t _i278;
+            for (_i278 = 0; _i278 < _size274; ++_i278)
             {
-              std::string _key278;
-              xfer += iprot->readString(_key278);
-              std::string& _val279 = this->parameters[_key278];
-              xfer += iprot->readString(_val279);
+              std::string _key279;
+              xfer += iprot->readString(_key279);
+              std::string& _val280 = this->parameters[_key279];
+              xfer += iprot->readString(_val280);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5951,11 +5975,11 @@ uint32_t Index::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 9);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter280;
-    for (_iter280 = this->parameters.begin(); _iter280 != this->parameters.end(); ++_iter280)
+    std::map<std::string, std::string> ::const_iterator _iter281;
+    for (_iter281 = this->parameters.begin(); _iter281 != this->parameters.end(); ++_iter281)
     {
-      xfer += oprot->writeString(_iter280->first);
-      xfer += oprot->writeString(_iter280->second);
+      xfer += oprot->writeString(_iter281->first);
+      xfer += oprot->writeString(_iter281->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -5985,20 +6009,7 @@ void swap(Index &a, Index &b) {
   swap(a.__isset, b.__isset);
 }
 
-Index::Index(const Index& other281) {
-  indexName = other281.indexName;
-  indexHandlerClass = other281.indexHandlerClass;
-  dbName = other281.dbName;
-  origTableName = other281.origTableName;
-  createTime = other281.createTime;
-  lastAccessTime = other281.lastAccessTime;
-  indexTableName = other281.indexTableName;
-  sd = other281.sd;
-  parameters = other281.parameters;
-  deferredRebuild = other281.deferredRebuild;
-  __isset = other281.__isset;
-}
-Index& Index::operator=(const Index& other282) {
+Index::Index(const Index& other282) {
   indexName = other282.indexName;
   indexHandlerClass = other282.indexHandlerClass;
   dbName = other282.dbName;
@@ -6010,6 +6021,19 @@ Index& Index::operator=(const Index& other282) {
   parameters = other282.parameters;
   deferredRebuild = other282.deferredRebuild;
   __isset = other282.__isset;
+}
+Index& Index::operator=(const Index& other283) {
+  indexName = other283.indexName;
+  indexHandlerClass = other283.indexHandlerClass;
+  dbName = other283.dbName;
+  origTableName = other283.origTableName;
+  createTime = other283.createTime;
+  lastAccessTime = other283.lastAccessTime;
+  indexTableName = other283.indexTableName;
+  sd = other283.sd;
+  parameters = other283.parameters;
+  deferredRebuild = other283.deferredRebuild;
+  __isset = other283.__isset;
   return *this;
 }
 void Index::printTo(std::ostream& out) const {
@@ -6160,19 +6184,19 @@ void swap(BooleanColumnStatsData &a, BooleanColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-BooleanColumnStatsData::BooleanColumnStatsData(const BooleanColumnStatsData& other283) {
-  numTrues = other283.numTrues;
-  numFalses = other283.numFalses;
-  numNulls = other283.numNulls;
-  bitVectors = other283.bitVectors;
-  __isset = other283.__isset;
-}
-BooleanColumnStatsData& BooleanColumnStatsData::operator=(const BooleanColumnStatsData& other284) {
+BooleanColumnStatsData::BooleanColumnStatsData(const BooleanColumnStatsData& other284) {
   numTrues = other284.numTrues;
   numFalses = other284.numFalses;
   numNulls = other284.numNulls;
   bitVectors = other284.bitVectors;
   __isset = other284.__isset;
+}
+BooleanColumnStatsData& BooleanColumnStatsData::operator=(const BooleanColumnStatsData& other285) {
+  numTrues = other285.numTrues;
+  numFalses = other285.numFalses;
+  numNulls = other285.numNulls;
+  bitVectors = other285.bitVectors;
+  __isset = other285.__isset;
   return *this;
 }
 void BooleanColumnStatsData::printTo(std::ostream& out) const {
@@ -6335,21 +6359,21 @@ void swap(DoubleColumnStatsData &a, DoubleColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-DoubleColumnStatsData::DoubleColumnStatsData(const DoubleColumnStatsData& other285) {
-  lowValue = other285.lowValue;
-  highValue = other285.highValue;
-  numNulls = other285.numNulls;
-  numDVs = other285.numDVs;
-  bitVectors = other285.bitVectors;
-  __isset = other285.__isset;
-}
-DoubleColumnStatsData& DoubleColumnStatsData::operator=(const DoubleColumnStatsData& other286) {
+DoubleColumnStatsData::DoubleColumnStatsData(const DoubleColumnStatsData& other286) {
   lowValue = other286.lowValue;
   highValue = other286.highValue;
   numNulls = other286.numNulls;
   numDVs = other286.numDVs;
   bitVectors = other286.bitVectors;
   __isset = other286.__isset;
+}
+DoubleColumnStatsData& DoubleColumnStatsData::operator=(const DoubleColumnStatsData& other287) {
+  lowValue = other287.lowValue;
+  highValue = other287.highValue;
+  numNulls = other287.numNulls;
+  numDVs = other287.numDVs;
+  bitVectors = other287.bitVectors;
+  __isset = other287.__isset;
   return *this;
 }
 void DoubleColumnStatsData::printTo(std::ostream& out) const {
@@ -6513,21 +6537,21 @@ void swap(LongColumnStatsData &a, LongColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-LongColumnStatsData::LongColumnStatsData(const LongColumnStatsData& other287) {
-  lowValue = other287.lowValue;
-  highValue = other287.highValue;
-  numNulls = other287.numNulls;
-  numDVs = other287.numDVs;
-  bitVectors = other287.bitVectors;
-  __isset = other287.__isset;
-}
-LongColumnStatsData& LongColumnStatsData::operator=(const LongColumnStatsData& other288) {
+LongColumnStatsData::LongColumnStatsData(const LongColumnStatsData& other288) {
   lowValue = other288.lowValue;
   highValue = other288.highValue;
   numNulls = other288.numNulls;
   numDVs = other288.numDVs;
   bitVectors = other288.bitVectors;
   __isset = other288.__isset;
+}
+LongColumnStatsData& LongColumnStatsData::operator=(const LongColumnStatsData& other289) {
+  lowValue = other289.lowValue;
+  highValue = other289.highValue;
+  numNulls = other289.numNulls;
+  numDVs = other289.numDVs;
+  bitVectors = other289.bitVectors;
+  __isset = other289.__isset;
   return *this;
 }
 void LongColumnStatsData::printTo(std::ostream& out) const {
@@ -6693,21 +6717,21 @@ void swap(StringColumnStatsData &a, StringColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-StringColumnStatsData::StringColumnStatsData(const StringColumnStatsData& other289) {
-  maxColLen = other289.maxColLen;
-  avgColLen = other289.avgColLen;
-  numNulls = other289.numNulls;
-  numDVs = other289.numDVs;
-  bitVectors = other289.bitVectors;
-  __isset = other289.__isset;
-}
-StringColumnStatsData& StringColumnStatsData::operator=(const StringColumnStatsData& other290) {
+StringColumnStatsData::StringColumnStatsData(const StringColumnStatsData& other290) {
   maxColLen = other290.maxColLen;
   avgColLen = other290.avgColLen;
   numNulls = other290.numNulls;
   numDVs = other290.numDVs;
   bitVectors = other290.bitVectors;
   __isset = other290.__isset;
+}
+StringColumnStatsData& StringColumnStatsData::operator=(const StringColumnStatsData& other291) {
+  maxColLen = other291.maxColLen;
+  avgColLen = other291.avgColLen;
+  numNulls = other291.numNulls;
+  numDVs = other291.numDVs;
+  bitVectors = other291.bitVectors;
+  __isset = other291.__isset;
   return *this;
 }
 void StringColumnStatsData::printTo(std::ostream& out) const {
@@ -6853,19 +6877,19 @@ void swap(BinaryColumnStatsData &a, BinaryColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-BinaryColumnStatsData::BinaryColumnStatsData(const BinaryColumnStatsData& other291) {
-  maxColLen = other291.maxColLen;
-  avgColLen = other291.avgColLen;
-  numNulls = other291.numNulls;
-  bitVectors = other291.bitVectors;
-  __isset = other291.__isset;
-}
-BinaryColumnStatsData& BinaryColumnStatsData::operator=(const BinaryColumnStatsData& other292) {
+BinaryColumnStatsData::BinaryColumnStatsData(const BinaryColumnStatsData& other292) {
   maxColLen = other292.maxColLen;
   avgColLen = other292.avgColLen;
   numNulls = other292.numNulls;
   bitVectors = other292.bitVectors;
   __isset = other292.__isset;
+}
+BinaryColumnStatsData& BinaryColumnStatsData::operator=(const BinaryColumnStatsData& other293) {
+  maxColLen = other293.maxColLen;
+  avgColLen = other293.avgColLen;
+  numNulls = other293.numNulls;
+  bitVectors = other293.bitVectors;
+  __isset = other293.__isset;
   return *this;
 }
 void BinaryColumnStatsData::printTo(std::ostream& out) const {
@@ -6970,13 +6994,13 @@ void swap(Decimal &a, Decimal &b) {
   swap(a.scale, b.scale);
 }
 
-Decimal::Decimal(const Decimal& other293) {
-  unscaled = other293.unscaled;
-  scale = other293.scale;
-}
-Decimal& Decimal::operator=(const Decimal& other294) {
+Decimal::Decimal(const Decimal& other294) {
   unscaled = other294.unscaled;
   scale = other294.scale;
+}
+Decimal& Decimal::operator=(const Decimal& other295) {
+  unscaled = other295.unscaled;
+  scale = other295.scale;
   return *this;
 }
 void Decimal::printTo(std::ostream& out) const {
@@ -7137,21 +7161,21 @@ void swap(DecimalColumnStatsData &a, DecimalColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-DecimalColumnStatsData::DecimalColumnStatsData(const DecimalColumnStatsData& other295) {
-  lowValue = other295.lowValue;
-  highValue = other295.highValue;
-  numNulls = other295.numNulls;
-  numDVs = other295.numDVs;
-  bitVectors = other295.bitVectors;
-  __isset = other295.__isset;
-}
-DecimalColumnStatsData& DecimalColumnStatsData::operator=(const DecimalColumnStatsData& other296) {
+DecimalColumnStatsData::DecimalColumnStatsData(const DecimalColumnStatsData& other296) {
   lowValue = other296.lowValue;
   highValue = other296.highValue;
   numNulls = other296.numNulls;
   numDVs = other296.numDVs;
   bitVectors = other296.bitVectors;
   __isset = other296.__isset;
+}
+DecimalColumnStatsData& DecimalColumnStatsData::operator=(const DecimalColumnStatsData& other297) {
+  lowValue = other297.lowValue;
+  highValue = other297.highValue;
+  numNulls = other297.numNulls;
+  numDVs = other297.numDVs;
+  bitVectors = other297.bitVectors;
+  __isset = other297.__isset;
   return *this;
 }
 void DecimalColumnStatsData::printTo(std::ostream& out) const {
@@ -7237,11 +7261,11 @@ void swap(Date &a, Date &b) {
   swap(a.daysSinceEpoch, b.daysSinceEpoch);
 }
 
-Date::Date(const Date& other297) {
-  daysSinceEpoch = other297.daysSinceEpoch;
-}
-Date& Date::operator=(const Date& other298) {
+Date::Date(const Date& other298) {
   daysSinceEpoch = other298.daysSinceEpoch;
+}
+Date& Date::operator=(const Date& other299) {
+  daysSinceEpoch = other299.daysSinceEpoch;
   return *this;
 }
 void Date::printTo(std::ostream& out) const {
@@ -7401,21 +7425,21 @@ void swap(DateColumnStatsData &a, DateColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-DateColumnStatsData::DateColumnStatsData(const DateColumnStatsData& other299) {
-  lowValue = other299.lowValue;
-  highValue = other299.highValue;
-  numNulls = other299.numNulls;
-  numDVs = other299.numDVs;
-  bitVectors = other299.bitVectors;
-  __isset = other299.__isset;
-}
-DateColumnStatsData& DateColumnStatsData::operator=(const DateColumnStatsData& other300) {
+DateColumnStatsData::DateColumnStatsData(const DateColumnStatsData& other300) {
   lowValue = other300.lowValue;
   highValue = other300.highValue;
   numNulls = other300.numNulls;
   numDVs = other300.numDVs;
   bitVectors = other300.bitVectors;
   __isset = other300.__isset;
+}
+DateColumnStatsData& DateColumnStatsData::operator=(const DateColumnStatsData& other301) {
+  lowValue = other301.lowValue;
+  highValue = other301.highValue;
+  numNulls = other301.numNulls;
+  numDVs = other301.numDVs;
+  bitVectors = other301.bitVectors;
+  __isset = other301.__isset;
   return *this;
 }
 void DateColumnStatsData::printTo(std::ostream& out) const {
@@ -7601,17 +7625,7 @@ void swap(ColumnStatisticsData &a, ColumnStatisticsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-ColumnStatisticsData::ColumnStatisticsData(const ColumnStatisticsData& other301) {
-  booleanStats = other301.booleanStats;
-  longStats = other301.longStats;
-  doubleStats = other301.doubleStats;
-  stringStats = other301.stringStats;
-  binaryStats = other301.binaryStats;
-  decimalStats = other301.decimalStats;
-  dateStats = other301.dateStats;
-  __isset = other301.__isset;
-}
-ColumnStatisticsData& ColumnStatisticsData::operator=(const ColumnStatisticsData& other302) {
+ColumnStatisticsData::ColumnStatisticsData(const ColumnStatisticsData& other302) {
   booleanStats = other302.booleanStats;
   longStats = other302.longStats;
   doubleStats = other302.doubleStats;
@@ -7620,6 +7634,16 @@ ColumnStatisticsData& ColumnStatisticsData::operator=(const ColumnStatisticsData
   decimalStats = other302.decimalStats;
   dateStats = other302.dateStats;
   __isset = other302.__isset;
+}
+ColumnStatisticsData& ColumnStatisticsData::operator=(const ColumnStatisticsData& other303) {
+  booleanStats = other303.booleanStats;
+  longStats = other303.longStats;
+  doubleStats = other303.doubleStats;
+  stringStats = other303.stringStats;
+  binaryStats = other303.binaryStats;
+  decimalStats = other303.decimalStats;
+  dateStats = other303.dateStats;
+  __isset = other303.__isset;
   return *this;
 }
 void ColumnStatisticsData::printTo(std::ostream& out) const {
@@ -7747,15 +7771,15 @@ void swap(ColumnStatisticsObj &a, ColumnStatisticsObj &b) {
   swap(a.statsData, b.statsData);
 }
 
-ColumnStatisticsObj::ColumnStatisticsObj(const ColumnStatisticsObj& other303) {
-  colName = other303.colName;
-  colType = other303.colType;
-  statsData = other303.statsData;
-}
-ColumnStatisticsObj& ColumnStatisticsObj::operator=(const ColumnStatisticsObj& other304) {
+ColumnStatisticsObj::ColumnStatisticsObj(const ColumnStatisticsObj& other304) {
   colName = other304.colName;
   colType = other304.colType;
   statsData = other304.statsData;
+}
+ColumnStatisticsObj& ColumnStatisticsObj::operator=(const ColumnStatisticsObj& other305) {
+  colName = other305.colName;
+  colType = other305.colType;
+  statsData = other305.statsData;
   return *this;
 }
 void ColumnStatisticsObj::printTo(std::ostream& out) const {
@@ -7918,21 +7942,21 @@ void swap(ColumnStatisticsDesc &a, ColumnStatisticsDesc &b) {
   swap(a.__isset, b.__isset);
 }
 
-ColumnStatisticsDesc::ColumnStatisticsDesc(const ColumnStatisticsDesc& other305) {
-  isTblLevel = other305.isTblLevel;
-  dbName = other305.dbName;
-  tableName = other305.tableName;
-  partName = other305.partName;
-  lastAnalyzed = other305.lastAnalyzed;
-  __isset = other305.__isset;
-}
-ColumnStatisticsDesc& ColumnStatisticsDesc::operator=(const ColumnStatisticsDesc& other306) {
+ColumnStatisticsDesc::ColumnStatisticsDesc(const ColumnStatisticsDesc& other306) {
   isTblLevel = other306.isTblLevel;
   dbName = other306.dbName;
   tableName = other306.tableName;
   partName = other306.partName;
   lastAnalyzed = other306.lastAnalyzed;
   __isset = other306.__isset;
+}
+ColumnStatisticsDesc& ColumnStatisticsDesc::operator=(const ColumnStatisticsDesc& other307) {
+  isTblLevel = other307.isTblLevel;
+  dbName = other307.dbName;
+  tableName = other307.tableName;
+  partName = other307.partName;
+  lastAnalyzed = other307.lastAnalyzed;
+  __isset = other307.__isset;
   return *this;
 }
 void ColumnStatisticsDesc::printTo(std::ostream& out) const {
@@ -7994,14 +8018,14 @@ uint32_t ColumnStatistics::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->statsObj.clear();
-            uint32_t _size307;
-            ::apache::thrift::protocol::TType _etype310;
-            xfer += iprot->readListBegin(_etype310, _size307);
-            this->statsObj.resize(_size307);
-            uint32_t _i311;
-            for (_i311 = 0; _i311 < _size307; ++_i311)
+            uint32_t _size308;
+            ::apache::thrift::protocol::TType _etype311;
+            xfer += iprot->readListBegin(_etype311, _size308);
+            this->statsObj.resize(_size308);
+            uint32_t _i312;
+            for (_i312 = 0; _i312 < _size308; ++_i312)
             {
-              xfer += this->statsObj[_i311].read(iprot);
+              xfer += this->statsObj[_i312].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8038,10 +8062,10 @@ uint32_t ColumnStatistics::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("statsObj", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->statsObj.size()));
-    std::vector<ColumnStatisticsObj> ::const_iterator _iter312;
-    for (_iter312 = this->statsObj.begin(); _iter312 != this->statsObj.end(); ++_iter312)
+    std::vector<ColumnStatisticsObj> ::const_iterator _iter313;
+    for (_iter313 = this->statsObj.begin(); _iter313 != this->statsObj.end(); ++_iter313)
     {
-      xfer += (*_iter312).write(oprot);
+      xfer += (*_iter313).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8058,13 +8082,13 @@ void swap(ColumnStatistics &a, ColumnStatistics &b) {
   swap(a.statsObj, b.statsObj);
 }
 
-ColumnStatistics::ColumnStatistics(const ColumnStatistics& other313) {
-  statsDesc = other313.statsDesc;
-  statsObj = other313.statsObj;
-}
-ColumnStatistics& ColumnStatistics::operator=(const ColumnStatistics& other314) {
+ColumnStatistics::ColumnStatistics(const ColumnStatistics& other314) {
   statsDesc = other314.statsDesc;
   statsObj = other314.statsObj;
+}
+ColumnStatistics& ColumnStatistics::operator=(const ColumnStatistics& other315) {
+  statsDesc = other315.statsDesc;
+  statsObj = other315.statsObj;
   return *this;
 }
 void ColumnStatistics::printTo(std::ostream& out) const {
@@ -8115,14 +8139,14 @@ uint32_t AggrStats::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->colStats.clear();
-            uint32_t _size315;
-            ::apache::thrift::protocol::TType _etype318;
-            xfer += iprot->readListBegin(_etype318, _size315);
-            this->colStats.resize(_size315);
-            uint32_t _i319;
-            for (_i319 = 0; _i319 < _size315; ++_i319)
+            uint32_t _size316;
+            ::apache::thrift::protocol::TType _etype319;
+            xfer += iprot->readListBegin(_etype319, _size316);
+            this->colStats.resize(_size316);
+            uint32_t _i320;
+            for (_i320 = 0; _i320 < _size316; ++_i320)
             {
-              xfer += this->colStats[_i319].read(iprot);
+              xfer += this->colStats[_i320].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8163,10 +8187,10 @@ uint32_t AggrStats::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("colStats", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->colStats.size()));
-    std::vector<ColumnStatisticsObj> ::const_iterator _iter320;
-    for (_iter320 = this->colStats.begin(); _iter320 != this->colStats.end(); ++_iter320)
+    std::vector<ColumnStatisticsObj> ::const_iterator _iter321;
+    for (_iter321 = this->colStats.begin(); _iter321 != this->colStats.end(); ++_iter321)
     {
-      xfer += (*_iter320).write(oprot);
+      xfer += (*_iter321).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8187,13 +8211,13 @@ void swap(AggrStats &a, AggrStats &b) {
   swap(a.partsFound, b.partsFound);
 }
 
-AggrStats::AggrStats(const AggrStats& other321) {
-  colStats = other321.colStats;
-  partsFound = other321.partsFound;
-}
-AggrStats& AggrStats::operator=(const AggrStats& other322) {
+AggrStats::AggrStats(const AggrStats& other322) {
   colStats = other322.colStats;
   partsFound = other322.partsFound;
+}
+AggrStats& AggrStats::operator=(const AggrStats& other323) {
+  colStats = other323.colStats;
+  partsFound = other323.partsFound;
   return *this;
 }
 void AggrStats::printTo(std::ostream& out) const {
@@ -8244,14 +8268,14 @@ uint32_t SetPartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->colStats.clear();
-            uint32_t _size323;
-            ::apache::thrift::protocol::TType _etype326;
-            xfer += iprot->readListBegin(_etype326, _size323);
-            this->colStats.resize(_size323);
-            uint32_t _i327;
-            for (_i327 = 0; _i327 < _size323; ++_i327)
+            uint32_t _size324;
+            ::apache::thrift::protocol::TType _etype327;
+            xfer += iprot->readListBegin(_etype327, _size324);
+            this->colStats.resize(_size324);
+            uint32_t _i328;
+            for (_i328 = 0; _i328 < _size324; ++_i328)
             {
-              xfer += this->colStats[_i327].read(iprot);
+              xfer += this->colStats[_i328].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8290,10 +8314,10 @@ uint32_t SetPartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("colStats", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->colStats.size()));
-    std::vector<ColumnStatistics> ::const_iterator _iter328;
-    for (_iter328 = this->colStats.begin(); _iter328 != this->colStats.end(); ++_iter328)
+    std::vector<ColumnStatistics> ::const_iterator _iter329;
+    for (_iter329 = this->colStats.begin(); _iter329 != this->colStats.end(); ++_iter329)
     {
-      xfer += (*_iter328).write(oprot);
+      xfer += (*_iter329).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8316,15 +8340,15 @@ void swap(SetPartitionsStatsRequest &a, SetPartitionsStatsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-SetPartitionsStatsRequest::SetPartitionsStatsRequest(const SetPartitionsStatsRequest& other329) {
-  colStats = other329.colStats;
-  needMerge = other329.needMerge;
-  __isset = other329.__isset;
-}
-SetPartitionsStatsRequest& SetPartitionsStatsRequest::operator=(const SetPartitionsStatsRequest& other330) {
+SetPartitionsStatsRequest::SetPartitionsStatsRequest(const SetPartitionsStatsRequest& other330) {
   colStats = other330.colStats;
   needMerge = other330.needMerge;
   __isset = other330.__isset;
+}
+SetPartitionsStatsRequest& SetPartitionsStatsRequest::operator=(const SetPartitionsStatsRequest& other331) {
+  colStats = other331.colStats;
+  needMerge = other331.needMerge;
+  __isset = other331.__isset;
   return *this;
 }
 void SetPartitionsStatsRequest::printTo(std::ostream& out) const {
@@ -8373,14 +8397,14 @@ uint32_t Schema::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fieldSchemas.clear();
-            uint32_t _size331;
-            ::apache::thrift::protocol::TType _etype334;
-            xfer += iprot->readListBegin(_etype334, _size331);
-            this->fieldSchemas.resize(_size331);
-            uint32_t _i335;
-            for (_i335 = 0; _i335 < _size331; ++_i335)
+            uint32_t _size332;
+            ::apache::thrift::protocol::TType _etype335;
+            xfer += iprot->readListBegin(_etype335, _size332);
+            this->fieldSchemas.resize(_size332);
+            uint32_t _i336;
+            for (_i336 = 0; _i336 < _size332; ++_i336)
             {
-              xfer += this->fieldSchemas[_i335].read(iprot);
+              xfer += this->fieldSchemas[_i336].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8393,17 +8417,17 @@ uint32_t Schema::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->properties.clear();
-            uint32_t _size336;
-            ::apache::thrift::protocol::TType _ktype337;
-            ::apache::thrift::protocol::TType _vtype338;
-            xfer += iprot->readMapBegin(_ktype337, _vtype338, _size336);
-            uint32_t _i340;
-            for (_i340 = 0; _i340 < _size336; ++_i340)
+            uint32_t _size337;
+            ::apache::thrift::protocol::TType _ktype338;
+            ::apache::thrift::protocol::TType _vtype339;
+            xfer += iprot->readMapBegin(_ktype338, _vtype339, _size337);
+            uint32_t _i341;
+            for (_i341 = 0; _i341 < _size337; ++_i341)
             {
-              std::string _key341;
-              xfer += iprot->readString(_key341);
-              std::string& _val342 = this->properties[_key341];
-              xfer += iprot->readString(_val342);
+              std::string _key342;
+              xfer += iprot->readString(_key342);
+              std::string& _val343 = this->properties[_key342];
+              xfer += iprot->readString(_val343);
             }
             xfer += iprot->readMapEnd();
           }
@@ -8432,10 +8456,10 @@ uint32_t Schema::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("fieldSchemas", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->fieldSchemas.size()));
-    std::vector<FieldSchema> ::const_iterator _iter343;
-    for (_iter343 = this->fieldSchemas.begin(); _iter343 != this->fieldSchemas.end(); ++_iter343)
+    std::vector<FieldSchema> ::const_iterator _iter344;
+    for (_iter344 = this->fieldSchemas.begin(); _iter344 != this->fieldSchemas.end(); ++_iter344)
     {
-      xfer += (*_iter343).write(oprot);
+      xfer += (*_iter344).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8444,11 +8468,11 @@ uint32_t Schema::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 2);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->properties.size()));
-    std::map<std::string, std::string> ::const_iterator _iter344;
-    for (_iter344 = this->properties.begin(); _iter344 != this->properties.end(); ++_iter344)
+    std::map<std::string, std::string> ::const_iterator _iter345;
+    for (_iter345 = this->properties.begin(); _iter345 != this->properties.end(); ++_iter345)
     {
-      xfer += oprot->writeString(_iter344->first);
-      xfer += oprot->writeString(_iter344->second);
+      xfer += oprot->writeString(_iter345->first);
+      xfer += oprot->writeString(_iter345->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -8466,15 +8490,15 @@ void swap(Schema &a, Schema &b) {
   swap(a.__isset, b.__isset);
 }
 
-Schema::Schema(const Schema& other345) {
-  fieldSchemas = other345.fieldSchemas;
-  properties = other345.properties;
-  __isset = other345.__isset;
-}
-Schema& Schema::operator=(const Schema& other346) {
+Schema::Schema(const Schema& other346) {
   fieldSchemas = other346.fieldSchemas;
   properties = other346.properties;
   __isset = other346.__isset;
+}
+Schema& Schema::operator=(const Schema& other347) {
+  fieldSchemas = other347.fieldSchemas;
+  properties = other347.properties;
+  __isset = other347.__isset;
   return *this;
 }
 void Schema::printTo(std::ostream& out) const {
@@ -8519,17 +8543,17 @@ uint32_t EnvironmentContext::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->properties.clear();
-            uint32_t _size347;
-            ::apache::thrift::protocol::TType _ktype348;
-            ::apache::thrift::protocol::TType _vtype349;
-            xfer += iprot->readMapBegin(_ktype348, _vtype349, _size347);
-            uint32_t _i351;
-            for (_i351 = 0; _i351 < _size347; ++_i351)
+            uint32_t _size348;
+            ::apache::thrift::protocol::TType _ktype349;
+            ::apache::thrift::protocol::TType _vtype350;
+            xfer += iprot->readMapBegin(_ktype349, _vtype350, _size348);
+            uint32_t _i352;
+            for (_i352 = 0; _i352 < _size348; ++_i352)
             {
-              std::string _key352;
-              xfer += iprot->readString(_key352);
-              std::string& _val353 = this->properties[_key352];
-              xfer += iprot->readString(_val353);
+              std::string _key353;
+              xfer += iprot->readString(_key353);
+              std::string& _val354 = this->properties[_key353];
+              xfer += iprot->readString(_val354);
             }
             xfer += iprot->readMapEnd();
           }
@@ -8558,11 +8582,11 @@ uint32_t EnvironmentContext::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->properties.size()));
-    std::map<std::string, std::string> ::const_iterator _iter354;
-    for (_iter354 = this->properties.begin(); _iter354 != this->properties.end(); ++_iter354)
+    std::map<std::string, std::string> ::const_iterator _iter355;
+    for (_iter355 = this->properties.begin(); _iter355 != this->properties.end(); ++_iter355)
     {
-      xfer += oprot->writeString(_iter354->first);
-      xfer += oprot->writeString(_iter354->second);
+      xfer += oprot->writeString(_iter355->first);
+      xfer += oprot->writeString(_iter355->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -8579,13 +8603,13 @@ void swap(EnvironmentContext &a, EnvironmentContext &b) {
   swap(a.__isset, b.__isset);
 }
 
-EnvironmentContext::EnvironmentContext(const EnvironmentContext& other355) {
-  properties = other355.properties;
-  __isset = other355.__isset;
-}
-EnvironmentContext& EnvironmentContext::operator=(const EnvironmentContext& other356) {
+EnvironmentContext::EnvironmentContext(const EnvironmentContext& other356) {
   properties = other356.properties;
   __isset = other356.__isset;
+}
+EnvironmentContext& EnvironmentContext::operator=(const EnvironmentContext& other357) {
+  properties = other357.properties;
+  __isset = other357.__isset;
   return *this;
 }
 void EnvironmentContext::printTo(std::ostream& out) const {
@@ -8687,13 +8711,13 @@ void swap(PrimaryKeysRequest &a, PrimaryKeysRequest &b) {
   swap(a.tbl_name, b.tbl_name);
 }
 
-PrimaryKeysRequest::PrimaryKeysRequest(const PrimaryKeysRequest& other357) {
-  db_name = other357.db_name;
-  tbl_name = other357.tbl_name;
-}
-PrimaryKeysRequest& PrimaryKeysRequest::operator=(const PrimaryKeysRequest& other358) {
+PrimaryKeysRequest::PrimaryKeysRequest(const PrimaryKeysRequest& other358) {
   db_name = other358.db_name;
   tbl_name = other358.tbl_name;
+}
+PrimaryKeysRequest& PrimaryKeysRequest::operator=(const PrimaryKeysRequest& other359) {
+  db_name = other359.db_name;
+  tbl_name = other359.tbl_name;
   return *this;
 }
 void PrimaryKeysRequest::printTo(std::ostream& out) const {
@@ -8739,14 +8763,14 @@ uint32_t PrimaryKeysResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size359;
-            ::apache::thrift::protocol::TType _etype362;
-            xfer += iprot->readListBegin(_etype362, _size359);
-            this->primaryKeys.resize(_size359);
-            uint32_t _i363;
-            for (_i363 = 0; _i363 < _size359; ++_i363)
+            uint32_t _size360;
+            ::apache::thrift::protocol::TType _etype363;
+            xfer += iprot->readListBegin(_etype363, _size360);
+            this->primaryKeys.resize(_size360);
+            uint32_t _i364;
+            for (_i364 = 0; _i364 < _size360; ++_i364)
             {
-              xfer += this->primaryKeys[_i363].read(iprot);
+              xfer += this->primaryKeys[_i364].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8777,10 +8801,10 @@ uint32_t PrimaryKeysResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter364;
-    for (_iter364 = this->primaryKeys.begin(); _iter364 != this->primaryKeys.end(); ++_iter364)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter365;
+    for (_iter365 = this->primaryKeys.begin(); _iter365 != this->primaryKeys.end(); ++_iter365)
     {
-      xfer += (*_iter364).write(oprot);
+      xfer += (*_iter365).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8796,11 +8820,11 @@ void swap(PrimaryKeysResponse &a, PrimaryKeysResponse &b) {
   swap(a.primaryKeys, b.primaryKeys);
 }
 
-PrimaryKeysResponse::PrimaryKeysResponse(const PrimaryKeysResponse& other365) {
-  primaryKeys = other365.primaryKeys;
-}
-PrimaryKeysResponse& PrimaryKeysResponse::operator=(const PrimaryKeysResponse& other366) {
+PrimaryKeysResponse::PrimaryKeysResponse(const PrimaryKeysResponse& other366) {
   primaryKeys = other366.primaryKeys;
+}
+PrimaryKeysResponse& PrimaryKeysResponse::operator=(const PrimaryKeysResponse& other367) {
+  primaryKeys = other367.primaryKeys;
   return *this;
 }
 void PrimaryKeysResponse::printTo(std::ostream& out) const {
@@ -8931,19 +8955,19 @@ void swap(ForeignKeysRequest &a, ForeignKeysRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ForeignKeysRequest::ForeignKeysRequest(const ForeignKeysRequest& other367) {
-  parent_db_name = other367.parent_db_name;
-  parent_tbl_name = other367.parent_tbl_name;
-  foreign_db_name = other367.foreign_db_name;
-  foreign_tbl_name = other367.foreign_tbl_name;
-  __isset = other367.__isset;
-}
-ForeignKeysRequest& ForeignKeysRequest::operator=(const ForeignKeysRequest& other368) {
+ForeignKeysRequest::ForeignKeysRequest(const ForeignKeysRequest& other368) {
   parent_db_name = other368.parent_db_name;
   parent_tbl_name = other368.parent_tbl_name;
   foreign_db_name = other368.foreign_db_name;
   foreign_tbl_name = other368.foreign_tbl_name;
   __isset = other368.__isset;
+}
+ForeignKeysRequest& ForeignKeysRequest::operator=(const ForeignKeysRequest& other369) {
+  parent_db_name = other369.parent_db_name;
+  parent_tbl_name = other369.parent_tbl_name;
+  foreign_db_name = other369.foreign_db_name;
+  foreign_tbl_name = other369.foreign_tbl_name;
+  __isset = other369.__isset;
   return *this;
 }
 void ForeignKeysRequest::printTo(std::ostream& out) const {
@@ -8991,14 +9015,14 @@ uint32_t ForeignKeysResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size369;
-            ::apache::thrift::protocol::TType _etype372;
-            xfer += iprot->readListBegin(_etype372, _size369);
-            this->foreignKeys.resize(_size369);
-            uint32_t _i373;
-            for (_i373 = 0; _i373 < _size369; ++_i373)
+            uint32_t _size370;
+            ::apache::thrift::protocol::TType _etype373;
+            xfer += iprot->readListBegin(_etype373, _size370);
+            this->foreignKeys.resize(_size370);
+            uint32_t _i374;
+            for (_i374 = 0; _i374 < _size370; ++_i374)
             {
-              xfer += this->foreignKeys[_i373].read(iprot);
+              xfer += this->foreignKeys[_i374].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9029,10 +9053,10 @@ uint32_t ForeignKeysResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter374;
-    for (_iter374 = this->foreignKeys.begin(); _iter374 != this->foreignKeys.end(); ++_iter374)
+    std::vector<SQLForeignKey> ::const_iterator _iter375;
+    for (_iter375 = this->foreignKeys.begin(); _iter375 != this->foreignKeys.end(); ++_iter375)
     {
-      xfer += (*_iter374).write(oprot);
+      xfer += (*_iter375).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9048,11 +9072,11 @@ void swap(ForeignKeysResponse &a, ForeignKeysResponse &b) {
   swap(a.foreignKeys, b.foreignKeys);
 }
 
-ForeignKeysResponse::ForeignKeysResponse(const ForeignKeysResponse& other375) {
-  foreignKeys = other375.foreignKeys;
-}
-ForeignKeysResponse& ForeignKeysResponse::operator=(const ForeignKeysResponse& other376) {
+ForeignKeysResponse::ForeignKeysResponse(const ForeignKeysResponse& other376) {
   foreignKeys = other376.foreignKeys;
+}
+ForeignKeysResponse& ForeignKeysResponse::operator=(const ForeignKeysResponse& other377) {
+  foreignKeys = other377.foreignKeys;
   return *this;
 }
 void ForeignKeysResponse::printTo(std::ostream& out) const {
@@ -9174,15 +9198,15 @@ void swap(DropConstraintRequest &a, DropConstraintRequest &b) {
   swap(a.constraintname, b.constraintname);
 }
 
-DropConstraintRequest::DropConstraintRequest(const DropConstraintRequest& other377) {
-  dbname = other377.dbname;
-  tablename = other377.tablename;
-  constraintname = other377.constraintname;
-}
-DropConstraintRequest& DropConstraintRequest::operator=(const DropConstraintRequest& other378) {
+DropConstraintRequest::DropConstraintRequest(const DropConstraintRequest& other378) {
   dbname = other378.dbname;
   tablename = other378.tablename;
   constraintname = other378.constraintname;
+}
+DropConstraintRequest& DropConstraintRequest::operator=(const DropConstraintRequest& other379) {
+  dbname = other379.dbname;
+  tablename = other379.tablename;
+  constraintname = other379.constraintname;
   return *this;
 }
 void DropConstraintRequest::printTo(std::ostream& out) const {
@@ -9229,14 +9253,14 @@ uint32_t AddPrimaryKeyRequest::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeyCols.clear();
-            uint32_t _size379;
-            ::apache::thrift::protocol::TType _etype382;
-            xfer += iprot->readListBegin(_etype382, _size379);
-            this->primaryKeyCols.resize(_size379);
-            uint32_t _i383;
-            for (_i383 = 0; _i383 < _size379; ++_i383)
+            uint32_t _size380;
+            ::apache::thrift::protocol::TType _etype383;
+            xfer += iprot->readListBegin(_etype383, _size380);
+            this->primaryKeyCols.resize(_size380);
+            uint32_t _i384;
+            for (_i384 = 0; _i384 < _size380; ++_i384)
             {
-              xfer += this->primaryKeyCols[_i383].read(iprot);
+              xfer += this->primaryKeyCols[_i384].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9267,10 +9291,10 @@ uint32_t AddPrimaryKeyRequest::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("primaryKeyCols", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeyCols.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter384;
-    for (_iter384 = this->primaryKeyCols.begin(); _iter384 != this->primaryKeyCols.end(); ++_iter384)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter385;
+    for (_iter385 = this->primaryKeyCols.begin(); _iter385 != this->primaryKeyCols.end(); ++_iter385)
     {
-      xfer += (*_iter384).write(oprot);
+      xfer += (*_iter385).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9286,11 +9310,11 @@ void swap(AddPrimaryKeyRequest &a, AddPrimaryKeyRequest &b) {
   swap(a.primaryKeyCols, b.primaryKeyCols);
 }
 
-AddPrimaryKeyRequest::AddPrimaryKeyRequest(const AddPrimaryKeyRequest& other385) {
-  primaryKeyCols = other385.primaryKeyCols;
-}
-AddPrimaryKeyRequest& AddPrimaryKeyRequest::operator=(const AddPrimaryKeyRequest& other386) {
+AddPrimaryKeyRequest::AddPrimaryKeyRequest(const AddPrimaryKeyRequest& other386) {
   primaryKeyCols = other386.primaryKeyCols;
+}
+AddPrimaryKeyRequest& AddPrimaryKeyRequest::operator=(const AddPrimaryKeyRequest& other387) {
+  primaryKeyCols = other387.primaryKeyCols;
   return *this;
 }
 void AddPrimaryKeyRequest::printTo(std::ostream& out) const {
@@ -9335,14 +9359,14 @@ uint32_t AddForeignKeyRequest::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeyCols.clear();
-            uint32_t _size387;
-            ::apache::thrift::protocol::TType _etype390;
-            xfer += iprot->readListBegin(_etype390, _size387);
-            this->foreignKeyCols.resize(_size387);
-            uint32_t _i391;
-            for (_i391 = 0; _i391 < _size387; ++_i391)
+            uint32_t _size388;
+            ::apache::thrift::protocol::TType _etype391;
+            xfer += iprot->readListBegin(_etype391, _size388);
+            this->foreignKeyCols.resize(_size388);
+            uint32_t _i392;
+            for (_i392 = 0; _i392 < _size388; ++_i392)
             {
-              xfer += this->foreignKeyCols[_i391].read(iprot);
+              xfer += this->foreignKeyCols[_i392].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9373,10 +9397,10 @@ uint32_t AddForeignKeyRequest::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("foreignKeyCols", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeyCols.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter392;
-    for (_iter392 = this->foreignKeyCols.begin(); _iter392 != this->foreignKeyCols.end(); ++_iter392)
+    std::vector<SQLForeignKey> ::const_iterator _iter393;
+    for (_iter393 = this->foreignKeyCols.begin(); _iter393 != this->foreignKeyCols.end(); ++_iter393)
     {
-      xfer += (*_iter392).write(oprot);
+      xfer += (*_iter393).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9392,11 +9416,11 @@ void swap(AddForeignKeyRequest &a, AddForeignKeyRequest &b) {
   swap(a.foreignKeyCols, b.foreignKeyCols);
 }
 
-AddForeignKeyRequest::AddForeignKeyRequest(const AddForeignKeyRequest& other393) {
-  foreignKeyCols = other393.foreignKeyCols;
-}
-AddForeignKeyRequest& AddForeignKeyRequest::operator=(const AddForeignKeyRequest& other394) {
+AddForeignKeyRequest::AddForeignKeyRequest(const AddForeignKeyRequest& other394) {
   foreignKeyCols = other394.foreignKeyCols;
+}
+AddForeignKeyRequest& AddForeignKeyRequest::operator=(const AddForeignKeyRequest& other395) {
+  foreignKeyCols = other395.foreignKeyCols;
   return *this;
 }
 void AddForeignKeyRequest::printTo(std::ostream& out) const {
@@ -9446,14 +9470,14 @@ uint32_t PartitionsByExprResult::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size395;
-            ::apache::thrift::protocol::TType _etype398;
-            xfer += iprot->readListBegin(_etype398, _size395);
-            this->partitions.resize(_size395);
-            uint32_t _i399;
-            for (_i399 = 0; _i399 < _size395; ++_i399)
+            uint32_t _size396;
+            ::apache::thrift::protocol::TType _etype399;
+            xfer += iprot->readListBegin(_etype399, _size396);
+            this->partitions.resize(_size396);
+            uint32_t _i400;
+            for (_i400 = 0; _i400 < _size396; ++_i400)
             {
-              xfer += this->partitions[_i399].read(iprot);
+              xfer += this->partitions[_i400].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9494,10 +9518,10 @@ uint32_t PartitionsByExprResult::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitions.size()));
-    std::vector<Partition> ::const_iterator _iter400;
-    for (_iter400 = this->partitions.begin(); _iter400 != this->partitions.end(); ++_iter400)
+    std::vector<Partition> ::const_iterator _iter401;
+    for (_iter401 = this->partitions.begin(); _iter401 != this->partitions.end(); ++_iter401)
     {
-      xfer += (*_iter400).write(oprot);
+      xfer += (*_iter401).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9518,13 +9542,13 @@ void swap(PartitionsByExprResult &a, PartitionsByExprResult &b) {
   swap(a.hasUnknownPartitions, b.hasUnknownPartitions);
 }
 
-PartitionsByExprResult::PartitionsByExprResult(const PartitionsByExprResult& other401) {
-  partitions = other401.partitions;
-  hasUnknownPartitions = other401.hasUnknownPartitions;
-}
-PartitionsByExprResult& PartitionsByExprResult::operator=(const PartitionsByExprResult& other402) {
+PartitionsByExprResult::PartitionsByExprResult(const PartitionsByExprResult& other402) {
   partitions = other402.partitions;
   hasUnknownPartitions = other402.hasUnknownPartitions;
+}
+PartitionsByExprResult& PartitionsByExprResult::operator=(const PartitionsByExprResult& other403) {
+  partitions = other403.partitions;
+  hasUnknownPartitions = other403.hasUnknownPartitions;
   return *this;
 }
 void PartitionsByExprResult::printTo(std::ostream& out) const {
@@ -9686,21 +9710,21 @@ void swap(PartitionsByExprRequest &a, PartitionsByExprRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionsByExprRequest::PartitionsByExprRequest(const PartitionsByExprRequest& other403) {
-  dbName = other403.dbName;
-  tblName = other403.tblName;
-  expr = other403.expr;
-  defaultPartitionName = other403.defaultPartitionName;
-  maxParts = other403.maxParts;
-  __isset = other403.__isset;
-}
-PartitionsByExprRequest& PartitionsByExprRequest::operator=(const PartitionsByExprRequest& other404) {
+PartitionsByExprRequest::PartitionsByExprRequest(const PartitionsByExprRequest& other404) {
   dbName = other404.dbName;
   tblName = other404.tblName;
   expr = other404.expr;
   defaultPartitionName = other404.defaultPartitionName;
   maxParts = other404.maxParts;
   __isset = other404.__isset;
+}
+PartitionsByExprRequest& PartitionsByExprRequest::operator=(const PartitionsByExprRequest& other405) {
+  dbName = other405.dbName;
+  tblName = other405.tblName;
+  expr = other405.expr;
+  defaultPartitionName = other405.defaultPartitionName;
+  maxParts = other405.maxParts;
+  __isset = other405.__isset;
   return *this;
 }
 void PartitionsByExprRequest::printTo(std::ostream& out) const {
@@ -9749,14 +9773,14 @@ uint32_t TableStatsResult::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tableStats.clear();
-            uint32_t _size405;
-            ::apache::thrift::protocol::TType _etype408;
-            xfer += iprot->readListBegin(_etype408, _size405);
-            this->tableStats.resize(_size405);
-            uint32_t _i409;
-            for (_i409 = 0; _i409 < _size405; ++_i409)
+            uint32_t _size406;
+            ::apache::thrift::protocol::TType _etype409;
+            xfer += iprot->readListBegin(_etype409, _size406);
+            this->tableStats.resize(_size406);
+            uint32_t _i410;
+            for (_i410 = 0; _i410 < _size406; ++_i410)
             {
-              xfer += this->tableStats[_i409].read(iprot);
+              xfer += this->tableStats[_i410].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9787,10 +9811,10 @@ uint32_t TableStatsResult::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("tableStats", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->tableStats.size()));
-    std::vector<ColumnStatisticsObj> ::const_iterator _iter410;
-    for (_iter410 = this->tableStats.begin(); _iter410 != this->tableStats.end(); ++_iter410)
+    std::vector<ColumnStatisticsObj> ::const_iterator _iter411;
+    for (_iter411 = this->tableStats.begin(); _iter411 != this->tableStats.end(); ++_iter411)
     {
-      xfer += (*_iter410).write(oprot);
+      xfer += (*_iter411).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9806,11 +9830,11 @@ void swap(TableStatsResult &a, TableStatsResult &b) {
   swap(a.tableStats, b.tableStats);
 }
 
-TableStatsResult::TableStatsResult(const TableStatsResult& other411) {
-  tableStats = other411.tableStats;
-}
-TableStatsResult& TableStatsResult::operator=(const TableStatsResult& other412) {
+TableStatsResult::TableStatsResult(const TableStatsResult& other412) {
   tableStats = other412.tableStats;
+}
+TableStatsResult& TableStatsResult::operator=(const TableStatsResult& other413) {
+  tableStats = other413.tableStats;
   return *this;
 }
 void TableStatsResult::printTo(std::ostream& out) const {
@@ -9855,26 +9879,26 @@ uint32_t PartitionsStatsResult::read(::apache::thrift::protocol::TProtocol* ipro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partStats.clear();
-            uint32_t _size413;
-            ::apache::thrift::protocol::TType _ktype414;
-            ::apache::thrift::protocol::TType _vtype415;
-            xfer += iprot->readMapBegin(_ktype414, _vtype415, _size413);
-            uint32_t _i417;
-            for (_i417 = 0; _i417 < _size413; ++_i417)
+            uint32_t _size414;
+            ::apache::thrift::protocol::TType _ktype415;
+            ::apache::thrift::protocol::TType _vtype416;
+            xfer += iprot->readMapBegin(_ktype415, _vtype416, _size414);
+            uint32_t _i418;
+            for (_i418 = 0; _i418 < _size414; ++_i418)
             {
-              std::string _key418;
-              xfer += iprot->readString(_key418);
-              std::vector<ColumnStatisticsObj> & _val419 = this->partStats[_key418];
+              std::string _key419;
+              xfer += iprot->readString(_key419);
+              std::vector<ColumnStatisticsObj> & _val420 = this->partStats[_key419];
               {
-                _val419.clear();
-                uint32_t _size420;
-                ::apache::thrift::protocol::TType _etype423;
-                xfer += iprot->readListBegin(_etype423, _size420);
-                _val419.resize(_size420);
-                uint32_t _i424;
-                for (_i424 = 0; _i424 < _size420; ++_i424)
+                _val420.clear();
+                uint32_t _size421;
+                ::apache::thrift::protocol::TType _etype424;
+                xfer += iprot->readListBegin(_etype424, _size421);
+                _val420.resize(_size421);
+                uint32_t _i425;
+                for (_i425 = 0; _i425 < _size421; ++_i425)
                 {
-                  xfer += _val419[_i424].read(iprot);
+                  xfer += _val420[_i425].read(iprot);
                 }
                 xfer += iprot->readListEnd();
               }
@@ -9908,16 +9932,16 @@ uint32_t PartitionsStatsResult::write(::apache::thrift::protocol::TProtocol* opr
   xfer += oprot->writeFieldBegin("partStats", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->partStats.size()));
-    std::map<std::string, std::vector<ColumnStatisticsObj> > ::const_iterator _iter425;
-    for (_iter425 = this->partStats.begin(); _iter425 != this->partStats.end(); ++_iter425)
+    std::map<std::string, std::vector<ColumnStatisticsObj> > ::const_iterator _iter426;
+    for (_iter426 = this->partStats.begin(); _iter426 != this->partStats.end(); ++_iter426)
     {
-      xfer += oprot->writeString(_iter425->first);
+      xfer += oprot->writeString(_iter426->first);
       {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter425->second.size()));
-        std::vector<ColumnStatisticsObj> ::const_iterator _iter426;
-        for (_iter426 = _iter425->second.begin(); _iter426 != _iter425->second.end(); ++_iter426)
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter426->second.size()));
+        std::vector<ColumnStatisticsObj> ::const_iterator _iter427;
+        for (_iter427 = _iter426->second.begin(); _iter427 != _iter426->second.end(); ++_iter427)
         {
-          xfer += (*_iter426).write(oprot);
+          xfer += (*_iter427).write(oprot);
         }
         xfer += oprot->writeListEnd();
       }
@@ -9936,11 +9960,11 @@ void swap(PartitionsStatsResult &a, PartitionsStatsResult &b) {
   swap(a.partStats, b.partStats);
 }
 
-PartitionsStatsResult::PartitionsStatsResult(const PartitionsStatsResult& other427) {
-  partStats = other427.partStats;
-}
-PartitionsStatsResult& PartitionsStatsResult::operator=(const PartitionsStatsResult& other428) {
+PartitionsStatsResult::PartitionsStatsResult(const PartitionsStatsResult& other428) {
   partStats = other428.partStats;
+}
+PartitionsStatsResult& PartitionsStatsResult::operator=(const PartitionsStatsResult& other429) {
+  partStats = other429.partStats;
   return *this;
 }
 void PartitionsStatsResult::printTo(std::ostream& out) const {
@@ -10011,14 +10035,14 @@ uint32_t TableStatsRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->colNames.clear();
-            uint32_t _size429;
-            ::apache::thrift::protocol::TType _etype432;
-            xfer += iprot->readListBegin(_etype432, _size429);
-            this->colNames.resize(_size429);
-            uint32_t _i433;
-            for (_i433 = 0; _i433 < _size429; ++_i433)
+            uint32_t _size430;
+            ::apache::thrift::protocol::TType _etype433;
+            xfer += iprot->readListBegin(_etype433, _size430);
+            this->colNames.resize(_size430);
+            uint32_t _i434;
+            for (_i434 = 0; _i434 < _size430; ++_i434)
             {
-              xfer += iprot->readString(this->colNames[_i433]);
+              xfer += iprot->readString(this->colNames[_i434]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10061,10 +10085,10 @@ uint32_t TableStatsRequest::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("colNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->colNames.size()));
-    std::vector<std::string> ::const_iterator _iter434;
-    for (_iter434 = this->colNames.begin(); _iter434 != this->colNames.end(); ++_iter434)
+    std::vector<std::string> ::const_iterator _iter435;
+    for (_iter435 = this->colNames.begin(); _iter435 != this->colNames.end(); ++_iter435)
     {
-      xfer += oprot->writeString((*_iter434));
+      xfer += oprot->writeString((*_iter435));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10082,15 +10106,15 @@ void swap(TableStatsRequest &a, TableStatsRequest &b) {
   swap(a.colNames, b.colNames);
 }
 
-TableStatsRequest::TableStatsRequest(const TableStatsRequest& other435) {
-  dbName = other435.dbName;
-  tblName = other435.tblName;
-  colNames = other435.colNames;
-}
-TableStatsRequest& TableStatsRequest::operator=(const TableStatsRequest& other436) {
+TableStatsRequest::TableStatsRequest(const TableStatsRequest& other436) {
   dbName = other436.dbName;
   tblName = other436.tblName;
   colNames = other436.colNames;
+}
+TableStatsRequest& TableStatsRequest::operator=(const TableStatsRequest& other437) {
+  dbName = other437.dbName;
+  tblName = other437.tblName;
+  colNames = other437.colNames;
   return *this;
 }
 void TableStatsRequest::printTo(std::ostream& out) const {
@@ -10168,14 +10192,14 @@ uint32_t PartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->colNames.clear();
-            uint32_t _size437;
-            ::apache::thrift::protocol::TType _etype440;
-            xfer += iprot->readListBegin(_etype440, _size437);
-            this->colNames.resize(_size437);
-            uint32_t _i441;
-            for (_i441 = 0; _i441 < _size437; ++_i441)
+            uint32_t _size438;
+            ::apache::thrift::protocol::TType _etype441;
+            xfer += iprot->readListBegin(_etype441, _size438);
+            this->colNames.resize(_size438);
+            uint32_t _i442;
+            for (_i442 = 0; _i442 < _size438; ++_i442)
             {
-              xfer += iprot->readString(this->colNames[_i441]);
+              xfer += iprot->readString(this->colNames[_i442]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10188,14 +10212,14 @@ uint32_t PartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size442;
-            ::apache::thrift::protocol::TType _etype445;
-            xfer += iprot->readListBegin(_etype445, _size442);
-            this->partNames.resize(_size442);
-            uint32_t _i446;
-            for (_i446 = 0; _i446 < _size442; ++_i446)
+            uint32_t _size443;
+            ::apache::thrift::protocol::TType _etype446;
+            xfer += iprot->readListBegin(_etype446, _size443);
+            this->partNames.resize(_size443);
+            uint32_t _i447;
+            for (_i447 = 0; _i447 < _size443; ++_i447)
             {
-              xfer += iprot->readString(this->partNames[_i446]);
+              xfer += iprot->readString(this->partNames[_i447]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10240,10 +10264,10 @@ uint32_t PartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("colNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->colNames.size()));
-    std::vector<std::string> ::const_iterator _iter447;
-    for (_iter447 = this->colNames.begin(); _iter447 != this->colNames.end(); ++_iter447)
+    std::vector<std::string> ::const_iterator _iter448;
+    for (_iter448 = this->colNames.begin(); _iter448 != this->colNames.end(); ++_iter448)
     {
-      xfer += oprot->writeString((*_iter447));
+      xfer += oprot->writeString((*_iter448));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10252,10 +10276,10 @@ uint32_t PartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-    std::vector<std::string> ::const_iterator _iter448;
-    for (_iter448 = this->partNames.begin(); _iter448 != this->partNames.end(); ++_iter448)
+    std::vector<std::string> ::const_iterator _iter449;
+    for (_iter449 = this->partNames.begin(); _iter449 != this->partNames.end(); ++_iter449)
     {
-      xfer += oprot->writeString((*_iter448));
+      xfer += oprot->writeString((*_iter449));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10274,17 +10298,17 @@ void swap(PartitionsStatsRequest &a, PartitionsStatsRequest &b) {
   swap(a.partNames, b.partNames);
 }
 
-PartitionsStatsRequest::PartitionsStatsRequest(const PartitionsStatsRequest& other449) {
-  dbName = other449.dbName;
-  tblName = other449.tblName;
-  colNames = other449.colNames;
-  partNames = other449.partNames;
-}
-PartitionsStatsRequest& PartitionsStatsRequest::operator=(const PartitionsStatsRequest& other450) {
+PartitionsStatsRequest::PartitionsStatsRequest(const PartitionsStatsRequest& other450) {
   dbName = other450.dbName;
   tblName = other450.tblName;
   colNames = other450.colNames;
   partNames = other450.partNames;
+}
+PartitionsStatsRequest& PartitionsStatsRequest::operator=(const PartitionsStatsRequest& other451) {
+  dbName = other451.dbName;
+  tblName = other451.tblName;
+  colNames = other451.colNames;
+  partNames = other451.partNames;
   return *this;
 }
 void PartitionsStatsRequest::printTo(std::ostream& out) const {
@@ -10332,14 +10356,14 @@ uint32_t AddPartitionsResult::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size451;
-            ::apache::thrift::protocol::TType _etype454;
-            xfer += iprot->readListBegin(_etype454, _size451);
-            this->partitions.resize(_size451);
-            uint32_t _i455;
-            for (_i455 = 0; _i455 < _size451; ++_i455)
+            uint32_t _size452;
+            ::apache::thrift::protocol::TType _etype455;
+            xfer += iprot->readListBegin(_etype455, _size452);
+            this->partitions.resize(_size452);
+            uint32_t _i456;
+            for (_i456 = 0; _i456 < _size452; ++_i456)
             {
-              xfer += this->partitions[_i455].read(iprot);
+              xfer += this->partitions[_i456].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10369,10 +10393,10 @@ uint32_t AddPartitionsResult::write(::apache::thrift::protocol::TProtocol* oprot
     xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitions.size()));
-      std::vector<Partition> ::const_iterator _iter456;
-      for (_iter456 = this->partitions.begin(); _iter456 != this->partitions.end(); ++_iter456)
+      std::vector<Partition> ::const_iterator _iter457;
+      for (_iter457 = this->partitions.begin(); _iter457 != this->partitions.end(); ++_iter457)
       {
-        xfer += (*_iter456).write(oprot);
+        xfer += (*_iter457).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -10389,13 +10413,13 @@ void swap(AddPartitionsResult &a, AddPartitionsResult &b) {
   swap(a.__isset, b.__isset);
 }
 
-AddPartitionsResult::AddPartitionsResult(const AddPartitionsResult& other457) {
-  partitions = other457.partitions;
-  __isset = other457.__isset;
-}
-AddPartitionsResult& AddPartitionsResult::operator=(const AddPartitionsResult& other458) {
+AddPartitionsResult::AddPartitionsResult(const AddPartitionsResult& other458) {
   partitions = other458.partitions;
   __isset = other458.__isset;
+}
+AddPartitionsResult& AddPartitionsResult::operator=(const AddPartitionsResult& other459) {
+  partitions = other459.partitions;
+  __isset = other459.__isset;
   return *this;
 }
 void AddPartitionsResult::printTo(std::ostream& out) const {
@@ -10476,14 +10500,14 @@ uint32_t AddPartitionsRequest::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->parts.clear();
-            uint32_t _size459;
-            ::apache::thrift::protocol::TType _etype462;
-            xfer += iprot->readListBegin(_etype462, _size459);
-            this->parts.resize(_size459);
-            uint32_t _i463;
-            for (_i463 = 0; _i463 < _size459; ++_i463)
+            uint32_t _size460;
+            ::apache::thrift::protocol::TType _etype463;
+            xfer += iprot->readListBegin(_etype463, _size460);
+            this->parts.resize(_size460);
+            uint32_t _i464;
+            for (_i464 = 0; _i464 < _size460; ++_i464)
             {
-              xfer += this->parts[_i463].read(iprot);
+              xfer += this->parts[_i464].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10544,10 +10568,10 @@ uint32_t AddPartitionsRequest::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("parts", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->parts.size()));
-    std::vector<Partition> ::const_iterator _iter464;
-    for (_iter464 = this->parts.begin(); _iter464 != this->parts.end(); ++_iter464)
+    std::vector<Partition> ::const_iterator _iter465;

<TRUNCATED>

[6/6] hive git commit: HIVE-19371: Add table ownerType to HMS thrift API (Sergio Pena, reviewed by Sahil Takiar, Vihang Karajgaonkar)

Posted by sp...@apache.org.
HIVE-19371: Add table ownerType to HMS thrift API (Sergio Pena, reviewed by Sahil Takiar, Vihang Karajgaonkar)


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

Branch: refs/heads/branch-2
Commit: 117c3b471819e2839c6d09034dadb92e12fe71ea
Parents: d988d4a
Author: Sergio Pena <se...@cloudera.com>
Authored: Mon May 14 13:09:50 2018 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Wed May 30 08:21:12 2018 -0500

----------------------------------------------------------------------
 metastore/if/hive_metastore.thrift              |    3 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2054 ++++++------
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 2932 +++++++++---------
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   14 +-
 .../apache/hadoop/hive/metastore/api/Table.java |  129 +-
 .../src/gen/thrift/gen-php/metastore/Types.php  |   23 +
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |   15 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |    7 +-
 .../results/clientpositive/create_view.q.out    |   16 +-
 .../create_view_partitioned.q.out               |    2 +-
 .../test/results/clientpositive/cteViews.q.out  |    8 +-
 .../llap/cbo_rp_unionDistinct_2.q.out           |    6 +-
 .../clientpositive/llap/subquery_views.q.out    |    6 +-
 .../clientpositive/llap/unionDistinct_2.q.out   |    6 +-
 .../clientpositive/tez/unionDistinct_2.q.out    |    6 +-
 15 files changed, 2712 insertions(+), 2515 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index 8a66bbc..ba117f5 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -305,7 +305,8 @@ struct Table {
   12: string tableType,                // table type enum, e.g. EXTERNAL_TABLE
   13: optional PrincipalPrivilegeSet privileges,
   14: optional bool temporary=false,
-  15: optional bool rewriteEnabled     // rewrite enabled or not
+  15: optional bool rewriteEnabled,     // rewrite enabled or not
+  16: optional PrincipalType ownerType = PrincipalType.USER // owner type of this table (default to USER for backward compatibility)
 }
 
 struct Partition {


[5/6] hive git commit: HIVE-19371: Add table ownerType to HMS thrift API (Sergio Pena, reviewed by Sahil Takiar, Vihang Karajgaonkar)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/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 155aab6..35aafe7 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -1240,14 +1240,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size848;
-            ::apache::thrift::protocol::TType _etype851;
-            xfer += iprot->readListBegin(_etype851, _size848);
-            this->success.resize(_size848);
-            uint32_t _i852;
-            for (_i852 = 0; _i852 < _size848; ++_i852)
+            uint32_t _size849;
+            ::apache::thrift::protocol::TType _etype852;
+            xfer += iprot->readListBegin(_etype852, _size849);
+            this->success.resize(_size849);
+            uint32_t _i853;
+            for (_i853 = 0; _i853 < _size849; ++_i853)
             {
-              xfer += iprot->readString(this->success[_i852]);
+              xfer += iprot->readString(this->success[_i853]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1286,10 +1286,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter853;
-      for (_iter853 = this->success.begin(); _iter853 != this->success.end(); ++_iter853)
+      std::vector<std::string> ::const_iterator _iter854;
+      for (_iter854 = this->success.begin(); _iter854 != this->success.end(); ++_iter854)
       {
-        xfer += oprot->writeString((*_iter853));
+        xfer += oprot->writeString((*_iter854));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1334,14 +1334,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size854;
-            ::apache::thrift::protocol::TType _etype857;
-            xfer += iprot->readListBegin(_etype857, _size854);
-            (*(this->success)).resize(_size854);
-            uint32_t _i858;
-            for (_i858 = 0; _i858 < _size854; ++_i858)
+            uint32_t _size855;
+            ::apache::thrift::protocol::TType _etype858;
+            xfer += iprot->readListBegin(_etype858, _size855);
+            (*(this->success)).resize(_size855);
+            uint32_t _i859;
+            for (_i859 = 0; _i859 < _size855; ++_i859)
             {
-              xfer += iprot->readString((*(this->success))[_i858]);
+              xfer += iprot->readString((*(this->success))[_i859]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1458,14 +1458,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size859;
-            ::apache::thrift::protocol::TType _etype862;
-            xfer += iprot->readListBegin(_etype862, _size859);
-            this->success.resize(_size859);
-            uint32_t _i863;
-            for (_i863 = 0; _i863 < _size859; ++_i863)
+            uint32_t _size860;
+            ::apache::thrift::protocol::TType _etype863;
+            xfer += iprot->readListBegin(_etype863, _size860);
+            this->success.resize(_size860);
+            uint32_t _i864;
+            for (_i864 = 0; _i864 < _size860; ++_i864)
             {
-              xfer += iprot->readString(this->success[_i863]);
+              xfer += iprot->readString(this->success[_i864]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1504,10 +1504,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter864;
-      for (_iter864 = this->success.begin(); _iter864 != this->success.end(); ++_iter864)
+      std::vector<std::string> ::const_iterator _iter865;
+      for (_iter865 = this->success.begin(); _iter865 != this->success.end(); ++_iter865)
       {
-        xfer += oprot->writeString((*_iter864));
+        xfer += oprot->writeString((*_iter865));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1552,14 +1552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size865;
-            ::apache::thrift::protocol::TType _etype868;
-            xfer += iprot->readListBegin(_etype868, _size865);
-            (*(this->success)).resize(_size865);
-            uint32_t _i869;
-            for (_i869 = 0; _i869 < _size865; ++_i869)
+            uint32_t _size866;
+            ::apache::thrift::protocol::TType _etype869;
+            xfer += iprot->readListBegin(_etype869, _size866);
+            (*(this->success)).resize(_size866);
+            uint32_t _i870;
+            for (_i870 = 0; _i870 < _size866; ++_i870)
             {
-              xfer += iprot->readString((*(this->success))[_i869]);
+              xfer += iprot->readString((*(this->success))[_i870]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2621,17 +2621,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size870;
-            ::apache::thrift::protocol::TType _ktype871;
-            ::apache::thrift::protocol::TType _vtype872;
-            xfer += iprot->readMapBegin(_ktype871, _vtype872, _size870);
-            uint32_t _i874;
-            for (_i874 = 0; _i874 < _size870; ++_i874)
+            uint32_t _size871;
+            ::apache::thrift::protocol::TType _ktype872;
+            ::apache::thrift::protocol::TType _vtype873;
+            xfer += iprot->readMapBegin(_ktype872, _vtype873, _size871);
+            uint32_t _i875;
+            for (_i875 = 0; _i875 < _size871; ++_i875)
             {
-              std::string _key875;
-              xfer += iprot->readString(_key875);
-              Type& _val876 = this->success[_key875];
-              xfer += _val876.read(iprot);
+              std::string _key876;
+              xfer += iprot->readString(_key876);
+              Type& _val877 = this->success[_key876];
+              xfer += _val877.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2670,11 +2670,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter877;
-      for (_iter877 = this->success.begin(); _iter877 != this->success.end(); ++_iter877)
+      std::map<std::string, Type> ::const_iterator _iter878;
+      for (_iter878 = this->success.begin(); _iter878 != this->success.end(); ++_iter878)
       {
-        xfer += oprot->writeString(_iter877->first);
-        xfer += _iter877->second.write(oprot);
+        xfer += oprot->writeString(_iter878->first);
+        xfer += _iter878->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -2719,17 +2719,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size878;
-            ::apache::thrift::protocol::TType _ktype879;
-            ::apache::thrift::protocol::TType _vtype880;
-            xfer += iprot->readMapBegin(_ktype879, _vtype880, _size878);
-            uint32_t _i882;
-            for (_i882 = 0; _i882 < _size878; ++_i882)
+            uint32_t _size879;
+            ::apache::thrift::protocol::TType _ktype880;
+            ::apache::thrift::protocol::TType _vtype881;
+            xfer += iprot->readMapBegin(_ktype880, _vtype881, _size879);
+            uint32_t _i883;
+            for (_i883 = 0; _i883 < _size879; ++_i883)
             {
-              std::string _key883;
-              xfer += iprot->readString(_key883);
-              Type& _val884 = (*(this->success))[_key883];
-              xfer += _val884.read(iprot);
+              std::string _key884;
+              xfer += iprot->readString(_key884);
+              Type& _val885 = (*(this->success))[_key884];
+              xfer += _val885.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2883,14 +2883,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size885;
-            ::apache::thrift::protocol::TType _etype888;
-            xfer += iprot->readListBegin(_etype888, _size885);
-            this->success.resize(_size885);
-            uint32_t _i889;
-            for (_i889 = 0; _i889 < _size885; ++_i889)
+            uint32_t _size886;
+            ::apache::thrift::protocol::TType _etype889;
+            xfer += iprot->readListBegin(_etype889, _size886);
+            this->success.resize(_size886);
+            uint32_t _i890;
+            for (_i890 = 0; _i890 < _size886; ++_i890)
             {
-              xfer += this->success[_i889].read(iprot);
+              xfer += this->success[_i890].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2945,10 +2945,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter890;
-      for (_iter890 = this->success.begin(); _iter890 != this->success.end(); ++_iter890)
+      std::vector<FieldSchema> ::const_iterator _iter891;
+      for (_iter891 = this->success.begin(); _iter891 != this->success.end(); ++_iter891)
       {
-        xfer += (*_iter890).write(oprot);
+        xfer += (*_iter891).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3001,14 +3001,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size891;
-            ::apache::thrift::protocol::TType _etype894;
-            xfer += iprot->readListBegin(_etype894, _size891);
-            (*(this->success)).resize(_size891);
-            uint32_t _i895;
-            for (_i895 = 0; _i895 < _size891; ++_i895)
+            uint32_t _size892;
+            ::apache::thrift::protocol::TType _etype895;
+            xfer += iprot->readListBegin(_etype895, _size892);
+            (*(this->success)).resize(_size892);
+            uint32_t _i896;
+            for (_i896 = 0; _i896 < _size892; ++_i896)
             {
-              xfer += (*(this->success))[_i895].read(iprot);
+              xfer += (*(this->success))[_i896].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3194,14 +3194,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size896;
-            ::apache::thrift::protocol::TType _etype899;
-            xfer += iprot->readListBegin(_etype899, _size896);
-            this->success.resize(_size896);
-            uint32_t _i900;
-            for (_i900 = 0; _i900 < _size896; ++_i900)
+            uint32_t _size897;
+            ::apache::thrift::protocol::TType _etype900;
+            xfer += iprot->readListBegin(_etype900, _size897);
+            this->success.resize(_size897);
+            uint32_t _i901;
+            for (_i901 = 0; _i901 < _size897; ++_i901)
             {
-              xfer += this->success[_i900].read(iprot);
+              xfer += this->success[_i901].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3256,10 +3256,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter901;
-      for (_iter901 = this->success.begin(); _iter901 != this->success.end(); ++_iter901)
+      std::vector<FieldSchema> ::const_iterator _iter902;
+      for (_iter902 = this->success.begin(); _iter902 != this->success.end(); ++_iter902)
       {
-        xfer += (*_iter901).write(oprot);
+        xfer += (*_iter902).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3312,14 +3312,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size902;
-            ::apache::thrift::protocol::TType _etype905;
-            xfer += iprot->readListBegin(_etype905, _size902);
-            (*(this->success)).resize(_size902);
-            uint32_t _i906;
-            for (_i906 = 0; _i906 < _size902; ++_i906)
+            uint32_t _size903;
+            ::apache::thrift::protocol::TType _etype906;
+            xfer += iprot->readListBegin(_etype906, _size903);
+            (*(this->success)).resize(_size903);
+            uint32_t _i907;
+            for (_i907 = 0; _i907 < _size903; ++_i907)
             {
-              xfer += (*(this->success))[_i906].read(iprot);
+              xfer += (*(this->success))[_i907].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3489,14 +3489,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size907;
-            ::apache::thrift::protocol::TType _etype910;
-            xfer += iprot->readListBegin(_etype910, _size907);
-            this->success.resize(_size907);
-            uint32_t _i911;
-            for (_i911 = 0; _i911 < _size907; ++_i911)
+            uint32_t _size908;
+            ::apache::thrift::protocol::TType _etype911;
+            xfer += iprot->readListBegin(_etype911, _size908);
+            this->success.resize(_size908);
+            uint32_t _i912;
+            for (_i912 = 0; _i912 < _size908; ++_i912)
             {
-              xfer += this->success[_i911].read(iprot);
+              xfer += this->success[_i912].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3551,10 +3551,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter912;
-      for (_iter912 = this->success.begin(); _iter912 != this->success.end(); ++_iter912)
+      std::vector<FieldSchema> ::const_iterator _iter913;
+      for (_iter913 = this->success.begin(); _iter913 != this->success.end(); ++_iter913)
       {
-        xfer += (*_iter912).write(oprot);
+        xfer += (*_iter913).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3607,14 +3607,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size913;
-            ::apache::thrift::protocol::TType _etype916;
-            xfer += iprot->readListBegin(_etype916, _size913);
-            (*(this->success)).resize(_size913);
-            uint32_t _i917;
-            for (_i917 = 0; _i917 < _size913; ++_i917)
+            uint32_t _size914;
+            ::apache::thrift::protocol::TType _etype917;
+            xfer += iprot->readListBegin(_etype917, _size914);
+            (*(this->success)).resize(_size914);
+            uint32_t _i918;
+            for (_i918 = 0; _i918 < _size914; ++_i918)
             {
-              xfer += (*(this->success))[_i917].read(iprot);
+              xfer += (*(this->success))[_i918].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3800,14 +3800,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size918;
-            ::apache::thrift::protocol::TType _etype921;
-            xfer += iprot->readListBegin(_etype921, _size918);
-            this->success.resize(_size918);
-            uint32_t _i922;
-            for (_i922 = 0; _i922 < _size918; ++_i922)
+            uint32_t _size919;
+            ::apache::thrift::protocol::TType _etype922;
+            xfer += iprot->readListBegin(_etype922, _size919);
+            this->success.resize(_size919);
+            uint32_t _i923;
+            for (_i923 = 0; _i923 < _size919; ++_i923)
             {
-              xfer += this->success[_i922].read(iprot);
+              xfer += this->success[_i923].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3862,10 +3862,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter923;
-      for (_iter923 = this->success.begin(); _iter923 != this->success.end(); ++_iter923)
+      std::vector<FieldSchema> ::const_iterator _iter924;
+      for (_iter924 = this->success.begin(); _iter924 != this->success.end(); ++_iter924)
       {
-        xfer += (*_iter923).write(oprot);
+        xfer += (*_iter924).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3918,14 +3918,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size924;
-            ::apache::thrift::protocol::TType _etype927;
-            xfer += iprot->readListBegin(_etype927, _size924);
-            (*(this->success)).resize(_size924);
-            uint32_t _i928;
-            for (_i928 = 0; _i928 < _size924; ++_i928)
+            uint32_t _size925;
+            ::apache::thrift::protocol::TType _etype928;
+            xfer += iprot->readListBegin(_etype928, _size925);
+            (*(this->success)).resize(_size925);
+            uint32_t _i929;
+            for (_i929 = 0; _i929 < _size925; ++_i929)
             {
-              xfer += (*(this->success))[_i928].read(iprot);
+              xfer += (*(this->success))[_i929].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4518,14 +4518,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size929;
-            ::apache::thrift::protocol::TType _etype932;
-            xfer += iprot->readListBegin(_etype932, _size929);
-            this->primaryKeys.resize(_size929);
-            uint32_t _i933;
-            for (_i933 = 0; _i933 < _size929; ++_i933)
+            uint32_t _size930;
+            ::apache::thrift::protocol::TType _etype933;
+            xfer += iprot->readListBegin(_etype933, _size930);
+            this->primaryKeys.resize(_size930);
+            uint32_t _i934;
+            for (_i934 = 0; _i934 < _size930; ++_i934)
             {
-              xfer += this->primaryKeys[_i933].read(iprot);
+              xfer += this->primaryKeys[_i934].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4538,14 +4538,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size934;
-            ::apache::thrift::protocol::TType _etype937;
-            xfer += iprot->readListBegin(_etype937, _size934);
-            this->foreignKeys.resize(_size934);
-            uint32_t _i938;
-            for (_i938 = 0; _i938 < _size934; ++_i938)
+            uint32_t _size935;
+            ::apache::thrift::protocol::TType _etype938;
+            xfer += iprot->readListBegin(_etype938, _size935);
+            this->foreignKeys.resize(_size935);
+            uint32_t _i939;
+            for (_i939 = 0; _i939 < _size935; ++_i939)
             {
-              xfer += this->foreignKeys[_i938].read(iprot);
+              xfer += this->foreignKeys[_i939].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4578,10 +4578,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter939;
-    for (_iter939 = this->primaryKeys.begin(); _iter939 != this->primaryKeys.end(); ++_iter939)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter940;
+    for (_iter940 = this->primaryKeys.begin(); _iter940 != this->primaryKeys.end(); ++_iter940)
     {
-      xfer += (*_iter939).write(oprot);
+      xfer += (*_iter940).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4590,10 +4590,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter940;
-    for (_iter940 = this->foreignKeys.begin(); _iter940 != this->foreignKeys.end(); ++_iter940)
+    std::vector<SQLForeignKey> ::const_iterator _iter941;
+    for (_iter941 = this->foreignKeys.begin(); _iter941 != this->foreignKeys.end(); ++_iter941)
     {
-      xfer += (*_iter940).write(oprot);
+      xfer += (*_iter941).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4621,10 +4621,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->primaryKeys)).size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter941;
-    for (_iter941 = (*(this->primaryKeys)).begin(); _iter941 != (*(this->primaryKeys)).end(); ++_iter941)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter942;
+    for (_iter942 = (*(this->primaryKeys)).begin(); _iter942 != (*(this->primaryKeys)).end(); ++_iter942)
     {
-      xfer += (*_iter941).write(oprot);
+      xfer += (*_iter942).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4633,10 +4633,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->foreignKeys)).size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter942;
-    for (_iter942 = (*(this->foreignKeys)).begin(); _iter942 != (*(this->foreignKeys)).end(); ++_iter942)
+    std::vector<SQLForeignKey> ::const_iterator _iter943;
+    for (_iter943 = (*(this->foreignKeys)).begin(); _iter943 != (*(this->foreignKeys)).end(); ++_iter943)
     {
-      xfer += (*_iter942).write(oprot);
+      xfer += (*_iter943).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -6055,14 +6055,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size943;
-            ::apache::thrift::protocol::TType _etype946;
-            xfer += iprot->readListBegin(_etype946, _size943);
-            this->success.resize(_size943);
-            uint32_t _i947;
-            for (_i947 = 0; _i947 < _size943; ++_i947)
+            uint32_t _size944;
+            ::apache::thrift::protocol::TType _etype947;
+            xfer += iprot->readListBegin(_etype947, _size944);
+            this->success.resize(_size944);
+            uint32_t _i948;
+            for (_i948 = 0; _i948 < _size944; ++_i948)
             {
-              xfer += iprot->readString(this->success[_i947]);
+              xfer += iprot->readString(this->success[_i948]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6101,10 +6101,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter948;
-      for (_iter948 = this->success.begin(); _iter948 != this->success.end(); ++_iter948)
+      std::vector<std::string> ::const_iterator _iter949;
+      for (_iter949 = this->success.begin(); _iter949 != this->success.end(); ++_iter949)
       {
-        xfer += oprot->writeString((*_iter948));
+        xfer += oprot->writeString((*_iter949));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6149,14 +6149,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size949;
-            ::apache::thrift::protocol::TType _etype952;
-            xfer += iprot->readListBegin(_etype952, _size949);
-            (*(this->success)).resize(_size949);
-            uint32_t _i953;
-            for (_i953 = 0; _i953 < _size949; ++_i953)
+            uint32_t _size950;
+            ::apache::thrift::protocol::TType _etype953;
+            xfer += iprot->readListBegin(_etype953, _size950);
+            (*(this->success)).resize(_size950);
+            uint32_t _i954;
+            for (_i954 = 0; _i954 < _size950; ++_i954)
             {
-              xfer += iprot->readString((*(this->success))[_i953]);
+              xfer += iprot->readString((*(this->success))[_i954]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6326,14 +6326,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size954;
-            ::apache::thrift::protocol::TType _etype957;
-            xfer += iprot->readListBegin(_etype957, _size954);
-            this->success.resize(_size954);
-            uint32_t _i958;
-            for (_i958 = 0; _i958 < _size954; ++_i958)
+            uint32_t _size955;
+            ::apache::thrift::protocol::TType _etype958;
+            xfer += iprot->readListBegin(_etype958, _size955);
+            this->success.resize(_size955);
+            uint32_t _i959;
+            for (_i959 = 0; _i959 < _size955; ++_i959)
             {
-              xfer += iprot->readString(this->success[_i958]);
+              xfer += iprot->readString(this->success[_i959]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6372,10 +6372,10 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_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 _iter959;
-      for (_iter959 = this->success.begin(); _iter959 != this->success.end(); ++_iter959)
+      std::vector<std::string> ::const_iterator _iter960;
+      for (_iter960 = this->success.begin(); _iter960 != this->success.end(); ++_iter960)
       {
-        xfer += oprot->writeString((*_iter959));
+        xfer += oprot->writeString((*_iter960));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6420,14 +6420,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size960;
-            ::apache::thrift::protocol::TType _etype963;
-            xfer += iprot->readListBegin(_etype963, _size960);
-            (*(this->success)).resize(_size960);
-            uint32_t _i964;
-            for (_i964 = 0; _i964 < _size960; ++_i964)
+            uint32_t _size961;
+            ::apache::thrift::protocol::TType _etype964;
+            xfer += iprot->readListBegin(_etype964, _size961);
+            (*(this->success)).resize(_size961);
+            uint32_t _i965;
+            for (_i965 = 0; _i965 < _size961; ++_i965)
             {
-              xfer += iprot->readString((*(this->success))[_i964]);
+              xfer += iprot->readString((*(this->success))[_i965]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6502,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 _size965;
-            ::apache::thrift::protocol::TType _etype968;
-            xfer += iprot->readListBegin(_etype968, _size965);
-            this->tbl_types.resize(_size965);
-            uint32_t _i969;
-            for (_i969 = 0; _i969 < _size965; ++_i969)
+            uint32_t _size966;
+            ::apache::thrift::protocol::TType _etype969;
+            xfer += iprot->readListBegin(_etype969, _size966);
+            this->tbl_types.resize(_size966);
+            uint32_t _i970;
+            for (_i970 = 0; _i970 < _size966; ++_i970)
             {
-              xfer += iprot->readString(this->tbl_types[_i969]);
+              xfer += iprot->readString(this->tbl_types[_i970]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6546,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 _iter970;
-    for (_iter970 = this->tbl_types.begin(); _iter970 != this->tbl_types.end(); ++_iter970)
+    std::vector<std::string> ::const_iterator _iter971;
+    for (_iter971 = this->tbl_types.begin(); _iter971 != this->tbl_types.end(); ++_iter971)
     {
-      xfer += oprot->writeString((*_iter970));
+      xfer += oprot->writeString((*_iter971));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6581,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 _iter971;
-    for (_iter971 = (*(this->tbl_types)).begin(); _iter971 != (*(this->tbl_types)).end(); ++_iter971)
+    std::vector<std::string> ::const_iterator _iter972;
+    for (_iter972 = (*(this->tbl_types)).begin(); _iter972 != (*(this->tbl_types)).end(); ++_iter972)
     {
-      xfer += oprot->writeString((*_iter971));
+      xfer += oprot->writeString((*_iter972));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6625,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 _size972;
-            ::apache::thrift::protocol::TType _etype975;
-            xfer += iprot->readListBegin(_etype975, _size972);
-            this->success.resize(_size972);
-            uint32_t _i976;
-            for (_i976 = 0; _i976 < _size972; ++_i976)
+            uint32_t _size973;
+            ::apache::thrift::protocol::TType _etype976;
+            xfer += iprot->readListBegin(_etype976, _size973);
+            this->success.resize(_size973);
+            uint32_t _i977;
+            for (_i977 = 0; _i977 < _size973; ++_i977)
             {
-              xfer += this->success[_i976].read(iprot);
+              xfer += this->success[_i977].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6671,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 _iter977;
-      for (_iter977 = this->success.begin(); _iter977 != this->success.end(); ++_iter977)
+      std::vector<TableMeta> ::const_iterator _iter978;
+      for (_iter978 = this->success.begin(); _iter978 != this->success.end(); ++_iter978)
       {
-        xfer += (*_iter977).write(oprot);
+        xfer += (*_iter978).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6719,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 _size978;
-            ::apache::thrift::protocol::TType _etype981;
-            xfer += iprot->readListBegin(_etype981, _size978);
-            (*(this->success)).resize(_size978);
-            uint32_t _i982;
-            for (_i982 = 0; _i982 < _size978; ++_i982)
+            uint32_t _size979;
+            ::apache::thrift::protocol::TType _etype982;
+            xfer += iprot->readListBegin(_etype982, _size979);
+            (*(this->success)).resize(_size979);
+            uint32_t _i983;
+            for (_i983 = 0; _i983 < _size979; ++_i983)
             {
-              xfer += (*(this->success))[_i982].read(iprot);
+              xfer += (*(this->success))[_i983].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6864,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 _size983;
-            ::apache::thrift::protocol::TType _etype986;
-            xfer += iprot->readListBegin(_etype986, _size983);
-            this->success.resize(_size983);
-            uint32_t _i987;
-            for (_i987 = 0; _i987 < _size983; ++_i987)
+            uint32_t _size984;
+            ::apache::thrift::protocol::TType _etype987;
+            xfer += iprot->readListBegin(_etype987, _size984);
+            this->success.resize(_size984);
+            uint32_t _i988;
+            for (_i988 = 0; _i988 < _size984; ++_i988)
             {
-              xfer += iprot->readString(this->success[_i987]);
+              xfer += iprot->readString(this->success[_i988]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6910,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 _iter988;
-      for (_iter988 = this->success.begin(); _iter988 != this->success.end(); ++_iter988)
+      std::vector<std::string> ::const_iterator _iter989;
+      for (_iter989 = this->success.begin(); _iter989 != this->success.end(); ++_iter989)
       {
-        xfer += oprot->writeString((*_iter988));
+        xfer += oprot->writeString((*_iter989));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6958,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 _size989;
-            ::apache::thrift::protocol::TType _etype992;
-            xfer += iprot->readListBegin(_etype992, _size989);
-            (*(this->success)).resize(_size989);
-            uint32_t _i993;
-            for (_i993 = 0; _i993 < _size989; ++_i993)
+            uint32_t _size990;
+            ::apache::thrift::protocol::TType _etype993;
+            xfer += iprot->readListBegin(_etype993, _size990);
+            (*(this->success)).resize(_size990);
+            uint32_t _i994;
+            for (_i994 = 0; _i994 < _size990; ++_i994)
             {
-              xfer += iprot->readString((*(this->success))[_i993]);
+              xfer += iprot->readString((*(this->success))[_i994]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7275,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 _size994;
-            ::apache::thrift::protocol::TType _etype997;
-            xfer += iprot->readListBegin(_etype997, _size994);
-            this->tbl_names.resize(_size994);
-            uint32_t _i998;
-            for (_i998 = 0; _i998 < _size994; ++_i998)
+            uint32_t _size995;
+            ::apache::thrift::protocol::TType _etype998;
+            xfer += iprot->readListBegin(_etype998, _size995);
+            this->tbl_names.resize(_size995);
+            uint32_t _i999;
+            for (_i999 = 0; _i999 < _size995; ++_i999)
             {
-              xfer += iprot->readString(this->tbl_names[_i998]);
+              xfer += iprot->readString(this->tbl_names[_i999]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7315,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 _iter999;
-    for (_iter999 = this->tbl_names.begin(); _iter999 != this->tbl_names.end(); ++_iter999)
+    std::vector<std::string> ::const_iterator _iter1000;
+    for (_iter1000 = this->tbl_names.begin(); _iter1000 != this->tbl_names.end(); ++_iter1000)
     {
-      xfer += oprot->writeString((*_iter999));
+      xfer += oprot->writeString((*_iter1000));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7346,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 _iter1000;
-    for (_iter1000 = (*(this->tbl_names)).begin(); _iter1000 != (*(this->tbl_names)).end(); ++_iter1000)
+    std::vector<std::string> ::const_iterator _iter1001;
+    for (_iter1001 = (*(this->tbl_names)).begin(); _iter1001 != (*(this->tbl_names)).end(); ++_iter1001)
     {
-      xfer += oprot->writeString((*_iter1000));
+      xfer += oprot->writeString((*_iter1001));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7390,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 _size1001;
-            ::apache::thrift::protocol::TType _etype1004;
-            xfer += iprot->readListBegin(_etype1004, _size1001);
-            this->success.resize(_size1001);
-            uint32_t _i1005;
-            for (_i1005 = 0; _i1005 < _size1001; ++_i1005)
+            uint32_t _size1002;
+            ::apache::thrift::protocol::TType _etype1005;
+            xfer += iprot->readListBegin(_etype1005, _size1002);
+            this->success.resize(_size1002);
+            uint32_t _i1006;
+            for (_i1006 = 0; _i1006 < _size1002; ++_i1006)
             {
-              xfer += this->success[_i1005].read(iprot);
+              xfer += this->success[_i1006].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7428,10 +7428,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 _iter1006;
-      for (_iter1006 = this->success.begin(); _iter1006 != this->success.end(); ++_iter1006)
+      std::vector<Table> ::const_iterator _iter1007;
+      for (_iter1007 = this->success.begin(); _iter1007 != this->success.end(); ++_iter1007)
       {
-        xfer += (*_iter1006).write(oprot);
+        xfer += (*_iter1007).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -7472,14 +7472,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 _size1007;
-            ::apache::thrift::protocol::TType _etype1010;
-            xfer += iprot->readListBegin(_etype1010, _size1007);
-            (*(this->success)).resize(_size1007);
-            uint32_t _i1011;
-            for (_i1011 = 0; _i1011 < _size1007; ++_i1011)
+            uint32_t _size1008;
+            ::apache::thrift::protocol::TType _etype1011;
+            xfer += iprot->readListBegin(_etype1011, _size1008);
+            (*(this->success)).resize(_size1008);
+            uint32_t _i1012;
+            for (_i1012 = 0; _i1012 < _size1008; ++_i1012)
             {
-              xfer += (*(this->success))[_i1011].read(iprot);
+              xfer += (*(this->success))[_i1012].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8115,14 +8115,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 _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 _size1013;
+            ::apache::thrift::protocol::TType _etype1016;
+            xfer += iprot->readListBegin(_etype1016, _size1013);
+            this->success.resize(_size1013);
+            uint32_t _i1017;
+            for (_i1017 = 0; _i1017 < _size1013; ++_i1017)
             {
-              xfer += iprot->readString(this->success[_i1016]);
+              xfer += iprot->readString(this->success[_i1017]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8177,10 +8177,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 _iter1017;
-      for (_iter1017 = this->success.begin(); _iter1017 != this->success.end(); ++_iter1017)
+      std::vector<std::string> ::const_iterator _iter1018;
+      for (_iter1018 = this->success.begin(); _iter1018 != this->success.end(); ++_iter1018)
       {
-        xfer += oprot->writeString((*_iter1017));
+        xfer += oprot->writeString((*_iter1018));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8233,14 +8233,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 _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 _size1019;
+            ::apache::thrift::protocol::TType _etype1022;
+            xfer += iprot->readListBegin(_etype1022, _size1019);
+            (*(this->success)).resize(_size1019);
+            uint32_t _i1023;
+            for (_i1023 = 0; _i1023 < _size1019; ++_i1023)
             {
-              xfer += iprot->readString((*(this->success))[_i1022]);
+              xfer += iprot->readString((*(this->success))[_i1023]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9574,14 +9574,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size1023;
-            ::apache::thrift::protocol::TType _etype1026;
-            xfer += iprot->readListBegin(_etype1026, _size1023);
-            this->new_parts.resize(_size1023);
-            uint32_t _i1027;
-            for (_i1027 = 0; _i1027 < _size1023; ++_i1027)
+            uint32_t _size1024;
+            ::apache::thrift::protocol::TType _etype1027;
+            xfer += iprot->readListBegin(_etype1027, _size1024);
+            this->new_parts.resize(_size1024);
+            uint32_t _i1028;
+            for (_i1028 = 0; _i1028 < _size1024; ++_i1028)
             {
-              xfer += this->new_parts[_i1027].read(iprot);
+              xfer += this->new_parts[_i1028].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9610,10 +9610,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 _iter1028;
-    for (_iter1028 = this->new_parts.begin(); _iter1028 != this->new_parts.end(); ++_iter1028)
+    std::vector<Partition> ::const_iterator _iter1029;
+    for (_iter1029 = this->new_parts.begin(); _iter1029 != this->new_parts.end(); ++_iter1029)
     {
-      xfer += (*_iter1028).write(oprot);
+      xfer += (*_iter1029).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9637,10 +9637,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 _iter1029;
-    for (_iter1029 = (*(this->new_parts)).begin(); _iter1029 != (*(this->new_parts)).end(); ++_iter1029)
+    std::vector<Partition> ::const_iterator _iter1030;
+    for (_iter1030 = (*(this->new_parts)).begin(); _iter1030 != (*(this->new_parts)).end(); ++_iter1030)
     {
-      xfer += (*_iter1029).write(oprot);
+      xfer += (*_iter1030).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9849,14 +9849,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 _size1030;
-            ::apache::thrift::protocol::TType _etype1033;
-            xfer += iprot->readListBegin(_etype1033, _size1030);
-            this->new_parts.resize(_size1030);
-            uint32_t _i1034;
-            for (_i1034 = 0; _i1034 < _size1030; ++_i1034)
+            uint32_t _size1031;
+            ::apache::thrift::protocol::TType _etype1034;
+            xfer += iprot->readListBegin(_etype1034, _size1031);
+            this->new_parts.resize(_size1031);
+            uint32_t _i1035;
+            for (_i1035 = 0; _i1035 < _size1031; ++_i1035)
             {
-              xfer += this->new_parts[_i1034].read(iprot);
+              xfer += this->new_parts[_i1035].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9885,10 +9885,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 _iter1035;
-    for (_iter1035 = this->new_parts.begin(); _iter1035 != this->new_parts.end(); ++_iter1035)
+    std::vector<PartitionSpec> ::const_iterator _iter1036;
+    for (_iter1036 = this->new_parts.begin(); _iter1036 != this->new_parts.end(); ++_iter1036)
     {
-      xfer += (*_iter1035).write(oprot);
+      xfer += (*_iter1036).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9912,10 +9912,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 _iter1036;
-    for (_iter1036 = (*(this->new_parts)).begin(); _iter1036 != (*(this->new_parts)).end(); ++_iter1036)
+    std::vector<PartitionSpec> ::const_iterator _iter1037;
+    for (_iter1037 = (*(this->new_parts)).begin(); _iter1037 != (*(this->new_parts)).end(); ++_iter1037)
     {
-      xfer += (*_iter1036).write(oprot);
+      xfer += (*_iter1037).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10140,14 +10140,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1037;
-            ::apache::thrift::protocol::TType _etype1040;
-            xfer += iprot->readListBegin(_etype1040, _size1037);
-            this->part_vals.resize(_size1037);
-            uint32_t _i1041;
-            for (_i1041 = 0; _i1041 < _size1037; ++_i1041)
+            uint32_t _size1038;
+            ::apache::thrift::protocol::TType _etype1041;
+            xfer += iprot->readListBegin(_etype1041, _size1038);
+            this->part_vals.resize(_size1038);
+            uint32_t _i1042;
+            for (_i1042 = 0; _i1042 < _size1038; ++_i1042)
             {
-              xfer += iprot->readString(this->part_vals[_i1041]);
+              xfer += iprot->readString(this->part_vals[_i1042]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10184,10 +10184,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 _iter1042;
-    for (_iter1042 = this->part_vals.begin(); _iter1042 != this->part_vals.end(); ++_iter1042)
+    std::vector<std::string> ::const_iterator _iter1043;
+    for (_iter1043 = this->part_vals.begin(); _iter1043 != this->part_vals.end(); ++_iter1043)
     {
-      xfer += oprot->writeString((*_iter1042));
+      xfer += oprot->writeString((*_iter1043));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10219,10 +10219,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 _iter1043;
-    for (_iter1043 = (*(this->part_vals)).begin(); _iter1043 != (*(this->part_vals)).end(); ++_iter1043)
+    std::vector<std::string> ::const_iterator _iter1044;
+    for (_iter1044 = (*(this->part_vals)).begin(); _iter1044 != (*(this->part_vals)).end(); ++_iter1044)
     {
-      xfer += oprot->writeString((*_iter1043));
+      xfer += oprot->writeString((*_iter1044));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10694,14 +10694,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1044;
-            ::apache::thrift::protocol::TType _etype1047;
-            xfer += iprot->readListBegin(_etype1047, _size1044);
-            this->part_vals.resize(_size1044);
-            uint32_t _i1048;
-            for (_i1048 = 0; _i1048 < _size1044; ++_i1048)
+            uint32_t _size1045;
+            ::apache::thrift::protocol::TType _etype1048;
+            xfer += iprot->readListBegin(_etype1048, _size1045);
+            this->part_vals.resize(_size1045);
+            uint32_t _i1049;
+            for (_i1049 = 0; _i1049 < _size1045; ++_i1049)
             {
-              xfer += iprot->readString(this->part_vals[_i1048]);
+              xfer += iprot->readString(this->part_vals[_i1049]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10746,10 +10746,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 _iter1049;
-    for (_iter1049 = this->part_vals.begin(); _iter1049 != this->part_vals.end(); ++_iter1049)
+    std::vector<std::string> ::const_iterator _iter1050;
+    for (_iter1050 = this->part_vals.begin(); _iter1050 != this->part_vals.end(); ++_iter1050)
     {
-      xfer += oprot->writeString((*_iter1049));
+      xfer += oprot->writeString((*_iter1050));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10785,10 +10785,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 _iter1050;
-    for (_iter1050 = (*(this->part_vals)).begin(); _iter1050 != (*(this->part_vals)).end(); ++_iter1050)
+    std::vector<std::string> ::const_iterator _iter1051;
+    for (_iter1051 = (*(this->part_vals)).begin(); _iter1051 != (*(this->part_vals)).end(); ++_iter1051)
     {
-      xfer += oprot->writeString((*_iter1050));
+      xfer += oprot->writeString((*_iter1051));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11591,14 +11591,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1051;
-            ::apache::thrift::protocol::TType _etype1054;
-            xfer += iprot->readListBegin(_etype1054, _size1051);
-            this->part_vals.resize(_size1051);
-            uint32_t _i1055;
-            for (_i1055 = 0; _i1055 < _size1051; ++_i1055)
+            uint32_t _size1052;
+            ::apache::thrift::protocol::TType _etype1055;
+            xfer += iprot->readListBegin(_etype1055, _size1052);
+            this->part_vals.resize(_size1052);
+            uint32_t _i1056;
+            for (_i1056 = 0; _i1056 < _size1052; ++_i1056)
             {
-              xfer += iprot->readString(this->part_vals[_i1055]);
+              xfer += iprot->readString(this->part_vals[_i1056]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11643,10 +11643,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 _iter1056;
-    for (_iter1056 = this->part_vals.begin(); _iter1056 != this->part_vals.end(); ++_iter1056)
+    std::vector<std::string> ::const_iterator _iter1057;
+    for (_iter1057 = this->part_vals.begin(); _iter1057 != this->part_vals.end(); ++_iter1057)
     {
-      xfer += oprot->writeString((*_iter1056));
+      xfer += oprot->writeString((*_iter1057));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11682,10 +11682,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 _iter1057;
-    for (_iter1057 = (*(this->part_vals)).begin(); _iter1057 != (*(this->part_vals)).end(); ++_iter1057)
+    std::vector<std::string> ::const_iterator _iter1058;
+    for (_iter1058 = (*(this->part_vals)).begin(); _iter1058 != (*(this->part_vals)).end(); ++_iter1058)
     {
-      xfer += oprot->writeString((*_iter1057));
+      xfer += oprot->writeString((*_iter1058));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11894,14 +11894,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1058;
-            ::apache::thrift::protocol::TType _etype1061;
-            xfer += iprot->readListBegin(_etype1061, _size1058);
-            this->part_vals.resize(_size1058);
-            uint32_t _i1062;
-            for (_i1062 = 0; _i1062 < _size1058; ++_i1062)
+            uint32_t _size1059;
+            ::apache::thrift::protocol::TType _etype1062;
+            xfer += iprot->readListBegin(_etype1062, _size1059);
+            this->part_vals.resize(_size1059);
+            uint32_t _i1063;
+            for (_i1063 = 0; _i1063 < _size1059; ++_i1063)
             {
-              xfer += iprot->readString(this->part_vals[_i1062]);
+              xfer += iprot->readString(this->part_vals[_i1063]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11954,10 +11954,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 _iter1063;
-    for (_iter1063 = this->part_vals.begin(); _iter1063 != this->part_vals.end(); ++_iter1063)
+    std::vector<std::string> ::const_iterator _iter1064;
+    for (_iter1064 = this->part_vals.begin(); _iter1064 != this->part_vals.end(); ++_iter1064)
     {
-      xfer += oprot->writeString((*_iter1063));
+      xfer += oprot->writeString((*_iter1064));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11997,10 +11997,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 _iter1064;
-    for (_iter1064 = (*(this->part_vals)).begin(); _iter1064 != (*(this->part_vals)).end(); ++_iter1064)
+    std::vector<std::string> ::const_iterator _iter1065;
+    for (_iter1065 = (*(this->part_vals)).begin(); _iter1065 != (*(this->part_vals)).end(); ++_iter1065)
     {
-      xfer += oprot->writeString((*_iter1064));
+      xfer += oprot->writeString((*_iter1065));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13006,14 +13006,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1065;
-            ::apache::thrift::protocol::TType _etype1068;
-            xfer += iprot->readListBegin(_etype1068, _size1065);
-            this->part_vals.resize(_size1065);
-            uint32_t _i1069;
-            for (_i1069 = 0; _i1069 < _size1065; ++_i1069)
+            uint32_t _size1066;
+            ::apache::thrift::protocol::TType _etype1069;
+            xfer += iprot->readListBegin(_etype1069, _size1066);
+            this->part_vals.resize(_size1066);
+            uint32_t _i1070;
+            for (_i1070 = 0; _i1070 < _size1066; ++_i1070)
             {
-              xfer += iprot->readString(this->part_vals[_i1069]);
+              xfer += iprot->readString(this->part_vals[_i1070]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13050,10 +13050,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 _iter1070;
-    for (_iter1070 = this->part_vals.begin(); _iter1070 != this->part_vals.end(); ++_iter1070)
+    std::vector<std::string> ::const_iterator _iter1071;
+    for (_iter1071 = this->part_vals.begin(); _iter1071 != this->part_vals.end(); ++_iter1071)
     {
-      xfer += oprot->writeString((*_iter1070));
+      xfer += oprot->writeString((*_iter1071));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13085,10 +13085,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 _iter1071;
-    for (_iter1071 = (*(this->part_vals)).begin(); _iter1071 != (*(this->part_vals)).end(); ++_iter1071)
+    std::vector<std::string> ::const_iterator _iter1072;
+    for (_iter1072 = (*(this->part_vals)).begin(); _iter1072 != (*(this->part_vals)).end(); ++_iter1072)
     {
-      xfer += oprot->writeString((*_iter1071));
+      xfer += oprot->writeString((*_iter1072));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13277,17 +13277,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size1072;
-            ::apache::thrift::protocol::TType _ktype1073;
-            ::apache::thrift::protocol::TType _vtype1074;
-            xfer += iprot->readMapBegin(_ktype1073, _vtype1074, _size1072);
-            uint32_t _i1076;
-            for (_i1076 = 0; _i1076 < _size1072; ++_i1076)
+            uint32_t _size1073;
+            ::apache::thrift::protocol::TType _ktype1074;
+            ::apache::thrift::protocol::TType _vtype1075;
+            xfer += iprot->readMapBegin(_ktype1074, _vtype1075, _size1073);
+            uint32_t _i1077;
+            for (_i1077 = 0; _i1077 < _size1073; ++_i1077)
             {
-              std::string _key1077;
-              xfer += iprot->readString(_key1077);
-              std::string& _val1078 = this->partitionSpecs[_key1077];
-              xfer += iprot->readString(_val1078);
+              std::string _key1078;
+              xfer += iprot->readString(_key1078);
+              std::string& _val1079 = this->partitionSpecs[_key1078];
+              xfer += iprot->readString(_val1079);
             }
             xfer += iprot->readMapEnd();
           }
@@ -13348,11 +13348,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 _iter1079;
-    for (_iter1079 = this->partitionSpecs.begin(); _iter1079 != this->partitionSpecs.end(); ++_iter1079)
+    std::map<std::string, std::string> ::const_iterator _iter1080;
+    for (_iter1080 = this->partitionSpecs.begin(); _iter1080 != this->partitionSpecs.end(); ++_iter1080)
     {
-      xfer += oprot->writeString(_iter1079->first);
-      xfer += oprot->writeString(_iter1079->second);
+      xfer += oprot->writeString(_iter1080->first);
+      xfer += oprot->writeString(_iter1080->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13392,11 +13392,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 _iter1080;
-    for (_iter1080 = (*(this->partitionSpecs)).begin(); _iter1080 != (*(this->partitionSpecs)).end(); ++_iter1080)
+    std::map<std::string, std::string> ::const_iterator _iter1081;
+    for (_iter1081 = (*(this->partitionSpecs)).begin(); _iter1081 != (*(this->partitionSpecs)).end(); ++_iter1081)
     {
-      xfer += oprot->writeString(_iter1080->first);
-      xfer += oprot->writeString(_iter1080->second);
+      xfer += oprot->writeString(_iter1081->first);
+      xfer += oprot->writeString(_iter1081->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13641,17 +13641,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size1081;
-            ::apache::thrift::protocol::TType _ktype1082;
-            ::apache::thrift::protocol::TType _vtype1083;
-            xfer += iprot->readMapBegin(_ktype1082, _vtype1083, _size1081);
-            uint32_t _i1085;
-            for (_i1085 = 0; _i1085 < _size1081; ++_i1085)
+            uint32_t _size1082;
+            ::apache::thrift::protocol::TType _ktype1083;
+            ::apache::thrift::protocol::TType _vtype1084;
+            xfer += iprot->readMapBegin(_ktype1083, _vtype1084, _size1082);
+            uint32_t _i1086;
+            for (_i1086 = 0; _i1086 < _size1082; ++_i1086)
             {
-              std::string _key1086;
-              xfer += iprot->readString(_key1086);
-              std::string& _val1087 = this->partitionSpecs[_key1086];
-              xfer += iprot->readString(_val1087);
+              std::string _key1087;
+              xfer += iprot->readString(_key1087);
+              std::string& _val1088 = this->partitionSpecs[_key1087];
+              xfer += iprot->readString(_val1088);
             }
             xfer += iprot->readMapEnd();
           }
@@ -13712,11 +13712,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 _iter1088;
-    for (_iter1088 = this->partitionSpecs.begin(); _iter1088 != this->partitionSpecs.end(); ++_iter1088)
+    std::map<std::string, std::string> ::const_iterator _iter1089;
+    for (_iter1089 = this->partitionSpecs.begin(); _iter1089 != this->partitionSpecs.end(); ++_iter1089)
     {
-      xfer += oprot->writeString(_iter1088->first);
-      xfer += oprot->writeString(_iter1088->second);
+      xfer += oprot->writeString(_iter1089->first);
+      xfer += oprot->writeString(_iter1089->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13756,11 +13756,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 _iter1089;
-    for (_iter1089 = (*(this->partitionSpecs)).begin(); _iter1089 != (*(this->partitionSpecs)).end(); ++_iter1089)
+    std::map<std::string, std::string> ::const_iterator _iter1090;
+    for (_iter1090 = (*(this->partitionSpecs)).begin(); _iter1090 != (*(this->partitionSpecs)).end(); ++_iter1090)
     {
-      xfer += oprot->writeString(_iter1089->first);
-      xfer += oprot->writeString(_iter1089->second);
+      xfer += oprot->writeString(_iter1090->first);
+      xfer += oprot->writeString(_iter1090->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13817,14 +13817,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1090;
-            ::apache::thrift::protocol::TType _etype1093;
-            xfer += iprot->readListBegin(_etype1093, _size1090);
-            this->success.resize(_size1090);
-            uint32_t _i1094;
-            for (_i1094 = 0; _i1094 < _size1090; ++_i1094)
+            uint32_t _size1091;
+            ::apache::thrift::protocol::TType _etype1094;
+            xfer += iprot->readListBegin(_etype1094, _size1091);
+            this->success.resize(_size1091);
+            uint32_t _i1095;
+            for (_i1095 = 0; _i1095 < _size1091; ++_i1095)
             {
-              xfer += this->success[_i1094].read(iprot);
+              xfer += this->success[_i1095].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13887,10 +13887,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 _iter1095;
-      for (_iter1095 = this->success.begin(); _iter1095 != this->success.end(); ++_iter1095)
+      std::vector<Partition> ::const_iterator _iter1096;
+      for (_iter1096 = this->success.begin(); _iter1096 != this->success.end(); ++_iter1096)
       {
-        xfer += (*_iter1095).write(oprot);
+        xfer += (*_iter1096).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13947,14 +13947,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1096;
-            ::apache::thrift::protocol::TType _etype1099;
-            xfer += iprot->readListBegin(_etype1099, _size1096);
-            (*(this->success)).resize(_size1096);
-            uint32_t _i1100;
-            for (_i1100 = 0; _i1100 < _size1096; ++_i1100)
+            uint32_t _size1097;
+            ::apache::thrift::protocol::TType _etype1100;
+            xfer += iprot->readListBegin(_etype1100, _size1097);
+            (*(this->success)).resize(_size1097);
+            uint32_t _i1101;
+            for (_i1101 = 0; _i1101 < _size1097; ++_i1101)
             {
-              xfer += (*(this->success))[_i1100].read(iprot);
+              xfer += (*(this->success))[_i1101].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14053,14 +14053,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 _size1101;
-            ::apache::thrift::protocol::TType _etype1104;
-            xfer += iprot->readListBegin(_etype1104, _size1101);
-            this->part_vals.resize(_size1101);
-            uint32_t _i1105;
-            for (_i1105 = 0; _i1105 < _size1101; ++_i1105)
+            uint32_t _size1102;
+            ::apache::thrift::protocol::TType _etype1105;
+            xfer += iprot->readListBegin(_etype1105, _size1102);
+            this->part_vals.resize(_size1102);
+            uint32_t _i1106;
+            for (_i1106 = 0; _i1106 < _size1102; ++_i1106)
             {
-              xfer += iprot->readString(this->part_vals[_i1105]);
+              xfer += iprot->readString(this->part_vals[_i1106]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14081,14 +14081,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 _size1106;
-            ::apache::thrift::protocol::TType _etype1109;
-            xfer += iprot->readListBegin(_etype1109, _size1106);
-            this->group_names.resize(_size1106);
-            uint32_t _i1110;
-            for (_i1110 = 0; _i1110 < _size1106; ++_i1110)
+            uint32_t _size1107;
+            ::apache::thrift::protocol::TType _etype1110;
+            xfer += iprot->readListBegin(_etype1110, _size1107);
+            this->group_names.resize(_size1107);
+            uint32_t _i1111;
+            for (_i1111 = 0; _i1111 < _size1107; ++_i1111)
             {
-              xfer += iprot->readString(this->group_names[_i1110]);
+              xfer += iprot->readString(this->group_names[_i1111]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14125,10 +14125,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 _iter1111;
-    for (_iter1111 = this->part_vals.begin(); _iter1111 != this->part_vals.end(); ++_iter1111)
+    std::vector<std::string> ::const_iterator _iter1112;
+    for (_iter1112 = this->part_vals.begin(); _iter1112 != this->part_vals.end(); ++_iter1112)
     {
-      xfer += oprot->writeString((*_iter1111));
+      xfer += oprot->writeString((*_iter1112));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14141,10 +14141,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 _iter1112;
-    for (_iter1112 = this->group_names.begin(); _iter1112 != this->group_names.end(); ++_iter1112)
+    std::vector<std::string> ::const_iterator _iter1113;
+    for (_iter1113 = this->group_names.begin(); _iter1113 != this->group_names.end(); ++_iter1113)
     {
-      xfer += oprot->writeString((*_iter1112));
+      xfer += oprot->writeString((*_iter1113));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14176,10 +14176,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 _iter1113;
-    for (_iter1113 = (*(this->part_vals)).begin(); _iter1113 != (*(this->part_vals)).end(); ++_iter1113)
+    std::vector<std::string> ::const_iterator _iter1114;
+    for (_iter1114 = (*(this->part_vals)).begin(); _iter1114 != (*(this->part_vals)).end(); ++_iter1114)
     {
-      xfer += oprot->writeString((*_iter1113));
+      xfer += oprot->writeString((*_iter1114));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14192,10 +14192,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 _iter1114;
-    for (_iter1114 = (*(this->group_names)).begin(); _iter1114 != (*(this->group_names)).end(); ++_iter1114)
+    std::vector<std::string> ::const_iterator _iter1115;
+    for (_iter1115 = (*(this->group_names)).begin(); _iter1115 != (*(this->group_names)).end(); ++_iter1115)
     {
-      xfer += oprot->writeString((*_iter1114));
+      xfer += oprot->writeString((*_iter1115));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14754,14 +14754,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1115;
-            ::apache::thrift::protocol::TType _etype1118;
-            xfer += iprot->readListBegin(_etype1118, _size1115);
-            this->success.resize(_size1115);
-            uint32_t _i1119;
-            for (_i1119 = 0; _i1119 < _size1115; ++_i1119)
+            uint32_t _size1116;
+            ::apache::thrift::protocol::TType _etype1119;
+            xfer += iprot->readListBegin(_etype1119, _size1116);
+            this->success.resize(_size1116);
+            uint32_t _i1120;
+            for (_i1120 = 0; _i1120 < _size1116; ++_i1120)
             {
-              xfer += this->success[_i1119].read(iprot);
+              xfer += this->success[_i1120].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14808,10 +14808,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 _iter1120;
-      for (_iter1120 = this->success.begin(); _iter1120 != this->success.end(); ++_iter1120)
+      std::vector<Partition> ::const_iterator _iter1121;
+      for (_iter1121 = this->success.begin(); _iter1121 != this->success.end(); ++_iter1121)
       {
-        xfer += (*_iter1120).write(oprot);
+        xfer += (*_iter1121).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14860,14 +14860,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1121;
-            ::apache::thrift::protocol::TType _etype1124;
-            xfer += iprot->readListBegin(_etype1124, _size1121);
-            (*(this->success)).resize(_size1121);
-            uint32_t _i1125;
-            for (_i1125 = 0; _i1125 < _size1121; ++_i1125)
+            uint32_t _size1122;
+            ::apache::thrift::protocol::TType _etype1125;
+            xfer += iprot->readListBegin(_etype1125, _size1122);
+            (*(this->success)).resize(_size1122);
+            uint32_t _i1126;
+            for (_i1126 = 0; _i1126 < _size1122; ++_i1126)
             {
-              xfer += (*(this->success))[_i1125].read(iprot);
+              xfer += (*(this->success))[_i1126].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14966,14 +14966,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 _size1126;
-            ::apache::thrift::protocol::TType _etype1129;
-            xfer += iprot->readListBegin(_etype1129, _size1126);
-            this->group_names.resize(_size1126);
-            uint32_t _i1130;
-            for (_i1130 = 0; _i1130 < _size1126; ++_i1130)
+            uint32_t _size1127;
+            ::apache::thrift::protocol::TType _etype1130;
+            xfer += iprot->readListBegin(_etype1130, _size1127);
+            this->group_names.resize(_size1127);
+            uint32_t _i1131;
+            for (_i1131 = 0; _i1131 < _size1127; ++_i1131)
             {
-              xfer += iprot->readString(this->group_names[_i1130]);
+              xfer += iprot->readString(this->group_names[_i1131]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15018,10 +15018,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 _iter1131;
-    for (_iter1131 = this->group_names.begin(); _iter1131 != this->group_names.end(); ++_iter1131)
+    std::vector<std::string> ::const_iterator _iter1132;
+    for (_iter1132 = this->group_names.begin(); _iter1132 != this->group_names.end(); ++_iter1132)
     {
-      xfer += oprot->writeString((*_iter1131));
+      xfer += oprot->writeString((*_iter1132));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15061,10 +15061,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 _iter1132;
-    for (_iter1132 = (*(this->group_names)).begin(); _iter1132 != (*(this->group_names)).end(); ++_iter1132)
+    std::vector<std::string> ::const_iterator _iter1133;
+    for (_iter1133 = (*(this->group_names)).begin(); _iter1133 != (*(this->group_names)).end(); ++_iter1133)
     {
-      xfer += oprot->writeString((*_iter1132));
+      xfer += oprot->writeString((*_iter1133));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15105,14 +15105,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1133;
-            ::apache::thrift::protocol::TType _etype1136;
-            xfer += iprot->readListBegin(_etype1136, _size1133);
-            this->success.resize(_size1133);
-            uint32_t _i1137;
-            for (_i1137 = 0; _i1137 < _size1133; ++_i1137)
+            uint32_t _size1134;
+            ::apache::thrift::protocol::TType _etype1137;
+            xfer += iprot->readListBegin(_etype1137, _size1134);
+            this->success.resize(_size1134);
+            uint32_t _i1138;
+            for (_i1138 = 0; _i1138 < _size1134; ++_i1138)
             {
-              xfer += this->success[_i1137].read(iprot);
+              xfer += this->success[_i1138].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15159,10 +15159,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 _iter1138;
-      for (_iter1138 = this->success.begin(); _iter1138 != this->success.end(); ++_iter1138)
+      std::vector<Partition> ::const_iterator _iter1139;
+      for (_iter1139 = this->success.begin(); _iter1139 != this->success.end(); ++_iter1139)
       {
-        xfer += (*_iter1138).write(oprot);
+        xfer += (*_iter1139).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -15211,14 +15211,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1139;
-            ::apache::thrift::protocol::TType _etype1142;
-            xfer += iprot->readListBegin(_etype1142, _size1139);
-            (*(this->success)).resize(_size1139);
-            uint32_t _i1143;
-            for (_i1143 = 0; _i1143 < _size1139; ++_i1143)
+            uint32_t _size1140;
+            ::apache::thrift::protocol::TType _etype1143;
+            xfer += iprot->readListBegin(_etype1143, _size1140);
+            (*(this->success)).resize(_size1140);
+            uint32_t _i1144;
+            for (_i1144 = 0; _i1144 < _size1140; ++_i1144)
             {
-              xfer += (*(this->success))[_i1143].read(iprot);
+              xfer += (*(this->success))[_i1144].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15396,14 +15396,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1144;
-            ::apache::thrift::protocol::TType _etype1147;
-            xfer += iprot->readListBegin(_etype1147, _size1144);
-            this->success.resize(_size1144);
-            uint32_t _i1148;
-            for (_i1148 = 0;

<TRUNCATED>

[3/6] hive git commit: HIVE-19371: Add table ownerType to HMS thrift API (Sergio Pena, reviewed by Sahil Takiar, Vihang Karajgaonkar)

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index c5bd146..feda941 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -2068,7 +2068,7 @@ inline std::ostream& operator<<(std::ostream& out, const StorageDescriptor& obj)
 }
 
 typedef struct _Table__isset {
-  _Table__isset() : tableName(false), dbName(false), owner(false), createTime(false), lastAccessTime(false), retention(false), sd(false), partitionKeys(false), parameters(false), viewOriginalText(false), viewExpandedText(false), tableType(false), privileges(false), temporary(true), rewriteEnabled(false) {}
+  _Table__isset() : tableName(false), dbName(false), owner(false), createTime(false), lastAccessTime(false), retention(false), sd(false), partitionKeys(false), parameters(false), viewOriginalText(false), viewExpandedText(false), tableType(false), privileges(false), temporary(true), rewriteEnabled(false), ownerType(true) {}
   bool tableName :1;
   bool dbName :1;
   bool owner :1;
@@ -2084,6 +2084,7 @@ typedef struct _Table__isset {
   bool privileges :1;
   bool temporary :1;
   bool rewriteEnabled :1;
+  bool ownerType :1;
 } _Table__isset;
 
 class Table {
@@ -2091,7 +2092,9 @@ class Table {
 
   Table(const Table&);
   Table& operator=(const Table&);
-  Table() : tableName(), dbName(), owner(), createTime(0), lastAccessTime(0), retention(0), viewOriginalText(), viewExpandedText(), tableType(), temporary(false), rewriteEnabled(0) {
+  Table() : tableName(), dbName(), owner(), createTime(0), lastAccessTime(0), retention(0), viewOriginalText(), viewExpandedText(), tableType(), temporary(false), rewriteEnabled(0), ownerType((PrincipalType::type)1) {
+    ownerType = (PrincipalType::type)1;
+
   }
 
   virtual ~Table() throw();
@@ -2110,6 +2113,7 @@ class Table {
   PrincipalPrivilegeSet privileges;
   bool temporary;
   bool rewriteEnabled;
+  PrincipalType::type ownerType;
 
   _Table__isset __isset;
 
@@ -2143,6 +2147,8 @@ class Table {
 
   void __set_rewriteEnabled(const bool val);
 
+  void __set_ownerType(const PrincipalType::type val);
+
   bool operator == (const Table & rhs) const
   {
     if (!(tableName == rhs.tableName))
@@ -2181,6 +2187,10 @@ class Table {
       return false;
     else if (__isset.rewriteEnabled && !(rewriteEnabled == rhs.rewriteEnabled))
       return false;
+    if (__isset.ownerType != rhs.__isset.ownerType)
+      return false;
+    else if (__isset.ownerType && !(ownerType == rhs.ownerType))
+      return false;
     return true;
   }
   bool operator != (const Table &rhs) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
index 800219f..ee752ab 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
@@ -53,6 +53,7 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
   private static final org.apache.thrift.protocol.TField PRIVILEGES_FIELD_DESC = new org.apache.thrift.protocol.TField("privileges", org.apache.thrift.protocol.TType.STRUCT, (short)13);
   private static final org.apache.thrift.protocol.TField TEMPORARY_FIELD_DESC = new org.apache.thrift.protocol.TField("temporary", org.apache.thrift.protocol.TType.BOOL, (short)14);
   private static final org.apache.thrift.protocol.TField REWRITE_ENABLED_FIELD_DESC = new org.apache.thrift.protocol.TField("rewriteEnabled", org.apache.thrift.protocol.TType.BOOL, (short)15);
+  private static final org.apache.thrift.protocol.TField OWNER_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("ownerType", org.apache.thrift.protocol.TType.I32, (short)16);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -75,6 +76,7 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
   private PrincipalPrivilegeSet privileges; // optional
   private boolean temporary; // optional
   private boolean rewriteEnabled; // optional
+  private PrincipalType ownerType; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -92,7 +94,12 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     TABLE_TYPE((short)12, "tableType"),
     PRIVILEGES((short)13, "privileges"),
     TEMPORARY((short)14, "temporary"),
-    REWRITE_ENABLED((short)15, "rewriteEnabled");
+    REWRITE_ENABLED((short)15, "rewriteEnabled"),
+    /**
+     * 
+     * @see PrincipalType
+     */
+    OWNER_TYPE((short)16, "ownerType");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -137,6 +144,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
           return TEMPORARY;
         case 15: // REWRITE_ENABLED
           return REWRITE_ENABLED;
+        case 16: // OWNER_TYPE
+          return OWNER_TYPE;
         default:
           return null;
       }
@@ -183,7 +192,7 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
   private static final int __TEMPORARY_ISSET_ID = 3;
   private static final int __REWRITEENABLED_ISSET_ID = 4;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.TEMPORARY,_Fields.REWRITE_ENABLED};
+  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.TEMPORARY,_Fields.REWRITE_ENABLED,_Fields.OWNER_TYPE};
   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);
@@ -220,6 +229,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     tmpMap.put(_Fields.REWRITE_ENABLED, new org.apache.thrift.meta_data.FieldMetaData("rewriteEnabled", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.OWNER_TYPE, new org.apache.thrift.meta_data.FieldMetaData("ownerType", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, PrincipalType.class)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Table.class, metaDataMap);
   }
@@ -227,6 +238,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
   public Table() {
     this.temporary = false;
 
+    this.ownerType = org.apache.hadoop.hive.metastore.api.PrincipalType.USER;
+
   }
 
   public Table(
@@ -306,6 +319,9 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     }
     this.temporary = other.temporary;
     this.rewriteEnabled = other.rewriteEnabled;
+    if (other.isSetOwnerType()) {
+      this.ownerType = other.ownerType;
+    }
   }
 
   public Table deepCopy() {
@@ -334,6 +350,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
 
     setRewriteEnabledIsSet(false);
     this.rewriteEnabled = false;
+    this.ownerType = org.apache.hadoop.hive.metastore.api.PrincipalType.USER;
+
   }
 
   public String getTableName() {
@@ -702,6 +720,37 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REWRITEENABLED_ISSET_ID, value);
   }
 
+  /**
+   * 
+   * @see PrincipalType
+   */
+  public PrincipalType getOwnerType() {
+    return this.ownerType;
+  }
+
+  /**
+   * 
+   * @see PrincipalType
+   */
+  public void setOwnerType(PrincipalType ownerType) {
+    this.ownerType = ownerType;
+  }
+
+  public void unsetOwnerType() {
+    this.ownerType = null;
+  }
+
+  /** Returns true if field ownerType is set (has been assigned a value) and false otherwise */
+  public boolean isSetOwnerType() {
+    return this.ownerType != null;
+  }
+
+  public void setOwnerTypeIsSet(boolean value) {
+    if (!value) {
+      this.ownerType = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case TABLE_NAME:
@@ -824,6 +873,14 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       }
       break;
 
+    case OWNER_TYPE:
+      if (value == null) {
+        unsetOwnerType();
+      } else {
+        setOwnerType((PrincipalType)value);
+      }
+      break;
+
     }
   }
 
@@ -874,6 +931,9 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     case REWRITE_ENABLED:
       return isRewriteEnabled();
 
+    case OWNER_TYPE:
+      return getOwnerType();
+
     }
     throw new IllegalStateException();
   }
@@ -915,6 +975,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       return isSetTemporary();
     case REWRITE_ENABLED:
       return isSetRewriteEnabled();
+    case OWNER_TYPE:
+      return isSetOwnerType();
     }
     throw new IllegalStateException();
   }
@@ -1067,6 +1129,15 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         return false;
     }
 
+    boolean this_present_ownerType = true && this.isSetOwnerType();
+    boolean that_present_ownerType = true && that.isSetOwnerType();
+    if (this_present_ownerType || that_present_ownerType) {
+      if (!(this_present_ownerType && that_present_ownerType))
+        return false;
+      if (!this.ownerType.equals(that.ownerType))
+        return false;
+    }
+
     return true;
   }
 
@@ -1149,6 +1220,11 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     if (present_rewriteEnabled)
       list.add(rewriteEnabled);
 
+    boolean present_ownerType = true && (isSetOwnerType());
+    list.add(present_ownerType);
+    if (present_ownerType)
+      list.add(ownerType.getValue());
+
     return list.hashCode();
   }
 
@@ -1310,6 +1386,16 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetOwnerType()).compareTo(other.isSetOwnerType());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetOwnerType()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ownerType, other.ownerType);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1435,6 +1521,16 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       sb.append(this.rewriteEnabled);
       first = false;
     }
+    if (isSetOwnerType()) {
+      if (!first) sb.append(", ");
+      sb.append("ownerType:");
+      if (this.ownerType == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.ownerType);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1631,6 +1727,14 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 16: // OWNER_TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.ownerType = org.apache.hadoop.hive.metastore.api.PrincipalType.findByValue(iprot.readI32());
+              struct.setOwnerTypeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1730,6 +1834,13 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         oprot.writeBool(struct.rewriteEnabled);
         oprot.writeFieldEnd();
       }
+      if (struct.ownerType != null) {
+        if (struct.isSetOwnerType()) {
+          oprot.writeFieldBegin(OWNER_TYPE_FIELD_DESC);
+          oprot.writeI32(struct.ownerType.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1793,7 +1904,10 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       if (struct.isSetRewriteEnabled()) {
         optionals.set(14);
       }
-      oprot.writeBitSet(optionals, 15);
+      if (struct.isSetOwnerType()) {
+        optionals.set(15);
+      }
+      oprot.writeBitSet(optionals, 16);
       if (struct.isSetTableName()) {
         oprot.writeString(struct.tableName);
       }
@@ -1852,12 +1966,15 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       if (struct.isSetRewriteEnabled()) {
         oprot.writeBool(struct.rewriteEnabled);
       }
+      if (struct.isSetOwnerType()) {
+        oprot.writeI32(struct.ownerType.getValue());
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, Table struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(15);
+      BitSet incoming = iprot.readBitSet(16);
       if (incoming.get(0)) {
         struct.tableName = iprot.readString();
         struct.setTableNameIsSet(true);
@@ -1941,6 +2058,10 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         struct.rewriteEnabled = iprot.readBool();
         struct.setRewriteEnabledIsSet(true);
       }
+      if (incoming.get(15)) {
+        struct.ownerType = org.apache.hadoop.hive.metastore.api.PrincipalType.findByValue(iprot.readI32());
+        struct.setOwnerTypeIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/Types.php b/metastore/src/gen/thrift/gen-php/metastore/Types.php
index d81ff27..04f9a2e 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -4567,6 +4567,10 @@ class Table {
    * @var bool
    */
   public $rewriteEnabled = null;
+  /**
+   * @var int
+   */
+  public $ownerType =   1;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -4646,6 +4650,10 @@ class Table {
           'var' => 'rewriteEnabled',
           'type' => TType::BOOL,
           ),
+        16 => array(
+          'var' => 'ownerType',
+          'type' => TType::I32,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -4694,6 +4702,9 @@ class Table {
       if (isset($vals['rewriteEnabled'])) {
         $this->rewriteEnabled = $vals['rewriteEnabled'];
       }
+      if (isset($vals['ownerType'])) {
+        $this->ownerType = $vals['ownerType'];
+      }
     }
   }
 
@@ -4847,6 +4858,13 @@ class Table {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 16:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->ownerType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -4966,6 +4984,11 @@ class Table {
       $xfer += $output->writeBool($this->rewriteEnabled);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->ownerType !== null) {
+      $xfer += $output->writeFieldBegin('ownerType', TType::I32, 16);
+      $xfer += $output->writeI32($this->ownerType);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;

http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 4c305cb..e58b4a4 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -3134,6 +3134,7 @@ class Table:
    - privileges
    - temporary
    - rewriteEnabled
+   - ownerType
   """
 
   thrift_spec = (
@@ -3153,9 +3154,10 @@ class Table:
     (13, TType.STRUCT, 'privileges', (PrincipalPrivilegeSet, PrincipalPrivilegeSet.thrift_spec), None, ), # 13
     (14, TType.BOOL, 'temporary', None, False, ), # 14
     (15, TType.BOOL, 'rewriteEnabled', None, None, ), # 15
+    (16, TType.I32, 'ownerType', None,     1, ), # 16
   )
 
-  def __init__(self, tableName=None, dbName=None, owner=None, createTime=None, lastAccessTime=None, retention=None, sd=None, partitionKeys=None, parameters=None, viewOriginalText=None, viewExpandedText=None, tableType=None, privileges=None, temporary=thrift_spec[14][4], rewriteEnabled=None,):
+  def __init__(self, tableName=None, dbName=None, owner=None, createTime=None, lastAccessTime=None, retention=None, sd=None, partitionKeys=None, parameters=None, viewOriginalText=None, viewExpandedText=None, tableType=None, privileges=None, temporary=thrift_spec[14][4], rewriteEnabled=None, ownerType=thrift_spec[16][4],):
     self.tableName = tableName
     self.dbName = dbName
     self.owner = owner
@@ -3171,6 +3173,7 @@ class Table:
     self.privileges = privileges
     self.temporary = temporary
     self.rewriteEnabled = rewriteEnabled
+    self.ownerType = ownerType
 
   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:
@@ -3270,6 +3273,11 @@ class Table:
           self.rewriteEnabled = iprot.readBool()
         else:
           iprot.skip(ftype)
+      elif fid == 16:
+        if ftype == TType.I32:
+          self.ownerType = iprot.readI32()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -3347,6 +3355,10 @@ class Table:
       oprot.writeFieldBegin('rewriteEnabled', TType.BOOL, 15)
       oprot.writeBool(self.rewriteEnabled)
       oprot.writeFieldEnd()
+    if self.ownerType is not None:
+      oprot.writeFieldBegin('ownerType', TType.I32, 16)
+      oprot.writeI32(self.ownerType)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -3371,6 +3383,7 @@ class Table:
     value = (value * 31) ^ hash(self.privileges)
     value = (value * 31) ^ hash(self.temporary)
     value = (value * 31) ^ hash(self.rewriteEnabled)
+    value = (value * 31) ^ hash(self.ownerType)
     return value
 
   def __repr__(self):

http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 8f550e8..30ff341 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -735,6 +735,7 @@ class Table
   PRIVILEGES = 13
   TEMPORARY = 14
   REWRITEENABLED = 15
+  OWNERTYPE = 16
 
   FIELDS = {
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'},
@@ -751,12 +752,16 @@ class Table
     TABLETYPE => {:type => ::Thrift::Types::STRING, :name => 'tableType'},
     PRIVILEGES => {:type => ::Thrift::Types::STRUCT, :name => 'privileges', :class => ::PrincipalPrivilegeSet, :optional => true},
     TEMPORARY => {:type => ::Thrift::Types::BOOL, :name => 'temporary', :default => false, :optional => true},
-    REWRITEENABLED => {:type => ::Thrift::Types::BOOL, :name => 'rewriteEnabled', :optional => true}
+    REWRITEENABLED => {:type => ::Thrift::Types::BOOL, :name => 'rewriteEnabled', :optional => true},
+    OWNERTYPE => {:type => ::Thrift::Types::I32, :name => 'ownerType', :default =>     1, :optional => true, :enum_class => ::PrincipalType}
   }
 
   def struct_fields; FIELDS; end
 
   def validate
+    unless @ownerType.nil? || ::PrincipalType::VALID_VALUES.include?(@ownerType)
+      raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field ownerType!')
+    end
   end
 
   ::Thrift::Struct.generate_accessors self

http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/ql/src/test/results/clientpositive/create_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_view.q.out b/ql/src/test/results/clientpositive/create_view.q.out
index d3b858a..ff2dc87 100644
--- a/ql/src/test/results/clientpositive/create_view.q.out
+++ b/ql/src/test/results/clientpositive/create_view.q.out
@@ -731,7 +731,7 @@ c                   	string
 	 	 
 #### A masked pattern was here ####
 FROM table1, viewExpandedText:SELECT `_c0` AS `c` FROM (SELECT `test_translate`('abc', 'a', 'b')	 	 
-FROM `default`.`table1`) `default.view8`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: DESCRIBE FORMATTED view8
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view8
@@ -809,7 +809,7 @@ m                   	int
 	 	 
 #### A masked pattern was here ####
 FROM src, viewExpandedText:SELECT `_c0` AS `m` FROM (SELECT `test_max`(length(`src`.`value`))	 	 
-FROM `default`.`src`) `default.view9`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: DESCRIBE FORMATTED view9
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view9
@@ -887,7 +887,7 @@ m                   	int
 	 	 
 #### A masked pattern was here ####
 FROM src, viewExpandedText:SELECT `_c0` AS `m` FROM (SELECT `test_max`(length(`src`.`value`))	 	 
-FROM `default`.`src`) `default.view9`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: DESCRIBE FORMATTED view9
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view9
@@ -1032,7 +1032,7 @@ boom                	int
 	 	 
 #### A masked pattern was here ####
 FROM table1, viewExpandedText:SELECT `test_explode`(array(1,2,3)) AS (`boom`)	 	 
-FROM `default`.`table1`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: DESCRIBE FORMATTED view11
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view11
@@ -1189,7 +1189,7 @@ key                 	int
 	 	 
 #### A masked pattern was here ####
 FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 ON key) s, viewExpandedText:SELECT `s`.`key`	 	 
-FROM `default`.`srcbucket` TABLESAMPLE (BUCKET 1 OUT OF 5 ON `key`) `s`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: DESCRIBE FORMATTED view13
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view13
@@ -1310,7 +1310,7 @@ JOIN
      (select 'tst1' as `key`, cast(count(1) as string) as `value` from `default`.`src` `s3`	 	 
                          UNION  ALL	 	 
       select `s4`.`key` as `key`, `s4`.`value` as `value` from `default`.`src` `s4` where `s4`.`key` < 10) `unionsrc2`	 	 
-ON (`unionsrc1`.`key` = `unionsrc2`.`key`), tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: DESCRIBE FORMATTED view14
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view14
@@ -1429,7 +1429,7 @@ value_count         	bigint
 FROM src	 	 
 GROUP BY key, viewExpandedText:SELECT `src`.`key`,COUNT(`src`.`value`) AS `value_count`	 	 
 FROM `default`.`src`	 	 
-GROUP BY `src`.`key`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: DESCRIBE FORMATTED view15
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view15
@@ -1515,7 +1515,7 @@ value               	string
 	 	 
 #### A masked pattern was here ####
 FROM src, viewExpandedText:SELECT DISTINCT `src`.`value`	 	 
-FROM `default`.`src`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: DESCRIBE FORMATTED view16
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view16

http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/ql/src/test/results/clientpositive/create_view_partitioned.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_view_partitioned.q.out b/ql/src/test/results/clientpositive/create_view_partitioned.q.out
index b1bf7d6..7f63797 100644
--- a/ql/src/test/results/clientpositive/create_view_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/create_view_partitioned.q.out
@@ -49,7 +49,7 @@ value               	string
 FROM src	 	 
 WHERE key=86, viewExpandedText:SELECT `src`.`key`, `src`.`value`	 	 
 FROM `default`.`src`	 	 
-WHERE `src`.`key`=86, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: DESCRIBE FORMATTED vp1
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@vp1

http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/ql/src/test/results/clientpositive/cteViews.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cteViews.q.out b/ql/src/test/results/clientpositive/cteViews.q.out
index 3a9bd43..9e186f4 100644
--- a/ql/src/test/results/clientpositive/cteViews.q.out
+++ b/ql/src/test/results/clientpositive/cteViews.q.out
@@ -46,7 +46,7 @@ key                 	string
 	 	 
 #### A masked pattern was here ####
 select key from cte, viewExpandedText:with cte as (select `src`.`key`, `src`.`value` from `default`.`src` order by key limit 5)	 	 
-select `cte`.`key` from cte, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: create database bug
 PREHOOK: type: CREATEDATABASE
 PREHOOK: Output: database:bug
@@ -121,7 +121,7 @@ value               	string
 	 	 
 #### A masked pattern was here ####
 select * from cte, viewExpandedText:with cte as (select `src`.`key`, `src`.`value` from `default`.`src`  order by `src`.`key` limit 5)	 	 
-select `cte`.`key`, `cte`.`value` from cte, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: create database bug
 PREHOOK: type: CREATEDATABASE
 PREHOOK: Output: database:bug
@@ -194,7 +194,7 @@ key                 	string
 	 	 
 #### A masked pattern was here ####
 select * from src1, viewExpandedText:with src1 as (select `src`.`key` from `default`.`src` order by key limit 5)	 	 
-select `src1`.`key` from src1, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: create database bug
 PREHOOK: type: CREATEDATABASE
 PREHOOK: Output: database:bug
@@ -259,7 +259,7 @@ key                 	string
 	 	 
 #### A masked pattern was here ####
 select * from src1 a where a.key is not null, viewExpandedText:with src1 as (select `src`.`key` from `default`.`src` order by key limit 5)	 	 
-select `a`.`key` from src1 a where `a`.`key` is not null, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src

http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/ql/src/test/results/clientpositive/llap/cbo_rp_unionDistinct_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/cbo_rp_unionDistinct_2.q.out b/ql/src/test/results/clientpositive/llap/cbo_rp_unionDistinct_2.q.out
index 3b3983f..733d4aa 100644
--- a/ql/src/test/results/clientpositive/llap/cbo_rp_unionDistinct_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/cbo_rp_unionDistinct_2.q.out
@@ -363,7 +363,7 @@ union
 select `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
 union all	 	 
 select `u3`.`key` as `key`, `u3`.`value` from `default`.`u3`	 	 
-) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1
@@ -435,7 +435,7 @@ select distinct * from u2
 select distinct `u1`.`key`, `u1`.`value` from `default`.`u1`	 	 
 union  	 	 
 select distinct `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
-) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1
@@ -502,7 +502,7 @@ select distinct * from u2
 select distinct `u1`.`key`, `u1`.`value` from `default`.`u1`	 	 
 union all 	 	 
 select distinct `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
-) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1

http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/ql/src/test/results/clientpositive/llap/subquery_views.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/subquery_views.q.out b/ql/src/test/results/clientpositive/llap/subquery_views.q.out
index bfd56e6..be53e93 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_views.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_views.q.out
@@ -41,7 +41,7 @@ from `default`.`src` `b`
 where exists	 	 
   (select `a`.`key` 	 	 
   from `default`.`src` `a` 	 	 
-  where `b`.`value` = `a`.`value`  and `a`.`key` = `b`.`key` and `a`.`value` > 'val_9'), tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: select * 
 from cv1 where cv1.key in (select key from cv1 c where c.key > '95')
 PREHOOK: type: QUERY
@@ -106,7 +106,7 @@ where `b`.`key` not in
   (select `a`.`key` 	 	 
   from `default`.`src` `a` 	 	 
   where `b`.`value` = `a`.`value`  and `a`.`key` = `b`.`key` and `a`.`value` > 'val_11'	 	 
-  ), tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: explain
 select * 
 from cv2 where cv2.key in (select key from cv2 c where c.key < '11')
@@ -572,7 +572,7 @@ having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.k
 from `default`.`src` `b`	 	 
 where `b`.`key` in (select `src`.`key` from `default`.`src` where `src`.`key` > '8')	 	 
 group by `b`.`key`, `b`.`value`	 	 
-having count(*) in (select count(*) from `default`.`src` `s1` where `s1`.`key` > '9' group by `s1`.`key` ), tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: select * from cv3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@cv3

http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/ql/src/test/results/clientpositive/llap/unionDistinct_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/unionDistinct_2.q.out b/ql/src/test/results/clientpositive/llap/unionDistinct_2.q.out
index 3b3983f..733d4aa 100644
--- a/ql/src/test/results/clientpositive/llap/unionDistinct_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/unionDistinct_2.q.out
@@ -363,7 +363,7 @@ union
 select `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
 union all	 	 
 select `u3`.`key` as `key`, `u3`.`value` from `default`.`u3`	 	 
-) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1
@@ -435,7 +435,7 @@ select distinct * from u2
 select distinct `u1`.`key`, `u1`.`value` from `default`.`u1`	 	 
 union  	 	 
 select distinct `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
-) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1
@@ -502,7 +502,7 @@ select distinct * from u2
 select distinct `u1`.`key`, `u1`.`value` from `default`.`u1`	 	 
 union all 	 	 
 select distinct `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
-) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1

http://git-wip-us.apache.org/repos/asf/hive/blob/117c3b47/ql/src/test/results/clientpositive/tez/unionDistinct_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/unionDistinct_2.q.out b/ql/src/test/results/clientpositive/tez/unionDistinct_2.q.out
index 3b3983f..733d4aa 100644
--- a/ql/src/test/results/clientpositive/tez/unionDistinct_2.q.out
+++ b/ql/src/test/results/clientpositive/tez/unionDistinct_2.q.out
@@ -363,7 +363,7 @@ union
 select `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
 union all	 	 
 select `u3`.`key` as `key`, `u3`.`value` from `default`.`u3`	 	 
-) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1
@@ -435,7 +435,7 @@ select distinct * from u2
 select distinct `u1`.`key`, `u1`.`value` from `default`.`u1`	 	 
 union  	 	 
 select distinct `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
-) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1
@@ -502,7 +502,7 @@ select distinct * from u2
 select distinct `u1`.`key`, `u1`.`value` from `default`.`u1`	 	 
 union all 	 	 
 select distinct `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
-) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
+#### A masked pattern was here ####
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1