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

[1/8] hive git commit: HIVE-15202 Concurrent compactions for the same partition may generate malformed folder structure (Eugene Koifman, reviewed by Wei Zheng)

Repository: hive
Updated Branches:
  refs/heads/master 76b311f59 -> 54b3abf52


http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/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 591485f..c735932 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -2281,6 +2281,29 @@ class CompactionRequest
   ::Thrift::Struct.generate_accessors self
 end
 
+class CompactionResponse
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  ID = 1
+  STATE = 2
+  ACCEPTED = 3
+
+  FIELDS = {
+    ID => {:type => ::Thrift::Types::I64, :name => 'id'},
+    STATE => {:type => ::Thrift::Types::STRING, :name => 'state'},
+    ACCEPTED => {:type => ::Thrift::Types::BOOL, :name => 'accepted'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field id is unset!') unless @id
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field state is unset!') unless @state
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field accepted is unset!') if @accepted.nil?
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class ShowCompactRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
 
@@ -2310,6 +2333,7 @@ class ShowCompactResponseElement
   METAINFO = 10
   ENDTIME = 11
   HADOOPJOBID = 12
+  ID = 13
 
   FIELDS = {
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbname'},
@@ -2323,7 +2347,8 @@ class ShowCompactResponseElement
     HIGHTESTTXNID => {:type => ::Thrift::Types::I64, :name => 'hightestTxnId', :optional => true},
     METAINFO => {:type => ::Thrift::Types::STRING, :name => 'metaInfo', :optional => true},
     ENDTIME => {:type => ::Thrift::Types::I64, :name => 'endTime', :optional => true},
-    HADOOPJOBID => {:type => ::Thrift::Types::STRING, :name => 'hadoopJobId', :default => %q"None", :optional => true}
+    HADOOPJOBID => {:type => ::Thrift::Types::STRING, :name => 'hadoopJobId', :default => %q"None", :optional => true},
+    ID => {:type => ::Thrift::Types::I64, :name => 'id', :optional => true}
   }
 
   def struct_fields; FIELDS; end

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index 86fb42a..7cdfc86 100644
--- a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -2367,6 +2367,21 @@ module ThriftHiveMetastore
       return
     end
 
+    def compact2(rqst)
+      send_compact2(rqst)
+      return recv_compact2()
+    end
+
+    def send_compact2(rqst)
+      send_message('compact2', Compact2_args, :rqst => rqst)
+    end
+
+    def recv_compact2()
+      result = receive_message(Compact2_result)
+      return result.success unless result.success.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'compact2 failed: unknown result')
+    end
+
     def show_compact(rqst)
       send_show_compact(rqst)
       return recv_show_compact()
@@ -4326,6 +4341,13 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'compact', seqid)
     end
 
+    def process_compact2(seqid, iprot, oprot)
+      args = read_args(iprot, Compact2_args)
+      result = Compact2_result.new()
+      result.success = @handler.compact2(args.rqst)
+      write_result(result, oprot, 'compact2', seqid)
+    end
+
     def process_show_compact(seqid, iprot, oprot)
       args = read_args(iprot, Show_compact_args)
       result = Show_compact_result.new()
@@ -9764,6 +9786,38 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Compact2_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    RQST = 1
+
+    FIELDS = {
+      RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::CompactionRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Compact2_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::CompactionResponse}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Show_compact_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     RQST = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 48bebb2..54fd544 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.metastore;
 
 import org.apache.hadoop.hive.metastore.api.ClientCapabilities;
 import org.apache.hadoop.hive.metastore.api.ClientCapability;
+import org.apache.hadoop.hive.metastore.api.CompactionResponse;
 import org.apache.hadoop.hive.metastore.api.GetTableRequest;
 import org.apache.hadoop.hive.metastore.api.GetTableResult;
 import org.apache.hadoop.hive.metastore.api.GetTablesRequest;
@@ -6399,10 +6400,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       throws TException {
       return getTxnHandler().heartbeatTxnRange(rqst);
     }
-
+    @Deprecated
     @Override
     public void compact(CompactionRequest rqst) throws TException {
-      getTxnHandler().compact(rqst);
+      compact2(rqst);
+    }
+    @Override
+    public CompactionResponse compact2(CompactionRequest rqst) throws TException {
+      return getTxnHandler().compact(rqst);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 823827d..4774899 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -2142,10 +2142,16 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     cr.setType(type);
     client.compact(cr);
   }
-
+  @Deprecated
   @Override
   public void compact(String dbname, String tableName, String partitionName, CompactionType type,
                       Map<String, String> tblproperties) throws TException {
+    compact2(dbname, tableName, partitionName, type, tblproperties);
+  }
+
+  @Override
+  public CompactionResponse compact2(String dbname, String tableName, String partitionName, CompactionType type,
+                      Map<String, String> tblproperties) throws TException {
     CompactionRequest cr = new CompactionRequest();
     if (dbname == null) cr.setDbname(DEFAULT_DATABASE_NAME);
     else cr.setDbname(dbname);
@@ -2153,9 +2159,8 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     if (partitionName != null) cr.setPartitionname(partitionName);
     cr.setType(type);
     cr.setProperties(tblproperties);
-    client.compact(cr);
+    return client.compact2(cr);
   }
-
   @Override
   public ShowCompactResponse showCompactions() throws TException {
     return client.show_compact(new ShowCompactRequest());

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index b770559..5ea000a 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CompactionResponse;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
@@ -1456,12 +1457,15 @@ public interface IMetaStoreClient {
   @Deprecated
   void compact(String dbname, String tableName, String partitionName,  CompactionType type)
       throws TException;
-
+  @Deprecated
+  void compact(String dbname, String tableName, String partitionName, CompactionType type,
+               Map<String, String> tblproperties) throws TException;
   /**
    * Send a request to compact a table or partition.  This will not block until the compaction is
    * complete.  It will instead put a request on the queue for that table or partition to be
    * compacted.  No checking is done on the dbname, tableName, or partitionName to make sure they
-   * refer to valid objects.  It is assumed this has already been done by the caller.
+   * refer to valid objects.  It is assumed this has already been done by the caller.  At most one
+   * Compaction can be scheduled/running for any given resource at a time.
    * @param dbname Name of the database the table is in.  If null, this will be assumed to be
    *               'default'.
    * @param tableName Name of the table to be compacted.  This cannot be null.  If partitionName
@@ -1469,10 +1473,11 @@ public interface IMetaStoreClient {
    * @param partitionName Name of the partition to be compacted
    * @param type Whether this is a major or minor compaction.
    * @param tblproperties the list of tblproperties to override for this compact. Can be null.
+   * @return id of newly scheduled compaction or id/state of one which is already scheduled/running
    * @throws TException
    */
-  void compact(String dbname, String tableName, String partitionName, CompactionType type,
-               Map<String, String> tblproperties) throws TException;
+  CompactionResponse compact2(String dbname, String tableName, String partitionName, CompactionType type,
+                              Map<String, String> tblproperties) throws TException;
 
   /**
    * Get a list of all current compactions.

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index 382df1f..6ea43b4 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -1375,18 +1375,49 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     stmt.executeUpdate(s);
     return id;
   }
-  public long compact(CompactionRequest rqst) throws MetaException {
+  @Override
+  public CompactionResponse compact(CompactionRequest rqst) throws MetaException {
     // Put a compaction request in the queue.
     try {
       Connection dbConn = null;
       Statement stmt = null;
+      TxnStore.MutexAPI.LockHandle handle = null;
       try {
         lockInternal();
+        /**
+         * MUTEX_KEY.CompactionScheduler lock ensures that there is only 1 entry in
+         * Initiated/Working state for any resource.  This ensures that we don't run concurrent
+         * compactions for any resource.
+         */
+        handle = getMutexAPI().acquireLock(MUTEX_KEY.CompactionScheduler.name());
         dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
-        
+
         long id = generateCompactionQueueId(stmt);
 
+        StringBuilder sb = new StringBuilder("select cq_id, cq_state from COMPACTION_QUEUE where").
+          append(" cq_state IN(").append(quoteChar(INITIATED_STATE)).
+            append(",").append(quoteChar(WORKING_STATE)).
+          append(") AND cq_database=").append(quoteString(rqst.getDbname())).
+          append(" AND cq_table=").append(quoteString(rqst.getTablename())).append(" AND ");
+        if(rqst.getPartitionname() == null) {
+          sb.append("cq_partition is null");
+        }
+        else {
+          sb.append("cq_partition=").append(quoteString(rqst.getPartitionname()));
+        }
+
+        LOG.debug("Going to execute query <" + sb.toString() + ">");
+        ResultSet rs = stmt.executeQuery(sb.toString());
+        if(rs.next()) {
+          long enqueuedId = rs.getLong(1);
+          String state = compactorStateToResponse(rs.getString(2).charAt(0));
+          LOG.info("Ignoring request to compact " + rqst.getDbname() + "/" + rqst.getTablename() +
+            "/" + rqst.getPartitionname() + " since it is already " + quoteString(state) +
+            " with id=" + enqueuedId);
+          return new CompactionResponse(enqueuedId, state, false);
+        }
+        close(rs);
         StringBuilder buf = new StringBuilder("insert into COMPACTION_QUEUE (cq_id, cq_database, " +
           "cq_table, ");
         String partName = rqst.getPartitionname();
@@ -1437,7 +1468,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         stmt.executeUpdate(s);
         LOG.debug("Going to commit");
         dbConn.commit();
-        return id;
+        return new CompactionResponse(id, INITIATED_RESPONSE, true);
       } catch (SQLException e) {
         LOG.debug("Going to rollback");
         rollbackDBConn(dbConn);
@@ -1447,6 +1478,9 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       } finally {
         closeStmt(stmt);
         closeDbConn(dbConn);
+        if(handle != null) {
+          handle.releaseLocks();
+        }
         unlockInternal();
       }
     } catch (RetryException e) {
@@ -1454,6 +1488,18 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     }
   }
 
+  private static String compactorStateToResponse(char s) {
+    switch (s) {
+      case INITIATED_STATE: return INITIATED_RESPONSE;
+      case WORKING_STATE: return WORKING_RESPONSE;
+      case READY_FOR_CLEANING: return CLEANING_RESPONSE;
+      case FAILED_STATE: return FAILED_RESPONSE;
+      case SUCCEEDED_STATE: return SUCCEEDED_RESPONSE;
+      case ATTEMPTED_STATE: return ATTEMPTED_RESPONSE;
+      default:
+        return Character.toString(s);
+    }
+  }
   public ShowCompactResponse showCompact(ShowCompactRequest rqst) throws MetaException {
     ShowCompactResponse response = new ShowCompactResponse(new ArrayList<ShowCompactResponseElement>());
     Connection dbConn = null;
@@ -1477,16 +1523,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           e.setDbname(rs.getString(1));
           e.setTablename(rs.getString(2));
           e.setPartitionname(rs.getString(3));
-          switch (rs.getString(4).charAt(0)) {
-            case INITIATED_STATE: e.setState(INITIATED_RESPONSE); break;
-            case WORKING_STATE: e.setState(WORKING_RESPONSE); break;
-            case READY_FOR_CLEANING: e.setState(CLEANING_RESPONSE); break;
-            case FAILED_STATE: e.setState(FAILED_RESPONSE); break;
-            case SUCCEEDED_STATE: e.setState(SUCCEEDED_RESPONSE); break;
-            case ATTEMPTED_STATE: e.setState(ATTEMPTED_RESPONSE); break;
-            default:
-              //do nothing to handle RU/D if we add another status
-          }
+          e.setState(compactorStateToResponse(rs.getString(4).charAt(0)));
           switch (rs.getString(5).charAt(0)) {
             case MAJOR_TYPE: e.setType(CompactionType.MAJOR); break;
             case MINOR_TYPE: e.setType(CompactionType.MINOR); break;

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
index 5b56aaf..3c06517 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
@@ -47,7 +47,8 @@ import java.util.Set;
 @InterfaceStability.Evolving
 public interface TxnStore {
 
-  public static enum MUTEX_KEY {Initiator, Cleaner, HouseKeeper, CompactionHistory, CheckLock, WriteSetCleaner}
+  public static enum MUTEX_KEY {Initiator, Cleaner, HouseKeeper, CompactionHistory, CheckLock,
+    WriteSetCleaner, CompactionScheduler}
   // Compactor states (Should really be enum)
   static final public String INITIATED_RESPONSE = "initiated";
   static final public String WORKING_RESPONSE = "working";
@@ -185,10 +186,10 @@ public interface TxnStore {
    * Submit a compaction request into the queue.  This is called when a user manually requests a
    * compaction.
    * @param rqst information on what to compact
-   * @return id of the compaction that has been started
+   * @return id of the compaction that has been started or existing id if this resource is already scheduled
    * @throws MetaException
    */
-  public long compact(CompactionRequest rqst) throws MetaException;
+  public CompactionResponse compact(CompactionRequest rqst) throws MetaException;
 
   /**
    * Show list of current compactions

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/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 cfece77..0ac9053 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
@@ -69,6 +69,7 @@ import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CompactionResponse;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -1809,8 +1810,15 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       }
       partName = partitions.get(0).getName();
     }
-    db.compact(tbl.getDbName(), tbl.getTableName(), partName, desc.getCompactionType(), desc.getProps());
-    console.printInfo("Compaction enqueued.");
+    CompactionResponse resp = db.compact2(tbl.getDbName(), tbl.getTableName(), partName,
+      desc.getCompactionType(), desc.getProps());
+    if(resp.isAccepted()) {
+      console.printInfo("Compaction enqueued with id " + resp.getId());
+    }
+    else {
+      console.printInfo("Compaction already enqueued with id " + resp.getId() +
+        "; State is " + resp.getState());
+    }
     return 0;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index 6627587..8f230fc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -86,6 +86,7 @@ import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CompactionResponse;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
@@ -3694,30 +3695,38 @@ private void constructOneLBLocationMap(FileStatus fSta,
   }
 
   /**
-   * Enqueue a compaction request.
+   * @deprecated use {@link #compact2(String, String, String, String, Map)}
+   */
+  public void compact(String dbname, String tableName, String partName, String compactType,
+                      Map<String, String> tblproperties) throws HiveException {
+    compact2(dbname, tableName, partName, compactType, tblproperties);
+  }
+  /**
+   * Enqueue a compaction request.  Only 1 compaction for a given resource (db/table/partSpec) can
+   * be scheduled/running at any given time.
    * @param dbname name of the database, if null default will be used.
    * @param tableName name of the table, cannot be null
    * @param partName name of the partition, if null table will be compacted (valid only for
    *                 non-partitioned tables).
    * @param compactType major or minor
    * @param tblproperties the list of tblproperties to overwrite for this compaction
+   * @return id of new request or id already existing request for specified resource
    * @throws HiveException
    */
-  public void compact(String dbname, String tableName, String partName, String compactType,
-                      Map<String, String> tblproperties)
+  public CompactionResponse compact2(String dbname, String tableName, String partName, String compactType,
+                                     Map<String, String> tblproperties)
       throws HiveException {
     try {
       CompactionType cr = null;
       if ("major".equals(compactType)) cr = CompactionType.MAJOR;
       else if ("minor".equals(compactType)) cr = CompactionType.MINOR;
       else throw new RuntimeException("Unknown compaction type " + compactType);
-      getMSC().compact(dbname, tableName, partName, cr, tblproperties);
+      return getMSC().compact2(dbname, tableName, partName, cr, tblproperties);
     } catch (Exception e) {
       LOG.error(StringUtils.stringifyException(e));
       throw new HiveException(e);
     }
   }
-
   public ShowCompactResponse showCompactions() throws HiveException {
     try {
       return getMSC().showCompactions();

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
index 8152c89..af4a1da 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.CompactionRequest;
+import org.apache.hadoop.hive.metastore.api.CompactionResponse;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Partition;
@@ -336,7 +337,10 @@ public class Initiator extends CompactorThread {
     if (ci.partName != null) rqst.setPartitionname(ci.partName);
     rqst.setRunas(runAs);
     LOG.info("Requesting compaction: " + rqst);
-    ci.id = txnHandler.compact(rqst);
+    CompactionResponse resp = txnHandler.compact(rqst);
+    if(resp.isAccepted()) {
+      ci.id = resp.getId();
+    }
   }
 
   // Because TABLE_NO_AUTO_COMPACT was originally assumed to be NO_AUTO_COMPACT and then was moved

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
index 2804e21..dbe1ce8 100644
--- a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
+++ b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
 import org.apache.hadoop.hive.metastore.api.CheckLockRequest;
 import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
 import org.apache.hadoop.hive.metastore.api.CompactionRequest;
+import org.apache.hadoop.hive.metastore.api.CompactionResponse;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.DataOperationType;
 import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
@@ -1142,6 +1143,38 @@ public class TestTxnHandler {
     assertEquals(0L, c.getStart());
   }
 
+  /**
+   * Once a Compaction for a given resource is scheduled/working, we should not
+   * schedule another one to prevent concurrent compactions for the same resource.
+   * @throws Exception
+   */
+  @Test
+  public void testCompactWhenAlreadyCompacting() throws Exception {
+    CompactionRequest rqst = new CompactionRequest("foo", "bar", CompactionType.MAJOR);
+    rqst.setPartitionname("ds=today");
+    CompactionResponse resp = txnHandler.compact(rqst);
+    Assert.assertEquals(resp, new CompactionResponse(1, TxnStore.INITIATED_RESPONSE, true));
+
+    ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
+    List<ShowCompactResponseElement> compacts = rsp.getCompacts();
+    assertEquals(1, compacts.size());
+
+    rqst.setType(CompactionType.MINOR);
+    resp = txnHandler.compact(rqst);
+    Assert.assertEquals(resp, new CompactionResponse(1, TxnStore.INITIATED_RESPONSE, false));
+
+    rsp = txnHandler.showCompact(new ShowCompactRequest());
+    compacts = rsp.getCompacts();
+    assertEquals(1, compacts.size());
+    ShowCompactResponseElement c = compacts.get(0);
+    assertEquals("foo", c.getDbname());
+    assertEquals("bar", c.getTablename());
+    assertEquals("ds=today", c.getPartitionname());
+    assertEquals(CompactionType.MAJOR, c.getType());
+    assertEquals("initiated", c.getState());
+    assertEquals(0L, c.getStart());
+  }
+
   @Test
   public void testCompactMinorNoPartition() throws Exception {
     CompactionRequest rqst = new CompactionRequest("foo", "bar", CompactionType.MINOR);


[5/8] hive git commit: HIVE-15202 Concurrent compactions for the same partition may generate malformed folder structure (Eugene Koifman, reviewed by Wei Zheng)

Posted by ek...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/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 09571b7..481a2c0 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 _size810;
-            ::apache::thrift::protocol::TType _etype813;
-            xfer += iprot->readListBegin(_etype813, _size810);
-            this->success.resize(_size810);
-            uint32_t _i814;
-            for (_i814 = 0; _i814 < _size810; ++_i814)
+            uint32_t _size812;
+            ::apache::thrift::protocol::TType _etype815;
+            xfer += iprot->readListBegin(_etype815, _size812);
+            this->success.resize(_size812);
+            uint32_t _i816;
+            for (_i816 = 0; _i816 < _size812; ++_i816)
             {
-              xfer += iprot->readString(this->success[_i814]);
+              xfer += iprot->readString(this->success[_i816]);
             }
             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 _iter815;
-      for (_iter815 = this->success.begin(); _iter815 != this->success.end(); ++_iter815)
+      std::vector<std::string> ::const_iterator _iter817;
+      for (_iter817 = this->success.begin(); _iter817 != this->success.end(); ++_iter817)
       {
-        xfer += oprot->writeString((*_iter815));
+        xfer += oprot->writeString((*_iter817));
       }
       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 _size816;
-            ::apache::thrift::protocol::TType _etype819;
-            xfer += iprot->readListBegin(_etype819, _size816);
-            (*(this->success)).resize(_size816);
-            uint32_t _i820;
-            for (_i820 = 0; _i820 < _size816; ++_i820)
+            uint32_t _size818;
+            ::apache::thrift::protocol::TType _etype821;
+            xfer += iprot->readListBegin(_etype821, _size818);
+            (*(this->success)).resize(_size818);
+            uint32_t _i822;
+            for (_i822 = 0; _i822 < _size818; ++_i822)
             {
-              xfer += iprot->readString((*(this->success))[_i820]);
+              xfer += iprot->readString((*(this->success))[_i822]);
             }
             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 _size821;
-            ::apache::thrift::protocol::TType _etype824;
-            xfer += iprot->readListBegin(_etype824, _size821);
-            this->success.resize(_size821);
-            uint32_t _i825;
-            for (_i825 = 0; _i825 < _size821; ++_i825)
+            uint32_t _size823;
+            ::apache::thrift::protocol::TType _etype826;
+            xfer += iprot->readListBegin(_etype826, _size823);
+            this->success.resize(_size823);
+            uint32_t _i827;
+            for (_i827 = 0; _i827 < _size823; ++_i827)
             {
-              xfer += iprot->readString(this->success[_i825]);
+              xfer += iprot->readString(this->success[_i827]);
             }
             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 _iter826;
-      for (_iter826 = this->success.begin(); _iter826 != this->success.end(); ++_iter826)
+      std::vector<std::string> ::const_iterator _iter828;
+      for (_iter828 = this->success.begin(); _iter828 != this->success.end(); ++_iter828)
       {
-        xfer += oprot->writeString((*_iter826));
+        xfer += oprot->writeString((*_iter828));
       }
       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 _size827;
-            ::apache::thrift::protocol::TType _etype830;
-            xfer += iprot->readListBegin(_etype830, _size827);
-            (*(this->success)).resize(_size827);
-            uint32_t _i831;
-            for (_i831 = 0; _i831 < _size827; ++_i831)
+            uint32_t _size829;
+            ::apache::thrift::protocol::TType _etype832;
+            xfer += iprot->readListBegin(_etype832, _size829);
+            (*(this->success)).resize(_size829);
+            uint32_t _i833;
+            for (_i833 = 0; _i833 < _size829; ++_i833)
             {
-              xfer += iprot->readString((*(this->success))[_i831]);
+              xfer += iprot->readString((*(this->success))[_i833]);
             }
             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 _size832;
-            ::apache::thrift::protocol::TType _ktype833;
-            ::apache::thrift::protocol::TType _vtype834;
-            xfer += iprot->readMapBegin(_ktype833, _vtype834, _size832);
-            uint32_t _i836;
-            for (_i836 = 0; _i836 < _size832; ++_i836)
+            uint32_t _size834;
+            ::apache::thrift::protocol::TType _ktype835;
+            ::apache::thrift::protocol::TType _vtype836;
+            xfer += iprot->readMapBegin(_ktype835, _vtype836, _size834);
+            uint32_t _i838;
+            for (_i838 = 0; _i838 < _size834; ++_i838)
             {
-              std::string _key837;
-              xfer += iprot->readString(_key837);
-              Type& _val838 = this->success[_key837];
-              xfer += _val838.read(iprot);
+              std::string _key839;
+              xfer += iprot->readString(_key839);
+              Type& _val840 = this->success[_key839];
+              xfer += _val840.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 _iter839;
-      for (_iter839 = this->success.begin(); _iter839 != this->success.end(); ++_iter839)
+      std::map<std::string, Type> ::const_iterator _iter841;
+      for (_iter841 = this->success.begin(); _iter841 != this->success.end(); ++_iter841)
       {
-        xfer += oprot->writeString(_iter839->first);
-        xfer += _iter839->second.write(oprot);
+        xfer += oprot->writeString(_iter841->first);
+        xfer += _iter841->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 _size840;
-            ::apache::thrift::protocol::TType _ktype841;
-            ::apache::thrift::protocol::TType _vtype842;
-            xfer += iprot->readMapBegin(_ktype841, _vtype842, _size840);
-            uint32_t _i844;
-            for (_i844 = 0; _i844 < _size840; ++_i844)
+            uint32_t _size842;
+            ::apache::thrift::protocol::TType _ktype843;
+            ::apache::thrift::protocol::TType _vtype844;
+            xfer += iprot->readMapBegin(_ktype843, _vtype844, _size842);
+            uint32_t _i846;
+            for (_i846 = 0; _i846 < _size842; ++_i846)
             {
-              std::string _key845;
-              xfer += iprot->readString(_key845);
-              Type& _val846 = (*(this->success))[_key845];
-              xfer += _val846.read(iprot);
+              std::string _key847;
+              xfer += iprot->readString(_key847);
+              Type& _val848 = (*(this->success))[_key847];
+              xfer += _val848.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 _size847;
-            ::apache::thrift::protocol::TType _etype850;
-            xfer += iprot->readListBegin(_etype850, _size847);
-            this->success.resize(_size847);
-            uint32_t _i851;
-            for (_i851 = 0; _i851 < _size847; ++_i851)
+            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 += this->success[_i851].read(iprot);
+              xfer += this->success[_i853].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 _iter852;
-      for (_iter852 = this->success.begin(); _iter852 != this->success.end(); ++_iter852)
+      std::vector<FieldSchema> ::const_iterator _iter854;
+      for (_iter854 = this->success.begin(); _iter854 != this->success.end(); ++_iter854)
       {
-        xfer += (*_iter852).write(oprot);
+        xfer += (*_iter854).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 _size853;
-            ::apache::thrift::protocol::TType _etype856;
-            xfer += iprot->readListBegin(_etype856, _size853);
-            (*(this->success)).resize(_size853);
-            uint32_t _i857;
-            for (_i857 = 0; _i857 < _size853; ++_i857)
+            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 += (*(this->success))[_i857].read(iprot);
+              xfer += (*(this->success))[_i859].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 _size858;
-            ::apache::thrift::protocol::TType _etype861;
-            xfer += iprot->readListBegin(_etype861, _size858);
-            this->success.resize(_size858);
-            uint32_t _i862;
-            for (_i862 = 0; _i862 < _size858; ++_i862)
+            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 += this->success[_i862].read(iprot);
+              xfer += this->success[_i864].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 _iter863;
-      for (_iter863 = this->success.begin(); _iter863 != this->success.end(); ++_iter863)
+      std::vector<FieldSchema> ::const_iterator _iter865;
+      for (_iter865 = this->success.begin(); _iter865 != this->success.end(); ++_iter865)
       {
-        xfer += (*_iter863).write(oprot);
+        xfer += (*_iter865).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 _size864;
-            ::apache::thrift::protocol::TType _etype867;
-            xfer += iprot->readListBegin(_etype867, _size864);
-            (*(this->success)).resize(_size864);
-            uint32_t _i868;
-            for (_i868 = 0; _i868 < _size864; ++_i868)
+            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 += (*(this->success))[_i868].read(iprot);
+              xfer += (*(this->success))[_i870].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 _size869;
-            ::apache::thrift::protocol::TType _etype872;
-            xfer += iprot->readListBegin(_etype872, _size869);
-            this->success.resize(_size869);
-            uint32_t _i873;
-            for (_i873 = 0; _i873 < _size869; ++_i873)
+            uint32_t _size871;
+            ::apache::thrift::protocol::TType _etype874;
+            xfer += iprot->readListBegin(_etype874, _size871);
+            this->success.resize(_size871);
+            uint32_t _i875;
+            for (_i875 = 0; _i875 < _size871; ++_i875)
             {
-              xfer += this->success[_i873].read(iprot);
+              xfer += this->success[_i875].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 _iter874;
-      for (_iter874 = this->success.begin(); _iter874 != this->success.end(); ++_iter874)
+      std::vector<FieldSchema> ::const_iterator _iter876;
+      for (_iter876 = this->success.begin(); _iter876 != this->success.end(); ++_iter876)
       {
-        xfer += (*_iter874).write(oprot);
+        xfer += (*_iter876).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 _size875;
-            ::apache::thrift::protocol::TType _etype878;
-            xfer += iprot->readListBegin(_etype878, _size875);
-            (*(this->success)).resize(_size875);
-            uint32_t _i879;
-            for (_i879 = 0; _i879 < _size875; ++_i879)
+            uint32_t _size877;
+            ::apache::thrift::protocol::TType _etype880;
+            xfer += iprot->readListBegin(_etype880, _size877);
+            (*(this->success)).resize(_size877);
+            uint32_t _i881;
+            for (_i881 = 0; _i881 < _size877; ++_i881)
             {
-              xfer += (*(this->success))[_i879].read(iprot);
+              xfer += (*(this->success))[_i881].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 _size880;
-            ::apache::thrift::protocol::TType _etype883;
-            xfer += iprot->readListBegin(_etype883, _size880);
-            this->success.resize(_size880);
-            uint32_t _i884;
-            for (_i884 = 0; _i884 < _size880; ++_i884)
+            uint32_t _size882;
+            ::apache::thrift::protocol::TType _etype885;
+            xfer += iprot->readListBegin(_etype885, _size882);
+            this->success.resize(_size882);
+            uint32_t _i886;
+            for (_i886 = 0; _i886 < _size882; ++_i886)
             {
-              xfer += this->success[_i884].read(iprot);
+              xfer += this->success[_i886].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 _iter885;
-      for (_iter885 = this->success.begin(); _iter885 != this->success.end(); ++_iter885)
+      std::vector<FieldSchema> ::const_iterator _iter887;
+      for (_iter887 = this->success.begin(); _iter887 != this->success.end(); ++_iter887)
       {
-        xfer += (*_iter885).write(oprot);
+        xfer += (*_iter887).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 _size886;
-            ::apache::thrift::protocol::TType _etype889;
-            xfer += iprot->readListBegin(_etype889, _size886);
-            (*(this->success)).resize(_size886);
-            uint32_t _i890;
-            for (_i890 = 0; _i890 < _size886; ++_i890)
+            uint32_t _size888;
+            ::apache::thrift::protocol::TType _etype891;
+            xfer += iprot->readListBegin(_etype891, _size888);
+            (*(this->success)).resize(_size888);
+            uint32_t _i892;
+            for (_i892 = 0; _i892 < _size888; ++_i892)
             {
-              xfer += (*(this->success))[_i890].read(iprot);
+              xfer += (*(this->success))[_i892].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 _size891;
-            ::apache::thrift::protocol::TType _etype894;
-            xfer += iprot->readListBegin(_etype894, _size891);
-            this->primaryKeys.resize(_size891);
-            uint32_t _i895;
-            for (_i895 = 0; _i895 < _size891; ++_i895)
+            uint32_t _size893;
+            ::apache::thrift::protocol::TType _etype896;
+            xfer += iprot->readListBegin(_etype896, _size893);
+            this->primaryKeys.resize(_size893);
+            uint32_t _i897;
+            for (_i897 = 0; _i897 < _size893; ++_i897)
             {
-              xfer += this->primaryKeys[_i895].read(iprot);
+              xfer += this->primaryKeys[_i897].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 _size896;
-            ::apache::thrift::protocol::TType _etype899;
-            xfer += iprot->readListBegin(_etype899, _size896);
-            this->foreignKeys.resize(_size896);
-            uint32_t _i900;
-            for (_i900 = 0; _i900 < _size896; ++_i900)
+            uint32_t _size898;
+            ::apache::thrift::protocol::TType _etype901;
+            xfer += iprot->readListBegin(_etype901, _size898);
+            this->foreignKeys.resize(_size898);
+            uint32_t _i902;
+            for (_i902 = 0; _i902 < _size898; ++_i902)
             {
-              xfer += this->foreignKeys[_i900].read(iprot);
+              xfer += this->foreignKeys[_i902].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 _iter901;
-    for (_iter901 = this->primaryKeys.begin(); _iter901 != this->primaryKeys.end(); ++_iter901)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter903;
+    for (_iter903 = this->primaryKeys.begin(); _iter903 != this->primaryKeys.end(); ++_iter903)
     {
-      xfer += (*_iter901).write(oprot);
+      xfer += (*_iter903).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 _iter902;
-    for (_iter902 = this->foreignKeys.begin(); _iter902 != this->foreignKeys.end(); ++_iter902)
+    std::vector<SQLForeignKey> ::const_iterator _iter904;
+    for (_iter904 = this->foreignKeys.begin(); _iter904 != this->foreignKeys.end(); ++_iter904)
     {
-      xfer += (*_iter902).write(oprot);
+      xfer += (*_iter904).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 _iter903;
-    for (_iter903 = (*(this->primaryKeys)).begin(); _iter903 != (*(this->primaryKeys)).end(); ++_iter903)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter905;
+    for (_iter905 = (*(this->primaryKeys)).begin(); _iter905 != (*(this->primaryKeys)).end(); ++_iter905)
     {
-      xfer += (*_iter903).write(oprot);
+      xfer += (*_iter905).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 _iter904;
-    for (_iter904 = (*(this->foreignKeys)).begin(); _iter904 != (*(this->foreignKeys)).end(); ++_iter904)
+    std::vector<SQLForeignKey> ::const_iterator _iter906;
+    for (_iter906 = (*(this->foreignKeys)).begin(); _iter906 != (*(this->foreignKeys)).end(); ++_iter906)
     {
-      xfer += (*_iter904).write(oprot);
+      xfer += (*_iter906).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 _size905;
-            ::apache::thrift::protocol::TType _etype908;
-            xfer += iprot->readListBegin(_etype908, _size905);
-            this->success.resize(_size905);
-            uint32_t _i909;
-            for (_i909 = 0; _i909 < _size905; ++_i909)
+            uint32_t _size907;
+            ::apache::thrift::protocol::TType _etype910;
+            xfer += iprot->readListBegin(_etype910, _size907);
+            this->success.resize(_size907);
+            uint32_t _i911;
+            for (_i911 = 0; _i911 < _size907; ++_i911)
             {
-              xfer += iprot->readString(this->success[_i909]);
+              xfer += iprot->readString(this->success[_i911]);
             }
             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 _iter910;
-      for (_iter910 = this->success.begin(); _iter910 != this->success.end(); ++_iter910)
+      std::vector<std::string> ::const_iterator _iter912;
+      for (_iter912 = this->success.begin(); _iter912 != this->success.end(); ++_iter912)
       {
-        xfer += oprot->writeString((*_iter910));
+        xfer += oprot->writeString((*_iter912));
       }
       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 _size911;
-            ::apache::thrift::protocol::TType _etype914;
-            xfer += iprot->readListBegin(_etype914, _size911);
-            (*(this->success)).resize(_size911);
-            uint32_t _i915;
-            for (_i915 = 0; _i915 < _size911; ++_i915)
+            uint32_t _size913;
+            ::apache::thrift::protocol::TType _etype916;
+            xfer += iprot->readListBegin(_etype916, _size913);
+            (*(this->success)).resize(_size913);
+            uint32_t _i917;
+            for (_i917 = 0; _i917 < _size913; ++_i917)
             {
-              xfer += iprot->readString((*(this->success))[_i915]);
+              xfer += iprot->readString((*(this->success))[_i917]);
             }
             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 _size916;
-            ::apache::thrift::protocol::TType _etype919;
-            xfer += iprot->readListBegin(_etype919, _size916);
-            this->success.resize(_size916);
-            uint32_t _i920;
-            for (_i920 = 0; _i920 < _size916; ++_i920)
+            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)
             {
-              xfer += iprot->readString(this->success[_i920]);
+              xfer += iprot->readString(this->success[_i922]);
             }
             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 _iter921;
-      for (_iter921 = this->success.begin(); _iter921 != this->success.end(); ++_iter921)
+      std::vector<std::string> ::const_iterator _iter923;
+      for (_iter923 = this->success.begin(); _iter923 != this->success.end(); ++_iter923)
       {
-        xfer += oprot->writeString((*_iter921));
+        xfer += oprot->writeString((*_iter923));
       }
       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 _size922;
-            ::apache::thrift::protocol::TType _etype925;
-            xfer += iprot->readListBegin(_etype925, _size922);
-            (*(this->success)).resize(_size922);
-            uint32_t _i926;
-            for (_i926 = 0; _i926 < _size922; ++_i926)
+            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)
             {
-              xfer += iprot->readString((*(this->success))[_i926]);
+              xfer += iprot->readString((*(this->success))[_i928]);
             }
             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 _size927;
-            ::apache::thrift::protocol::TType _etype930;
-            xfer += iprot->readListBegin(_etype930, _size927);
-            this->tbl_types.resize(_size927);
-            uint32_t _i931;
-            for (_i931 = 0; _i931 < _size927; ++_i931)
+            uint32_t _size929;
+            ::apache::thrift::protocol::TType _etype932;
+            xfer += iprot->readListBegin(_etype932, _size929);
+            this->tbl_types.resize(_size929);
+            uint32_t _i933;
+            for (_i933 = 0; _i933 < _size929; ++_i933)
             {
-              xfer += iprot->readString(this->tbl_types[_i931]);
+              xfer += iprot->readString(this->tbl_types[_i933]);
             }
             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 _iter932;
-    for (_iter932 = this->tbl_types.begin(); _iter932 != this->tbl_types.end(); ++_iter932)
+    std::vector<std::string> ::const_iterator _iter934;
+    for (_iter934 = this->tbl_types.begin(); _iter934 != this->tbl_types.end(); ++_iter934)
     {
-      xfer += oprot->writeString((*_iter932));
+      xfer += oprot->writeString((*_iter934));
     }
     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 _iter933;
-    for (_iter933 = (*(this->tbl_types)).begin(); _iter933 != (*(this->tbl_types)).end(); ++_iter933)
+    std::vector<std::string> ::const_iterator _iter935;
+    for (_iter935 = (*(this->tbl_types)).begin(); _iter935 != (*(this->tbl_types)).end(); ++_iter935)
     {
-      xfer += oprot->writeString((*_iter933));
+      xfer += oprot->writeString((*_iter935));
     }
     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 _size934;
-            ::apache::thrift::protocol::TType _etype937;
-            xfer += iprot->readListBegin(_etype937, _size934);
-            this->success.resize(_size934);
-            uint32_t _i938;
-            for (_i938 = 0; _i938 < _size934; ++_i938)
+            uint32_t _size936;
+            ::apache::thrift::protocol::TType _etype939;
+            xfer += iprot->readListBegin(_etype939, _size936);
+            this->success.resize(_size936);
+            uint32_t _i940;
+            for (_i940 = 0; _i940 < _size936; ++_i940)
             {
-              xfer += this->success[_i938].read(iprot);
+              xfer += this->success[_i940].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 _iter939;
-      for (_iter939 = this->success.begin(); _iter939 != this->success.end(); ++_iter939)
+      std::vector<TableMeta> ::const_iterator _iter941;
+      for (_iter941 = this->success.begin(); _iter941 != this->success.end(); ++_iter941)
       {
-        xfer += (*_iter939).write(oprot);
+        xfer += (*_iter941).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 _size940;
-            ::apache::thrift::protocol::TType _etype943;
-            xfer += iprot->readListBegin(_etype943, _size940);
-            (*(this->success)).resize(_size940);
-            uint32_t _i944;
-            for (_i944 = 0; _i944 < _size940; ++_i944)
+            uint32_t _size942;
+            ::apache::thrift::protocol::TType _etype945;
+            xfer += iprot->readListBegin(_etype945, _size942);
+            (*(this->success)).resize(_size942);
+            uint32_t _i946;
+            for (_i946 = 0; _i946 < _size942; ++_i946)
             {
-              xfer += (*(this->success))[_i944].read(iprot);
+              xfer += (*(this->success))[_i946].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 _size945;
-            ::apache::thrift::protocol::TType _etype948;
-            xfer += iprot->readListBegin(_etype948, _size945);
-            this->success.resize(_size945);
-            uint32_t _i949;
-            for (_i949 = 0; _i949 < _size945; ++_i949)
+            uint32_t _size947;
+            ::apache::thrift::protocol::TType _etype950;
+            xfer += iprot->readListBegin(_etype950, _size947);
+            this->success.resize(_size947);
+            uint32_t _i951;
+            for (_i951 = 0; _i951 < _size947; ++_i951)
             {
-              xfer += iprot->readString(this->success[_i949]);
+              xfer += iprot->readString(this->success[_i951]);
             }
             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 _iter950;
-      for (_iter950 = this->success.begin(); _iter950 != this->success.end(); ++_iter950)
+      std::vector<std::string> ::const_iterator _iter952;
+      for (_iter952 = this->success.begin(); _iter952 != this->success.end(); ++_iter952)
       {
-        xfer += oprot->writeString((*_iter950));
+        xfer += oprot->writeString((*_iter952));
       }
       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 _size951;
-            ::apache::thrift::protocol::TType _etype954;
-            xfer += iprot->readListBegin(_etype954, _size951);
-            (*(this->success)).resize(_size951);
-            uint32_t _i955;
-            for (_i955 = 0; _i955 < _size951; ++_i955)
+            uint32_t _size953;
+            ::apache::thrift::protocol::TType _etype956;
+            xfer += iprot->readListBegin(_etype956, _size953);
+            (*(this->success)).resize(_size953);
+            uint32_t _i957;
+            for (_i957 = 0; _i957 < _size953; ++_i957)
             {
-              xfer += iprot->readString((*(this->success))[_i955]);
+              xfer += iprot->readString((*(this->success))[_i957]);
             }
             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 _size956;
-            ::apache::thrift::protocol::TType _etype959;
-            xfer += iprot->readListBegin(_etype959, _size956);
-            this->tbl_names.resize(_size956);
-            uint32_t _i960;
-            for (_i960 = 0; _i960 < _size956; ++_i960)
+            uint32_t _size958;
+            ::apache::thrift::protocol::TType _etype961;
+            xfer += iprot->readListBegin(_etype961, _size958);
+            this->tbl_names.resize(_size958);
+            uint32_t _i962;
+            for (_i962 = 0; _i962 < _size958; ++_i962)
             {
-              xfer += iprot->readString(this->tbl_names[_i960]);
+              xfer += iprot->readString(this->tbl_names[_i962]);
             }
             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 _iter961;
-    for (_iter961 = this->tbl_names.begin(); _iter961 != this->tbl_names.end(); ++_iter961)
+    std::vector<std::string> ::const_iterator _iter963;
+    for (_iter963 = this->tbl_names.begin(); _iter963 != this->tbl_names.end(); ++_iter963)
     {
-      xfer += oprot->writeString((*_iter961));
+      xfer += oprot->writeString((*_iter963));
     }
     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 _iter962;
-    for (_iter962 = (*(this->tbl_names)).begin(); _iter962 != (*(this->tbl_names)).end(); ++_iter962)
+    std::vector<std::string> ::const_iterator _iter964;
+    for (_iter964 = (*(this->tbl_names)).begin(); _iter964 != (*(this->tbl_names)).end(); ++_iter964)
     {
-      xfer += oprot->writeString((*_iter962));
+      xfer += oprot->writeString((*_iter964));
     }
     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 _size963;
-            ::apache::thrift::protocol::TType _etype966;
-            xfer += iprot->readListBegin(_etype966, _size963);
-            this->success.resize(_size963);
-            uint32_t _i967;
-            for (_i967 = 0; _i967 < _size963; ++_i967)
+            uint32_t _size965;
+            ::apache::thrift::protocol::TType _etype968;
+            xfer += iprot->readListBegin(_etype968, _size965);
+            this->success.resize(_size965);
+            uint32_t _i969;
+            for (_i969 = 0; _i969 < _size965; ++_i969)
             {
-              xfer += this->success[_i967].read(iprot);
+              xfer += this->success[_i969].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 _iter968;
-      for (_iter968 = this->success.begin(); _iter968 != this->success.end(); ++_iter968)
+      std::vector<Table> ::const_iterator _iter970;
+      for (_iter970 = this->success.begin(); _iter970 != this->success.end(); ++_iter970)
       {
-        xfer += (*_iter968).write(oprot);
+        xfer += (*_iter970).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 _size969;
-            ::apache::thrift::protocol::TType _etype972;
-            xfer += iprot->readListBegin(_etype972, _size969);
-            (*(this->success)).resize(_size969);
-            uint32_t _i973;
-            for (_i973 = 0; _i973 < _size969; ++_i973)
+            uint32_t _size971;
+            ::apache::thrift::protocol::TType _etype974;
+            xfer += iprot->readListBegin(_etype974, _size971);
+            (*(this->success)).resize(_size971);
+            uint32_t _i975;
+            for (_i975 = 0; _i975 < _size971; ++_i975)
             {
-              xfer += (*(this->success))[_i973].read(iprot);
+              xfer += (*(this->success))[_i975].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 _size974;
-            ::apache::thrift::protocol::TType _etype977;
-            xfer += iprot->readListBegin(_etype977, _size974);
-            this->success.resize(_size974);
-            uint32_t _i978;
-            for (_i978 = 0; _i978 < _size974; ++_i978)
+            uint32_t _size976;
+            ::apache::thrift::protocol::TType _etype979;
+            xfer += iprot->readListBegin(_etype979, _size976);
+            this->success.resize(_size976);
+            uint32_t _i980;
+            for (_i980 = 0; _i980 < _size976; ++_i980)
             {
-              xfer += iprot->readString(this->success[_i978]);
+              xfer += iprot->readString(this->success[_i980]);
             }
             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 _iter979;
-      for (_iter979 = this->success.begin(); _iter979 != this->success.end(); ++_iter979)
+      std::vector<std::string> ::const_iterator _iter981;
+      for (_iter981 = this->success.begin(); _iter981 != this->success.end(); ++_iter981)
       {
-        xfer += oprot->writeString((*_iter979));
+        xfer += oprot->writeString((*_iter981));
       }
       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 _size980;
-            ::apache::thrift::protocol::TType _etype983;
-            xfer += iprot->readListBegin(_etype983, _size980);
-            (*(this->success)).resize(_size980);
-            uint32_t _i984;
-            for (_i984 = 0; _i984 < _size980; ++_i984)
+            uint32_t _size982;
+            ::apache::thrift::protocol::TType _etype985;
+            xfer += iprot->readListBegin(_etype985, _size982);
+            (*(this->success)).resize(_size982);
+            uint32_t _i986;
+            for (_i986 = 0; _i986 < _size982; ++_i986)
             {
-              xfer += iprot->readString((*(this->success))[_i984]);
+              xfer += iprot->readString((*(this->success))[_i986]);
             }
             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 _size985;
-            ::apache::thrift::protocol::TType _etype988;
-            xfer += iprot->readListBegin(_etype988, _size985);
-            this->new_parts.resize(_size985);
-            uint32_t _i989;
-            for (_i989 = 0; _i989 < _size985; ++_i989)
+            uint32_t _size987;
+            ::apache::thrift::protocol::TType _etype990;
+            xfer += iprot->readListBegin(_etype990, _size987);
+            this->new_parts.resize(_size987);
+            uint32_t _i991;
+            for (_i991 = 0; _i991 < _size987; ++_i991)
             {
-              xfer += this->new_parts[_i989].read(iprot);
+              xfer += this->new_parts[_i991].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 _iter990;
-    for (_iter990 = this->new_parts.begin(); _iter990 != this->new_parts.end(); ++_iter990)
+    std::vector<Partition> ::const_iterator _iter992;
+    for (_iter992 = this->new_parts.begin(); _iter992 != this->new_parts.end(); ++_iter992)
     {
-      xfer += (*_iter990).write(oprot);
+      xfer += (*_iter992).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 _iter991;
-    for (_iter991 = (*(this->new_parts)).begin(); _iter991 != (*(this->new_parts)).end(); ++_iter991)
+    std::vector<Partition> ::const_iterator _iter993;
+    for (_iter993 = (*(this->new_parts)).begin(); _iter993 != (*(this->new_parts)).end(); ++_iter993)
     {
-      xfer += (*_iter991).write(oprot);
+      xfer += (*_iter993).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 _size992;
-            ::apache::thrift::protocol::TType _etype995;
-            xfer += iprot->readListBegin(_etype995, _size992);
-            this->new_parts.resize(_size992);
-            uint32_t _i996;
-            for (_i996 = 0; _i996 < _size992; ++_i996)
+            uint32_t _size994;
+            ::apache::thrift::protocol::TType _etype997;
+            xfer += iprot->readListBegin(_etype997, _size994);
+            this->new_parts.resize(_size994);
+            uint32_t _i998;
+            for (_i998 = 0; _i998 < _size994; ++_i998)
             {
-              xfer += this->new_parts[_i996].read(iprot);
+              xfer += this->new_parts[_i998].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 _iter997;
-    for (_iter997 = this->new_parts.begin(); _iter997 != this->new_parts.end(); ++_iter997)
+    std::vector<PartitionSpec> ::const_iterator _iter999;
+    for (_iter999 = this->new_parts.begin(); _iter999 != this->new_parts.end(); ++_iter999)
     {
-      xfer += (*_iter997).write(oprot);
+      xfer += (*_iter999).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 _iter998;
-    for (_iter998 = (*(this->new_parts)).begin(); _iter998 != (*(this->new_parts)).end(); ++_iter998)
+    std::vector<PartitionSpec> ::const_iterator _iter1000;
+    for (_iter1000 = (*(this->new_parts)).begin(); _iter1000 != (*(this->new_parts)).end(); ++_iter1000)
     {
-      xfer += (*_iter998).write(oprot);
+      xfer += (*_iter1000).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 _size999;
-            ::apache::thrift::protocol::TType _etype1002;
-            xfer += iprot->readListBegin(_etype1002, _size999);
-            this->part_vals.resize(_size999);
-            uint32_t _i1003;
-            for (_i1003 = 0; _i1003 < _size999; ++_i1003)
+            uint32_t _size1001;
+            ::apache::thrift::protocol::TType _etype1004;
+            xfer += iprot->readListBegin(_etype1004, _size1001);
+            this->part_vals.resize(_size1001);
+            uint32_t _i1005;
+            for (_i1005 = 0; _i1005 < _size1001; ++_i1005)
             {
-              xfer += iprot->readString(this->part_vals[_i1003]);
+              xfer += iprot->readString(this->part_vals[_i1005]);
             }
             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 _iter1004;
-    for (_iter1004 = this->part_vals.begin(); _iter1004 != this->part_vals.end(); ++_iter1004)
+    std::vector<std::string> ::const_iterator _iter1006;
+    for (_iter1006 = this->part_vals.begin(); _iter1006 != this->part_vals.end(); ++_iter1006)
     {
-      xfer += oprot->writeString((*_iter1004));
+      xfer += oprot->writeString((*_iter1006));
     }
     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 _iter1005;
-    for (_iter1005 = (*(this->part_vals)).begin(); _iter1005 != (*(this->part_vals)).end(); ++_iter1005)
+    std::vector<std::string> ::const_iterator _iter1007;
+    for (_iter1007 = (*(this->part_vals)).begin(); _iter1007 != (*(this->part_vals)).end(); ++_iter1007)
     {
-      xfer += oprot->writeString((*_iter1005));
+      xfer += oprot->writeString((*_iter1007));
     }
     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 _size1006;
-            ::apache::thrift::protocol::TType _etype1009;
-            xfer += iprot->readListBegin(_etype1009, _size1006);
-            this->part_vals.resize(_size1006);
-            uint32_t _i1010;
-            for (_i1010 = 0; _i1010 < _size1006; ++_i1010)
+            uint32_t _size1008;
+            ::apache::thrift::protocol::TType _etype1011;
+            xfer += iprot->readListBegin(_etype1011, _size1008);
+            this->part_vals.resize(_size1008);
+            uint32_t _i1012;
+            for (_i1012 = 0; _i1012 < _size1008; ++_i1012)
             {
-              xfer += iprot->readString(this->part_vals[_i1010]);
+              xfer += iprot->readString(this->part_vals[_i1012]);
             }
             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 _iter1011;
-    for (_iter1011 = this->part_vals.begin(); _iter1011 != this->part_vals.end(); ++_iter1011)
+    std::vector<std::string> ::const_iterator _iter1013;
+    for (_iter1013 = this->part_vals.begin(); _iter1013 != this->part_vals.end(); ++_iter1013)
     {
-      xfer += oprot->writeString((*_iter1011));
+      xfer += oprot->writeString((*_iter1013));
     }
     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 _iter1012;
-    for (_iter1012 = (*(this->part_vals)).begin(); _iter1012 != (*(this->part_vals)).end(); ++_iter1012)
+    std::vector<std::string> ::const_iterator _iter1014;
+    for (_iter1014 = (*(this->part_vals)).begin(); _iter1014 != (*(this->part_vals)).end(); ++_iter1014)
     {
-      xfer += oprot->writeString((*_iter1012));
+      xfer += oprot->writeString((*_iter1014));
     }
     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 _size1013;
-            ::apache::thrift::protocol::TType _etype1016;
-            xfer += iprot->readListBegin(_etype1016, _size1013);
-            this->part_vals.resize(_size1013);
-            uint32_t _i1017;
-            for (_i1017 = 0; _i1017 < _size1013; ++_i1017)
+            uint32_t _size1015;
+            ::apache::thrift::protocol::TType _etype1018;
+            xfer += iprot->readListBegin(_etype1018, _size1015);
+            this->part_vals.resize(_size1015);
+            uint32_t _i1019;
+            for (_i1019 = 0; _i1019 < _size1015; ++_i1019)
             {
-              xfer += iprot->readString(this->part_vals[_i1017]);
+              xfer += iprot->readString(this->part_vals[_i1019]);
             }
             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 _iter1018;
-    for (_iter1018 = this->part_vals.begin(); _iter1018 != this->part_vals.end(); ++_iter1018)
+    std::vector<std::string> ::const_iterator _iter1020;
+    for (_iter1020 = this->part_vals.begin(); _iter1020 != this->part_vals.end(); ++_iter1020)
     {
-      xfer += oprot->writeString((*_iter1018));
+      xfer += oprot->writeString((*_iter1020));
     }
     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 _iter1019;
-    for (_iter1019 = (*(this->part_vals)).begin(); _iter1019 != (*(this->part_vals)).end(); ++_iter1019)
+    std::vector<std::string> ::const_iterator _iter1021;
+    for (_iter1021 = (*(this->part_vals)).begin(); _iter1021 != (*(this->part_vals)).end(); ++_iter1021)
     {
-      xfer += oprot->writeString((*_iter1019));
+      xfer += oprot->writeString((*_iter1021));
     }
     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 _size1020;
-            ::apache::thrift::protocol::TType _etype1023;
-            xfer += iprot->readListBegin(_etype1023, _size1020);
-            this->part_vals.resize(_size1020);
-            uint32_t _i1024;
-            for (_i1024 = 0; _i1024 < _size1020; ++_i1024)
+            uint32_t _size1022;
+            ::apache::thrift::protocol::TType _etype1025;
+            xfer += iprot->readListBegin(_etype1025, _size1022);
+            this->part_vals.resize(_size1022);
+            uint32_t _i1026;
+            for (_i1026 = 0; _i1026 < _size1022; ++_i1026)
             {
-              xfer += iprot->readString(this->part_vals[_i1024]);
+              xfer += iprot->readString(this->part_vals[_i1026]);
             }
             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 _iter1025;
-    for (_iter1025 = this->part_vals.begin(); _iter1025 != this->part_vals.end(); ++_iter1025)
+    std::vector<std::string> ::const_iterator _iter1027;
+    for (_iter1027 = this->part_vals.begin(); _iter1027 != this->part_vals.end(); ++_iter1027)
     {
-      xfer += oprot->writeString((*_iter1025));
+      xfer += oprot->writeString((*_iter1027));
     }
     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 _iter1026;
-    for (_iter1026 = (*(this->part_vals)).begin(); _iter1026 != (*(this->part_vals)).end(); ++_iter1026)
+    std::vector<std::string> ::const_iterator _iter1028;
+    for (_iter1028 = (*(this->part_vals)).begin(); _iter1028 != (*(this->part_vals)).end(); ++_iter1028)
     {
-      xfer += oprot->writeString((*_iter1026));
+      xfer += oprot->writeString((*_iter1028));
     }
     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 _size1027;
-            ::apache::thrift::protocol::TType _etype1030;
-            xfer += iprot->readListBegin(_etype1030, _size1027);
-            this->part_vals.resize(_size1027);
-            uint32_t _i1031;
-            for (_i1031 = 0; _i1031 < _size1027; ++_i1031)
+            uint32_t _size1029;
+            ::apache::thrift::protocol::TType _etype1032;
+            xfer += iprot->readListBegin(_etype1032, _size1029);
+            this->part_vals.resize(_size1029);
+            uint32_t _i1033;
+            for (_i1033 = 0; _i1033 < _size1029; ++_i1033)
             {
-              xfer += iprot->readString(this->part_vals[_i1031]);
+              xfer += iprot->readString(this->part_vals[_i1033]);
             }
             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 _iter1032;
-    for (_iter1032 = this->part_vals.begin(); _iter1032 != this->part_vals.end(); ++_iter1032)
+    std::vector<std::string> ::const_iterator _iter1034;
+    for (_iter1034 = this->part_vals.begin(); _iter1034 != this->part_vals.end(); ++_iter1034)
     {
-      xfer += oprot->writeString((*_iter1032));
+      xfer += oprot->writeString((*_iter1034));
     }
     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 _iter1033;
-    for (_iter1033 = (*(this->part_vals)).begin(); _iter1033 != (*(this->part_vals)).end(); ++_iter1033)
+    std::vector<std::string> ::const_iterator _iter1035;
+    for (_iter1035 = (*(this->part_vals)).begin(); _iter1035 != (*(this->part_vals)).end(); ++_iter1035)
     {
-      xfer += oprot->writeString((*_iter1033));
+      xfer += oprot->writeString((*_iter1035));
     }
     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 _size1034;
-            ::apache::thrift::protocol::TType _ktype1035;
-            ::apache::thrift::protocol::TType _vtype1036;
-            xfer += iprot->readMapBegin(_ktype1035, _vtype1036, _size1034);
-            uint32_t _i1038;
-            for (_i1038 = 0; _i1038 < _size1034; ++_i1038)
+            uint32_t _size1036;
+            ::apache::thrift::protocol::TType _ktype1037;
+            ::apache::thrift::protocol::TType _vtype1038;
+            xfer += iprot->readMapBegin(_ktype1037, _vtype1038, _size1036);
+            uint32_t _i1040;
+            for (_i1040 = 0; _i1040 < _size1036; ++_i1040)
             {
-              std::string _key1039;
-              xfer += iprot->readString(_key1039);
-              std::string& _val1040 = this->partitionSpecs[_key1039];
-              xfer += iprot->readString(_val1040);
+              std::string _key1041;
+              xfer += iprot->readString(_key1041);
+              std::string& _val1042 = this->partitionSpecs[_key1041];
+              xfer += iprot->readString(_val1042);
             }
             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 _iter1041;
-    for (_iter1041 = this->partitionSpecs.begin(); _iter1041 != this->partitionSpecs.end(); ++_iter1041)
+    std::map<std::string, std::string> ::const_iterator _iter1043;
+    for (_iter1043 = this->partitionSpecs.begin(); _iter1043 != this->partitionSpecs.end(); ++_iter1043)
     {
-      xfer += oprot->writeString(_iter1041->first);
-      xfer += oprot->writeString(_iter1041->second);
+      xfer += oprot->writeString(_iter1043->first);
+      xfer += oprot->writeString(_iter1043->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 _iter1042;
-    for (_iter1042 = (*(this->partitionSpecs)).begin(); _iter1042 != (*(this->partitionSpecs)).end(); ++_iter1042)
+    std::map<std::string, std::string> ::const_iterator _iter1044;
+    for (_iter1044 = (*(this->partitionSpecs)).begin(); _iter1044 != (*(this->partitionSpecs)).end(); ++_iter1044)
     {
-      xfer += oprot->writeString(_iter1042->first);
-      xfer += oprot->writeString(_iter1042->second);
+      xfer += oprot->writeString(_iter1044->first);
+      xfer += oprot->writeString(_iter1044->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 _size1043;
-            ::apache::thrift::protocol::TType _ktype1044;
-            ::apache::thrift::protocol::TType _vtype1045;
-            xfer += iprot->readMapBegin(_ktype1044, _vtype1045, _size1043);
-            uint32_t _i1047;
-            for (_i1047 = 0; _i1047 < _size1043; ++_i1047)
+            uint32_t _size1045;
+            ::apache::thrift::protocol::TType _ktype1046;
+            ::apache::thrift::protocol::TType _vtype1047;
+            xfer += iprot->readMapBegin(_ktype1046, _vtype1047, _size1045);
+            uint32_t _i1049;
+            for (_i1049 = 0; _i1049 < _size1045; ++_i1049)
             {
-              std::string _key1048;
-              xfer += iprot->readString(_key1048);
-              std::string& _val1049 = this->partitionSpecs[_key1048];
-              xfer += iprot->readString(_val1049);
+              std::string _key1050;
+              xfer += iprot->readString(_key1050);
+              std::string& _val1051 = this->partitionSpecs[_key1050];
+              xfer += iprot->readString(_val1051);
             }
             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 _iter1050;
-    for (_iter1050 = this->partitionSpecs.begin(); _iter1050 != this->partitionSpecs.end(); ++_iter1050)
+    std::map<std::string, std::string> ::const_iterator _iter1052;
+    for (_iter1052 = this->partitionSpecs.begin(); _iter1052 != this->partitionSpecs.end(); ++_iter1052)
     {
-      xfer += oprot->writeString(_iter1050->first);
-      xfer += oprot->writeString(_iter1050->second);
+      xfer += oprot->writeString(_iter1052->first);
+      xfer += oprot->writeString(_iter1052->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 _iter1051;
-    for (_iter1051 = (*(this->partitionSpecs)).begin(); _iter1051 != (*(this->partitionSpecs)).end(); ++_iter1051)
+    std::map<std::string, std::string> ::const_iterator _iter1053;
+    for (_iter1053 = (*(this->partitionSpecs)).begin(); _iter1053 != (*(this->partitionSpecs)).end(); ++_iter1053)
     {
-      xfer += oprot->writeString(_iter1051->first);
-      xfer += oprot->writeString(_iter1051->second);
+      xfer += oprot->writeString(_iter1053->first);
+      xfer += oprot->writeString(_iter1053->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 _size1052;
-            ::apache::thrift::protocol::TType _etype1055;
-            xfer += iprot->readListBegin(_etype1055, _size1052);
-            this->success.resize(_size1052);
-            uint32_t _i1056;
-            for (_i1056 = 0; _i1056 < _size1052; ++_i1056)
+            uint32_t _size1054;
+            ::apache::thrift::protocol::TType _etype1057;
+            xfer += iprot->readListBegin(_etype1057, _size1054);
+            this->success.resize(_size1054);
+            uint32_t _i1058;
+            for (_i1058 = 0; _i1058 < _size1054; ++_i1058)
             {
-              xfer += this->success[_i1056].read(iprot);
+              xfer += this->success[_i1058].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 _iter1057;
-      for (_iter1057 = this->success.begin(); _iter1057 != this->success.end(); ++_iter1057)
+      std::vector<Partition> ::const_iterator _iter1059;
+      for (_iter1059 = this->success.begin(); _iter1059 != this->success.end(); ++_iter1059)
       {
-        xfer += (*_iter1057).write(oprot);
+        xfer += (*_iter1059).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 _size1058;
-            ::apache::thrift::protocol::TType _etype1061;
-            xfer += iprot->readListBegin(_etype1061, _size1058);
-            (*(this->success)).resize(_size1058);
-            uint32_t _i1062;
-            for (_i1062 = 0; _i1062 < _size1058; ++_i1062)
+            uint32_t _size1060;
+            ::apache::thrift::protocol::TType _etype1063;
+            xfer += iprot->readListBegin(_etype1063, _size1060);
+            (*(this->success)).resize(_size1060);
+            uint32_t _i1064;
+            for (_i1064 = 0; _i1064 < _size1060; ++_i1064)
             {
-              xfer += (*(this->success))[_i1062].read(iprot);
+              xfer += (*(this->success))[_i1064].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 _size1063;
-            ::apache::thrift::protocol::TType _etype1066;
-            xfer += iprot->readListBegin(_etype1066, _size1063);
-            this->part_vals.resize(_size1063);
-            uint32_t _i1067;
-            for (_i1067 = 0; _i1067 < _size1063; ++_i1067)
+            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)
             {
-              xfer += iprot->readString(this->part_vals[_i1067]);
+              xfer += iprot->readString(this->part_vals[_i1069]);
             }
             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 _size1068;
-            ::apache::thrift::protocol::TType _etype1071;
-            xfer += iprot->readListBegin(_etype1071, _size1068);
-            this->group_names.resize(_size1068);
-            uint32_t _i1072;
-            for (_i1072 = 0; _i1072 < _size1068; ++_i1072)
+            uint32_t _size1070;
+            ::apache::thrift::protocol::TType _etype1073;
+            xfer += iprot->readListBegin(_etype1073, _size1070);
+            this->group_names.resize(_size1070);
+            uint32_t _i1074;
+            for (_i1074 = 0; _i1074 < _size1070; ++_i1074)
             {
-              xfer += iprot->readString(this->group_names[_i1072]);
+              xfer += iprot->readString(this->group_names[_i1074]);
             }
             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 _iter1073;
-    for (_iter1073 = this->part_vals.begin(); _iter1073 != this->part_vals.end(); ++_iter1073)
+    std::vector<std::string> ::const_iterator _iter1075;
+    for (_iter1075 = this->part_vals.begin(); _iter1075 != this->part_vals.end(); ++_iter1075)
     {
-      xfer += oprot->writeString((*_iter1073));
+      xfer += oprot->writeString((*_iter1075));
     }
     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 _iter1074;
-    for (_iter1074 = this->group_names.begin(); _iter1074 != this->group_names.end(); ++_iter1074)
+    std::vector<std::string> ::const_iterator _iter1076;
+    for (_iter1076 = this->group_names.begin(); _iter1076 != this->group_names.end(); ++_iter1076)
     {
-      xfer += oprot->writeString((*_iter1074));
+      xfer += oprot->writeString((*_iter1076));
     }
     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 _iter1075;
-    for (_iter1075 = (*(this->part_vals)).begin(); _iter1075 != (*(this->part_vals)).end(); ++_iter1075)
+    std::vector<std::string> ::const_iterator _iter1077;
+    for (_iter1077 = (*(this->part_vals)).begin(); _iter1077 != (*(this->part_vals)).end(); ++_iter1077)
     {
-      xfer += oprot->writeString((*_iter1075));
+      xfer += oprot->writeString((*_iter1077));
     }
     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 _iter1076;
-    for (_iter1076 = (*(this->group_names)).begin(); _iter1076 != (*(this->group_names)).end(); ++_iter1076)
+    std::vector<std::string> ::const_iterator _iter1078;
+    for (_iter1078 = (*(this->group_names)).begin(); _iter1078 != (*(this->group_names)).end(); ++_iter1078)
     {
-      xfer += oprot->writeString((*_iter1076));
+      xfer += oprot->writeString((*_iter1078));
     }
     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 _size1077;
-            ::apache::thrift::protocol::TType _etype1080;
-            xfer += iprot->readListBegin(_etype1080, _size1077);
-            this->success.resize(_size1077);
-            uint32_t _i1081;
-            for (_i1081 = 0; _i1081 < _size1077; ++_i1081)
+            uint32_t _size1079;
+            ::apache::thrift::protocol::TType _etype1082;
+            xfer += iprot->readListBegin(_etype1082, _size1079);
+            this->success.resize(_size1079);
+            uint32_t _i1083;
+            for (_i1083 = 0; _i1083 < _size1079; ++_i1083)
             {
-              xfer += this->success[_i1081].read(iprot);
+              xfer += this->success[_i1083].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 _iter1082;
-      for (_iter1082 = this->success.begin(); _iter1082 != this->success.end(); ++_iter1082)
+      std::vector<Partition> ::const_iterator _iter1084;
+      for (_iter1084 = this->success.begin(); _iter1084 != this->success.end(); ++_iter1084)
       {
-        xfer += (*_iter1082).write(oprot);
+        xfer += (*_iter1084).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 _size1083;
-            ::apache::thrift::protocol::TType _etype1086;
-            xfer += iprot->readListBegin(_etype1086, _size1083);
-            (*(this->success)).resize(_size1083);
-            uint32_t _i1087;
-            for (_i1087 = 0; _i1087 < _size1083; ++_i1087)
+            uint32_t _size1085;
+            ::apache::thrift::protocol::TType _etype1088;
+            xfer += iprot->readListBegin(_etype1088, _size1085);
+            (*(this->success)).resize(_size1085);
+            uint32_t _i1089;
+            for (_i1089 = 0; _i1089 < _size1085; ++_i1089)
             {
-              xfer += (*(this->success))[_i1087].read(iprot);
+              xfer += (*(this->success))[_i1089].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 _size1088;
-            ::apache::thrift::protocol::TType _etype1091;
-            xfer += iprot->readListBegin(_etype1091, _size1088);
-            this->group_names.resize(_size1088);
-            uint32_t _i1092;
-            for (_i1092 = 0; _i1092 < _size1088; ++_i1092)
+            uint32_t _size1090;
+            ::apache::thrift::protocol::TType _etype1093;
+            xfer += iprot->readListBegin(_etype1093, _size1090);
+            this->group_names.resize(_size1090);
+            uint32_t _i1094;
+            for (_i1094 = 0; _i1094 < _size1090; ++_i1094)
             {
-              xfer += iprot->readString(this->group_names[_i1092]);
+              xfer += iprot->readString(this->group_names[_i1094]);
             }
             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 _iter1093;
-    for (_iter1093 = this->group_names.begin(); _iter1093 != this->group_names.end(); ++_iter1093)
+    std::vector<std::string> ::const_iterator _iter1095;
+    for (_iter1095 = this->group_names.begin(); _iter1095 != this->group_names.end(); ++_iter1095)
     {
-      xfer += oprot->writeString((*_iter1093));
+      xfer += oprot->writeString((*_iter1095));
     }
     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 _iter1094;
-    for (_iter1094 = (*(this->group_names)).begin(); _iter1094 != (*(this->group_names)).end(); ++_iter1094)
+    std::vector<std::string> ::const_iterator _iter1096;
+    for (_iter1096 = (*(this->group_names)).begin(); _iter1096 != (*(this->group_names)).end(); ++_iter1096)
     {
-      xfer += oprot->writeString((*_iter1094));
+      xfer += oprot->writeString((*_iter1096));
     }
     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 _size1095;
-            ::apache::thrift::protocol::TType _etype1098;
-            xfer += iprot->readListBegin(_etype1098, _size1095);
-            this->success.resize(_size1095);
-            uint32_t _i1099;
-            for (_i1099 = 0; _i1099 < _size1095; ++_i1099)
+            uint32_t _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[_i1099].read(iprot);
+              xfer += this->success[_i1101].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 _iter1100;
-      for (_iter1100 = this->success.begin(); _iter1100 != this->success.end(); ++_iter1100)
+      std::vector<Partition> ::const_iterator _iter1102;
+      for (_iter1102 = this->success.begin(); _iter1102 != this->success.end(); ++_iter1102)
       {
-        xfer += (*_iter1100).write(oprot);
+        xfer += (*_iter1102).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 _size1101;
-            ::apache::thrift::protocol::TType _etype1104;
-            xfer += iprot->readListBegin(_etype1104, _size1101);
-            (*(this->success)).resize(_size1101);
-            uint32_t _i1105;
-            for (_i1105 = 0; _i1105 < _size1101; ++_i1105)
+            uint32_t _size1103;
+            ::apache::thrift::protocol::TType _etype1106;
+            xfer += iprot->readListBegin(_etype1106, _size1103);
+            (*(this->success)).resize(_size1103);
+            uint32_t _i1107;
+            for (_i1107 = 0; _i1107 < _size1103; ++_i1107)
             {
-              xfer += (*(this->success))[_i1105].read(iprot);
+              xfer += (*(this->success))[_i1107].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 _size1106;
-            ::apache::thrift::protocol::TType _etype1109;
-            xfer += iprot->readListBegin(_etype1109, _size1106);
-            this->success.resize(_size1106);
-            uint32_t _i1110;
-            for (_i1110 = 0; _i1110 < _size1106; ++_i1110)
+            uint32_t _size1108;
+            ::apache::thrift::protocol::TType _etype1111;
+            xfer += iprot->readListBegin(_etype1111, _size1108);
+            this

<TRUNCATED>

[2/8] hive git commit: HIVE-15202 Concurrent compactions for the same partition may generate malformed folder structure (Eugene Koifman, reviewed by Wei Zheng)

Posted by ek...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 317da4b..d41a99c 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -324,6 +324,8 @@ public class ThriftHiveMetastore {
 
     public void compact(CompactionRequest rqst) throws org.apache.thrift.TException;
 
+    public CompactionResponse compact2(CompactionRequest rqst) throws org.apache.thrift.TException;
+
     public ShowCompactResponse show_compact(ShowCompactRequest rqst) throws org.apache.thrift.TException;
 
     public void add_dynamic_partitions(AddDynamicPartitions rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException;
@@ -632,6 +634,8 @@ public class ThriftHiveMetastore {
 
     public void compact(CompactionRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void compact2(CompactionRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void show_compact(ShowCompactRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void add_dynamic_partitions(AddDynamicPartitions rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -4804,6 +4808,29 @@ public class ThriftHiveMetastore {
       return;
     }
 
+    public CompactionResponse compact2(CompactionRequest rqst) throws org.apache.thrift.TException
+    {
+      send_compact2(rqst);
+      return recv_compact2();
+    }
+
+    public void send_compact2(CompactionRequest rqst) throws org.apache.thrift.TException
+    {
+      compact2_args args = new compact2_args();
+      args.setRqst(rqst);
+      sendBase("compact2", args);
+    }
+
+    public CompactionResponse recv_compact2() throws org.apache.thrift.TException
+    {
+      compact2_result result = new compact2_result();
+      receiveBase(result, "compact2");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "compact2 failed: unknown result");
+    }
+
     public ShowCompactResponse show_compact(ShowCompactRequest rqst) throws org.apache.thrift.TException
     {
       send_show_compact(rqst);
@@ -10080,6 +10107,38 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void compact2(CompactionRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      compact2_call method_call = new compact2_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class compact2_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private CompactionRequest rqst;
+      public compact2_call(CompactionRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.rqst = rqst;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("compact2", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        compact2_args args = new compact2_args();
+        args.setRqst(rqst);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public CompactionResponse getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_compact2();
+      }
+    }
+
     public void show_compact(ShowCompactRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       show_compact_call method_call = new show_compact_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
@@ -10580,6 +10639,7 @@ public class ThriftHiveMetastore {
       processMap.put("heartbeat", new heartbeat());
       processMap.put("heartbeat_txn_range", new heartbeat_txn_range());
       processMap.put("compact", new compact());
+      processMap.put("compact2", new compact2());
       processMap.put("show_compact", new show_compact());
       processMap.put("add_dynamic_partitions", new add_dynamic_partitions());
       processMap.put("get_next_notification", new get_next_notification());
@@ -14244,6 +14304,26 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class compact2<I extends Iface> extends org.apache.thrift.ProcessFunction<I, compact2_args> {
+      public compact2() {
+        super("compact2");
+      }
+
+      public compact2_args getEmptyArgsInstance() {
+        return new compact2_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public compact2_result getResult(I iface, compact2_args args) throws org.apache.thrift.TException {
+        compact2_result result = new compact2_result();
+        result.success = iface.compact2(args.rqst);
+        return result;
+      }
+    }
+
     public static class show_compact<I extends Iface> extends org.apache.thrift.ProcessFunction<I, show_compact_args> {
       public show_compact() {
         super("show_compact");
@@ -14624,6 +14704,7 @@ public class ThriftHiveMetastore {
       processMap.put("heartbeat", new heartbeat());
       processMap.put("heartbeat_txn_range", new heartbeat_txn_range());
       processMap.put("compact", new compact());
+      processMap.put("compact2", new compact2());
       processMap.put("show_compact", new show_compact());
       processMap.put("add_dynamic_partitions", new add_dynamic_partitions());
       processMap.put("get_next_notification", new get_next_notification());
@@ -23317,6 +23398,57 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class compact2<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compact2_args, CompactionResponse> {
+      public compact2() {
+        super("compact2");
+      }
+
+      public compact2_args getEmptyArgsInstance() {
+        return new compact2_args();
+      }
+
+      public AsyncMethodCallback<CompactionResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<CompactionResponse>() { 
+          public void onComplete(CompactionResponse o) {
+            compact2_result result = new compact2_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            compact2_result result = new compact2_result();
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, compact2_args args, org.apache.thrift.async.AsyncMethodCallback<CompactionResponse> resultHandler) throws TException {
+        iface.compact2(args.rqst,resultHandler);
+      }
+    }
+
     public static class show_compact<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, show_compact_args, ShowCompactResponse> {
       public show_compact() {
         super("show_compact");
@@ -170424,86 +170556,759 @@ public class ThriftHiveMetastore {
         return _fieldName;
       }
     }
-
-    // isset id assignments
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, HeartbeatTxnRangeResponse.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(heartbeat_txn_range_result.class, metaDataMap);
+    }
+
+    public heartbeat_txn_range_result() {
+    }
+
+    public heartbeat_txn_range_result(
+      HeartbeatTxnRangeResponse success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public heartbeat_txn_range_result(heartbeat_txn_range_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new HeartbeatTxnRangeResponse(other.success);
+      }
+    }
+
+    public heartbeat_txn_range_result deepCopy() {
+      return new heartbeat_txn_range_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public HeartbeatTxnRangeResponse getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(HeartbeatTxnRangeResponse success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((HeartbeatTxnRangeResponse)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof heartbeat_txn_range_result)
+        return this.equals((heartbeat_txn_range_result)that);
+      return false;
+    }
+
+    public boolean equals(heartbeat_txn_range_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (isSetSuccess());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(heartbeat_txn_range_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("heartbeat_txn_range_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class heartbeat_txn_range_resultStandardSchemeFactory implements SchemeFactory {
+      public heartbeat_txn_range_resultStandardScheme getScheme() {
+        return new heartbeat_txn_range_resultStandardScheme();
+      }
+    }
+
+    private static class heartbeat_txn_range_resultStandardScheme extends StandardScheme<heartbeat_txn_range_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, heartbeat_txn_range_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new HeartbeatTxnRangeResponse();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, heartbeat_txn_range_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class heartbeat_txn_range_resultTupleSchemeFactory implements SchemeFactory {
+      public heartbeat_txn_range_resultTupleScheme getScheme() {
+        return new heartbeat_txn_range_resultTupleScheme();
+      }
+    }
+
+    private static class heartbeat_txn_range_resultTupleScheme extends TupleScheme<heartbeat_txn_range_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, heartbeat_txn_range_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, heartbeat_txn_range_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = new HeartbeatTxnRangeResponse();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class compact_args implements org.apache.thrift.TBase<compact_args, compact_args._Fields>, java.io.Serializable, Cloneable, Comparable<compact_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compact_args");
+
+    private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new compact_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new compact_argsTupleSchemeFactory());
+    }
+
+    private CompactionRequest rqst; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      RQST((short)1, "rqst");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // RQST
+            return RQST;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CompactionRequest.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compact_args.class, metaDataMap);
+    }
+
+    public compact_args() {
+    }
+
+    public compact_args(
+      CompactionRequest rqst)
+    {
+      this();
+      this.rqst = rqst;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public compact_args(compact_args other) {
+      if (other.isSetRqst()) {
+        this.rqst = new CompactionRequest(other.rqst);
+      }
+    }
+
+    public compact_args deepCopy() {
+      return new compact_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.rqst = null;
+    }
+
+    public CompactionRequest getRqst() {
+      return this.rqst;
+    }
+
+    public void setRqst(CompactionRequest rqst) {
+      this.rqst = rqst;
+    }
+
+    public void unsetRqst() {
+      this.rqst = null;
+    }
+
+    /** Returns true if field rqst is set (has been assigned a value) and false otherwise */
+    public boolean isSetRqst() {
+      return this.rqst != null;
+    }
+
+    public void setRqstIsSet(boolean value) {
+      if (!value) {
+        this.rqst = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case RQST:
+        if (value == null) {
+          unsetRqst();
+        } else {
+          setRqst((CompactionRequest)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case RQST:
+        return getRqst();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case RQST:
+        return isSetRqst();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof compact_args)
+        return this.equals((compact_args)that);
+      return false;
+    }
+
+    public boolean equals(compact_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_rqst = true && this.isSetRqst();
+      boolean that_present_rqst = true && that.isSetRqst();
+      if (this_present_rqst || that_present_rqst) {
+        if (!(this_present_rqst && that_present_rqst))
+          return false;
+        if (!this.rqst.equals(that.rqst))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_rqst = true && (isSetRqst());
+      list.add(present_rqst);
+      if (present_rqst)
+        list.add(rqst);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(compact_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetRqst()).compareTo(other.isSetRqst());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRqst()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, other.rqst);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("compact_args(");
+      boolean first = true;
+
+      sb.append("rqst:");
+      if (this.rqst == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.rqst);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (rqst != null) {
+        rqst.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class compact_argsStandardSchemeFactory implements SchemeFactory {
+      public compact_argsStandardScheme getScheme() {
+        return new compact_argsStandardScheme();
+      }
+    }
+
+    private static class compact_argsStandardScheme extends StandardScheme<compact_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, compact_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // RQST
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.rqst = new CompactionRequest();
+                struct.rqst.read(iprot);
+                struct.setRqstIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, compact_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.rqst != null) {
+          oprot.writeFieldBegin(RQST_FIELD_DESC);
+          struct.rqst.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class compact_argsTupleSchemeFactory implements SchemeFactory {
+      public compact_argsTupleScheme getScheme() {
+        return new compact_argsTupleScheme();
+      }
+    }
+
+    private static class compact_argsTupleScheme extends TupleScheme<compact_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, compact_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetRqst()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetRqst()) {
+          struct.rqst.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, compact_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.rqst = new CompactionRequest();
+          struct.rqst.read(iprot);
+          struct.setRqstIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class compact_result implements org.apache.thrift.TBase<compact_result, compact_result._Fields>, java.io.Serializable, Cloneable, Comparable<compact_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compact_result");
+
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new compact_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new compact_resultTupleSchemeFactory());
+    }
+
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+;
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, HeartbeatTxnRangeResponse.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(heartbeat_txn_range_result.class, metaDataMap);
-    }
-
-    public heartbeat_txn_range_result() {
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compact_result.class, metaDataMap);
     }
 
-    public heartbeat_txn_range_result(
-      HeartbeatTxnRangeResponse success)
-    {
-      this();
-      this.success = success;
+    public compact_result() {
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public heartbeat_txn_range_result(heartbeat_txn_range_result other) {
-      if (other.isSetSuccess()) {
-        this.success = new HeartbeatTxnRangeResponse(other.success);
-      }
+    public compact_result(compact_result other) {
     }
 
-    public heartbeat_txn_range_result deepCopy() {
-      return new heartbeat_txn_range_result(this);
+    public compact_result deepCopy() {
+      return new compact_result(this);
     }
 
     @Override
     public void clear() {
-      this.success = null;
-    }
-
-    public HeartbeatTxnRangeResponse getSuccess() {
-      return this.success;
-    }
-
-    public void setSuccess(HeartbeatTxnRangeResponse success) {
-      this.success = success;
-    }
-
-    public void unsetSuccess() {
-      this.success = null;
-    }
-
-    /** Returns true if field success is set (has been assigned a value) and false otherwise */
-    public boolean isSetSuccess() {
-      return this.success != null;
-    }
-
-    public void setSuccessIsSet(boolean value) {
-      if (!value) {
-        this.success = null;
-      }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
-      case SUCCESS:
-        if (value == null) {
-          unsetSuccess();
-        } else {
-          setSuccess((HeartbeatTxnRangeResponse)value);
-        }
-        break;
-
       }
     }
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
-      case SUCCESS:
-        return getSuccess();
-
       }
       throw new IllegalStateException();
     }
@@ -170515,8 +171320,6 @@ public class ThriftHiveMetastore {
       }
 
       switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
       }
       throw new IllegalStateException();
     }
@@ -170525,24 +171328,15 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof heartbeat_txn_range_result)
-        return this.equals((heartbeat_txn_range_result)that);
+      if (that instanceof compact_result)
+        return this.equals((compact_result)that);
       return false;
     }
 
-    public boolean equals(heartbeat_txn_range_result that) {
+    public boolean equals(compact_result that) {
       if (that == null)
         return false;
 
-      boolean this_present_success = true && this.isSetSuccess();
-      boolean that_present_success = true && that.isSetSuccess();
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
-          return false;
-        if (!this.success.equals(that.success))
-          return false;
-      }
-
       return true;
     }
 
@@ -170550,32 +171344,17 @@ public class ThriftHiveMetastore {
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
-      boolean present_success = true && (isSetSuccess());
-      list.add(present_success);
-      if (present_success)
-        list.add(success);
-
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(heartbeat_txn_range_result other) {
+    public int compareTo(compact_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSuccess()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
       return 0;
     }
 
@@ -170593,16 +171372,9 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("heartbeat_txn_range_result(");
+      StringBuilder sb = new StringBuilder("compact_result(");
       boolean first = true;
 
-      sb.append("success:");
-      if (this.success == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.success);
-      }
-      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -170610,9 +171382,6 @@ public class ThriftHiveMetastore {
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
-      if (success != null) {
-        success.validate();
-      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -170631,15 +171400,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class heartbeat_txn_range_resultStandardSchemeFactory implements SchemeFactory {
-      public heartbeat_txn_range_resultStandardScheme getScheme() {
-        return new heartbeat_txn_range_resultStandardScheme();
+    private static class compact_resultStandardSchemeFactory implements SchemeFactory {
+      public compact_resultStandardScheme getScheme() {
+        return new compact_resultStandardScheme();
       }
     }
 
-    private static class heartbeat_txn_range_resultStandardScheme extends StandardScheme<heartbeat_txn_range_result> {
+    private static class compact_resultStandardScheme extends StandardScheme<compact_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, heartbeat_txn_range_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, compact_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -170649,15 +171418,6 @@ public class ThriftHiveMetastore {
             break;
           }
           switch (schemeField.id) {
-            case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new HeartbeatTxnRangeResponse();
-                struct.success.read(iprot);
-                struct.setSuccessIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -170667,65 +171427,46 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, heartbeat_txn_range_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, compact_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.success != null) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          struct.success.write(oprot);
-          oprot.writeFieldEnd();
-        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class heartbeat_txn_range_resultTupleSchemeFactory implements SchemeFactory {
-      public heartbeat_txn_range_resultTupleScheme getScheme() {
-        return new heartbeat_txn_range_resultTupleScheme();
+    private static class compact_resultTupleSchemeFactory implements SchemeFactory {
+      public compact_resultTupleScheme getScheme() {
+        return new compact_resultTupleScheme();
       }
     }
 
-    private static class heartbeat_txn_range_resultTupleScheme extends TupleScheme<heartbeat_txn_range_result> {
+    private static class compact_resultTupleScheme extends TupleScheme<compact_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, heartbeat_txn_range_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, compact_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
-        BitSet optionals = new BitSet();
-        if (struct.isSetSuccess()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetSuccess()) {
-          struct.success.write(oprot);
-        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, heartbeat_txn_range_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, compact_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          struct.success = new HeartbeatTxnRangeResponse();
-          struct.success.read(iprot);
-          struct.setSuccessIsSet(true);
-        }
       }
     }
 
   }
 
-  public static class compact_args implements org.apache.thrift.TBase<compact_args, compact_args._Fields>, java.io.Serializable, Cloneable, Comparable<compact_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compact_args");
+  public static class compact2_args implements org.apache.thrift.TBase<compact2_args, compact2_args._Fields>, java.io.Serializable, Cloneable, Comparable<compact2_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compact2_args");
 
     private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new compact_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new compact_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new compact2_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new compact2_argsTupleSchemeFactory());
     }
 
     private CompactionRequest rqst; // required
@@ -170795,13 +171536,13 @@ public class ThriftHiveMetastore {
       tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CompactionRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compact_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compact2_args.class, metaDataMap);
     }
 
-    public compact_args() {
+    public compact2_args() {
     }
 
-    public compact_args(
+    public compact2_args(
       CompactionRequest rqst)
     {
       this();
@@ -170811,14 +171552,14 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public compact_args(compact_args other) {
+    public compact2_args(compact2_args other) {
       if (other.isSetRqst()) {
         this.rqst = new CompactionRequest(other.rqst);
       }
     }
 
-    public compact_args deepCopy() {
-      return new compact_args(this);
+    public compact2_args deepCopy() {
+      return new compact2_args(this);
     }
 
     @Override
@@ -170888,12 +171629,12 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof compact_args)
-        return this.equals((compact_args)that);
+      if (that instanceof compact2_args)
+        return this.equals((compact2_args)that);
       return false;
     }
 
-    public boolean equals(compact_args that) {
+    public boolean equals(compact2_args that) {
       if (that == null)
         return false;
 
@@ -170922,7 +171663,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(compact_args other) {
+    public int compareTo(compact2_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -170956,7 +171697,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("compact_args(");
+      StringBuilder sb = new StringBuilder("compact2_args(");
       boolean first = true;
 
       sb.append("rqst:");
@@ -170994,15 +171735,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class compact_argsStandardSchemeFactory implements SchemeFactory {
-      public compact_argsStandardScheme getScheme() {
-        return new compact_argsStandardScheme();
+    private static class compact2_argsStandardSchemeFactory implements SchemeFactory {
+      public compact2_argsStandardScheme getScheme() {
+        return new compact2_argsStandardScheme();
       }
     }
 
-    private static class compact_argsStandardScheme extends StandardScheme<compact_args> {
+    private static class compact2_argsStandardScheme extends StandardScheme<compact2_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, compact_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, compact2_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -171030,7 +171771,7 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, compact_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, compact2_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -171045,16 +171786,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class compact_argsTupleSchemeFactory implements SchemeFactory {
-      public compact_argsTupleScheme getScheme() {
-        return new compact_argsTupleScheme();
+    private static class compact2_argsTupleSchemeFactory implements SchemeFactory {
+      public compact2_argsTupleScheme getScheme() {
+        return new compact2_argsTupleScheme();
       }
     }
 
-    private static class compact_argsTupleScheme extends TupleScheme<compact_args> {
+    private static class compact2_argsTupleScheme extends TupleScheme<compact2_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, compact_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, compact2_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetRqst()) {
@@ -171067,7 +171808,7 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, compact_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, compact2_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -171080,20 +171821,22 @@ public class ThriftHiveMetastore {
 
   }
 
-  public static class compact_result implements org.apache.thrift.TBase<compact_result, compact_result._Fields>, java.io.Serializable, Cloneable, Comparable<compact_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compact_result");
+  public static class compact2_result implements org.apache.thrift.TBase<compact2_result, compact2_result._Fields>, java.io.Serializable, Cloneable, Comparable<compact2_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compact2_result");
 
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new compact_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new compact_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new compact2_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new compact2_resultTupleSchemeFactory());
     }
 
+    private CompactionResponse success; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-;
+      SUCCESS((short)0, "success");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -171108,6 +171851,8 @@ public class ThriftHiveMetastore {
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
           default:
             return null;
         }
@@ -171146,37 +171891,86 @@ public class ThriftHiveMetastore {
         return _fieldName;
       }
     }
+
+    // isset id assignments
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CompactionResponse.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compact_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compact2_result.class, metaDataMap);
     }
 
-    public compact_result() {
+    public compact2_result() {
+    }
+
+    public compact2_result(
+      CompactionResponse success)
+    {
+      this();
+      this.success = success;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public compact_result(compact_result other) {
+    public compact2_result(compact2_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new CompactionResponse(other.success);
+      }
     }
 
-    public compact_result deepCopy() {
-      return new compact_result(this);
+    public compact2_result deepCopy() {
+      return new compact2_result(this);
     }
 
     @Override
     public void clear() {
+      this.success = null;
+    }
+
+    public CompactionResponse getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(CompactionResponse success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((CompactionResponse)value);
+        }
+        break;
+
       }
     }
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
       }
       throw new IllegalStateException();
     }
@@ -171188,6 +171982,8 @@ public class ThriftHiveMetastore {
       }
 
       switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
       }
       throw new IllegalStateException();
     }
@@ -171196,15 +171992,24 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof compact_result)
-        return this.equals((compact_result)that);
+      if (that instanceof compact2_result)
+        return this.equals((compact2_result)that);
       return false;
     }
 
-    public boolean equals(compact_result that) {
+    public boolean equals(compact2_result that) {
       if (that == null)
         return false;
 
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
       return true;
     }
 
@@ -171212,17 +172017,32 @@ public class ThriftHiveMetastore {
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
+      boolean present_success = true && (isSetSuccess());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(compact_result other) {
+    public int compareTo(compact2_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -171240,9 +172060,16 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("compact_result(");
+      StringBuilder sb = new StringBuilder("compact2_result(");
       boolean first = true;
 
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -171250,6 +172077,9 @@ public class ThriftHiveMetastore {
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -171268,15 +172098,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class compact_resultStandardSchemeFactory implements SchemeFactory {
-      public compact_resultStandardScheme getScheme() {
-        return new compact_resultStandardScheme();
+    private static class compact2_resultStandardSchemeFactory implements SchemeFactory {
+      public compact2_resultStandardScheme getScheme() {
+        return new compact2_resultStandardScheme();
       }
     }
 
-    private static class compact_resultStandardScheme extends StandardScheme<compact_result> {
+    private static class compact2_resultStandardScheme extends StandardScheme<compact2_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, compact_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, compact2_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -171286,6 +172116,15 @@ public class ThriftHiveMetastore {
             break;
           }
           switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new CompactionResponse();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -171295,32 +172134,51 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, compact_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, compact2_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class compact_resultTupleSchemeFactory implements SchemeFactory {
-      public compact_resultTupleScheme getScheme() {
-        return new compact_resultTupleScheme();
+    private static class compact2_resultTupleSchemeFactory implements SchemeFactory {
+      public compact2_resultTupleScheme getScheme() {
+        return new compact2_resultTupleScheme();
       }
     }
 
-    private static class compact_resultTupleScheme extends TupleScheme<compact_result> {
+    private static class compact2_resultTupleScheme extends TupleScheme<compact2_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, compact_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, compact2_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, compact_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, compact2_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = new CompactionResponse();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index e5abb66..0087205 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -1131,6 +1131,11 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function compact(\metastore\CompactionRequest $rqst);
   /**
+   * @param \metastore\CompactionRequest $rqst
+   * @return \metastore\CompactionResponse
+   */
+  public function compact2(\metastore\CompactionRequest $rqst);
+  /**
    * @param \metastore\ShowCompactRequest $rqst
    * @return \metastore\ShowCompactResponse
    */
@@ -9266,6 +9271,57 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     return;
   }
 
+  public function compact2(\metastore\CompactionRequest $rqst)
+  {
+    $this->send_compact2($rqst);
+    return $this->recv_compact2();
+  }
+
+  public function send_compact2(\metastore\CompactionRequest $rqst)
+  {
+    $args = new \metastore\ThriftHiveMetastore_compact2_args();
+    $args->rqst = $rqst;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'compact2', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('compact2', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_compact2()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_compact2_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_compact2_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    throw new \Exception("compact2 failed: unknown result");
+  }
+
   public function show_compact(\metastore\ShowCompactRequest $rqst)
   {
     $this->send_show_compact($rqst);
@@ -43126,6 +43182,166 @@ class ThriftHiveMetastore_compact_result {
 
 }
 
+class ThriftHiveMetastore_compact2_args {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\CompactionRequest
+   */
+  public $rqst = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'rqst',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\CompactionRequest',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['rqst'])) {
+        $this->rqst = $vals['rqst'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_compact2_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->rqst = new \metastore\CompactionRequest();
+            $xfer += $this->rqst->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_compact2_args');
+    if ($this->rqst !== null) {
+      if (!is_object($this->rqst)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('rqst', TType::STRUCT, 1);
+      $xfer += $this->rqst->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_compact2_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\CompactionResponse
+   */
+  public $success = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\CompactionResponse',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_compact2_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::STRUCT) {
+            $this->success = new \metastore\CompactionResponse();
+            $xfer += $this->success->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_compact2_result');
+    if ($this->success !== null) {
+      if (!is_object($this->success)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0);
+      $xfer += $this->success->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class ThriftHiveMetastore_show_compact_args {
   static $_TSPEC;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/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 ecb6254..b9af4ef 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -14673,6 +14673,127 @@ class CompactionRequest {
 
 }
 
+class CompactionResponse {
+  static $_TSPEC;
+
+  /**
+   * @var int
+   */
+  public $id = null;
+  /**
+   * @var string
+   */
+  public $state = null;
+  /**
+   * @var bool
+   */
+  public $accepted = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'id',
+          'type' => TType::I64,
+          ),
+        2 => array(
+          'var' => 'state',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'accepted',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['id'])) {
+        $this->id = $vals['id'];
+      }
+      if (isset($vals['state'])) {
+        $this->state = $vals['state'];
+      }
+      if (isset($vals['accepted'])) {
+        $this->accepted = $vals['accepted'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'CompactionResponse';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->id);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->state);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->accepted);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('CompactionResponse');
+    if ($this->id !== null) {
+      $xfer += $output->writeFieldBegin('id', TType::I64, 1);
+      $xfer += $output->writeI64($this->id);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->state !== null) {
+      $xfer += $output->writeFieldBegin('state', TType::STRING, 2);
+      $xfer += $output->writeString($this->state);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->accepted !== null) {
+      $xfer += $output->writeFieldBegin('accepted', TType::BOOL, 3);
+      $xfer += $output->writeBool($this->accepted);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class ShowCompactRequest {
   static $_TSPEC;
 
@@ -14774,6 +14895,10 @@ class ShowCompactResponseElement {
    * @var string
    */
   public $hadoopJobId = "None";
+  /**
+   * @var int
+   */
+  public $id = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -14826,6 +14951,10 @@ class ShowCompactResponseElement {
           'var' => 'hadoopJobId',
           'type' => TType::STRING,
           ),
+        13 => array(
+          'var' => 'id',
+          'type' => TType::I64,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -14865,6 +14994,9 @@ class ShowCompactResponseElement {
       if (isset($vals['hadoopJobId'])) {
         $this->hadoopJobId = $vals['hadoopJobId'];
       }
+      if (isset($vals['id'])) {
+        $this->id = $vals['id'];
+      }
     }
   }
 
@@ -14971,6 +15103,13 @@ class ShowCompactResponseElement {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 13:
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->id);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -15044,6 +15183,11 @@ class ShowCompactResponseElement {
       $xfer += $output->writeString($this->hadoopJobId);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->id !== null) {
+      $xfer += $output->writeFieldBegin('id', TType::I64, 13);
+      $xfer += $output->writeI64($this->id);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index 81bebd8..5c24707 100755
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -165,6 +165,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  void heartbeat(HeartbeatRequest ids)')
   print('  HeartbeatTxnRangeResponse heartbeat_txn_range(HeartbeatTxnRangeRequest txns)')
   print('  void compact(CompactionRequest rqst)')
+  print('  CompactionResponse compact2(CompactionRequest rqst)')
   print('  ShowCompactResponse show_compact(ShowCompactRequest rqst)')
   print('  void add_dynamic_partitions(AddDynamicPartitions rqst)')
   print('  NotificationEventResponse get_next_notification(NotificationEventRequest rqst)')
@@ -1091,6 +1092,12 @@ elif cmd == 'compact':
     sys.exit(1)
   pp.pprint(client.compact(eval(args[0]),))
 
+elif cmd == 'compact2':
+  if len(args) != 1:
+    print('compact2 requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.compact2(eval(args[0]),))
+
 elif cmd == 'show_compact':
   if len(args) != 1:
     print('show_compact requires 1 args')

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index dfebd90..bc51b75 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -1154,6 +1154,13 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def compact2(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    pass
+
   def show_compact(self, rqst):
     """
     Parameters:
@@ -6391,6 +6398,37 @@ class Client(fb303.FacebookService.Client, Iface):
     iprot.readMessageEnd()
     return
 
+  def compact2(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    self.send_compact2(rqst)
+    return self.recv_compact2()
+
+  def send_compact2(self, rqst):
+    self._oprot.writeMessageBegin('compact2', TMessageType.CALL, self._seqid)
+    args = compact2_args()
+    args.rqst = rqst
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_compact2(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = compact2_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "compact2 failed: unknown result")
+
   def show_compact(self, rqst):
     """
     Parameters:
@@ -6867,6 +6905,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["heartbeat"] = Processor.process_heartbeat
     self._processMap["heartbeat_txn_range"] = Processor.process_heartbeat_txn_range
     self._processMap["compact"] = Processor.process_compact
+    self._processMap["compact2"] = Processor.process_compact2
     self._processMap["show_compact"] = Processor.process_show_compact
     self._processMap["add_dynamic_partitions"] = Processor.process_add_dynamic_partitions
     self._processMap["get_next_notification"] = Processor.process_get_next_notification
@@ -10392,6 +10431,25 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_compact2(self, seqid, iprot, oprot):
+    args = compact2_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = compact2_result()
+    try:
+      result.success = self._handler.compact2(args.rqst)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("compact2", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_show_compact(self, seqid, iprot, oprot):
     args = show_compact_args()
     args.read(iprot)
@@ -34979,6 +35037,137 @@ class compact_result:
   def __ne__(self, other):
     return not (self == other)
 
+class compact2_args:
+  """
+  Attributes:
+   - rqst
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'rqst', (CompactionRequest, CompactionRequest.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, rqst=None,):
+    self.rqst = rqst
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.rqst = CompactionRequest()
+          self.rqst.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('compact2_args')
+    if self.rqst is not None:
+      oprot.writeFieldBegin('rqst', TType.STRUCT, 1)
+      self.rqst.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.rqst)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class compact2_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (CompactionResponse, CompactionResponse.thrift_spec), None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = CompactionResponse()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('compact2_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class show_compact_args:
   """
   Attributes:

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/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 d420f06..21c0390 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -10185,6 +10185,103 @@ class CompactionRequest:
   def __ne__(self, other):
     return not (self == other)
 
+class CompactionResponse:
+  """
+  Attributes:
+   - id
+   - state
+   - accepted
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I64, 'id', None, None, ), # 1
+    (2, TType.STRING, 'state', None, None, ), # 2
+    (3, TType.BOOL, 'accepted', None, None, ), # 3
+  )
+
+  def __init__(self, id=None, state=None, accepted=None,):
+    self.id = id
+    self.state = state
+    self.accepted = accepted
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.I64:
+          self.id = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.state = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.BOOL:
+          self.accepted = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('CompactionResponse')
+    if self.id is not None:
+      oprot.writeFieldBegin('id', TType.I64, 1)
+      oprot.writeI64(self.id)
+      oprot.writeFieldEnd()
+    if self.state is not None:
+      oprot.writeFieldBegin('state', TType.STRING, 2)
+      oprot.writeString(self.state)
+      oprot.writeFieldEnd()
+    if self.accepted is not None:
+      oprot.writeFieldBegin('accepted', TType.BOOL, 3)
+      oprot.writeBool(self.accepted)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.id is None:
+      raise TProtocol.TProtocolException(message='Required field id is unset!')
+    if self.state is None:
+      raise TProtocol.TProtocolException(message='Required field state is unset!')
+    if self.accepted is None:
+      raise TProtocol.TProtocolException(message='Required field accepted is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.id)
+    value = (value * 31) ^ hash(self.state)
+    value = (value * 31) ^ hash(self.accepted)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class ShowCompactRequest:
 
   thrift_spec = (
@@ -10246,6 +10343,7 @@ class ShowCompactResponseElement:
    - metaInfo
    - endTime
    - hadoopJobId
+   - id
   """
 
   thrift_spec = (
@@ -10262,9 +10360,10 @@ class ShowCompactResponseElement:
     (10, TType.STRING, 'metaInfo', None, None, ), # 10
     (11, TType.I64, 'endTime', None, None, ), # 11
     (12, TType.STRING, 'hadoopJobId', None, "None", ), # 12
+    (13, TType.I64, 'id', None, None, ), # 13
   )
 
-  def __init__(self, dbname=None, tablename=None, partitionname=None, type=None, state=None, workerid=None, start=None, runAs=None, hightestTxnId=None, metaInfo=None, endTime=None, hadoopJobId=thrift_spec[12][4],):
+  def __init__(self, dbname=None, tablename=None, partitionname=None, type=None, state=None, workerid=None, start=None, runAs=None, hightestTxnId=None, metaInfo=None, endTime=None, hadoopJobId=thrift_spec[12][4], id=None,):
     self.dbname = dbname
     self.tablename = tablename
     self.partitionname = partitionname
@@ -10277,6 +10376,7 @@ class ShowCompactResponseElement:
     self.metaInfo = metaInfo
     self.endTime = endTime
     self.hadoopJobId = hadoopJobId
+    self.id = id
 
   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:
@@ -10347,6 +10447,11 @@ class ShowCompactResponseElement:
           self.hadoopJobId = iprot.readString()
         else:
           iprot.skip(ftype)
+      elif fid == 13:
+        if ftype == TType.I64:
+          self.id = iprot.readI64()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -10405,6 +10510,10 @@ class ShowCompactResponseElement:
       oprot.writeFieldBegin('hadoopJobId', TType.STRING, 12)
       oprot.writeString(self.hadoopJobId)
       oprot.writeFieldEnd()
+    if self.id is not None:
+      oprot.writeFieldBegin('id', TType.I64, 13)
+      oprot.writeI64(self.id)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -10434,6 +10543,7 @@ class ShowCompactResponseElement:
     value = (value * 31) ^ hash(self.metaInfo)
     value = (value * 31) ^ hash(self.endTime)
     value = (value * 31) ^ hash(self.hadoopJobId)
+    value = (value * 31) ^ hash(self.id)
     return value
 
   def __repr__(self):


[8/8] hive git commit: HIVE-15309 Miscellaneous logging clean up (Eugene Koifman, reviewed by Wei Zheng)

Posted by ek...@apache.org.
HIVE-15309 Miscellaneous logging clean up (Eugene Koifman, reviewed by Wei Zheng)


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

Branch: refs/heads/master
Commit: 54b3abf5246765e9b173033bc9b16aec9965b00d
Parents: 185026a
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Wed Nov 30 16:27:00 2016 -0800
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Wed Nov 30 16:27:00 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/metastore/txn/TxnHandler.java    | 12 ++++++------
 orc/src/java/org/apache/orc/impl/OrcAcidUtils.java      |  3 +++
 2 files changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/54b3abf5/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index f2b1048..b0fa836 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -2824,9 +2824,9 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         deletedLocks += stmt.executeUpdate(query);
       }
       if(deletedLocks > 0) {
-        Collections.sort(extLockIDs);////easier to read logs
-        LOG.info("Deleted " + deletedLocks + " ext locks from HIVE_LOCKS due to timeout (vs. " +
-            extLockIDs.size() + " found. List: " + extLockIDs + ") maxHeartbeatTime=" + maxHeartbeatTime);
+        Collections.sort(extLockIDs);//easier to read logs
+        LOG.info("Deleted " + deletedLocks + " int locks from HIVE_LOCKS due to timeout (" +
+          "HL_LOCK_EXT_ID list:  " + extLockIDs + ") maxHeartbeatTime=" + maxHeartbeatTime);
       }
       LOG.debug("Going to commit");
       dbConn.commit();
@@ -3236,7 +3236,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
      * production code as possible.
      * In particular, with Derby we always run in a single process with a single metastore and
      * the absence of For Update is handled via a Semaphore.  The later would strictly speaking
-     * make the SQL statments below unnecessary (for Derby), but then they would not be tested.
+     * make the SQL statements below unnecessary (for Derby), but then they would not be tested.
      */
     Connection dbConn = null;
     Statement stmt = null;
@@ -3277,7 +3277,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           derbySemaphore =  derbyKey2Lock.get(key);
           derbySemaphore.acquire();
         }
-        LOG.info(quoteString(key) + " locked by " + quoteString(TxnHandler.hostname));
+        LOG.debug(quoteString(key) + " locked by " + quoteString(TxnHandler.hostname));
         //OK, so now we have a lock
         return new LockHandleImpl(dbConn, stmt, rs, key, derbySemaphore);
       } catch (SQLException ex) {
@@ -3334,7 +3334,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         derbySemaphore.release();
       }
       for(String key : keys) {
-        LOG.info(quoteString(key) + " unlocked by " + quoteString(TxnHandler.hostname));
+        LOG.debug(quoteString(key) + " unlocked by " + quoteString(TxnHandler.hostname));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/54b3abf5/orc/src/java/org/apache/orc/impl/OrcAcidUtils.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/OrcAcidUtils.java b/orc/src/java/org/apache/orc/impl/OrcAcidUtils.java
index 72c7f54..7ca9e1d 100644
--- a/orc/src/java/org/apache/orc/impl/OrcAcidUtils.java
+++ b/orc/src/java/org/apache/orc/impl/OrcAcidUtils.java
@@ -54,6 +54,9 @@ public class OrcAcidUtils {
                                         Path deltaFile) throws IOException {
     Path lengths = getSideFile(deltaFile);
     long result = Long.MAX_VALUE;
+    if(!fs.exists(lengths)) {
+      return result;
+    }
     try (FSDataInputStream stream = fs.open(lengths)) {
       result = -1;
       while (stream.available() > 0) {


[7/8] hive git commit: HIVE-15308 Create ACID table failed intermittently: due to Postgres (SQLState=25P02, ErrorCode=0) (Eugene Koifman, reviewed by Wei Zheng)

Posted by ek...@apache.org.
HIVE-15308 Create ACID table failed intermittently: due to Postgres (SQLState=25P02, ErrorCode=0) (Eugene Koifman, reviewed by Wei Zheng)


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

Branch: refs/heads/master
Commit: 185026a750c5cc2e5207403515c99fdf21e7dbfe
Parents: 7b24ff7
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Wed Nov 30 16:25:11 2016 -0800
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Wed Nov 30 16:25:11 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java   | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/185026a7/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index 6ea43b4..f2b1048 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -3260,6 +3260,11 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
             if (!isDuplicateKeyError(ex)) {
               throw new RuntimeException("Unable to lock " + quoteString(key) + " due to: " + getMessage(ex), ex);
             }
+            //if here, it means a concrurrent acquireLock() inserted the 'key'
+
+            //rollback is done for the benefit of Postgres which throws (SQLState=25P02, ErrorCode=0) if
+            //you attempt any stmt in a txn which had an error.
+            dbConn.rollback();
           }
           rs = stmt.executeQuery(sqlStmt);
           if (!rs.next()) {


[3/8] hive git commit: HIVE-15202 Concurrent compactions for the same partition may generate malformed folder structure (Eugene Koifman, reviewed by Wei Zheng)

Posted by ek...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/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 70fe2c3..6838133 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -346,6 +346,8 @@ class HeartbeatTxnRangeResponse;
 
 class CompactionRequest;
 
+class CompactionResponse;
+
 class ShowCompactRequest;
 
 class ShowCompactResponseElement;
@@ -6001,6 +6003,56 @@ inline std::ostream& operator<<(std::ostream& out, const CompactionRequest& obj)
 }
 
 
+class CompactionResponse {
+ public:
+
+  CompactionResponse(const CompactionResponse&);
+  CompactionResponse& operator=(const CompactionResponse&);
+  CompactionResponse() : id(0), state(), accepted(0) {
+  }
+
+  virtual ~CompactionResponse() throw();
+  int64_t id;
+  std::string state;
+  bool accepted;
+
+  void __set_id(const int64_t val);
+
+  void __set_state(const std::string& val);
+
+  void __set_accepted(const bool val);
+
+  bool operator == (const CompactionResponse & rhs) const
+  {
+    if (!(id == rhs.id))
+      return false;
+    if (!(state == rhs.state))
+      return false;
+    if (!(accepted == rhs.accepted))
+      return false;
+    return true;
+  }
+  bool operator != (const CompactionResponse &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const CompactionResponse & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(CompactionResponse &a, CompactionResponse &b);
+
+inline std::ostream& operator<<(std::ostream& out, const CompactionResponse& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
 class ShowCompactRequest {
  public:
 
@@ -6036,7 +6088,7 @@ inline std::ostream& operator<<(std::ostream& out, const ShowCompactRequest& obj
 }
 
 typedef struct _ShowCompactResponseElement__isset {
-  _ShowCompactResponseElement__isset() : partitionname(false), workerid(false), start(false), runAs(false), hightestTxnId(false), metaInfo(false), endTime(false), hadoopJobId(true) {}
+  _ShowCompactResponseElement__isset() : partitionname(false), workerid(false), start(false), runAs(false), hightestTxnId(false), metaInfo(false), endTime(false), hadoopJobId(true), id(false) {}
   bool partitionname :1;
   bool workerid :1;
   bool start :1;
@@ -6045,6 +6097,7 @@ typedef struct _ShowCompactResponseElement__isset {
   bool metaInfo :1;
   bool endTime :1;
   bool hadoopJobId :1;
+  bool id :1;
 } _ShowCompactResponseElement__isset;
 
 class ShowCompactResponseElement {
@@ -6052,7 +6105,7 @@ class ShowCompactResponseElement {
 
   ShowCompactResponseElement(const ShowCompactResponseElement&);
   ShowCompactResponseElement& operator=(const ShowCompactResponseElement&);
-  ShowCompactResponseElement() : dbname(), tablename(), partitionname(), type((CompactionType::type)0), state(), workerid(), start(0), runAs(), hightestTxnId(0), metaInfo(), endTime(0), hadoopJobId("None") {
+  ShowCompactResponseElement() : dbname(), tablename(), partitionname(), type((CompactionType::type)0), state(), workerid(), start(0), runAs(), hightestTxnId(0), metaInfo(), endTime(0), hadoopJobId("None"), id(0) {
   }
 
   virtual ~ShowCompactResponseElement() throw();
@@ -6068,6 +6121,7 @@ class ShowCompactResponseElement {
   std::string metaInfo;
   int64_t endTime;
   std::string hadoopJobId;
+  int64_t id;
 
   _ShowCompactResponseElement__isset __isset;
 
@@ -6095,6 +6149,8 @@ class ShowCompactResponseElement {
 
   void __set_hadoopJobId(const std::string& val);
 
+  void __set_id(const int64_t val);
+
   bool operator == (const ShowCompactResponseElement & rhs) const
   {
     if (!(dbname == rhs.dbname))
@@ -6137,6 +6193,10 @@ class ShowCompactResponseElement {
       return false;
     else if (__isset.hadoopJobId && !(hadoopJobId == rhs.hadoopJobId))
       return false;
+    if (__isset.id != rhs.__isset.id)
+      return false;
+    else if (__isset.id && !(id == rhs.id))
+      return false;
     return true;
   }
   bool operator != (const ShowCompactResponseElement &rhs) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionResponse.java
new file mode 100644
index 0000000..157ff53
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionResponse.java
@@ -0,0 +1,583 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class CompactionResponse implements org.apache.thrift.TBase<CompactionResponse, CompactionResponse._Fields>, java.io.Serializable, Cloneable, Comparable<CompactionResponse> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CompactionResponse");
+
+  private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("state", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField ACCEPTED_FIELD_DESC = new org.apache.thrift.protocol.TField("accepted", org.apache.thrift.protocol.TType.BOOL, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new CompactionResponseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new CompactionResponseTupleSchemeFactory());
+  }
+
+  private long id; // required
+  private String state; // required
+  private boolean accepted; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    ID((short)1, "id"),
+    STATE((short)2, "state"),
+    ACCEPTED((short)3, "accepted");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ID
+          return ID;
+        case 2: // STATE
+          return STATE;
+        case 3: // ACCEPTED
+          return ACCEPTED;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __ID_ISSET_ID = 0;
+  private static final int __ACCEPTED_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.STATE, new org.apache.thrift.meta_data.FieldMetaData("state", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ACCEPTED, new org.apache.thrift.meta_data.FieldMetaData("accepted", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CompactionResponse.class, metaDataMap);
+  }
+
+  public CompactionResponse() {
+  }
+
+  public CompactionResponse(
+    long id,
+    String state,
+    boolean accepted)
+  {
+    this();
+    this.id = id;
+    setIdIsSet(true);
+    this.state = state;
+    this.accepted = accepted;
+    setAcceptedIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public CompactionResponse(CompactionResponse other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.id = other.id;
+    if (other.isSetState()) {
+      this.state = other.state;
+    }
+    this.accepted = other.accepted;
+  }
+
+  public CompactionResponse deepCopy() {
+    return new CompactionResponse(this);
+  }
+
+  @Override
+  public void clear() {
+    setIdIsSet(false);
+    this.id = 0;
+    this.state = null;
+    setAcceptedIsSet(false);
+    this.accepted = false;
+  }
+
+  public long getId() {
+    return this.id;
+  }
+
+  public void setId(long id) {
+    this.id = id;
+    setIdIsSet(true);
+  }
+
+  public void unsetId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ID_ISSET_ID);
+  }
+
+  /** Returns true if field id is set (has been assigned a value) and false otherwise */
+  public boolean isSetId() {
+    return EncodingUtils.testBit(__isset_bitfield, __ID_ISSET_ID);
+  }
+
+  public void setIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ID_ISSET_ID, value);
+  }
+
+  public String getState() {
+    return this.state;
+  }
+
+  public void setState(String state) {
+    this.state = state;
+  }
+
+  public void unsetState() {
+    this.state = null;
+  }
+
+  /** Returns true if field state is set (has been assigned a value) and false otherwise */
+  public boolean isSetState() {
+    return this.state != null;
+  }
+
+  public void setStateIsSet(boolean value) {
+    if (!value) {
+      this.state = null;
+    }
+  }
+
+  public boolean isAccepted() {
+    return this.accepted;
+  }
+
+  public void setAccepted(boolean accepted) {
+    this.accepted = accepted;
+    setAcceptedIsSet(true);
+  }
+
+  public void unsetAccepted() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ACCEPTED_ISSET_ID);
+  }
+
+  /** Returns true if field accepted is set (has been assigned a value) and false otherwise */
+  public boolean isSetAccepted() {
+    return EncodingUtils.testBit(__isset_bitfield, __ACCEPTED_ISSET_ID);
+  }
+
+  public void setAcceptedIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ACCEPTED_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ID:
+      if (value == null) {
+        unsetId();
+      } else {
+        setId((Long)value);
+      }
+      break;
+
+    case STATE:
+      if (value == null) {
+        unsetState();
+      } else {
+        setState((String)value);
+      }
+      break;
+
+    case ACCEPTED:
+      if (value == null) {
+        unsetAccepted();
+      } else {
+        setAccepted((Boolean)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ID:
+      return getId();
+
+    case STATE:
+      return getState();
+
+    case ACCEPTED:
+      return isAccepted();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ID:
+      return isSetId();
+    case STATE:
+      return isSetState();
+    case ACCEPTED:
+      return isSetAccepted();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof CompactionResponse)
+      return this.equals((CompactionResponse)that);
+    return false;
+  }
+
+  public boolean equals(CompactionResponse that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_id = true;
+    boolean that_present_id = true;
+    if (this_present_id || that_present_id) {
+      if (!(this_present_id && that_present_id))
+        return false;
+      if (this.id != that.id)
+        return false;
+    }
+
+    boolean this_present_state = true && this.isSetState();
+    boolean that_present_state = true && that.isSetState();
+    if (this_present_state || that_present_state) {
+      if (!(this_present_state && that_present_state))
+        return false;
+      if (!this.state.equals(that.state))
+        return false;
+    }
+
+    boolean this_present_accepted = true;
+    boolean that_present_accepted = true;
+    if (this_present_accepted || that_present_accepted) {
+      if (!(this_present_accepted && that_present_accepted))
+        return false;
+      if (this.accepted != that.accepted)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_id = true;
+    list.add(present_id);
+    if (present_id)
+      list.add(id);
+
+    boolean present_state = true && (isSetState());
+    list.add(present_state);
+    if (present_state)
+      list.add(state);
+
+    boolean present_accepted = true;
+    list.add(present_accepted);
+    if (present_accepted)
+      list.add(accepted);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(CompactionResponse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetId()).compareTo(other.isSetId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetState()).compareTo(other.isSetState());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetState()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.state, other.state);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetAccepted()).compareTo(other.isSetAccepted());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetAccepted()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.accepted, other.accepted);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("CompactionResponse(");
+    boolean first = true;
+
+    sb.append("id:");
+    sb.append(this.id);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("state:");
+    if (this.state == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.state);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("accepted:");
+    sb.append(this.accepted);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'id' is unset! Struct:" + toString());
+    }
+
+    if (!isSetState()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'state' is unset! Struct:" + toString());
+    }
+
+    if (!isSetAccepted()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'accepted' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class CompactionResponseStandardSchemeFactory implements SchemeFactory {
+    public CompactionResponseStandardScheme getScheme() {
+      return new CompactionResponseStandardScheme();
+    }
+  }
+
+  private static class CompactionResponseStandardScheme extends StandardScheme<CompactionResponse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, CompactionResponse struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.id = iprot.readI64();
+              struct.setIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // STATE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.state = iprot.readString();
+              struct.setStateIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // ACCEPTED
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.accepted = iprot.readBool();
+              struct.setAcceptedIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, CompactionResponse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(ID_FIELD_DESC);
+      oprot.writeI64(struct.id);
+      oprot.writeFieldEnd();
+      if (struct.state != null) {
+        oprot.writeFieldBegin(STATE_FIELD_DESC);
+        oprot.writeString(struct.state);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(ACCEPTED_FIELD_DESC);
+      oprot.writeBool(struct.accepted);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class CompactionResponseTupleSchemeFactory implements SchemeFactory {
+    public CompactionResponseTupleScheme getScheme() {
+      return new CompactionResponseTupleScheme();
+    }
+  }
+
+  private static class CompactionResponseTupleScheme extends TupleScheme<CompactionResponse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, CompactionResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI64(struct.id);
+      oprot.writeString(struct.state);
+      oprot.writeBool(struct.accepted);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, CompactionResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.id = iprot.readI64();
+      struct.setIdIsSet(true);
+      struct.state = iprot.readString();
+      struct.setStateIsSet(true);
+      struct.accepted = iprot.readBool();
+      struct.setAcceptedIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java
index 212ecd6..78b4f7c 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java
@@ -50,6 +50,7 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
   private static final org.apache.thrift.protocol.TField META_INFO_FIELD_DESC = new org.apache.thrift.protocol.TField("metaInfo", org.apache.thrift.protocol.TType.STRING, (short)10);
   private static final org.apache.thrift.protocol.TField END_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("endTime", org.apache.thrift.protocol.TType.I64, (short)11);
   private static final org.apache.thrift.protocol.TField HADOOP_JOB_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("hadoopJobId", org.apache.thrift.protocol.TType.STRING, (short)12);
+  private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I64, (short)13);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -69,6 +70,7 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
   private String metaInfo; // optional
   private long endTime; // optional
   private String hadoopJobId; // optional
+  private long id; // 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 {
@@ -87,7 +89,8 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
     HIGHTEST_TXN_ID((short)9, "hightestTxnId"),
     META_INFO((short)10, "metaInfo"),
     END_TIME((short)11, "endTime"),
-    HADOOP_JOB_ID((short)12, "hadoopJobId");
+    HADOOP_JOB_ID((short)12, "hadoopJobId"),
+    ID((short)13, "id");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -126,6 +129,8 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
           return END_TIME;
         case 12: // HADOOP_JOB_ID
           return HADOOP_JOB_ID;
+        case 13: // ID
+          return ID;
         default:
           return null;
       }
@@ -169,8 +174,9 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
   private static final int __START_ISSET_ID = 0;
   private static final int __HIGHTESTTXNID_ISSET_ID = 1;
   private static final int __ENDTIME_ISSET_ID = 2;
+  private static final int __ID_ISSET_ID = 3;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.PARTITIONNAME,_Fields.WORKERID,_Fields.START,_Fields.RUN_AS,_Fields.HIGHTEST_TXN_ID,_Fields.META_INFO,_Fields.END_TIME,_Fields.HADOOP_JOB_ID};
+  private static final _Fields optionals[] = {_Fields.PARTITIONNAME,_Fields.WORKERID,_Fields.START,_Fields.RUN_AS,_Fields.HIGHTEST_TXN_ID,_Fields.META_INFO,_Fields.END_TIME,_Fields.HADOOP_JOB_ID,_Fields.ID};
   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);
@@ -198,6 +204,8 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.HADOOP_JOB_ID, new org.apache.thrift.meta_data.FieldMetaData("hadoopJobId", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ShowCompactResponseElement.class, metaDataMap);
   }
@@ -255,6 +263,7 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
     if (other.isSetHadoopJobId()) {
       this.hadoopJobId = other.hadoopJobId;
     }
+    this.id = other.id;
   }
 
   public ShowCompactResponseElement deepCopy() {
@@ -279,6 +288,8 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
     this.endTime = 0;
     this.hadoopJobId = "None";
 
+    setIdIsSet(false);
+    this.id = 0;
   }
 
   public String getDbname() {
@@ -562,6 +573,28 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
     }
   }
 
+  public long getId() {
+    return this.id;
+  }
+
+  public void setId(long id) {
+    this.id = id;
+    setIdIsSet(true);
+  }
+
+  public void unsetId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ID_ISSET_ID);
+  }
+
+  /** Returns true if field id is set (has been assigned a value) and false otherwise */
+  public boolean isSetId() {
+    return EncodingUtils.testBit(__isset_bitfield, __ID_ISSET_ID);
+  }
+
+  public void setIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ID_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case DBNAME:
@@ -660,6 +693,14 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
       }
       break;
 
+    case ID:
+      if (value == null) {
+        unsetId();
+      } else {
+        setId((Long)value);
+      }
+      break;
+
     }
   }
 
@@ -701,6 +742,9 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
     case HADOOP_JOB_ID:
       return getHadoopJobId();
 
+    case ID:
+      return getId();
+
     }
     throw new IllegalStateException();
   }
@@ -736,6 +780,8 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
       return isSetEndTime();
     case HADOOP_JOB_ID:
       return isSetHadoopJobId();
+    case ID:
+      return isSetId();
     }
     throw new IllegalStateException();
   }
@@ -861,6 +907,15 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
         return false;
     }
 
+    boolean this_present_id = true && this.isSetId();
+    boolean that_present_id = true && that.isSetId();
+    if (this_present_id || that_present_id) {
+      if (!(this_present_id && that_present_id))
+        return false;
+      if (this.id != that.id)
+        return false;
+    }
+
     return true;
   }
 
@@ -928,6 +983,11 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
     if (present_hadoopJobId)
       list.add(hadoopJobId);
 
+    boolean present_id = true && (isSetId());
+    list.add(present_id);
+    if (present_id)
+      list.add(id);
+
     return list.hashCode();
   }
 
@@ -1059,6 +1119,16 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetId()).compareTo(other.isSetId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1178,6 +1248,12 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
       }
       first = false;
     }
+    if (isSetId()) {
+      if (!first) sb.append(", ");
+      sb.append("id:");
+      sb.append(this.id);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1335,6 +1411,14 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 13: // ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.id = iprot.readI64();
+              struct.setIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1418,6 +1502,11 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
           oprot.writeFieldEnd();
         }
       }
+      if (struct.isSetId()) {
+        oprot.writeFieldBegin(ID_FIELD_DESC);
+        oprot.writeI64(struct.id);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1464,7 +1553,10 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
       if (struct.isSetHadoopJobId()) {
         optionals.set(7);
       }
-      oprot.writeBitSet(optionals, 8);
+      if (struct.isSetId()) {
+        optionals.set(8);
+      }
+      oprot.writeBitSet(optionals, 9);
       if (struct.isSetPartitionname()) {
         oprot.writeString(struct.partitionname);
       }
@@ -1489,6 +1581,9 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
       if (struct.isSetHadoopJobId()) {
         oprot.writeString(struct.hadoopJobId);
       }
+      if (struct.isSetId()) {
+        oprot.writeI64(struct.id);
+      }
     }
 
     @Override
@@ -1502,7 +1597,7 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
       struct.setTypeIsSet(true);
       struct.state = iprot.readString();
       struct.setStateIsSet(true);
-      BitSet incoming = iprot.readBitSet(8);
+      BitSet incoming = iprot.readBitSet(9);
       if (incoming.get(0)) {
         struct.partitionname = iprot.readString();
         struct.setPartitionnameIsSet(true);
@@ -1535,6 +1630,10 @@ public class ShowCompactResponseElement implements org.apache.thrift.TBase<ShowC
         struct.hadoopJobId = iprot.readString();
         struct.setHadoopJobIdIsSet(true);
       }
+      if (incoming.get(8)) {
+        struct.id = iprot.readI64();
+        struct.setIdIsSet(true);
+      }
     }
   }
 


[6/8] hive git commit: HIVE-15202 Concurrent compactions for the same partition may generate malformed folder structure (Eugene Koifman, reviewed by Wei Zheng)

Posted by ek...@apache.org.
HIVE-15202 Concurrent compactions for the same partition may generate malformed folder structure (Eugene Koifman, reviewed by Wei Zheng)


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

Branch: refs/heads/master
Commit: 7b24ff7470d8bbda43b0dcf68c05afebaeb9bfe5
Parents: 76b311f
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Wed Nov 30 16:23:15 2016 -0800
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Wed Nov 30 16:23:15 2016 -0800

----------------------------------------------------------------------
 metastore/if/hive_metastore.thrift              |    8 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2437 ++++++++++--------
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  126 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 1186 +++++----
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   64 +-
 .../hive/metastore/api/CompactionResponse.java  |  583 +++++
 .../api/ShowCompactResponseElement.java         |  107 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 1236 +++++++--
 .../gen-php/metastore/ThriftHiveMetastore.php   |  216 ++
 .../src/gen/thrift/gen-php/metastore/Types.php  |  144 ++
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       |  189 ++
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  112 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   27 +-
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   54 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |    9 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   11 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |   13 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |   63 +-
 .../hadoop/hive/metastore/txn/TxnStore.java     |    7 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   12 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   19 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java |    6 +-
 .../hive/metastore/txn/TestTxnHandler.java      |   33 +
 25 files changed, 4901 insertions(+), 1773 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index 4e92f54..baab31b 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -750,6 +750,12 @@ struct CompactionRequest {
     6: optional map<string, string> properties
 }
 
+struct CompactionResponse {
+    1: required i64 id,
+    2: required string state,
+    3: required bool accepted
+}
+
 struct ShowCompactRequest {
 }
 
@@ -766,6 +772,7 @@ struct ShowCompactResponseElement {
     10: optional string metaInfo,
     11: optional i64 endTime,
     12: optional string hadoopJobId = "None",
+    13: optional i64 id,
 }
 
 struct ShowCompactResponse {
@@ -1460,6 +1467,7 @@ service ThriftHiveMetastore extends fb303.FacebookService
   void heartbeat(1:HeartbeatRequest ids) throws (1:NoSuchLockException o1, 2:NoSuchTxnException o2, 3:TxnAbortedException o3)
   HeartbeatTxnRangeResponse heartbeat_txn_range(1:HeartbeatTxnRangeRequest txns)
   void compact(1:CompactionRequest rqst) 
+  CompactionResponse compact2(1:CompactionRequest rqst) 
   ShowCompactResponse show_compact(1:ShowCompactRequest rqst)
   void add_dynamic_partitions(1:AddDynamicPartitions rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2)
 


[4/8] hive git commit: HIVE-15202 Concurrent compactions for the same partition may generate malformed folder structure (Eugene Koifman, reviewed by Wei Zheng)

Posted by ek...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index 1e1acaa..fca5ae5 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -163,6 +163,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void heartbeat(const HeartbeatRequest& ids) = 0;
   virtual void heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const HeartbeatTxnRangeRequest& txns) = 0;
   virtual void compact(const CompactionRequest& rqst) = 0;
+  virtual void compact2(CompactionResponse& _return, const CompactionRequest& rqst) = 0;
   virtual void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) = 0;
   virtual void add_dynamic_partitions(const AddDynamicPartitions& rqst) = 0;
   virtual void get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst) = 0;
@@ -654,6 +655,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void compact(const CompactionRequest& /* rqst */) {
     return;
   }
+  void compact2(CompactionResponse& /* _return */, const CompactionRequest& /* rqst */) {
+    return;
+  }
   void show_compact(ShowCompactResponse& /* _return */, const ShowCompactRequest& /* rqst */) {
     return;
   }
@@ -18383,6 +18387,110 @@ class ThriftHiveMetastore_compact_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_compact2_args__isset {
+  _ThriftHiveMetastore_compact2_args__isset() : rqst(false) {}
+  bool rqst :1;
+} _ThriftHiveMetastore_compact2_args__isset;
+
+class ThriftHiveMetastore_compact2_args {
+ public:
+
+  ThriftHiveMetastore_compact2_args(const ThriftHiveMetastore_compact2_args&);
+  ThriftHiveMetastore_compact2_args& operator=(const ThriftHiveMetastore_compact2_args&);
+  ThriftHiveMetastore_compact2_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_compact2_args() throw();
+  CompactionRequest rqst;
+
+  _ThriftHiveMetastore_compact2_args__isset __isset;
+
+  void __set_rqst(const CompactionRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_compact2_args & rhs) const
+  {
+    if (!(rqst == rhs.rqst))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_compact2_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_compact2_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_compact2_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_compact2_pargs() throw();
+  const CompactionRequest* rqst;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_compact2_result__isset {
+  _ThriftHiveMetastore_compact2_result__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_compact2_result__isset;
+
+class ThriftHiveMetastore_compact2_result {
+ public:
+
+  ThriftHiveMetastore_compact2_result(const ThriftHiveMetastore_compact2_result&);
+  ThriftHiveMetastore_compact2_result& operator=(const ThriftHiveMetastore_compact2_result&);
+  ThriftHiveMetastore_compact2_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_compact2_result() throw();
+  CompactionResponse success;
+
+  _ThriftHiveMetastore_compact2_result__isset __isset;
+
+  void __set_success(const CompactionResponse& val);
+
+  bool operator == (const ThriftHiveMetastore_compact2_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_compact2_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_compact2_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_compact2_presult__isset {
+  _ThriftHiveMetastore_compact2_presult__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_compact2_presult__isset;
+
+class ThriftHiveMetastore_compact2_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_compact2_presult() throw();
+  CompactionResponse* success;
+
+  _ThriftHiveMetastore_compact2_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_show_compact_args__isset {
   _ThriftHiveMetastore_show_compact_args__isset() : rqst(false) {}
   bool rqst :1;
@@ -19927,6 +20035,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void compact(const CompactionRequest& rqst);
   void send_compact(const CompactionRequest& rqst);
   void recv_compact();
+  void compact2(CompactionResponse& _return, const CompactionRequest& rqst);
+  void send_compact2(const CompactionRequest& rqst);
+  void recv_compact2(CompactionResponse& _return);
   void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst);
   void send_show_compact(const ShowCompactRequest& rqst);
   void recv_show_compact(ShowCompactResponse& _return);
@@ -20111,6 +20222,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_heartbeat(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_heartbeat_txn_range(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_compact(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_compact2(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_show_compact(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_add_dynamic_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_next_notification(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -20267,6 +20379,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["heartbeat"] = &ThriftHiveMetastoreProcessor::process_heartbeat;
     processMap_["heartbeat_txn_range"] = &ThriftHiveMetastoreProcessor::process_heartbeat_txn_range;
     processMap_["compact"] = &ThriftHiveMetastoreProcessor::process_compact;
+    processMap_["compact2"] = &ThriftHiveMetastoreProcessor::process_compact2;
     processMap_["show_compact"] = &ThriftHiveMetastoreProcessor::process_show_compact;
     processMap_["add_dynamic_partitions"] = &ThriftHiveMetastoreProcessor::process_add_dynamic_partitions;
     processMap_["get_next_notification"] = &ThriftHiveMetastoreProcessor::process_get_next_notification;
@@ -21660,6 +21773,16 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     ifaces_[i]->compact(rqst);
   }
 
+  void compact2(CompactionResponse& _return, const CompactionRequest& rqst) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->compact2(_return, rqst);
+    }
+    ifaces_[i]->compact2(_return, rqst);
+    return;
+  }
+
   void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -22207,6 +22330,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void compact(const CompactionRequest& rqst);
   int32_t send_compact(const CompactionRequest& rqst);
   void recv_compact(const int32_t seqid);
+  void compact2(CompactionResponse& _return, const CompactionRequest& rqst);
+  int32_t send_compact2(const CompactionRequest& rqst);
+  void recv_compact2(CompactionResponse& _return, const int32_t seqid);
   void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst);
   int32_t send_show_compact(const ShowCompactRequest& rqst);
   void recv_show_compact(ShowCompactResponse& _return, const int32_t seqid);

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index 7a3fa4b..dfa462d 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -727,6 +727,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("compact\n");
   }
 
+  void compact2(CompactionResponse& _return, const CompactionRequest& rqst) {
+    // Your implementation goes here
+    printf("compact2\n");
+  }
+
   void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) {
     // Your implementation goes here
     printf("show_compact\n");

http://git-wip-us.apache.org/repos/asf/hive/blob/7b24ff74/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 974ddb5..1fae3bc 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -14713,6 +14713,138 @@ void CompactionRequest::printTo(std::ostream& out) const {
 }
 
 
+CompactionResponse::~CompactionResponse() throw() {
+}
+
+
+void CompactionResponse::__set_id(const int64_t val) {
+  this->id = val;
+}
+
+void CompactionResponse::__set_state(const std::string& val) {
+  this->state = val;
+}
+
+void CompactionResponse::__set_accepted(const bool val) {
+  this->accepted = val;
+}
+
+uint32_t CompactionResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_id = false;
+  bool isset_state = false;
+  bool isset_accepted = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->id);
+          isset_id = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->state);
+          isset_state = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->accepted);
+          isset_accepted = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_id)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_state)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_accepted)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t CompactionResponse::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("CompactionResponse");
+
+  xfer += oprot->writeFieldBegin("id", ::apache::thrift::protocol::T_I64, 1);
+  xfer += oprot->writeI64(this->id);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("state", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->state);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("accepted", ::apache::thrift::protocol::T_BOOL, 3);
+  xfer += oprot->writeBool(this->accepted);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(CompactionResponse &a, CompactionResponse &b) {
+  using ::std::swap;
+  swap(a.id, b.id);
+  swap(a.state, b.state);
+  swap(a.accepted, b.accepted);
+}
+
+CompactionResponse::CompactionResponse(const CompactionResponse& other613) {
+  id = other613.id;
+  state = other613.state;
+  accepted = other613.accepted;
+}
+CompactionResponse& CompactionResponse::operator=(const CompactionResponse& other614) {
+  id = other614.id;
+  state = other614.state;
+  accepted = other614.accepted;
+  return *this;
+}
+void CompactionResponse::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "CompactionResponse(";
+  out << "id=" << to_string(id);
+  out << ", " << "state=" << to_string(state);
+  out << ", " << "accepted=" << to_string(accepted);
+  out << ")";
+}
+
+
 ShowCompactRequest::~ShowCompactRequest() throw() {
 }
 
@@ -14761,11 +14893,11 @@ void swap(ShowCompactRequest &a, ShowCompactRequest &b) {
   (void) b;
 }
 
-ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other613) {
-  (void) other613;
+ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other615) {
+  (void) other615;
 }
-ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other614) {
-  (void) other614;
+ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other616) {
+  (void) other616;
   return *this;
 }
 void ShowCompactRequest::printTo(std::ostream& out) const {
@@ -14835,6 +14967,11 @@ void ShowCompactResponseElement::__set_hadoopJobId(const std::string& val) {
 __isset.hadoopJobId = true;
 }
 
+void ShowCompactResponseElement::__set_id(const int64_t val) {
+  this->id = val;
+__isset.id = true;
+}
+
 uint32_t ShowCompactResponseElement::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -14886,9 +15023,9 @@ uint32_t ShowCompactResponseElement::read(::apache::thrift::protocol::TProtocol*
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast615;
-          xfer += iprot->readI32(ecast615);
-          this->type = (CompactionType::type)ecast615;
+          int32_t ecast617;
+          xfer += iprot->readI32(ecast617);
+          this->type = (CompactionType::type)ecast617;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -14958,6 +15095,14 @@ uint32_t ShowCompactResponseElement::read(::apache::thrift::protocol::TProtocol*
           xfer += iprot->skip(ftype);
         }
         break;
+      case 13:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->id);
+          this->__isset.id = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -15039,6 +15184,11 @@ uint32_t ShowCompactResponseElement::write(::apache::thrift::protocol::TProtocol
     xfer += oprot->writeString(this->hadoopJobId);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.id) {
+    xfer += oprot->writeFieldBegin("id", ::apache::thrift::protocol::T_I64, 13);
+    xfer += oprot->writeI64(this->id);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -15058,38 +15208,41 @@ void swap(ShowCompactResponseElement &a, ShowCompactResponseElement &b) {
   swap(a.metaInfo, b.metaInfo);
   swap(a.endTime, b.endTime);
   swap(a.hadoopJobId, b.hadoopJobId);
+  swap(a.id, b.id);
   swap(a.__isset, b.__isset);
 }
 
-ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other616) {
-  dbname = other616.dbname;
-  tablename = other616.tablename;
-  partitionname = other616.partitionname;
-  type = other616.type;
-  state = other616.state;
-  workerid = other616.workerid;
-  start = other616.start;
-  runAs = other616.runAs;
-  hightestTxnId = other616.hightestTxnId;
-  metaInfo = other616.metaInfo;
-  endTime = other616.endTime;
-  hadoopJobId = other616.hadoopJobId;
-  __isset = other616.__isset;
-}
-ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other617) {
-  dbname = other617.dbname;
-  tablename = other617.tablename;
-  partitionname = other617.partitionname;
-  type = other617.type;
-  state = other617.state;
-  workerid = other617.workerid;
-  start = other617.start;
-  runAs = other617.runAs;
-  hightestTxnId = other617.hightestTxnId;
-  metaInfo = other617.metaInfo;
-  endTime = other617.endTime;
-  hadoopJobId = other617.hadoopJobId;
-  __isset = other617.__isset;
+ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other618) {
+  dbname = other618.dbname;
+  tablename = other618.tablename;
+  partitionname = other618.partitionname;
+  type = other618.type;
+  state = other618.state;
+  workerid = other618.workerid;
+  start = other618.start;
+  runAs = other618.runAs;
+  hightestTxnId = other618.hightestTxnId;
+  metaInfo = other618.metaInfo;
+  endTime = other618.endTime;
+  hadoopJobId = other618.hadoopJobId;
+  id = other618.id;
+  __isset = other618.__isset;
+}
+ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other619) {
+  dbname = other619.dbname;
+  tablename = other619.tablename;
+  partitionname = other619.partitionname;
+  type = other619.type;
+  state = other619.state;
+  workerid = other619.workerid;
+  start = other619.start;
+  runAs = other619.runAs;
+  hightestTxnId = other619.hightestTxnId;
+  metaInfo = other619.metaInfo;
+  endTime = other619.endTime;
+  hadoopJobId = other619.hadoopJobId;
+  id = other619.id;
+  __isset = other619.__isset;
   return *this;
 }
 void ShowCompactResponseElement::printTo(std::ostream& out) const {
@@ -15107,6 +15260,7 @@ void ShowCompactResponseElement::printTo(std::ostream& out) const {
   out << ", " << "metaInfo="; (__isset.metaInfo ? (out << to_string(metaInfo)) : (out << "<null>"));
   out << ", " << "endTime="; (__isset.endTime ? (out << to_string(endTime)) : (out << "<null>"));
   out << ", " << "hadoopJobId="; (__isset.hadoopJobId ? (out << to_string(hadoopJobId)) : (out << "<null>"));
+  out << ", " << "id="; (__isset.id ? (out << to_string(id)) : (out << "<null>"));
   out << ")";
 }
 
@@ -15145,14 +15299,14 @@ uint32_t ShowCompactResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->compacts.clear();
-            uint32_t _size618;
-            ::apache::thrift::protocol::TType _etype621;
-            xfer += iprot->readListBegin(_etype621, _size618);
-            this->compacts.resize(_size618);
-            uint32_t _i622;
-            for (_i622 = 0; _i622 < _size618; ++_i622)
+            uint32_t _size620;
+            ::apache::thrift::protocol::TType _etype623;
+            xfer += iprot->readListBegin(_etype623, _size620);
+            this->compacts.resize(_size620);
+            uint32_t _i624;
+            for (_i624 = 0; _i624 < _size620; ++_i624)
             {
-              xfer += this->compacts[_i622].read(iprot);
+              xfer += this->compacts[_i624].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15183,10 +15337,10 @@ uint32_t ShowCompactResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("compacts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->compacts.size()));
-    std::vector<ShowCompactResponseElement> ::const_iterator _iter623;
-    for (_iter623 = this->compacts.begin(); _iter623 != this->compacts.end(); ++_iter623)
+    std::vector<ShowCompactResponseElement> ::const_iterator _iter625;
+    for (_iter625 = this->compacts.begin(); _iter625 != this->compacts.end(); ++_iter625)
     {
-      xfer += (*_iter623).write(oprot);
+      xfer += (*_iter625).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -15202,11 +15356,11 @@ void swap(ShowCompactResponse &a, ShowCompactResponse &b) {
   swap(a.compacts, b.compacts);
 }
 
-ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other624) {
-  compacts = other624.compacts;
+ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other626) {
+  compacts = other626.compacts;
 }
-ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other625) {
-  compacts = other625.compacts;
+ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other627) {
+  compacts = other627.compacts;
   return *this;
 }
 void ShowCompactResponse::printTo(std::ostream& out) const {
@@ -15295,14 +15449,14 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionnames.clear();
-            uint32_t _size626;
-            ::apache::thrift::protocol::TType _etype629;
-            xfer += iprot->readListBegin(_etype629, _size626);
-            this->partitionnames.resize(_size626);
-            uint32_t _i630;
-            for (_i630 = 0; _i630 < _size626; ++_i630)
+            uint32_t _size628;
+            ::apache::thrift::protocol::TType _etype631;
+            xfer += iprot->readListBegin(_etype631, _size628);
+            this->partitionnames.resize(_size628);
+            uint32_t _i632;
+            for (_i632 = 0; _i632 < _size628; ++_i632)
             {
-              xfer += iprot->readString(this->partitionnames[_i630]);
+              xfer += iprot->readString(this->partitionnames[_i632]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15313,9 +15467,9 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast631;
-          xfer += iprot->readI32(ecast631);
-          this->operationType = (DataOperationType::type)ecast631;
+          int32_t ecast633;
+          xfer += iprot->readI32(ecast633);
+          this->operationType = (DataOperationType::type)ecast633;
           this->__isset.operationType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -15361,10 +15515,10 @@ uint32_t AddDynamicPartitions::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("partitionnames", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionnames.size()));
-    std::vector<std::string> ::const_iterator _iter632;
-    for (_iter632 = this->partitionnames.begin(); _iter632 != this->partitionnames.end(); ++_iter632)
+    std::vector<std::string> ::const_iterator _iter634;
+    for (_iter634 = this->partitionnames.begin(); _iter634 != this->partitionnames.end(); ++_iter634)
     {
-      xfer += oprot->writeString((*_iter632));
+      xfer += oprot->writeString((*_iter634));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15390,21 +15544,21 @@ void swap(AddDynamicPartitions &a, AddDynamicPartitions &b) {
   swap(a.__isset, b.__isset);
 }
 
-AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other633) {
-  txnid = other633.txnid;
-  dbname = other633.dbname;
-  tablename = other633.tablename;
-  partitionnames = other633.partitionnames;
-  operationType = other633.operationType;
-  __isset = other633.__isset;
-}
-AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other634) {
-  txnid = other634.txnid;
-  dbname = other634.dbname;
-  tablename = other634.tablename;
-  partitionnames = other634.partitionnames;
-  operationType = other634.operationType;
-  __isset = other634.__isset;
+AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other635) {
+  txnid = other635.txnid;
+  dbname = other635.dbname;
+  tablename = other635.tablename;
+  partitionnames = other635.partitionnames;
+  operationType = other635.operationType;
+  __isset = other635.__isset;
+}
+AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other636) {
+  txnid = other636.txnid;
+  dbname = other636.dbname;
+  tablename = other636.tablename;
+  partitionnames = other636.partitionnames;
+  operationType = other636.operationType;
+  __isset = other636.__isset;
   return *this;
 }
 void AddDynamicPartitions::printTo(std::ostream& out) const {
@@ -15510,15 +15664,15 @@ void swap(NotificationEventRequest &a, NotificationEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other635) {
-  lastEvent = other635.lastEvent;
-  maxEvents = other635.maxEvents;
-  __isset = other635.__isset;
+NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other637) {
+  lastEvent = other637.lastEvent;
+  maxEvents = other637.maxEvents;
+  __isset = other637.__isset;
 }
-NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other636) {
-  lastEvent = other636.lastEvent;
-  maxEvents = other636.maxEvents;
-  __isset = other636.__isset;
+NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other638) {
+  lastEvent = other638.lastEvent;
+  maxEvents = other638.maxEvents;
+  __isset = other638.__isset;
   return *this;
 }
 void NotificationEventRequest::printTo(std::ostream& out) const {
@@ -15700,23 +15854,23 @@ void swap(NotificationEvent &a, NotificationEvent &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEvent::NotificationEvent(const NotificationEvent& other637) {
-  eventId = other637.eventId;
-  eventTime = other637.eventTime;
-  eventType = other637.eventType;
-  dbName = other637.dbName;
-  tableName = other637.tableName;
-  message = other637.message;
-  __isset = other637.__isset;
-}
-NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other638) {
-  eventId = other638.eventId;
-  eventTime = other638.eventTime;
-  eventType = other638.eventType;
-  dbName = other638.dbName;
-  tableName = other638.tableName;
-  message = other638.message;
-  __isset = other638.__isset;
+NotificationEvent::NotificationEvent(const NotificationEvent& other639) {
+  eventId = other639.eventId;
+  eventTime = other639.eventTime;
+  eventType = other639.eventType;
+  dbName = other639.dbName;
+  tableName = other639.tableName;
+  message = other639.message;
+  __isset = other639.__isset;
+}
+NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other640) {
+  eventId = other640.eventId;
+  eventTime = other640.eventTime;
+  eventType = other640.eventType;
+  dbName = other640.dbName;
+  tableName = other640.tableName;
+  message = other640.message;
+  __isset = other640.__isset;
   return *this;
 }
 void NotificationEvent::printTo(std::ostream& out) const {
@@ -15766,14 +15920,14 @@ uint32_t NotificationEventResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->events.clear();
-            uint32_t _size639;
-            ::apache::thrift::protocol::TType _etype642;
-            xfer += iprot->readListBegin(_etype642, _size639);
-            this->events.resize(_size639);
-            uint32_t _i643;
-            for (_i643 = 0; _i643 < _size639; ++_i643)
+            uint32_t _size641;
+            ::apache::thrift::protocol::TType _etype644;
+            xfer += iprot->readListBegin(_etype644, _size641);
+            this->events.resize(_size641);
+            uint32_t _i645;
+            for (_i645 = 0; _i645 < _size641; ++_i645)
             {
-              xfer += this->events[_i643].read(iprot);
+              xfer += this->events[_i645].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15804,10 +15958,10 @@ uint32_t NotificationEventResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("events", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->events.size()));
-    std::vector<NotificationEvent> ::const_iterator _iter644;
-    for (_iter644 = this->events.begin(); _iter644 != this->events.end(); ++_iter644)
+    std::vector<NotificationEvent> ::const_iterator _iter646;
+    for (_iter646 = this->events.begin(); _iter646 != this->events.end(); ++_iter646)
     {
-      xfer += (*_iter644).write(oprot);
+      xfer += (*_iter646).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -15823,11 +15977,11 @@ void swap(NotificationEventResponse &a, NotificationEventResponse &b) {
   swap(a.events, b.events);
 }
 
-NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other645) {
-  events = other645.events;
+NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other647) {
+  events = other647.events;
 }
-NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other646) {
-  events = other646.events;
+NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other648) {
+  events = other648.events;
   return *this;
 }
 void NotificationEventResponse::printTo(std::ostream& out) const {
@@ -15909,11 +16063,11 @@ void swap(CurrentNotificationEventId &a, CurrentNotificationEventId &b) {
   swap(a.eventId, b.eventId);
 }
 
-CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other647) {
-  eventId = other647.eventId;
+CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other649) {
+  eventId = other649.eventId;
 }
-CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other648) {
-  eventId = other648.eventId;
+CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other650) {
+  eventId = other650.eventId;
   return *this;
 }
 void CurrentNotificationEventId::printTo(std::ostream& out) const {
@@ -15958,14 +16112,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->filesAdded.clear();
-            uint32_t _size649;
-            ::apache::thrift::protocol::TType _etype652;
-            xfer += iprot->readListBegin(_etype652, _size649);
-            this->filesAdded.resize(_size649);
-            uint32_t _i653;
-            for (_i653 = 0; _i653 < _size649; ++_i653)
+            uint32_t _size651;
+            ::apache::thrift::protocol::TType _etype654;
+            xfer += iprot->readListBegin(_etype654, _size651);
+            this->filesAdded.resize(_size651);
+            uint32_t _i655;
+            for (_i655 = 0; _i655 < _size651; ++_i655)
             {
-              xfer += iprot->readString(this->filesAdded[_i653]);
+              xfer += iprot->readString(this->filesAdded[_i655]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15996,10 +16150,10 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("filesAdded", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->filesAdded.size()));
-    std::vector<std::string> ::const_iterator _iter654;
-    for (_iter654 = this->filesAdded.begin(); _iter654 != this->filesAdded.end(); ++_iter654)
+    std::vector<std::string> ::const_iterator _iter656;
+    for (_iter656 = this->filesAdded.begin(); _iter656 != this->filesAdded.end(); ++_iter656)
     {
-      xfer += oprot->writeString((*_iter654));
+      xfer += oprot->writeString((*_iter656));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16015,11 +16169,11 @@ void swap(InsertEventRequestData &a, InsertEventRequestData &b) {
   swap(a.filesAdded, b.filesAdded);
 }
 
-InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other655) {
-  filesAdded = other655.filesAdded;
+InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other657) {
+  filesAdded = other657.filesAdded;
 }
-InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other656) {
-  filesAdded = other656.filesAdded;
+InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other658) {
+  filesAdded = other658.filesAdded;
   return *this;
 }
 void InsertEventRequestData::printTo(std::ostream& out) const {
@@ -16099,13 +16253,13 @@ void swap(FireEventRequestData &a, FireEventRequestData &b) {
   swap(a.__isset, b.__isset);
 }
 
-FireEventRequestData::FireEventRequestData(const FireEventRequestData& other657) {
-  insertData = other657.insertData;
-  __isset = other657.__isset;
+FireEventRequestData::FireEventRequestData(const FireEventRequestData& other659) {
+  insertData = other659.insertData;
+  __isset = other659.__isset;
 }
-FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other658) {
-  insertData = other658.insertData;
-  __isset = other658.__isset;
+FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other660) {
+  insertData = other660.insertData;
+  __isset = other660.__isset;
   return *this;
 }
 void FireEventRequestData::printTo(std::ostream& out) const {
@@ -16202,14 +16356,14 @@ uint32_t FireEventRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionVals.clear();
-            uint32_t _size659;
-            ::apache::thrift::protocol::TType _etype662;
-            xfer += iprot->readListBegin(_etype662, _size659);
-            this->partitionVals.resize(_size659);
-            uint32_t _i663;
-            for (_i663 = 0; _i663 < _size659; ++_i663)
+            uint32_t _size661;
+            ::apache::thrift::protocol::TType _etype664;
+            xfer += iprot->readListBegin(_etype664, _size661);
+            this->partitionVals.resize(_size661);
+            uint32_t _i665;
+            for (_i665 = 0; _i665 < _size661; ++_i665)
             {
-              xfer += iprot->readString(this->partitionVals[_i663]);
+              xfer += iprot->readString(this->partitionVals[_i665]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16261,10 +16415,10 @@ uint32_t FireEventRequest::write(::apache::thrift::protocol::TProtocol* oprot) c
     xfer += oprot->writeFieldBegin("partitionVals", ::apache::thrift::protocol::T_LIST, 5);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionVals.size()));
-      std::vector<std::string> ::const_iterator _iter664;
-      for (_iter664 = this->partitionVals.begin(); _iter664 != this->partitionVals.end(); ++_iter664)
+      std::vector<std::string> ::const_iterator _iter666;
+      for (_iter666 = this->partitionVals.begin(); _iter666 != this->partitionVals.end(); ++_iter666)
       {
-        xfer += oprot->writeString((*_iter664));
+        xfer += oprot->writeString((*_iter666));
       }
       xfer += oprot->writeListEnd();
     }
@@ -16285,21 +16439,21 @@ void swap(FireEventRequest &a, FireEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-FireEventRequest::FireEventRequest(const FireEventRequest& other665) {
-  successful = other665.successful;
-  data = other665.data;
-  dbName = other665.dbName;
-  tableName = other665.tableName;
-  partitionVals = other665.partitionVals;
-  __isset = other665.__isset;
-}
-FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other666) {
-  successful = other666.successful;
-  data = other666.data;
-  dbName = other666.dbName;
-  tableName = other666.tableName;
-  partitionVals = other666.partitionVals;
-  __isset = other666.__isset;
+FireEventRequest::FireEventRequest(const FireEventRequest& other667) {
+  successful = other667.successful;
+  data = other667.data;
+  dbName = other667.dbName;
+  tableName = other667.tableName;
+  partitionVals = other667.partitionVals;
+  __isset = other667.__isset;
+}
+FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other668) {
+  successful = other668.successful;
+  data = other668.data;
+  dbName = other668.dbName;
+  tableName = other668.tableName;
+  partitionVals = other668.partitionVals;
+  __isset = other668.__isset;
   return *this;
 }
 void FireEventRequest::printTo(std::ostream& out) const {
@@ -16362,11 +16516,11 @@ void swap(FireEventResponse &a, FireEventResponse &b) {
   (void) b;
 }
 
-FireEventResponse::FireEventResponse(const FireEventResponse& other667) {
-  (void) other667;
+FireEventResponse::FireEventResponse(const FireEventResponse& other669) {
+  (void) other669;
 }
-FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other668) {
-  (void) other668;
+FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other670) {
+  (void) other670;
   return *this;
 }
 void FireEventResponse::printTo(std::ostream& out) const {
@@ -16466,15 +16620,15 @@ void swap(MetadataPpdResult &a, MetadataPpdResult &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetadataPpdResult::MetadataPpdResult(const MetadataPpdResult& other669) {
-  metadata = other669.metadata;
-  includeBitset = other669.includeBitset;
-  __isset = other669.__isset;
+MetadataPpdResult::MetadataPpdResult(const MetadataPpdResult& other671) {
+  metadata = other671.metadata;
+  includeBitset = other671.includeBitset;
+  __isset = other671.__isset;
 }
-MetadataPpdResult& MetadataPpdResult::operator=(const MetadataPpdResult& other670) {
-  metadata = other670.metadata;
-  includeBitset = other670.includeBitset;
-  __isset = other670.__isset;
+MetadataPpdResult& MetadataPpdResult::operator=(const MetadataPpdResult& other672) {
+  metadata = other672.metadata;
+  includeBitset = other672.includeBitset;
+  __isset = other672.__isset;
   return *this;
 }
 void MetadataPpdResult::printTo(std::ostream& out) const {
@@ -16525,17 +16679,17 @@ uint32_t GetFileMetadataByExprResult::read(::apache::thrift::protocol::TProtocol
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->metadata.clear();
-            uint32_t _size671;
-            ::apache::thrift::protocol::TType _ktype672;
-            ::apache::thrift::protocol::TType _vtype673;
-            xfer += iprot->readMapBegin(_ktype672, _vtype673, _size671);
-            uint32_t _i675;
-            for (_i675 = 0; _i675 < _size671; ++_i675)
+            uint32_t _size673;
+            ::apache::thrift::protocol::TType _ktype674;
+            ::apache::thrift::protocol::TType _vtype675;
+            xfer += iprot->readMapBegin(_ktype674, _vtype675, _size673);
+            uint32_t _i677;
+            for (_i677 = 0; _i677 < _size673; ++_i677)
             {
-              int64_t _key676;
-              xfer += iprot->readI64(_key676);
-              MetadataPpdResult& _val677 = this->metadata[_key676];
-              xfer += _val677.read(iprot);
+              int64_t _key678;
+              xfer += iprot->readI64(_key678);
+              MetadataPpdResult& _val679 = this->metadata[_key678];
+              xfer += _val679.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -16576,11 +16730,11 @@ uint32_t GetFileMetadataByExprResult::write(::apache::thrift::protocol::TProtoco
   xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_I64, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->metadata.size()));
-    std::map<int64_t, MetadataPpdResult> ::const_iterator _iter678;
-    for (_iter678 = this->metadata.begin(); _iter678 != this->metadata.end(); ++_iter678)
+    std::map<int64_t, MetadataPpdResult> ::const_iterator _iter680;
+    for (_iter680 = this->metadata.begin(); _iter680 != this->metadata.end(); ++_iter680)
     {
-      xfer += oprot->writeI64(_iter678->first);
-      xfer += _iter678->second.write(oprot);
+      xfer += oprot->writeI64(_iter680->first);
+      xfer += _iter680->second.write(oprot);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -16601,13 +16755,13 @@ void swap(GetFileMetadataByExprResult &a, GetFileMetadataByExprResult &b) {
   swap(a.isSupported, b.isSupported);
 }
 
-GetFileMetadataByExprResult::GetFileMetadataByExprResult(const GetFileMetadataByExprResult& other679) {
-  metadata = other679.metadata;
-  isSupported = other679.isSupported;
+GetFileMetadataByExprResult::GetFileMetadataByExprResult(const GetFileMetadataByExprResult& other681) {
+  metadata = other681.metadata;
+  isSupported = other681.isSupported;
 }
-GetFileMetadataByExprResult& GetFileMetadataByExprResult::operator=(const GetFileMetadataByExprResult& other680) {
-  metadata = other680.metadata;
-  isSupported = other680.isSupported;
+GetFileMetadataByExprResult& GetFileMetadataByExprResult::operator=(const GetFileMetadataByExprResult& other682) {
+  metadata = other682.metadata;
+  isSupported = other682.isSupported;
   return *this;
 }
 void GetFileMetadataByExprResult::printTo(std::ostream& out) const {
@@ -16668,14 +16822,14 @@ uint32_t GetFileMetadataByExprRequest::read(::apache::thrift::protocol::TProtoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size681;
-            ::apache::thrift::protocol::TType _etype684;
-            xfer += iprot->readListBegin(_etype684, _size681);
-            this->fileIds.resize(_size681);
-            uint32_t _i685;
-            for (_i685 = 0; _i685 < _size681; ++_i685)
+            uint32_t _size683;
+            ::apache::thrift::protocol::TType _etype686;
+            xfer += iprot->readListBegin(_etype686, _size683);
+            this->fileIds.resize(_size683);
+            uint32_t _i687;
+            for (_i687 = 0; _i687 < _size683; ++_i687)
             {
-              xfer += iprot->readI64(this->fileIds[_i685]);
+              xfer += iprot->readI64(this->fileIds[_i687]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16702,9 +16856,9 @@ uint32_t GetFileMetadataByExprRequest::read(::apache::thrift::protocol::TProtoco
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast686;
-          xfer += iprot->readI32(ecast686);
-          this->type = (FileMetadataExprType::type)ecast686;
+          int32_t ecast688;
+          xfer += iprot->readI32(ecast688);
+          this->type = (FileMetadataExprType::type)ecast688;
           this->__isset.type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -16734,10 +16888,10 @@ uint32_t GetFileMetadataByExprRequest::write(::apache::thrift::protocol::TProtoc
   xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
-    std::vector<int64_t> ::const_iterator _iter687;
-    for (_iter687 = this->fileIds.begin(); _iter687 != this->fileIds.end(); ++_iter687)
+    std::vector<int64_t> ::const_iterator _iter689;
+    for (_iter689 = this->fileIds.begin(); _iter689 != this->fileIds.end(); ++_iter689)
     {
-      xfer += oprot->writeI64((*_iter687));
+      xfer += oprot->writeI64((*_iter689));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16771,19 +16925,19 @@ void swap(GetFileMetadataByExprRequest &a, GetFileMetadataByExprRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetFileMetadataByExprRequest::GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest& other688) {
-  fileIds = other688.fileIds;
-  expr = other688.expr;
-  doGetFooters = other688.doGetFooters;
-  type = other688.type;
-  __isset = other688.__isset;
+GetFileMetadataByExprRequest::GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest& other690) {
+  fileIds = other690.fileIds;
+  expr = other690.expr;
+  doGetFooters = other690.doGetFooters;
+  type = other690.type;
+  __isset = other690.__isset;
 }
-GetFileMetadataByExprRequest& GetFileMetadataByExprRequest::operator=(const GetFileMetadataByExprRequest& other689) {
-  fileIds = other689.fileIds;
-  expr = other689.expr;
-  doGetFooters = other689.doGetFooters;
-  type = other689.type;
-  __isset = other689.__isset;
+GetFileMetadataByExprRequest& GetFileMetadataByExprRequest::operator=(const GetFileMetadataByExprRequest& other691) {
+  fileIds = other691.fileIds;
+  expr = other691.expr;
+  doGetFooters = other691.doGetFooters;
+  type = other691.type;
+  __isset = other691.__isset;
   return *this;
 }
 void GetFileMetadataByExprRequest::printTo(std::ostream& out) const {
@@ -16836,17 +16990,17 @@ uint32_t GetFileMetadataResult::read(::apache::thrift::protocol::TProtocol* ipro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->metadata.clear();
-            uint32_t _size690;
-            ::apache::thrift::protocol::TType _ktype691;
-            ::apache::thrift::protocol::TType _vtype692;
-            xfer += iprot->readMapBegin(_ktype691, _vtype692, _size690);
-            uint32_t _i694;
-            for (_i694 = 0; _i694 < _size690; ++_i694)
+            uint32_t _size692;
+            ::apache::thrift::protocol::TType _ktype693;
+            ::apache::thrift::protocol::TType _vtype694;
+            xfer += iprot->readMapBegin(_ktype693, _vtype694, _size692);
+            uint32_t _i696;
+            for (_i696 = 0; _i696 < _size692; ++_i696)
             {
-              int64_t _key695;
-              xfer += iprot->readI64(_key695);
-              std::string& _val696 = this->metadata[_key695];
-              xfer += iprot->readBinary(_val696);
+              int64_t _key697;
+              xfer += iprot->readI64(_key697);
+              std::string& _val698 = this->metadata[_key697];
+              xfer += iprot->readBinary(_val698);
             }
             xfer += iprot->readMapEnd();
           }
@@ -16887,11 +17041,11 @@ uint32_t GetFileMetadataResult::write(::apache::thrift::protocol::TProtocol* opr
   xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_I64, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->metadata.size()));
-    std::map<int64_t, std::string> ::const_iterator _iter697;
-    for (_iter697 = this->metadata.begin(); _iter697 != this->metadata.end(); ++_iter697)
+    std::map<int64_t, std::string> ::const_iterator _iter699;
+    for (_iter699 = this->metadata.begin(); _iter699 != this->metadata.end(); ++_iter699)
     {
-      xfer += oprot->writeI64(_iter697->first);
-      xfer += oprot->writeBinary(_iter697->second);
+      xfer += oprot->writeI64(_iter699->first);
+      xfer += oprot->writeBinary(_iter699->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -16912,13 +17066,13 @@ void swap(GetFileMetadataResult &a, GetFileMetadataResult &b) {
   swap(a.isSupported, b.isSupported);
 }
 
-GetFileMetadataResult::GetFileMetadataResult(const GetFileMetadataResult& other698) {
-  metadata = other698.metadata;
-  isSupported = other698.isSupported;
+GetFileMetadataResult::GetFileMetadataResult(const GetFileMetadataResult& other700) {
+  metadata = other700.metadata;
+  isSupported = other700.isSupported;
 }
-GetFileMetadataResult& GetFileMetadataResult::operator=(const GetFileMetadataResult& other699) {
-  metadata = other699.metadata;
-  isSupported = other699.isSupported;
+GetFileMetadataResult& GetFileMetadataResult::operator=(const GetFileMetadataResult& other701) {
+  metadata = other701.metadata;
+  isSupported = other701.isSupported;
   return *this;
 }
 void GetFileMetadataResult::printTo(std::ostream& out) const {
@@ -16964,14 +17118,14 @@ uint32_t GetFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size700;
-            ::apache::thrift::protocol::TType _etype703;
-            xfer += iprot->readListBegin(_etype703, _size700);
-            this->fileIds.resize(_size700);
-            uint32_t _i704;
-            for (_i704 = 0; _i704 < _size700; ++_i704)
+            uint32_t _size702;
+            ::apache::thrift::protocol::TType _etype705;
+            xfer += iprot->readListBegin(_etype705, _size702);
+            this->fileIds.resize(_size702);
+            uint32_t _i706;
+            for (_i706 = 0; _i706 < _size702; ++_i706)
             {
-              xfer += iprot->readI64(this->fileIds[_i704]);
+              xfer += iprot->readI64(this->fileIds[_i706]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17002,10 +17156,10 @@ uint32_t GetFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
-    std::vector<int64_t> ::const_iterator _iter705;
-    for (_iter705 = this->fileIds.begin(); _iter705 != this->fileIds.end(); ++_iter705)
+    std::vector<int64_t> ::const_iterator _iter707;
+    for (_iter707 = this->fileIds.begin(); _iter707 != this->fileIds.end(); ++_iter707)
     {
-      xfer += oprot->writeI64((*_iter705));
+      xfer += oprot->writeI64((*_iter707));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17021,11 +17175,11 @@ void swap(GetFileMetadataRequest &a, GetFileMetadataRequest &b) {
   swap(a.fileIds, b.fileIds);
 }
 
-GetFileMetadataRequest::GetFileMetadataRequest(const GetFileMetadataRequest& other706) {
-  fileIds = other706.fileIds;
+GetFileMetadataRequest::GetFileMetadataRequest(const GetFileMetadataRequest& other708) {
+  fileIds = other708.fileIds;
 }
-GetFileMetadataRequest& GetFileMetadataRequest::operator=(const GetFileMetadataRequest& other707) {
-  fileIds = other707.fileIds;
+GetFileMetadataRequest& GetFileMetadataRequest::operator=(const GetFileMetadataRequest& other709) {
+  fileIds = other709.fileIds;
   return *this;
 }
 void GetFileMetadataRequest::printTo(std::ostream& out) const {
@@ -17084,11 +17238,11 @@ void swap(PutFileMetadataResult &a, PutFileMetadataResult &b) {
   (void) b;
 }
 
-PutFileMetadataResult::PutFileMetadataResult(const PutFileMetadataResult& other708) {
-  (void) other708;
+PutFileMetadataResult::PutFileMetadataResult(const PutFileMetadataResult& other710) {
+  (void) other710;
 }
-PutFileMetadataResult& PutFileMetadataResult::operator=(const PutFileMetadataResult& other709) {
-  (void) other709;
+PutFileMetadataResult& PutFileMetadataResult::operator=(const PutFileMetadataResult& other711) {
+  (void) other711;
   return *this;
 }
 void PutFileMetadataResult::printTo(std::ostream& out) const {
@@ -17142,14 +17296,14 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size710;
-            ::apache::thrift::protocol::TType _etype713;
-            xfer += iprot->readListBegin(_etype713, _size710);
-            this->fileIds.resize(_size710);
-            uint32_t _i714;
-            for (_i714 = 0; _i714 < _size710; ++_i714)
+            uint32_t _size712;
+            ::apache::thrift::protocol::TType _etype715;
+            xfer += iprot->readListBegin(_etype715, _size712);
+            this->fileIds.resize(_size712);
+            uint32_t _i716;
+            for (_i716 = 0; _i716 < _size712; ++_i716)
             {
-              xfer += iprot->readI64(this->fileIds[_i714]);
+              xfer += iprot->readI64(this->fileIds[_i716]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17162,14 +17316,14 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->metadata.clear();
-            uint32_t _size715;
-            ::apache::thrift::protocol::TType _etype718;
-            xfer += iprot->readListBegin(_etype718, _size715);
-            this->metadata.resize(_size715);
-            uint32_t _i719;
-            for (_i719 = 0; _i719 < _size715; ++_i719)
+            uint32_t _size717;
+            ::apache::thrift::protocol::TType _etype720;
+            xfer += iprot->readListBegin(_etype720, _size717);
+            this->metadata.resize(_size717);
+            uint32_t _i721;
+            for (_i721 = 0; _i721 < _size717; ++_i721)
             {
-              xfer += iprot->readBinary(this->metadata[_i719]);
+              xfer += iprot->readBinary(this->metadata[_i721]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17180,9 +17334,9 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast720;
-          xfer += iprot->readI32(ecast720);
-          this->type = (FileMetadataExprType::type)ecast720;
+          int32_t ecast722;
+          xfer += iprot->readI32(ecast722);
+          this->type = (FileMetadataExprType::type)ecast722;
           this->__isset.type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -17212,10 +17366,10 @@ uint32_t PutFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
-    std::vector<int64_t> ::const_iterator _iter721;
-    for (_iter721 = this->fileIds.begin(); _iter721 != this->fileIds.end(); ++_iter721)
+    std::vector<int64_t> ::const_iterator _iter723;
+    for (_iter723 = this->fileIds.begin(); _iter723 != this->fileIds.end(); ++_iter723)
     {
-      xfer += oprot->writeI64((*_iter721));
+      xfer += oprot->writeI64((*_iter723));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17224,10 +17378,10 @@ uint32_t PutFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->metadata.size()));
-    std::vector<std::string> ::const_iterator _iter722;
-    for (_iter722 = this->metadata.begin(); _iter722 != this->metadata.end(); ++_iter722)
+    std::vector<std::string> ::const_iterator _iter724;
+    for (_iter724 = this->metadata.begin(); _iter724 != this->metadata.end(); ++_iter724)
     {
-      xfer += oprot->writeBinary((*_iter722));
+      xfer += oprot->writeBinary((*_iter724));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17251,17 +17405,17 @@ void swap(PutFileMetadataRequest &a, PutFileMetadataRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-PutFileMetadataRequest::PutFileMetadataRequest(const PutFileMetadataRequest& other723) {
-  fileIds = other723.fileIds;
-  metadata = other723.metadata;
-  type = other723.type;
-  __isset = other723.__isset;
+PutFileMetadataRequest::PutFileMetadataRequest(const PutFileMetadataRequest& other725) {
+  fileIds = other725.fileIds;
+  metadata = other725.metadata;
+  type = other725.type;
+  __isset = other725.__isset;
 }
-PutFileMetadataRequest& PutFileMetadataRequest::operator=(const PutFileMetadataRequest& other724) {
-  fileIds = other724.fileIds;
-  metadata = other724.metadata;
-  type = other724.type;
-  __isset = other724.__isset;
+PutFileMetadataRequest& PutFileMetadataRequest::operator=(const PutFileMetadataRequest& other726) {
+  fileIds = other726.fileIds;
+  metadata = other726.metadata;
+  type = other726.type;
+  __isset = other726.__isset;
   return *this;
 }
 void PutFileMetadataRequest::printTo(std::ostream& out) const {
@@ -17322,11 +17476,11 @@ void swap(ClearFileMetadataResult &a, ClearFileMetadataResult &b) {
   (void) b;
 }
 
-ClearFileMetadataResult::ClearFileMetadataResult(const ClearFileMetadataResult& other725) {
-  (void) other725;
+ClearFileMetadataResult::ClearFileMetadataResult(const ClearFileMetadataResult& other727) {
+  (void) other727;
 }
-ClearFileMetadataResult& ClearFileMetadataResult::operator=(const ClearFileMetadataResult& other726) {
-  (void) other726;
+ClearFileMetadataResult& ClearFileMetadataResult::operator=(const ClearFileMetadataResult& other728) {
+  (void) other728;
   return *this;
 }
 void ClearFileMetadataResult::printTo(std::ostream& out) const {
@@ -17370,14 +17524,14 @@ uint32_t ClearFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* i
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size727;
-            ::apache::thrift::protocol::TType _etype730;
-            xfer += iprot->readListBegin(_etype730, _size727);
-            this->fileIds.resize(_size727);
-            uint32_t _i731;
-            for (_i731 = 0; _i731 < _size727; ++_i731)
+            uint32_t _size729;
+            ::apache::thrift::protocol::TType _etype732;
+            xfer += iprot->readListBegin(_etype732, _size729);
+            this->fileIds.resize(_size729);
+            uint32_t _i733;
+            for (_i733 = 0; _i733 < _size729; ++_i733)
             {
-              xfer += iprot->readI64(this->fileIds[_i731]);
+              xfer += iprot->readI64(this->fileIds[_i733]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17408,10 +17562,10 @@ uint32_t ClearFileMetadataRequest::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
-    std::vector<int64_t> ::const_iterator _iter732;
-    for (_iter732 = this->fileIds.begin(); _iter732 != this->fileIds.end(); ++_iter732)
+    std::vector<int64_t> ::const_iterator _iter734;
+    for (_iter734 = this->fileIds.begin(); _iter734 != this->fileIds.end(); ++_iter734)
     {
-      xfer += oprot->writeI64((*_iter732));
+      xfer += oprot->writeI64((*_iter734));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17427,11 +17581,11 @@ void swap(ClearFileMetadataRequest &a, ClearFileMetadataRequest &b) {
   swap(a.fileIds, b.fileIds);
 }
 
-ClearFileMetadataRequest::ClearFileMetadataRequest(const ClearFileMetadataRequest& other733) {
-  fileIds = other733.fileIds;
+ClearFileMetadataRequest::ClearFileMetadataRequest(const ClearFileMetadataRequest& other735) {
+  fileIds = other735.fileIds;
 }
-ClearFileMetadataRequest& ClearFileMetadataRequest::operator=(const ClearFileMetadataRequest& other734) {
-  fileIds = other734.fileIds;
+ClearFileMetadataRequest& ClearFileMetadataRequest::operator=(const ClearFileMetadataRequest& other736) {
+  fileIds = other736.fileIds;
   return *this;
 }
 void ClearFileMetadataRequest::printTo(std::ostream& out) const {
@@ -17513,11 +17667,11 @@ void swap(CacheFileMetadataResult &a, CacheFileMetadataResult &b) {
   swap(a.isSupported, b.isSupported);
 }
 
-CacheFileMetadataResult::CacheFileMetadataResult(const CacheFileMetadataResult& other735) {
-  isSupported = other735.isSupported;
+CacheFileMetadataResult::CacheFileMetadataResult(const CacheFileMetadataResult& other737) {
+  isSupported = other737.isSupported;
 }
-CacheFileMetadataResult& CacheFileMetadataResult::operator=(const CacheFileMetadataResult& other736) {
-  isSupported = other736.isSupported;
+CacheFileMetadataResult& CacheFileMetadataResult::operator=(const CacheFileMetadataResult& other738) {
+  isSupported = other738.isSupported;
   return *this;
 }
 void CacheFileMetadataResult::printTo(std::ostream& out) const {
@@ -17658,19 +17812,19 @@ void swap(CacheFileMetadataRequest &a, CacheFileMetadataRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CacheFileMetadataRequest::CacheFileMetadataRequest(const CacheFileMetadataRequest& other737) {
-  dbName = other737.dbName;
-  tblName = other737.tblName;
-  partName = other737.partName;
-  isAllParts = other737.isAllParts;
-  __isset = other737.__isset;
+CacheFileMetadataRequest::CacheFileMetadataRequest(const CacheFileMetadataRequest& other739) {
+  dbName = other739.dbName;
+  tblName = other739.tblName;
+  partName = other739.partName;
+  isAllParts = other739.isAllParts;
+  __isset = other739.__isset;
 }
-CacheFileMetadataRequest& CacheFileMetadataRequest::operator=(const CacheFileMetadataRequest& other738) {
-  dbName = other738.dbName;
-  tblName = other738.tblName;
-  partName = other738.partName;
-  isAllParts = other738.isAllParts;
-  __isset = other738.__isset;
+CacheFileMetadataRequest& CacheFileMetadataRequest::operator=(const CacheFileMetadataRequest& other740) {
+  dbName = other740.dbName;
+  tblName = other740.tblName;
+  partName = other740.partName;
+  isAllParts = other740.isAllParts;
+  __isset = other740.__isset;
   return *this;
 }
 void CacheFileMetadataRequest::printTo(std::ostream& out) const {
@@ -17718,14 +17872,14 @@ uint32_t GetAllFunctionsResponse::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->functions.clear();
-            uint32_t _size739;
-            ::apache::thrift::protocol::TType _etype742;
-            xfer += iprot->readListBegin(_etype742, _size739);
-            this->functions.resize(_size739);
-            uint32_t _i743;
-            for (_i743 = 0; _i743 < _size739; ++_i743)
+            uint32_t _size741;
+            ::apache::thrift::protocol::TType _etype744;
+            xfer += iprot->readListBegin(_etype744, _size741);
+            this->functions.resize(_size741);
+            uint32_t _i745;
+            for (_i745 = 0; _i745 < _size741; ++_i745)
             {
-              xfer += this->functions[_i743].read(iprot);
+              xfer += this->functions[_i745].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -17755,10 +17909,10 @@ uint32_t GetAllFunctionsResponse::write(::apache::thrift::protocol::TProtocol* o
     xfer += oprot->writeFieldBegin("functions", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->functions.size()));
-      std::vector<Function> ::const_iterator _iter744;
-      for (_iter744 = this->functions.begin(); _iter744 != this->functions.end(); ++_iter744)
+      std::vector<Function> ::const_iterator _iter746;
+      for (_iter746 = this->functions.begin(); _iter746 != this->functions.end(); ++_iter746)
       {
-        xfer += (*_iter744).write(oprot);
+        xfer += (*_iter746).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -17775,13 +17929,13 @@ void swap(GetAllFunctionsResponse &a, GetAllFunctionsResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other745) {
-  functions = other745.functions;
-  __isset = other745.__isset;
+GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other747) {
+  functions = other747.functions;
+  __isset = other747.__isset;
 }
-GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other746) {
-  functions = other746.functions;
-  __isset = other746.__isset;
+GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other748) {
+  functions = other748.functions;
+  __isset = other748.__isset;
   return *this;
 }
 void GetAllFunctionsResponse::printTo(std::ostream& out) const {
@@ -17826,16 +17980,16 @@ uint32_t ClientCapabilities::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->values.clear();
-            uint32_t _size747;
-            ::apache::thrift::protocol::TType _etype750;
-            xfer += iprot->readListBegin(_etype750, _size747);
-            this->values.resize(_size747);
-            uint32_t _i751;
-            for (_i751 = 0; _i751 < _size747; ++_i751)
+            uint32_t _size749;
+            ::apache::thrift::protocol::TType _etype752;
+            xfer += iprot->readListBegin(_etype752, _size749);
+            this->values.resize(_size749);
+            uint32_t _i753;
+            for (_i753 = 0; _i753 < _size749; ++_i753)
             {
-              int32_t ecast752;
-              xfer += iprot->readI32(ecast752);
-              this->values[_i751] = (ClientCapability::type)ecast752;
+              int32_t ecast754;
+              xfer += iprot->readI32(ecast754);
+              this->values[_i753] = (ClientCapability::type)ecast754;
             }
             xfer += iprot->readListEnd();
           }
@@ -17866,10 +18020,10 @@ uint32_t ClientCapabilities::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I32, static_cast<uint32_t>(this->values.size()));
-    std::vector<ClientCapability::type> ::const_iterator _iter753;
-    for (_iter753 = this->values.begin(); _iter753 != this->values.end(); ++_iter753)
+    std::vector<ClientCapability::type> ::const_iterator _iter755;
+    for (_iter755 = this->values.begin(); _iter755 != this->values.end(); ++_iter755)
     {
-      xfer += oprot->writeI32((int32_t)(*_iter753));
+      xfer += oprot->writeI32((int32_t)(*_iter755));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17885,11 +18039,11 @@ void swap(ClientCapabilities &a, ClientCapabilities &b) {
   swap(a.values, b.values);
 }
 
-ClientCapabilities::ClientCapabilities(const ClientCapabilities& other754) {
-  values = other754.values;
+ClientCapabilities::ClientCapabilities(const ClientCapabilities& other756) {
+  values = other756.values;
 }
-ClientCapabilities& ClientCapabilities::operator=(const ClientCapabilities& other755) {
-  values = other755.values;
+ClientCapabilities& ClientCapabilities::operator=(const ClientCapabilities& other757) {
+  values = other757.values;
   return *this;
 }
 void ClientCapabilities::printTo(std::ostream& out) const {
@@ -18011,17 +18165,17 @@ void swap(GetTableRequest &a, GetTableRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetTableRequest::GetTableRequest(const GetTableRequest& other756) {
-  dbName = other756.dbName;
-  tblName = other756.tblName;
-  capabilities = other756.capabilities;
-  __isset = other756.__isset;
+GetTableRequest::GetTableRequest(const GetTableRequest& other758) {
+  dbName = other758.dbName;
+  tblName = other758.tblName;
+  capabilities = other758.capabilities;
+  __isset = other758.__isset;
 }
-GetTableRequest& GetTableRequest::operator=(const GetTableRequest& other757) {
-  dbName = other757.dbName;
-  tblName = other757.tblName;
-  capabilities = other757.capabilities;
-  __isset = other757.__isset;
+GetTableRequest& GetTableRequest::operator=(const GetTableRequest& other759) {
+  dbName = other759.dbName;
+  tblName = other759.tblName;
+  capabilities = other759.capabilities;
+  __isset = other759.__isset;
   return *this;
 }
 void GetTableRequest::printTo(std::ostream& out) const {
@@ -18105,11 +18259,11 @@ void swap(GetTableResult &a, GetTableResult &b) {
   swap(a.table, b.table);
 }
 
-GetTableResult::GetTableResult(const GetTableResult& other758) {
-  table = other758.table;
+GetTableResult::GetTableResult(const GetTableResult& other760) {
+  table = other760.table;
 }
-GetTableResult& GetTableResult::operator=(const GetTableResult& other759) {
-  table = other759.table;
+GetTableResult& GetTableResult::operator=(const GetTableResult& other761) {
+  table = other761.table;
   return *this;
 }
 void GetTableResult::printTo(std::ostream& out) const {
@@ -18172,14 +18326,14 @@ uint32_t GetTablesRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tblNames.clear();
-            uint32_t _size760;
-            ::apache::thrift::protocol::TType _etype763;
-            xfer += iprot->readListBegin(_etype763, _size760);
-            this->tblNames.resize(_size760);
-            uint32_t _i764;
-            for (_i764 = 0; _i764 < _size760; ++_i764)
+            uint32_t _size762;
+            ::apache::thrift::protocol::TType _etype765;
+            xfer += iprot->readListBegin(_etype765, _size762);
+            this->tblNames.resize(_size762);
+            uint32_t _i766;
+            for (_i766 = 0; _i766 < _size762; ++_i766)
             {
-              xfer += iprot->readString(this->tblNames[_i764]);
+              xfer += iprot->readString(this->tblNames[_i766]);
             }
             xfer += iprot->readListEnd();
           }
@@ -18223,10 +18377,10 @@ uint32_t GetTablesRequest::write(::apache::thrift::protocol::TProtocol* oprot) c
     xfer += oprot->writeFieldBegin("tblNames", ::apache::thrift::protocol::T_LIST, 2);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tblNames.size()));
-      std::vector<std::string> ::const_iterator _iter765;
-      for (_iter765 = this->tblNames.begin(); _iter765 != this->tblNames.end(); ++_iter765)
+      std::vector<std::string> ::const_iterator _iter767;
+      for (_iter767 = this->tblNames.begin(); _iter767 != this->tblNames.end(); ++_iter767)
       {
-        xfer += oprot->writeString((*_iter765));
+        xfer += oprot->writeString((*_iter767));
       }
       xfer += oprot->writeListEnd();
     }
@@ -18250,17 +18404,17 @@ void swap(GetTablesRequest &a, GetTablesRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetTablesRequest::GetTablesRequest(const GetTablesRequest& other766) {
-  dbName = other766.dbName;
-  tblNames = other766.tblNames;
-  capabilities = other766.capabilities;
-  __isset = other766.__isset;
+GetTablesRequest::GetTablesRequest(const GetTablesRequest& other768) {
+  dbName = other768.dbName;
+  tblNames = other768.tblNames;
+  capabilities = other768.capabilities;
+  __isset = other768.__isset;
 }
-GetTablesRequest& GetTablesRequest::operator=(const GetTablesRequest& other767) {
-  dbName = other767.dbName;
-  tblNames = other767.tblNames;
-  capabilities = other767.capabilities;
-  __isset = other767.__isset;
+GetTablesRequest& GetTablesRequest::operator=(const GetTablesRequest& other769) {
+  dbName = other769.dbName;
+  tblNames = other769.tblNames;
+  capabilities = other769.capabilities;
+  __isset = other769.__isset;
   return *this;
 }
 void GetTablesRequest::printTo(std::ostream& out) const {
@@ -18307,14 +18461,14 @@ uint32_t GetTablesResult::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tables.clear();
-            uint32_t _size768;
-            ::apache::thrift::protocol::TType _etype771;
-            xfer += iprot->readListBegin(_etype771, _size768);
-            this->tables.resize(_size768);
-            uint32_t _i772;
-            for (_i772 = 0; _i772 < _size768; ++_i772)
+            uint32_t _size770;
+            ::apache::thrift::protocol::TType _etype773;
+            xfer += iprot->readListBegin(_etype773, _size770);
+            this->tables.resize(_size770);
+            uint32_t _i774;
+            for (_i774 = 0; _i774 < _size770; ++_i774)
             {
-              xfer += this->tables[_i772].read(iprot);
+              xfer += this->tables[_i774].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -18345,10 +18499,10 @@ uint32_t GetTablesResult::write(::apache::thrift::protocol::TProtocol* oprot) co
   xfer += oprot->writeFieldBegin("tables", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->tables.size()));
-    std::vector<Table> ::const_iterator _iter773;
-    for (_iter773 = this->tables.begin(); _iter773 != this->tables.end(); ++_iter773)
+    std::vector<Table> ::const_iterator _iter775;
+    for (_iter775 = this->tables.begin(); _iter775 != this->tables.end(); ++_iter775)
     {
-      xfer += (*_iter773).write(oprot);
+      xfer += (*_iter775).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -18364,11 +18518,11 @@ void swap(GetTablesResult &a, GetTablesResult &b) {
   swap(a.tables, b.tables);
 }
 
-GetTablesResult::GetTablesResult(const GetTablesResult& other774) {
-  tables = other774.tables;
+GetTablesResult::GetTablesResult(const GetTablesResult& other776) {
+  tables = other776.tables;
 }
-GetTablesResult& GetTablesResult::operator=(const GetTablesResult& other775) {
-  tables = other775.tables;
+GetTablesResult& GetTablesResult::operator=(const GetTablesResult& other777) {
+  tables = other777.tables;
   return *this;
 }
 void GetTablesResult::printTo(std::ostream& out) const {
@@ -18510,19 +18664,19 @@ void swap(TableMeta &a, TableMeta &b) {
   swap(a.__isset, b.__isset);
 }
 
-TableMeta::TableMeta(const TableMeta& other776) {
-  dbName = other776.dbName;
-  tableName = other776.tableName;
-  tableType = other776.tableType;
-  comments = other776.comments;
-  __isset = other776.__isset;
+TableMeta::TableMeta(const TableMeta& other778) {
+  dbName = other778.dbName;
+  tableName = other778.tableName;
+  tableType = other778.tableType;
+  comments = other778.comments;
+  __isset = other778.__isset;
 }
-TableMeta& TableMeta::operator=(const TableMeta& other777) {
-  dbName = other777.dbName;
-  tableName = other777.tableName;
-  tableType = other777.tableType;
-  comments = other777.comments;
-  __isset = other777.__isset;
+TableMeta& TableMeta::operator=(const TableMeta& other779) {
+  dbName = other779.dbName;
+  tableName = other779.tableName;
+  tableType = other779.tableType;
+  comments = other779.comments;
+  __isset = other779.__isset;
   return *this;
 }
 void TableMeta::printTo(std::ostream& out) const {
@@ -18605,13 +18759,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other778) : TException() {
-  message = other778.message;
-  __isset = other778.__isset;
+MetaException::MetaException(const MetaException& other780) : TException() {
+  message = other780.message;
+  __isset = other780.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other779) {
-  message = other779.message;
-  __isset = other779.__isset;
+MetaException& MetaException::operator=(const MetaException& other781) {
+  message = other781.message;
+  __isset = other781.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -18702,13 +18856,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other780) : TException() {
-  message = other780.message;
-  __isset = other780.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other782) : TException() {
+  message = other782.message;
+  __isset = other782.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other781) {
-  message = other781.message;
-  __isset = other781.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other783) {
+  message = other783.message;
+  __isset = other783.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -18799,13 +18953,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other782) : TException() {
-  message = other782.message;
-  __isset = other782.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other784) : TException() {
+  message = other784.message;
+  __isset = other784.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other783) {
-  message = other783.message;
-  __isset = other783.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other785) {
+  message = other785.message;
+  __isset = other785.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -18896,13 +19050,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other784) : TException() {
-  message = other784.message;
-  __isset = other784.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other786) : TException() {
+  message = other786.message;
+  __isset = other786.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other785) {
-  message = other785.message;
-  __isset = other785.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other787) {
+  message = other787.message;
+  __isset = other787.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -18993,13 +19147,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other786) : TException() {
-  message = other786.message;
-  __isset = other786.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other788) : TException() {
+  message = other788.message;
+  __isset = other788.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other787) {
-  message = other787.message;
-  __isset = other787.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other789) {
+  message = other789.message;
+  __isset = other789.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -19090,13 +19244,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other788) : TException() {
-  message = other788.message;
-  __isset = other788.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other790) : TException() {
+  message = other790.message;
+  __isset = other790.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other789) {
-  message = other789.message;
-  __isset = other789.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other791) {
+  message = other791.message;
+  __isset = other791.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -19187,13 +19341,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other790) : TException() {
-  message = other790.message;
-  __isset = other790.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other792) : TException() {
+  message = other792.message;
+  __isset = other792.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other791) {
-  message = other791.message;
-  __isset = other791.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other793) {
+  message = other793.message;
+  __isset = other793.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -19284,13 +19438,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other792) : TException() {
-  message = other792.message;
-  __isset = other792.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other794) : TException() {
+  message = other794.message;
+  __isset = other794.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other793) {
-  message = other793.message;
-  __isset = other793.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other795) {
+  message = other795.message;
+  __isset = other795.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -19381,13 +19535,13 @@ void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other794) : TException() {
-  message = other794.message;
-  __isset = other794.__isset;
+IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other796) : TException() {
+  message = other796.message;
+  __isset = other796.__isset;
 }
-IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other795) {
-  message = other795.message;
-  __isset = other795.__isset;
+IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other797) {
+  message = other797.message;
+  __isset = other797.__isset;
   return *this;
 }
 void IndexAlreadyExistsException::printTo(std::ostream& out) const {
@@ -19478,13 +19632,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other796) : TException() {
-  message = other796.message;
-  __isset = other796.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other798) : TException() {
+  message = other798.message;
+  __isset = other798.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other797) {
-  message = other797.message;
-  __isset = other797.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other799) {
+  message = other799.message;
+  __isset = other799.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -19575,13 +19729,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other798) : TException() {
-  message = other798.message;
-  __isset = other798.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other800) : TException() {
+  message = other800.message;
+  __isset = other800.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other799) {
-  message = other799.message;
-  __isset = other799.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other801) {
+  message = other801.message;
+  __isset = other801.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -19672,13 +19826,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other800) : TException() {
-  message = other800.message;
-  __isset = other800.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other802) : TException() {
+  message = other802.message;
+  __isset = other802.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other801) {
-  message = other801.message;
-  __isset = other801.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other803) {
+  message = other803.message;
+  __isset = other803.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -19769,13 +19923,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other802) : TException() {
-  message = other802.message;
-  __isset = other802.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other804) : TException() {
+  message = other804.message;
+  __isset = other804.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other803) {
-  message = other803.message;
-  __isset = other803.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other805) {
+  message = other805.message;
+  __isset = other805.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -19866,13 +20020,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other804) : TException() {
-  message = other804.message;
-  __isset = other804.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other806) : TException() {
+  message = other806.message;
+  __isset = other806.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other805) {
-  message = other805.message;
-  __isset = other805.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other807) {
+  message = other807.message;
+  __isset = other807.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -19963,13 +20117,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other806) : TException() {
-  message = other806.message;
-  __isset = other806.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other808) : TException() {
+  message = other808.message;
+  __isset = other808.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other807) {
-  message = other807.message;
-  __isset = other807.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other809) {
+  message = other809.message;
+  __isset = other809.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -20060,13 +20214,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other808) : TException() {
-  message = other808.message;
-  __isset = other808.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other810) : TException() {
+  message = other810.message;
+  __isset = other810.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other809) {
-  message = other809.message;
-  __isset = other809.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other811) {
+  message = other811.message;
+  __isset = other811.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {