You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2018/07/14 06:07:16 UTC

[01/12] hive git commit: HIVE-20006: Make materializations invalidation cache work with multiple active remote metastores (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master ab9e954d4 -> 1b5903b03


http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index 3785f89..9dd3787 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -26,10 +26,10 @@ import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.sql.Savepoint;
 import java.sql.Statement;
+import java.time.Instant;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.BitSet;
-import java.util.Calendar;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -40,7 +40,6 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 import java.util.SortedSet;
-import java.util.TimeZone;
 import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Semaphore;
@@ -59,11 +58,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.ValidReadTxnList;
 import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
 import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
 import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.common.classification.RetrySemantics;
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
-import org.apache.hadoop.hive.metastore.MaterializationsInvalidationCache;
-import org.apache.hadoop.hive.metastore.MaterializationsRebuildLockHandler;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.MetaStoreListenerNotifier;
 import org.apache.hadoop.hive.metastore.TransactionalMetaStoreEventListener;
@@ -869,10 +867,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
   @RetrySemantics.Idempotent("No-op if already committed")
   public void commitTxn(CommitTxnRequest rqst)
     throws NoSuchTxnException, TxnAbortedException, MetaException {
-    MaterializationsRebuildLockHandler materializationsRebuildLockHandler =
-        MaterializationsRebuildLockHandler.get();
-    List<TransactionRegistryInfo> txnComponents = new ArrayList<>();
-    boolean isUpdateDelete = false;
+    char isUpdateDelete = 'N';
     long txnid = rqst.getTxnid();
     long sourceTxnId = -1;
 
@@ -936,7 +931,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
                   "tc_operation_type " + conflictSQLSuffix));
         }
         if (rs != null && rs.next()) {
-          isUpdateDelete = true;
+          isUpdateDelete = 'Y';
           close(rs);
           //if here it means currently committing txn performed update/delete and we should check WW conflict
           /**
@@ -1033,8 +1028,8 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           // Move the record from txn_components into completed_txn_components so that the compactor
           // knows where to look to compact.
           s = "insert into COMPLETED_TXN_COMPONENTS (ctc_txnid, ctc_database, " +
-                  "ctc_table, ctc_partition, ctc_writeid) select tc_txnid, tc_database, tc_table, " +
-                  "tc_partition, tc_writeid from TXN_COMPONENTS where tc_txnid = " + txnid;
+                  "ctc_table, ctc_partition, ctc_writeid, ctc_update_delete) select tc_txnid, tc_database, tc_table, " +
+                  "tc_partition, tc_writeid, '" + isUpdateDelete + "' from TXN_COMPONENTS where tc_txnid = " + txnid;
           LOG.debug("Going to execute insert <" + s + ">");
 
           if ((stmt.executeUpdate(s)) < 1) {
@@ -1050,10 +1045,11 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
               rows.add(txnid + "," + quoteString(writeEventInfo.getDatabase()) + "," +
                       quoteString(writeEventInfo.getTable()) + "," +
                       quoteString(writeEventInfo.getPartition()) + "," +
-                      writeEventInfo.getWriteId());
+                      writeEventInfo.getWriteId() + "," +
+                      "'" + isUpdateDelete + "'");
             }
             List<String> queries = sqlGenerator.createInsertValuesStmt("COMPLETED_TXN_COMPONENTS " +
-                    "(ctc_txnid," + " ctc_database, ctc_table, ctc_partition, ctc_writeid)", rows);
+                    "(ctc_txnid," + " ctc_database, ctc_table, ctc_partition, ctc_writeid, ctc_update_delete)", rows);
             for (String q : queries) {
               LOG.debug("Going to execute insert  <" + q + "> ");
               stmt.execute(q);
@@ -1066,18 +1062,6 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           stmt.executeUpdate(s);
         }
 
-        // Obtain information that we need to update registry
-        s = "select ctc_database, ctc_table, ctc_writeid, ctc_timestamp from COMPLETED_TXN_COMPONENTS" +
-                " where ctc_txnid = " + txnid;
-
-        LOG.debug("Going to extract table modification information for invalidation cache <" + s + ">");
-        rs = stmt.executeQuery(s);
-        while (rs.next()) {
-          // We only enter in this loop if the transaction actually affected any table
-          txnComponents.add(new TransactionRegistryInfo(rs.getString(1), rs.getString(2),
-              rs.getLong(3), rs.getTimestamp(4, Calendar.getInstance(TimeZone.getTimeZone("UTC"))).getTime()));
-        }
-
         // cleanup all txn related metadata
         s = "delete from TXN_COMPONENTS where tc_txnid = " + txnid;
         LOG.debug("Going to execute update <" + s + ">");
@@ -1092,29 +1076,19 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         LOG.debug("Going to execute update <" + s + ">");
         stmt.executeUpdate(s);
         LOG.info("Removed committed transaction: (" + txnid + ") from MIN_HISTORY_LEVEL");
+
+        s = "delete from MATERIALIZATION_REBUILD_LOCKS where mrl_txn_id = " + txnid;
+        LOG.debug("Going to execute update <" + s + ">");
+        stmt.executeUpdate(s);
+
         if (transactionalListeners != null) {
           MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
                   EventMessage.EventType.COMMIT_TXN, new CommitTxnEvent(txnid, null), dbConn, sqlGenerator);
         }
 
-        MaterializationsInvalidationCache materializationsInvalidationCache =
-            MaterializationsInvalidationCache.get();
-        for (TransactionRegistryInfo info : txnComponents) {
-          if (materializationsInvalidationCache.containsMaterialization(info.dbName, info.tblName) &&
-              !materializationsRebuildLockHandler.readyToCommitResource(info.dbName, info.tblName, txnid)) {
-            throw new MetaException(
-                "Another process is rebuilding the materialized view " + info.fullyQualifiedName);
-          }
-        }
         LOG.debug("Going to commit");
         close(rs);
         dbConn.commit();
-
-        // Update registry with modifications
-        for (TransactionRegistryInfo info : txnComponents) {
-          materializationsInvalidationCache.notifyTableModification(
-              info.dbName, info.tblName, info.writeId, info.timestamp, isUpdateDelete);
-        }
       } catch (SQLException e) {
         LOG.debug("Going to rollback");
         rollbackDBConn(dbConn);
@@ -1125,9 +1099,6 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         close(commitIdRs);
         close(lockHandle, stmt, dbConn);
         unlockInternal();
-        for (TransactionRegistryInfo info : txnComponents) {
-          materializationsRebuildLockHandler.unlockResource(info.dbName, info.tblName, txnid);
-        }
       }
     } catch (RetryException e) {
       commitTxn(rqst);
@@ -1694,16 +1665,30 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
   }
 
   /**
-   * Gets the information of the first transaction for the given table
-   * after the transaction with the input id was committed (if any). 
+   * Get invalidation info for the materialization. Currently, the materialization information
+   * only contains information about whether there was update/delete operations on the source
+   * tables used by the materialization since it was created.
    */
   @Override
   @RetrySemantics.ReadOnly
-  public BasicTxnInfo getFirstCompletedTransactionForTableAfterCommit(
-      String inputDbName, String inputTableName, ValidWriteIdList txnList)
-          throws MetaException {
-    final List<Long> openTxns = Arrays.asList(ArrayUtils.toObject(txnList.getInvalidWriteIds()));
+  public Materialization getMaterializationInvalidationInfo(
+      CreationMetadata creationMetadata, String validTxnListStr) throws MetaException {
+    if (creationMetadata.getTablesUsed().isEmpty()) {
+      // Bail out
+      LOG.warn("Materialization creation metadata does not contain any table");
+      return null;
+    }
+
+    // Parse validTxnList
+    final ValidReadTxnList validTxnList =
+        new ValidReadTxnList(validTxnListStr);
+
+    // Parse validReaderWriteIdList from creation metadata
+    final ValidTxnWriteIdList validReaderWriteIdList =
+        new ValidTxnWriteIdList(creationMetadata.getValidTxnList());
 
+    // We are composing a query that returns a single row if an update happened after
+    // the materialization was created. Otherwise, query returns 0 rows.
     Connection dbConn = null;
     Statement stmt = null;
     ResultSet rs = null;
@@ -1711,32 +1696,207 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
       stmt = dbConn.createStatement();
       stmt.setMaxRows(1);
-      String s = "select ctc_timestamp, ctc_writeid, ctc_database, ctc_table "
-          + "from COMPLETED_TXN_COMPONENTS "
-          + "where ctc_database=" + quoteString(inputDbName) + " and ctc_table=" + quoteString(inputTableName)
-          + " and ctc_writeid > " + txnList.getHighWatermark()
-          + (txnList.getInvalidWriteIds().length == 0 ?
-              " " : " or ctc_writeid IN(" + StringUtils.join(",", openTxns) + ") ")
-          + "order by ctc_timestamp asc";
+      StringBuilder query = new StringBuilder();
+      // compose a query that select transactions containing an update...
+      query.append("select ctc_update_delete from COMPLETED_TXN_COMPONENTS where ctc_update_delete='Y' AND (");
+      int i = 0;
+      for (String fullyQualifiedName : creationMetadata.getTablesUsed()) {
+        // ...for each of the tables that are part of the materialized view,
+        // where the transaction had to be committed after the materialization was created...
+        if (i != 0) {
+          query.append("OR");
+        }
+        String[] names = TxnUtils.getDbTableName(fullyQualifiedName);
+        query.append(" (ctc_database=" + quoteString(names[0]) + " AND ctc_table=" + quoteString(names[1]));
+        ValidWriteIdList tblValidWriteIdList =
+            validReaderWriteIdList.getTableValidWriteIdList(fullyQualifiedName);
+        if (tblValidWriteIdList == null) {
+          LOG.warn("ValidWriteIdList for table {} not present in creation metadata, this should not happen");
+          return null;
+        }
+        query.append(" AND (ctc_writeid > " + tblValidWriteIdList.getHighWatermark());
+        query.append(tblValidWriteIdList.getInvalidWriteIds().length == 0 ? ") " :
+            " OR ctc_writeid IN(" + StringUtils.join(",",
+                Arrays.asList(ArrayUtils.toObject(tblValidWriteIdList.getInvalidWriteIds()))) + ") ");
+        query.append(") ");
+        i++;
+      }
+      // ... and where the transaction has already been committed as per snapshot taken
+      // when we are running current query
+      query.append(") AND ctc_txnid <= " + validTxnList.getHighWatermark());
+      query.append(validTxnList.getInvalidTransactions().length == 0 ? " " :
+          " AND ctc_txnid NOT IN(" + StringUtils.join(",",
+              Arrays.asList(ArrayUtils.toObject(validTxnList.getInvalidTransactions()))) + ") ");
+
+      // Execute query
+      String s = query.toString();
       if (LOG.isDebugEnabled()) {
         LOG.debug("Going to execute query <" + s + ">");
       }
       rs = stmt.executeQuery(s);
 
-      if(!rs.next()) {
-        return new BasicTxnInfo(true);
+      return new Materialization(rs.next());
+    } catch (SQLException ex) {
+      LOG.warn("getMaterializationInvalidationInfo failed due to " + getMessage(ex), ex);
+      throw new MetaException("Unable to retrieve materialization invalidation information due to " +
+          StringUtils.stringifyException(ex));
+    } finally {
+      close(rs, stmt, dbConn);
+    }
+  }
+
+  @Override
+  public LockResponse lockMaterializationRebuild(String dbName, String tableName, long txnId)
+      throws MetaException {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Acquiring lock for materialization rebuild with txnId={} for {}", txnId, Warehouse.getQualifiedName(dbName,tableName));
+    }
+
+    TxnStore.MutexAPI.LockHandle handle = null;
+    Connection dbConn = null;
+    Statement stmt = null;
+    ResultSet rs = null;
+    try {
+      lockInternal();
+      /**
+       * MUTEX_KEY.MaterializationRebuild lock ensures that there is only 1 entry in
+       * Initiated/Working state for any resource. This ensures we do not run concurrent
+       * rebuild operations on any materialization.
+       */
+      handle = getMutexAPI().acquireLock(MUTEX_KEY.MaterializationRebuild.name());
+      dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+      stmt = dbConn.createStatement();
+
+      String selectQ = "select mrl_txn_id from MATERIALIZATION_REBUILD_LOCKS where" +
+          " mrl_db_name =" + quoteString(dbName) +
+          " AND mrl_tbl_name=" + quoteString(tableName);
+      LOG.debug("Going to execute query <" + selectQ + ">");
+      rs = stmt.executeQuery(selectQ);
+      if(rs.next()) {
+        LOG.info("Ignoring request to rebuild " + dbName + "/" + tableName +
+            " since it is already being rebuilt");
+        return new LockResponse(txnId, LockState.NOT_ACQUIRED);
       }
-      final BasicTxnInfo txnInfo = new BasicTxnInfo(false);
-      txnInfo.setTime(rs.getTimestamp(1, Calendar.getInstance(TimeZone.getTimeZone("UTC"))).getTime());
-      txnInfo.setTxnid(rs.getLong(2));
-      txnInfo.setDbname(rs.getString(3));
-      txnInfo.setTablename(rs.getString(4));
-      return txnInfo;
+      String insertQ = "insert into MATERIALIZATION_REBUILD_LOCKS " +
+          "(mrl_txn_id, mrl_db_name, mrl_tbl_name, mrl_last_heartbeat) values (" + txnId +
+          ", '" + dbName + "', '" + tableName + "', " + Instant.now().toEpochMilli() + ")";
+      LOG.debug("Going to execute update <" + insertQ + ">");
+      stmt.executeUpdate(insertQ);
+      LOG.debug("Going to commit");
+      dbConn.commit();
+      return new LockResponse(txnId, LockState.ACQUIRED);
     } catch (SQLException ex) {
-      LOG.warn("getLastCompletedTransactionForTable failed due to " + getMessage(ex), ex);
-      throw new MetaException("Unable to retrieve commits information due to " + StringUtils.stringifyException(ex));
+      LOG.warn("lockMaterializationRebuild failed due to " + getMessage(ex), ex);
+      throw new MetaException("Unable to retrieve materialization invalidation information due to " +
+          StringUtils.stringifyException(ex));
     } finally {
       close(rs, stmt, dbConn);
+      if(handle != null) {
+        handle.releaseLocks();
+      }
+      unlockInternal();
+    }
+  }
+
+  @Override
+  public boolean heartbeatLockMaterializationRebuild(String dbName, String tableName, long txnId)
+      throws MetaException {
+    try {
+      Connection dbConn = null;
+      Statement stmt = null;
+      ResultSet rs = null;
+      try {
+        lockInternal();
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+        stmt = dbConn.createStatement();
+        String s = "update MATERIALIZATION_REBUILD_LOCKS" +
+            " set mrl_last_heartbeat = " + Instant.now().toEpochMilli() +
+            " where mrl_txn_id = " + txnId +
+            " AND mrl_db_name =" + quoteString(dbName) +
+            " AND mrl_tbl_name=" + quoteString(tableName);
+        LOG.debug("Going to execute update <" + s + ">");
+        int rc = stmt.executeUpdate(s);
+        if (rc < 1) {
+          LOG.debug("Going to rollback");
+          dbConn.rollback();
+          LOG.info("No lock found for rebuild of " + Warehouse.getQualifiedName(dbName, tableName) +
+              " when trying to heartbeat");
+          // It could not be renewed, return that information
+          return false;
+        }
+        LOG.debug("Going to commit");
+        dbConn.commit();
+        // It could be renewed, return that information
+        return true;
+      } catch (SQLException e) {
+        LOG.debug("Going to rollback");
+        rollbackDBConn(dbConn);
+        checkRetryable(dbConn, e,
+            "heartbeatLockMaterializationRebuild(" + Warehouse.getQualifiedName(dbName, tableName) + ", " + txnId + ")");
+        throw new MetaException("Unable to heartbeat rebuild lock due to " +
+            StringUtils.stringifyException(e));
+      } finally {
+        close(rs, stmt, dbConn);
+        unlockInternal();
+      }
+    } catch (RetryException e) {
+      return heartbeatLockMaterializationRebuild(dbName, tableName ,txnId);
+    }
+  }
+
+  @Override
+  public long cleanupMaterializationRebuildLocks(ValidTxnList validTxnList, long timeout) throws MetaException {
+    try {
+      // Aux values
+      long cnt = 0L;
+      List<Long> txnIds = new ArrayList<>();
+      long timeoutTime = Instant.now().toEpochMilli() - timeout;
+
+      Connection dbConn = null;
+      Statement stmt = null;
+      ResultSet rs = null;
+      try {
+        lockInternal();
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+        stmt = dbConn.createStatement();
+
+        String selectQ = "select mrl_txn_id, mrl_last_heartbeat from MATERIALIZATION_REBUILD_LOCKS";
+        LOG.debug("Going to execute query <" + selectQ + ">");
+        rs = stmt.executeQuery(selectQ);
+        while(rs.next()) {
+          long lastHeartbeat = rs.getLong(2);
+          if (lastHeartbeat < timeoutTime) {
+            // The heartbeat has timeout, double check whether we can remove it
+            long txnId = rs.getLong(1);
+            if (validTxnList.isTxnValid(txnId) || validTxnList.isTxnAborted(txnId)) {
+              // Txn was committed (but notification was not received) or it was aborted.
+              // Either case, we can clean it up
+              txnIds.add(txnId);
+            }
+          }
+        }
+        if (!txnIds.isEmpty()) {
+          String deleteQ = "delete from MATERIALIZATION_REBUILD_LOCKS where" +
+              " mrl_txn_id IN(" + StringUtils.join(",", txnIds) + ") ";
+          LOG.debug("Going to execute update <" + deleteQ + ">");
+          cnt = stmt.executeUpdate(deleteQ);
+        }
+        LOG.debug("Going to commit");
+        dbConn.commit();
+        return cnt;
+      } catch (SQLException e) {
+        LOG.debug("Going to rollback");
+        rollbackDBConn(dbConn);
+        checkRetryable(dbConn, e, "cleanupMaterializationRebuildLocks");
+        throw new MetaException("Unable to clean rebuild locks due to " +
+            StringUtils.stringifyException(e));
+      } finally {
+        close(rs, stmt, dbConn);
+        unlockInternal();
+      }
+    } catch (RetryException e) {
+      return cleanupMaterializationRebuildLocks(validTxnList, timeout);
     }
   }
 
@@ -2009,6 +2169,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
   private static String normalizeCase(String s) {
     return s == null ? null : s.toLowerCase();
   }
+
   private LockResponse checkLockWithRetry(Connection dbConn, long extLockId, long txnId)
     throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, MetaException {
     try {
@@ -4887,20 +5048,4 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     }
   };
 
-  private class TransactionRegistryInfo {
-    final String dbName;
-    final String tblName;
-    final String fullyQualifiedName;
-    final long writeId;
-    final long timestamp;
-
-    public TransactionRegistryInfo (String dbName, String tblName, long writeId, long timestamp) {
-      this.dbName = dbName;
-      this.tblName = tblName;
-      this.fullyQualifiedName = Warehouse.getQualifiedName(dbName, tblName);
-      this.writeId = writeId;
-      this.timestamp = timestamp;
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
index d972d10..33f24fb 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
@@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.common.classification.RetrySemantics;
 import org.apache.hadoop.hive.metastore.api.*;
@@ -29,6 +30,7 @@ import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import java.sql.SQLException;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 /**
@@ -41,7 +43,7 @@ public interface TxnStore extends Configurable {
 
   enum MUTEX_KEY {
     Initiator, Cleaner, HouseKeeper, CompactionHistory, CheckLock,
-    WriteSetCleaner, CompactionScheduler, WriteIdAllocator
+    WriteSetCleaner, CompactionScheduler, WriteIdAllocator, MaterializationRebuild
   }
   // Compactor states (Should really be enum)
   String INITIATED_RESPONSE = "initiated";
@@ -128,21 +130,33 @@ public interface TxnStore extends Configurable {
   void replTableWriteIdState(ReplTblWriteIdStateRequest rqst) throws MetaException;
 
   /**
-   * Get the first transaction corresponding to given database and table after transactions
-   * referenced in the transaction snapshot.
-   * @return
+   * Get invalidation info for the materialization. Currently, the materialization information
+   * only contains information about whether there was update/delete operations on the source
+   * tables used by the materialization since it was created.
+   * @param cm creation metadata for the materialization
+   * @param validTxnList valid transaction list for snapshot taken for current query
    * @throws MetaException
    */
   @RetrySemantics.Idempotent
-  BasicTxnInfo getFirstCompletedTransactionForTableAfterCommit(
-      String inputDbName, String inputTableName, ValidWriteIdList txnList)
+  Materialization getMaterializationInvalidationInfo(
+      final CreationMetadata cm, final String validTxnList)
           throws MetaException;
-  /**
-   * Gets the list of valid write ids for the given table wrt to current txn
-   * @param rqst info on transaction and list of table names associated with given transaction
-   * @throws NoSuchTxnException
-   * @throws MetaException
-   */
+
+  LockResponse lockMaterializationRebuild(String dbName, String tableName, long txnId)
+      throws MetaException;
+
+  boolean heartbeatLockMaterializationRebuild(String dbName, String tableName, long txnId)
+      throws MetaException;
+
+  long cleanupMaterializationRebuildLocks(ValidTxnList validTxnList, long timeout)
+      throws MetaException;
+
+    /**
+     * Gets the list of valid write ids for the given table wrt to current txn
+     * @param rqst info on transaction and list of table names associated with given transaction
+     * @throws NoSuchTxnException
+     * @throws MetaException
+     */
   @RetrySemantics.ReadOnly
   GetValidWriteIdsResponse getValidWriteIds(GetValidWriteIdsRequest rqst)
           throws NoSuchTxnException,  MetaException;

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/derby/hive-schema-3.1.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/derby/hive-schema-3.1.0.derby.sql b/standalone-metastore/metastore-common/src/main/sql/derby/hive-schema-3.1.0.derby.sql
index a696d06..5e8693e 100644
--- a/standalone-metastore/metastore-common/src/main/sql/derby/hive-schema-3.1.0.derby.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/derby/hive-schema-3.1.0.derby.sql
@@ -197,7 +197,8 @@ CREATE TABLE "APP"."MV_CREATION_METADATA" (
   "CAT_NAME" VARCHAR(256) NOT NULL,
   "DB_NAME" VARCHAR(128) NOT NULL,
   "TBL_NAME" VARCHAR(256) NOT NULL,
-  "TXN_LIST" CLOB
+  "TXN_LIST" CLOB,
+  "MATERIALIZATION_TIME" BIGINT NOT NULL
 );
 
 CREATE TABLE "APP"."MV_TABLES_USED" (
@@ -526,7 +527,8 @@ CREATE TABLE COMPLETED_TXN_COMPONENTS (
   CTC_TABLE varchar(256),
   CTC_PARTITION varchar(767),
   CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL,
-  CTC_WRITEID bigint
+  CTC_WRITEID bigint,
+  CTC_UPDATE_DELETE char(1) NOT NULL
 );
 
 CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
@@ -645,6 +647,14 @@ CREATE TABLE MIN_HISTORY_LEVEL (
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID BIGINT NOT NULL,
+  MRL_DB_NAME VARCHAR(128) NOT NULL,
+  MRL_TBL_NAME VARCHAR(256) NOT NULL,
+  MRL_LAST_HEARTBEAT BIGINT NOT NULL,
+  PRIMARY KEY(MRL_TXN_ID)
+);
+
 CREATE TABLE "APP"."I_SCHEMA" (
   "SCHEMA_ID" bigint primary key,
   "SCHEMA_TYPE" integer not null,

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/derby/hive-schema-4.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/derby/hive-schema-4.0.0.derby.sql b/standalone-metastore/metastore-common/src/main/sql/derby/hive-schema-4.0.0.derby.sql
index 7cab4fb..5ba71c4 100644
--- a/standalone-metastore/metastore-common/src/main/sql/derby/hive-schema-4.0.0.derby.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/derby/hive-schema-4.0.0.derby.sql
@@ -197,7 +197,8 @@ CREATE TABLE "APP"."MV_CREATION_METADATA" (
   "CAT_NAME" VARCHAR(256) NOT NULL,
   "DB_NAME" VARCHAR(128) NOT NULL,
   "TBL_NAME" VARCHAR(256) NOT NULL,
-  "TXN_LIST" CLOB
+  "TXN_LIST" CLOB,
+  "MATERIALIZATION_TIME" BIGINT NOT NULL
 );
 
 CREATE TABLE "APP"."MV_TABLES_USED" (
@@ -526,7 +527,8 @@ CREATE TABLE COMPLETED_TXN_COMPONENTS (
   CTC_TABLE varchar(256),
   CTC_PARTITION varchar(767),
   CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL,
-  CTC_WRITEID bigint
+  CTC_WRITEID bigint,
+  CTC_UPDATE_DELETE char(1) NOT NULL
 );
 
 CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
@@ -645,6 +647,14 @@ CREATE TABLE MIN_HISTORY_LEVEL (
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID BIGINT NOT NULL,
+  MRL_DB_NAME VARCHAR(128) NOT NULL,
+  MRL_TBL_NAME VARCHAR(256) NOT NULL,
+  MRL_LAST_HEARTBEAT BIGINT NOT NULL,
+  PRIMARY KEY(MRL_TXN_ID)
+);
+
 CREATE TABLE "APP"."I_SCHEMA" (
   "SCHEMA_ID" bigint primary key,
   "SCHEMA_TYPE" integer not null,

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql b/standalone-metastore/metastore-common/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql
index 7058ab0..2b200f2 100644
--- a/standalone-metastore/metastore-common/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql
@@ -45,5 +45,24 @@ CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
 );
 INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
 
+-- HIVE-19027
+-- add column MATERIALIZATION_TIME (bigint) to MV_CREATION_METADATA table
+ALTER TABLE "APP"."MV_CREATION_METADATA" ADD COLUMN "MATERIALIZATION_TIME" BIGINT;
+UPDATE "APP"."MV_CREATION_METADATA" SET "MATERIALIZATION_TIME" = 0;
+ALTER TABLE "APP"."MV_CREATION_METADATA" ALTER COLUMN "MATERIALIZATION_TIME" NOT NULL;
+
+-- add column CTC_UPDATE_DELETE (char) to COMPLETED_TXN_COMPONENTS table
+ALTER TABLE COMPLETED_TXN_COMPONENTS ADD COLUMN CTC_UPDATE_DELETE char(1);
+UPDATE COMPLETED_TXN_COMPONENTS SET CTC_UPDATE_DELETE = 'N';
+ALTER TABLE COMPLETED_TXN_COMPONENTS ALTER COLUMN CTC_UPDATE_DELETE NOT NULL;
+
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID BIGINT NOT NULL,
+  MRL_DB_NAME VARCHAR(128) NOT NULL,
+  MRL_TBL_NAME VARCHAR(256) NOT NULL,
+  MRL_LAST_HEARTBEAT BIGINT NOT NULL,
+  PRIMARY KEY(MRL_TXN_ID)
+);
+
 -- This needs to be the last thing done.  Insert any changes above this line.
 UPDATE "APP".VERSION SET SCHEMA_VERSION='3.1.0', VERSION_COMMENT='Hive release version 3.1.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql b/standalone-metastore/metastore-common/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql
index d7722dc..446ee6e 100644
--- a/standalone-metastore/metastore-common/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql
@@ -388,7 +388,8 @@ CREATE TABLE MV_CREATION_METADATA
     CAT_NAME nvarchar(256) NOT NULL,
     DB_NAME nvarchar(128) NOT NULL,
     TBL_NAME nvarchar(256) NOT NULL,
-    TXN_LIST text NULL
+    TXN_LIST text NULL,
+    MATERIALIZATION_TIME bigint NOT NULL
 );
 
 ALTER TABLE MV_CREATION_METADATA ADD CONSTRAINT MV_CREATION_METADATA_PK PRIMARY KEY (MV_CREATION_METADATA_ID);
@@ -1034,7 +1035,8 @@ CREATE TABLE COMPLETED_TXN_COMPONENTS(
 	CTC_TABLE nvarchar(128) NULL,
 	CTC_PARTITION nvarchar(767) NULL,
     CTC_TIMESTAMP datetime2 DEFAULT CURRENT_TIMESTAMP NOT NULL,
-    CTC_WRITEID bigint
+    CTC_WRITEID bigint,
+    CTC_UPDATE_DELETE char(1) NOT NULL
 );
 
 CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
@@ -1191,6 +1193,17 @@ PRIMARY KEY CLUSTERED
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID bigint NOT NULL,
+  MRL_DB_NAME nvarchar(128) NOT NULL,
+  MRL_TBL_NAME nvarchar(256) NOT NULL,
+  MRL_LAST_HEARTBEAT bigint NOT NULL,
+PRIMARY KEY CLUSTERED
+(
+    MRL_TXN_ID ASC
+)
+);
+
 CREATE TABLE "I_SCHEMA" (
   "SCHEMA_ID" bigint primary key,
   "SCHEMA_TYPE" int not null,

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql b/standalone-metastore/metastore-common/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql
index a81fc40..bbc8ea2 100644
--- a/standalone-metastore/metastore-common/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql
@@ -248,7 +248,6 @@ CREATE TABLE TAB_COL_STATS
 ALTER TABLE TAB_COL_STATS ADD CONSTRAINT TAB_COL_STATS_PK PRIMARY KEY (CS_ID);
 CREATE INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS (CAT_NAME, DB_NAME, TABLE_NAME, COLUMN_NAME);
 
-
 -- Table TYPES for classes [org.apache.hadoop.hive.metastore.model.MType]
 CREATE TABLE TYPES
 (
@@ -389,7 +388,8 @@ CREATE TABLE MV_CREATION_METADATA
     CAT_NAME nvarchar(256) NOT NULL,
     DB_NAME nvarchar(128) NOT NULL,
     TBL_NAME nvarchar(256) NOT NULL,
-    TXN_LIST text NULL
+    TXN_LIST text NULL,
+    MATERIALIZATION_TIME bigint NOT NULL
 );
 
 ALTER TABLE MV_CREATION_METADATA ADD CONSTRAINT MV_CREATION_METADATA_PK PRIMARY KEY (MV_CREATION_METADATA_ID);
@@ -1035,7 +1035,8 @@ CREATE TABLE COMPLETED_TXN_COMPONENTS(
 	CTC_TABLE nvarchar(128) NULL,
 	CTC_PARTITION nvarchar(767) NULL,
     CTC_TIMESTAMP datetime2 DEFAULT CURRENT_TIMESTAMP NOT NULL,
-    CTC_WRITEID bigint
+    CTC_WRITEID bigint,
+    CTC_UPDATE_DELETE char(1) NOT NULL
 );
 
 CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
@@ -1192,6 +1193,17 @@ PRIMARY KEY CLUSTERED
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID bigint NOT NULL,
+  MRL_DB_NAME nvarchar(128) NOT NULL,
+  MRL_TBL_NAME nvarchar(256) NOT NULL,
+  MRL_LAST_HEARTBEAT bigint NOT NULL,
+PRIMARY KEY CLUSTERED
+(
+    MRL_TXN_ID ASC
+)
+);
+
 CREATE TABLE "I_SCHEMA" (
   "SCHEMA_ID" bigint primary key,
   "SCHEMA_TYPE" int not null,

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql b/standalone-metastore/metastore-common/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql
index 41f23f7..d44cfdb 100644
--- a/standalone-metastore/metastore-common/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql
@@ -46,6 +46,25 @@ CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
 ALTER TABLE TXN_WRITE_NOTIFICATION_LOG ADD CONSTRAINT TXN_WRITE_NOTIFICATION_LOG_PK PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION);
 INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
 
+-- HIVE-19027
+-- add column MATERIALIZATION_TIME (bigint) to MV_CREATION_METADATA table
+ALTER TABLE MV_CREATION_METADATA ADD MATERIALIZATION_TIME bigint NOT NULL DEFAULT(0);
+
+-- add column CTC_UPDATE_DELETE (char) to COMPLETED_TXN_COMPONENTS table
+ALTER TABLE COMPLETED_TXN_COMPONENTS ADD CTC_UPDATE_DELETE char(1) NOT NULL DEFAULT('N');
+
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID bigint NOT NULL,
+  MRL_DB_NAME nvarchar(128) NOT NULL,
+  MRL_TBL_NAME nvarchar(256) NOT NULL,
+  MRL_LAST_HEARTBEAT bigint NOT NULL,
+PRIMARY KEY CLUSTERED
+(
+    MRL_TXN_ID ASC
+)
+);
+
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.1.0', VERSION_COMMENT='Hive release version 3.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 3.0.0 to 3.1.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql b/standalone-metastore/metastore-common/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql
index 29d4a43..75612a7 100644
--- a/standalone-metastore/metastore-common/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql
@@ -603,6 +603,7 @@ CREATE TABLE IF NOT EXISTS `MV_CREATION_METADATA` (
   `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `TBL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `TXN_LIST` TEXT DEFAULT NULL,
+  `MATERIALIZATION_TIME` bigint(20) NOT NULL,
   PRIMARY KEY (`MV_CREATION_METADATA_ID`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 /*!40101 SET character_set_client = @saved_cs_client */;
@@ -1006,7 +1007,8 @@ CREATE TABLE COMPLETED_TXN_COMPONENTS (
   CTC_TABLE varchar(256),
   CTC_PARTITION varchar(767),
   CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL,
-  CTC_WRITEID bigint
+  CTC_WRITEID bigint,
+  CTC_UPDATE_DELETE char(1) NOT NULL
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
 CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION) USING BTREE;
@@ -1124,6 +1126,14 @@ CREATE TABLE MIN_HISTORY_LEVEL (
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID bigint NOT NULL,
+  MRL_DB_NAME VARCHAR(128) NOT NULL,
+  MRL_TBL_NAME VARCHAR(256) NOT NULL,
+  MRL_LAST_HEARTBEAT bigint NOT NULL,
+  PRIMARY KEY(MRL_TXN_ID)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
 CREATE TABLE `I_SCHEMA` (
   `SCHEMA_ID` BIGINT PRIMARY KEY,
   `SCHEMA_TYPE` INTEGER NOT NULL,

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql b/standalone-metastore/metastore-common/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
index 968f4a4..d53e7fc 100644
--- a/standalone-metastore/metastore-common/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
@@ -603,6 +603,7 @@ CREATE TABLE IF NOT EXISTS `MV_CREATION_METADATA` (
   `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `TBL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `TXN_LIST` TEXT DEFAULT NULL,
+  `MATERIALIZATION_TIME` bigint(20) NOT NULL,
   PRIMARY KEY (`MV_CREATION_METADATA_ID`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 /*!40101 SET character_set_client = @saved_cs_client */;
@@ -1006,7 +1007,8 @@ CREATE TABLE COMPLETED_TXN_COMPONENTS (
   CTC_TABLE varchar(256),
   CTC_PARTITION varchar(767),
   CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL,
-  CTC_WRITEID bigint
+  CTC_WRITEID bigint,
+  CTC_UPDATE_DELETE char(1) NOT NULL
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
 CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION) USING BTREE;
@@ -1124,6 +1126,14 @@ CREATE TABLE MIN_HISTORY_LEVEL (
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID bigint NOT NULL,
+  MRL_DB_NAME VARCHAR(128) NOT NULL,
+  MRL_TBL_NAME VARCHAR(256) NOT NULL,
+  MRL_LAST_HEARTBEAT bigint NOT NULL,
+  PRIMARY KEY(MRL_TXN_ID)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
 CREATE TABLE `I_SCHEMA` (
   `SCHEMA_ID` BIGINT PRIMARY KEY,
   `SCHEMA_TYPE` INTEGER NOT NULL,

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql b/standalone-metastore/metastore-common/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql
index e103bef..7752e89 100644
--- a/standalone-metastore/metastore-common/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql
@@ -46,6 +46,26 @@ CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 INSERT INTO `SEQUENCE_TABLE` (`SEQUENCE_NAME`, `NEXT_VAL`) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
 
+-- HIVE-19027
+-- add column MATERIALIZATION_TIME (bigint) to MV_CREATION_METADATA table
+ALTER TABLE `MV_CREATION_METADATA` ADD `MATERIALIZATION_TIME` BIGINT;
+UPDATE `MV_CREATION_METADATA` SET `MATERIALIZATION_TIME` = 0;
+ALTER TABLE `MV_CREATION_METADATA` MODIFY COLUMN `MATERIALIZATION_TIME` BIGINT NOT NULL;
+
+-- add column CTC_UPDATE_DELETE (char) to COMPLETED_TXN_COMPONENTS table
+ALTER TABLE COMPLETED_TXN_COMPONENTS ADD CTC_UPDATE_DELETE char(1);
+UPDATE COMPLETED_TXN_COMPONENTS SET CTC_UPDATE_DELETE = 'N';
+ALTER TABLE COMPLETED_TXN_COMPONENTS MODIFY COLUMN CTC_UPDATE_DELETE char(1) NOT NULL;
+
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID BIGINT NOT NULL,
+  MRL_DB_NAME VARCHAR(128) NOT NULL,
+  MRL_TBL_NAME VARCHAR(256) NOT NULL,
+  MRL_LAST_HEARTBEAT BIGINT NOT NULL,
+  PRIMARY KEY(MRL_TXN_ID)
+);
+
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.1.0', VERSION_COMMENT='Hive release version 3.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 3.0.0 to 3.1.0' AS ' ';

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql b/standalone-metastore/metastore-common/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql
index 9adea31..a4720c8 100644
--- a/standalone-metastore/metastore-common/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql
@@ -410,7 +410,8 @@ CREATE TABLE MV_CREATION_METADATA
     CAT_NAME VARCHAR2(256) NOT NULL,
     DB_NAME VARCHAR2(128) NOT NULL,
     TBL_NAME VARCHAR2(256) NOT NULL,
-    TXN_LIST CLOB NULL
+    TXN_LIST CLOB NULL,
+    MATERIALIZATION_TIME NUMBER NOT NULL
 );
 
 ALTER TABLE MV_CREATION_METADATA ADD CONSTRAINT MV_CREATION_METADATA_PK PRIMARY KEY (MV_CREATION_METADATA_ID);
@@ -983,7 +984,8 @@ CREATE TABLE COMPLETED_TXN_COMPONENTS (
   CTC_TABLE VARCHAR2(256),
   CTC_PARTITION VARCHAR2(767),
   CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL,
-  CTC_WRITEID NUMBER(19)
+  CTC_WRITEID NUMBER(19),
+  CTC_UPDATE_DELETE CHAR(1) NOT NULL
 ) ROWDEPENDENCIES;
 
 CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
@@ -1100,6 +1102,14 @@ CREATE TABLE MIN_HISTORY_LEVEL (
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID NUMBER NOT NULL,
+  MRL_DB_NAME VARCHAR(128) NOT NULL,
+  MRL_TBL_NAME VARCHAR(256) NOT NULL,
+  MRL_LAST_HEARTBEAT NUMBER NOT NULL,
+  PRIMARY KEY(MRL_TXN_ID)
+);
+
 CREATE TABLE "I_SCHEMA" (
   "SCHEMA_ID" number primary key,
   "SCHEMA_TYPE" number not null,

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql b/standalone-metastore/metastore-common/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
index faca669..e58ee33 100644
--- a/standalone-metastore/metastore-common/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
@@ -410,7 +410,8 @@ CREATE TABLE MV_CREATION_METADATA
     CAT_NAME VARCHAR2(256) NOT NULL,
     DB_NAME VARCHAR2(128) NOT NULL,
     TBL_NAME VARCHAR2(256) NOT NULL,
-    TXN_LIST CLOB NULL
+    TXN_LIST CLOB NULL,
+    MATERIALIZATION_TIME NUMBER NOT NULL
 );
 
 ALTER TABLE MV_CREATION_METADATA ADD CONSTRAINT MV_CREATION_METADATA_PK PRIMARY KEY (MV_CREATION_METADATA_ID);
@@ -983,7 +984,8 @@ CREATE TABLE COMPLETED_TXN_COMPONENTS (
   CTC_TABLE VARCHAR2(256),
   CTC_PARTITION VARCHAR2(767),
   CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL,
-  CTC_WRITEID NUMBER(19)
+  CTC_WRITEID NUMBER(19),
+  CTC_UPDATE_DELETE CHAR(1) NOT NULL
 ) ROWDEPENDENCIES;
 
 CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
@@ -1100,6 +1102,14 @@ CREATE TABLE MIN_HISTORY_LEVEL (
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID NUMBER NOT NULL,
+  MRL_DB_NAME VARCHAR(128) NOT NULL,
+  MRL_TBL_NAME VARCHAR(256) NOT NULL,
+  MRL_LAST_HEARTBEAT NUMBER NOT NULL,
+  PRIMARY KEY(MRL_TXN_ID)
+);
+
 CREATE TABLE "I_SCHEMA" (
   "SCHEMA_ID" number primary key,
   "SCHEMA_TYPE" number not null,

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql b/standalone-metastore/metastore-common/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql
index cf8699b..e4efe4d 100644
--- a/standalone-metastore/metastore-common/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql
@@ -46,6 +46,25 @@ CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
 );
 INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
 
+-- HIVE-19027
+-- add column MATERIALIZATION_TIME (bigint) to MV_CREATION_METADATA table
+ALTER TABLE MV_CREATION_METADATA ADD MATERIALIZATION_TIME NUMBER NULL;
+UPDATE MV_CREATION_METADATA SET MATERIALIZATION_TIME = 0;
+ALTER TABLE MV_CREATION_METADATA MODIFY(MATERIALIZATION_TIME NOT NULL);
+
+-- add column CTC_UPDATE_DELETE (char) to COMPLETED_TXN_COMPONENTS table
+ALTER TABLE COMPLETED_TXN_COMPONENTS ADD CTC_UPDATE_DELETE char(1) NULL;
+UPDATE COMPLETED_TXN_COMPONENTS SET CTC_UPDATE_DELETE = 'N';
+ALTER TABLE COMPLETED_TXN_COMPONENTS MODIFY(CTC_UPDATE_DELETE NOT NULL);
+
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID NUMBER NOT NULL,
+  MRL_DB_NAME VARCHAR(128) NOT NULL,
+  MRL_TBL_NAME VARCHAR(256) NOT NULL,
+  MRL_LAST_HEARTBEAT NUMBER NOT NULL,
+  PRIMARY KEY(MRL_TXN_ID)
+);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.1.0', VERSION_COMMENT='Hive release version 3.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 3.0.0 to 3.1.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql b/standalone-metastore/metastore-common/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql
index 7a8a419..a74c388 100644
--- a/standalone-metastore/metastore-common/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql
@@ -404,7 +404,8 @@ CREATE TABLE "MV_CREATION_METADATA" (
     "CAT_NAME" character varying(256) NOT NULL,
     "DB_NAME" character varying(128) NOT NULL,
     "TBL_NAME" character varying(256) NOT NULL,
-    "TXN_LIST" text
+    "TXN_LIST" text,
+    "MATERIALIZATION_TIME" bigint NOT NULL
 );
 
 --
@@ -1673,7 +1674,8 @@ CREATE TABLE COMPLETED_TXN_COMPONENTS (
   CTC_TABLE varchar(256),
   CTC_PARTITION varchar(767),
   CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL,
-  CTC_WRITEID bigint
+  CTC_WRITEID bigint,
+  CTC_UPDATE_DELETE char(1) NOT NULL
 );
 
 CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX ON COMPLETED_TXN_COMPONENTS USING btree (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
@@ -1790,6 +1792,14 @@ CREATE TABLE MIN_HISTORY_LEVEL (
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID bigint NOT NULL,
+  MRL_DB_NAME varchar(128) NOT NULL,
+  MRL_TBL_NAME varchar(256) NOT NULL,
+  MRL_LAST_HEARTBEAT bigint NOT NULL,
+  PRIMARY KEY(MRL_TXN_ID)
+);
+
 CREATE TABLE "I_SCHEMA" (
   "SCHEMA_ID" bigint primary key,
   "SCHEMA_TYPE" integer not null,

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql b/standalone-metastore/metastore-common/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
index 2e7ac5a..5d1a525 100644
--- a/standalone-metastore/metastore-common/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
@@ -404,7 +404,8 @@ CREATE TABLE "MV_CREATION_METADATA" (
     "CAT_NAME" character varying(256) NOT NULL,
     "DB_NAME" character varying(128) NOT NULL,
     "TBL_NAME" character varying(256) NOT NULL,
-    "TXN_LIST" text
+    "TXN_LIST" text,
+    "MATERIALIZATION_TIME" bigint NOT NULL
 );
 
 --
@@ -1282,6 +1283,11 @@ CREATE INDEX "TBLS_N50" ON "TBLS" USING btree ("SD_ID");
 
 CREATE INDEX "TBL_COL_PRIVS_N49" ON "TBL_COL_PRIVS" USING btree ("TBL_ID");
 
+--
+-- Name: TAB_COL_STATS_IDX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE INDEX "TAB_COL_STATS_IDX" ON "TAB_COL_STATS" USING btree ("CAT_NAME", "DB_NAME","TABLE_NAME","COLUMN_NAME");
 
 --
 -- Name: TBL_PRIVS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
@@ -1303,13 +1309,6 @@ CREATE INDEX "TYPE_FIELDS_N49" ON "TYPE_FIELDS" USING btree ("TYPE_NAME");
 CREATE INDEX "TAB_COL_STATS_N49" ON "TAB_COL_STATS" USING btree ("TBL_ID");
 
 --
--- Name: TAB_COL_STATS_IDX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
---
-
-CREATE INDEX "TAB_COL_STATS_IDX" ON "TAB_COL_STATS" USING btree ("CAT_NAME", "DB_NAME","TABLE_NAME","COLUMN_NAME");
-
-
---
 -- Name: PART_COL_STATS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
 --
 
@@ -1675,7 +1674,8 @@ CREATE TABLE COMPLETED_TXN_COMPONENTS (
   CTC_TABLE varchar(256),
   CTC_PARTITION varchar(767),
   CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL,
-  CTC_WRITEID bigint
+  CTC_WRITEID bigint,
+  CTC_UPDATE_DELETE char(1) NOT NULL
 );
 
 CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX ON COMPLETED_TXN_COMPONENTS USING btree (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
@@ -1792,6 +1792,14 @@ CREATE TABLE MIN_HISTORY_LEVEL (
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID bigint NOT NULL,
+  MRL_DB_NAME varchar(128) NOT NULL,
+  MRL_TBL_NAME varchar(256) NOT NULL,
+  MRL_LAST_HEARTBEAT bigint NOT NULL,
+  PRIMARY KEY(MRL_TXN_ID)
+);
+
 CREATE TABLE "I_SCHEMA" (
   "SCHEMA_ID" bigint primary key,
   "SCHEMA_TYPE" integer not null,

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql b/standalone-metastore/metastore-common/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql
index 445c3a2..dadf065 100644
--- a/standalone-metastore/metastore-common/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql
+++ b/standalone-metastore/metastore-common/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql
@@ -48,6 +48,25 @@ CREATE TABLE "TXN_WRITE_NOTIFICATION_LOG" (
 );
 INSERT INTO "SEQUENCE_TABLE" ("SEQUENCE_NAME", "NEXT_VAL") VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
 
+-- HIVE-19027
+-- add column MATERIALIZATION_TIME (bigint) to MV_CREATION_METADATA table
+ALTER TABLE "MV_CREATION_METADATA" ADD COLUMN "MATERIALIZATION_TIME" bigint NULL;
+UPDATE "MV_CREATION_METADATA" SET "MATERIALIZATION_TIME" = 0;
+ALTER TABLE "MV_CREATION_METADATA" ALTER COLUMN "MATERIALIZATION_TIME" SET NOT NULL;
+
+-- add column CTC_UPDATE_DELETE (char) to COMPLETED_TXN_COMPONENTS table
+ALTER TABLE COMPLETED_TXN_COMPONENTS ADD COLUMN CTC_UPDATE_DELETE char(1) NULL;
+UPDATE COMPLETED_TXN_COMPONENTS SET CTC_UPDATE_DELETE = 'N';
+ALTER TABLE COMPLETED_TXN_COMPONENTS ALTER COLUMN CTC_UPDATE_DELETE SET NOT NULL;
+
+CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (
+  MRL_TXN_ID bigint NOT NULL,
+  MRL_DB_NAME varchar(128) NOT NULL,
+  MRL_TBL_NAME varchar(256) NOT NULL,
+  MRL_LAST_HEARTBEAT bigint NOT NULL,
+  PRIMARY KEY(MRL_TXN_ID)
+);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE "VERSION" SET "SCHEMA_VERSION"='3.1.0', "VERSION_COMMENT"='Hive release version 3.1.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 3.0.0 to 3.1.0';

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
index 1ca6454..8965059 100644
--- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
@@ -1073,6 +1073,7 @@ struct CreationMetadata {
     3: required string tblName,
     4: required set<string> tablesUsed,
     5: optional string validTxnList,
+    6: optional i64 materializationTime
 }
 
 struct NotificationEventRequest {
@@ -1273,10 +1274,7 @@ struct TableMeta {
 }
 
 struct Materialization {
-  1: required set<string> tablesUsed;
-  2: optional string validTxnList
-  3: optional i64 invalidationTime;
-  4: optional bool sourceTablesUpdateDeleteModified;
+  1: required bool sourceTablesUpdateDeleteModified;
 }
 
 // Data types for workload management.
@@ -1728,7 +1726,7 @@ service ThriftHiveMetastore extends fb303.FacebookService
   GetTableResult get_table_req(1:GetTableRequest req) throws (1:MetaException o1, 2:NoSuchObjectException o2)
   GetTablesResult get_table_objects_by_name_req(1:GetTablesRequest req)
 				   throws (1:MetaException o1, 2:InvalidOperationException o2, 3:UnknownDBException o3)
-  map<string, Materialization> get_materialization_invalidation_info(1:string dbname, 2:list<string> tbl_names)
+  Materialization get_materialization_invalidation_info(1:CreationMetadata creation_metadata, 2:string validTxnList)
 				   throws (1:MetaException o1, 2:InvalidOperationException o2, 3:UnknownDBException o3)
   void update_creation_metadata(1: string catName, 2:string dbname, 3:string tbl_name, 4:CreationMetadata creation_metadata)
                    throws (1:MetaException o1, 2:InvalidOperationException o2, 3:UnknownDBException o3)

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java b/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
index 53c4d24..d91f737 100644
--- a/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
+++ b/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
@@ -164,8 +164,6 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
       // instantiate the metastore server handler directly instead of connecting
       // through the network
       client = HiveMetaStore.newRetryingHMSHandler("hive client", this.conf, true);
-      // Initialize materializations invalidation cache (only for local metastore)
-      MaterializationsInvalidationCache.get().init(conf, (IHMSHandler) client);
       isConnected = true;
       snapshotActiveConf();
       return;
@@ -1442,10 +1440,9 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
 
   /** {@inheritDoc} */
   @Override
-  public Map<String, Materialization> getMaterializationsInvalidationInfo(String dbName, List<String> viewNames)
+  public Materialization getMaterializationInvalidationInfo(CreationMetadata cm, String validTxnList)
       throws MetaException, InvalidOperationException, UnknownDBException, TException {
-    return client.get_materialization_invalidation_info(
-        dbName, filterHook.filterTableNames(null, dbName, viewNames));
+    return client.get_materialization_invalidation_info(cm, validTxnList);
   }
 
   /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMaterializationsCacheCleaner.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMaterializationsCacheCleaner.java b/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMaterializationsCacheCleaner.java
deleted file mode 100644
index 8debcce..0000000
--- a/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMaterializationsCacheCleaner.java
+++ /dev/null
@@ -1,328 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.metastore;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.api.BasicTxnInfo;
-import org.apache.hadoop.hive.metastore.api.CreationMetadata;
-import org.apache.hadoop.hive.metastore.api.Materialization;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.junit.Assert;
-import org.junit.FixMethodOrder;
-import org.junit.Test;
-import org.junit.runners.MethodSorters;
-
-import java.util.Map;
-
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Unit tests for {@link org.apache.hadoop.hive.metastore.MaterializationsInvalidationCache}.
- * The tests focus on arrival of notifications (possibly out of order) and the logic
- * to clean up the materializations cache. Tests need to be executed in a certain order
- * to avoid interactions among them, as the invalidation cache is a singleton.
- */
-@FixMethodOrder(MethodSorters.NAME_ASCENDING)
-public class TestMetaStoreMaterializationsCacheCleaner {
-
-  private static final String DB_NAME = "hive3252";
-  private static final String TBL_NAME_1 = "tmptbl1";
-  private static final String TBL_NAME_2 = "tmptbl2";
-  private static final String TBL_NAME_3 = "tmptbl3";
-  private static final String MV_NAME_1 = "mv1";
-  private static final String MV_NAME_2 = "mv2";
-
-
-  @Test
-  public void testCleanerScenario1() throws Exception {
-    // create mock raw store
-    Configuration conf = new Configuration();
-    conf.set("metastore.materializations.invalidation.impl", "DISABLE");
-    // create mock handler
-    final IHMSHandler handler = mock(IHMSHandler.class);
-    // initialize invalidation cache (set conf to disable)
-    MaterializationsInvalidationCache.get().init(conf, handler);
-
-    // This is a dummy test, invalidation cache is not supposed to
-    // record any information.
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_1, 1, 1, false);
-    int id = 2;
-    BasicTxnInfo txn2 = createTxnInfo(DB_NAME, TBL_NAME_1, id);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_1, id, id, false);
-    // Create tbl2 (nothing to do)
-    id = 3;
-    BasicTxnInfo txn3 = createTxnInfo(DB_NAME, TBL_NAME_1, id);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_2, id, id, false);
-    // Cleanup (current = 4, duration = 4) -> Does nothing
-    long removed = MaterializationsInvalidationCache.get().cleanup(0L);
-    Assert.assertEquals(0L, removed);
-    // Create mv1
-    Table mv1 = mock(Table.class);
-    when(mv1.getDbName()).thenReturn(DB_NAME);
-    when(mv1.getTableName()).thenReturn(MV_NAME_1);
-    CreationMetadata mockCM1 = new CreationMetadata(
-        DEFAULT_CATALOG_NAME, DB_NAME, MV_NAME_1,
-        ImmutableSet.of(
-            DB_NAME + "." + TBL_NAME_1,
-            DB_NAME + "." + TBL_NAME_2));
-    // Create txn list (highWatermark=4;minOpenTxn=Long.MAX_VALUE)
-    mockCM1.setValidTxnList("3:" + Long.MAX_VALUE + "::");
-    when(mv1.getCreationMetadata()).thenReturn(mockCM1);
-    MaterializationsInvalidationCache.get().createMaterializedView(mockCM1.getDbName(), mockCM1.getTblName(),
-        mockCM1.getTablesUsed(), mockCM1.getValidTxnList());
-    // Format <txnId>$<table_name>:<hwm>:<minOpenWriteId>:<open_writeids>:<abort_writeids>$<table_name>
-    Map<String, Materialization> invalidationInfos =
-        MaterializationsInvalidationCache.get().getMaterializationInvalidationInfo(
-            DB_NAME, ImmutableList.of(MV_NAME_1));
-    Assert.assertTrue(invalidationInfos.isEmpty());
-    id = 10;
-    BasicTxnInfo txn10 = createTxnInfo(DB_NAME, TBL_NAME_2, id);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_2, id, id, false);
-    id = 9;
-    BasicTxnInfo txn9 = createTxnInfo(DB_NAME, TBL_NAME_1, id);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_1, id, id, false);
-    // Cleanup (current = 12, duration = 4) -> Removes txn1, txn2, txn3
-    removed = MaterializationsInvalidationCache.get().cleanup(8L);
-    Assert.assertEquals(0L, removed);
-    invalidationInfos =
-        MaterializationsInvalidationCache.get().getMaterializationInvalidationInfo(
-            DB_NAME, ImmutableList.of(MV_NAME_1));
-    Assert.assertTrue(invalidationInfos.isEmpty());
-    // Create mv2
-    Table mv2 = mock(Table.class);
-    when(mv2.getDbName()).thenReturn(DB_NAME);
-    when(mv2.getTableName()).thenReturn(MV_NAME_2);
-    CreationMetadata mockCM2 = new CreationMetadata(
-        DEFAULT_CATALOG_NAME, DB_NAME, MV_NAME_2,
-        ImmutableSet.of(
-            DB_NAME + "." + TBL_NAME_1,
-            DB_NAME + "." + TBL_NAME_2));
-    // Create txn list (highWatermark=10;minOpenTxn=Long.MAX_VALUE)
-    mockCM2.setValidTxnList("10:" + Long.MAX_VALUE + "::");
-    when(mv2.getCreationMetadata()).thenReturn(mockCM2);
-    MaterializationsInvalidationCache.get().createMaterializedView(mockCM2.getDbName(), mockCM2.getTblName(),
-        mockCM2.getTablesUsed(), mockCM2.getValidTxnList());
-    when(mv2.getCreationMetadata()).thenReturn(mockCM2);
-    invalidationInfos =
-        MaterializationsInvalidationCache.get().getMaterializationInvalidationInfo(
-            DB_NAME, ImmutableList.of(MV_NAME_1, MV_NAME_2));
-    Assert.assertTrue(invalidationInfos.isEmpty());
-    // Create tbl3 (nothing to do)
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_3, 11, 11, false);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_3, 18, 18, false);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_1, 14, 14, false);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_1, 17, 17, false);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_2, 16, 16, false);
-    // Cleanup (current = 20, duration = 4) -> Removes txn10, txn11
-    removed = MaterializationsInvalidationCache.get().cleanup(16L);
-    Assert.assertEquals(0L, removed);
-    invalidationInfos =
-        MaterializationsInvalidationCache.get().getMaterializationInvalidationInfo(
-            DB_NAME, ImmutableList.of(MV_NAME_1, MV_NAME_2));
-    Assert.assertTrue(invalidationInfos.isEmpty());
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_1, 12, 12, false);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_2, 15, 15, false);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_2, 7, 7, false);
-    invalidationInfos =
-        MaterializationsInvalidationCache.get().getMaterializationInvalidationInfo(
-            DB_NAME, ImmutableList.of(MV_NAME_1, MV_NAME_2));
-    Assert.assertTrue(invalidationInfos.isEmpty());
-    // Cleanup (current = 24, duration = 4) -> Removes txn9, txn14, txn15, txn16, txn17, txn18
-    removed = MaterializationsInvalidationCache.get().cleanup(20L);
-    Assert.assertEquals(0L, removed);
-    invalidationInfos =
-        MaterializationsInvalidationCache.get().getMaterializationInvalidationInfo(
-            DB_NAME, ImmutableList.of(MV_NAME_1, MV_NAME_2));
-    Assert.assertTrue(invalidationInfos.isEmpty());
-    // Cleanup (current = 28, duration = 4) -> Removes txn9
-    removed = MaterializationsInvalidationCache.get().cleanup(24L);
-    Assert.assertEquals(0L, removed);
-  }
-
-  @Test
-  public void testCleanerScenario2() throws Exception {
-    // create mock raw store
-    Configuration conf = new Configuration();
-    conf.set("metastore.materializations.invalidation.impl", "DEFAULT");
-    // create mock handler
-    final IHMSHandler handler = mock(IHMSHandler.class);
-    // initialize invalidation cache (set conf to default)
-    MaterializationsInvalidationCache.get().init(conf, handler);
-
-    // Scenario consists of the following steps:
-    // Create tbl1
-    // (t = 1) Insert row in tbl1
-    // (t = 2) Insert row in tbl1
-    // Create tbl2
-    // (t = 3) Insert row in tbl2
-    // Cleanup (current = 4, duration = 4) -> Does nothing
-    // Create mv1
-    // (t = 10) Insert row in tbl2
-    // (t = 9) Insert row in tbl1 (out of order)
-    // Cleanup (current = 12, duration = 4) -> Removes txn1, txn2, txn3
-    // Create mv2
-    // Create tbl3
-    // (t = 11) Insert row in tbl3
-    // (t = 18) Insert row in tbl3
-    // (t = 14) Insert row in tbl1
-    // (t = 17) Insert row in tbl1
-    // (t = 16) Insert row in tbl2
-    // Cleanup (current = 20, duration = 4) -> Removes txn10, txn11
-    // (t = 12) Insert row in tbl1
-    // (t = 15) Insert row in tbl2
-    // (t = 7) Insert row in tbl2
-    // Cleanup (current = 24, duration = 4) -> Removes txn9, txn14, txn15, txn16, txn17, txn18
-    // Create tbl1 (nothing to do)
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_1, 1, 1, false);
-    int id = 2;
-    BasicTxnInfo txn2 = createTxnInfo(DB_NAME, TBL_NAME_1, id);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_1, id, id, false);
-    // Create tbl2 (nothing to do)
-    id = 3;
-    BasicTxnInfo txn3 = createTxnInfo(DB_NAME, TBL_NAME_1, id);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_2, id, id, false);
-    // Cleanup (current = 4, duration = 4) -> Does nothing
-    long removed = MaterializationsInvalidationCache.get().cleanup(0L);
-    Assert.assertEquals(0L, removed);
-    // Create mv1
-    Table mv1 = mock(Table.class);
-    when(mv1.getDbName()).thenReturn(DB_NAME);
-    when(mv1.getTableName()).thenReturn(MV_NAME_1);
-    CreationMetadata mockCM1 = new CreationMetadata(
-        DEFAULT_CATALOG_NAME, DB_NAME, MV_NAME_1,
-        ImmutableSet.of(
-            DB_NAME + "." + TBL_NAME_1,
-            DB_NAME + "." + TBL_NAME_2));
-    // Create txn list (highWatermark=4;minOpenTxn=Long.MAX_VALUE)
-    mockCM1.setValidTxnList("3$" + DB_NAME + "." + TBL_NAME_1 + ":3:" + Long.MAX_VALUE + "::" +
-        "$" + DB_NAME + "." + TBL_NAME_2 + ":3:" + Long.MAX_VALUE + "::");
-    when(mv1.getCreationMetadata()).thenReturn(mockCM1);
-    MaterializationsInvalidationCache.get().createMaterializedView(mockCM1.getDbName(), mockCM1.getTblName(),
-        mockCM1.getTablesUsed(), mockCM1.getValidTxnList());
-    Map<String, Materialization> invalidationInfos =
-        MaterializationsInvalidationCache.get().getMaterializationInvalidationInfo(
-            DB_NAME, ImmutableList.of(MV_NAME_1));
-    Assert.assertEquals(0L, invalidationInfos.get(MV_NAME_1).getInvalidationTime());
-    id = 10;
-    BasicTxnInfo txn10 = createTxnInfo(DB_NAME, TBL_NAME_2, id);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_2, id, id, false);
-    id = 9;
-    BasicTxnInfo txn9 = createTxnInfo(DB_NAME, TBL_NAME_1, id);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_1, id, id, false);
-    // Cleanup (current = 12, duration = 4) -> Removes txn1, txn2, txn3
-    removed = MaterializationsInvalidationCache.get().cleanup(8L);
-    Assert.assertEquals(3L, removed);
-    invalidationInfos =
-        MaterializationsInvalidationCache.get().getMaterializationInvalidationInfo(
-            DB_NAME, ImmutableList.of(MV_NAME_1));
-    Assert.assertEquals(9L, invalidationInfos.get(MV_NAME_1).getInvalidationTime());
-    // Create mv2
-    Table mv2 = mock(Table.class);
-    when(mv2.getDbName()).thenReturn(DB_NAME);
-    when(mv2.getTableName()).thenReturn(MV_NAME_2);
-    CreationMetadata mockCM2 = new CreationMetadata(
-        DEFAULT_CATALOG_NAME, DB_NAME, MV_NAME_2,
-        ImmutableSet.of(
-            DB_NAME + "." + TBL_NAME_1,
-            DB_NAME + "." + TBL_NAME_2));
-    // Create txn list (highWatermark=10;minOpenTxn=Long.MAX_VALUE)
-    mockCM2.setValidTxnList("10$" + DB_NAME + "." + TBL_NAME_1 + ":10:" + Long.MAX_VALUE + "::" +
-        "$" + DB_NAME + "." + TBL_NAME_2 + ":10:" + Long.MAX_VALUE + "::");
-    when(mv2.getCreationMetadata()).thenReturn(mockCM2);
-    MaterializationsInvalidationCache.get().createMaterializedView(mockCM2.getDbName(), mockCM2.getTblName(),
-        mockCM2.getTablesUsed(), mockCM2.getValidTxnList());
-    when(mv2.getCreationMetadata()).thenReturn(mockCM2);
-    invalidationInfos =
-        MaterializationsInvalidationCache.get().getMaterializationInvalidationInfo(
-            DB_NAME, ImmutableList.of(MV_NAME_1, MV_NAME_2));
-    Assert.assertEquals(9L, invalidationInfos.get(MV_NAME_1).getInvalidationTime());
-    Assert.assertEquals(0L, invalidationInfos.get(MV_NAME_2).getInvalidationTime());
-    // Create tbl3 (nothing to do)
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_3, 11, 11, false);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_3, 18, 18, false);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_1, 14, 14, false);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_1, 17, 17, false);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_2, 16, 16, false);
-    // Cleanup (current = 20, duration = 4) -> Removes txn10, txn11
-    removed = MaterializationsInvalidationCache.get().cleanup(16L);
-    Assert.assertEquals(2L, removed);
-    invalidationInfos =
-        MaterializationsInvalidationCache.get().getMaterializationInvalidationInfo(
-            DB_NAME, ImmutableList.of(MV_NAME_1, MV_NAME_2));
-    Assert.assertEquals(9L, invalidationInfos.get(MV_NAME_1).getInvalidationTime());
-    Assert.assertEquals(14L, invalidationInfos.get(MV_NAME_2).getInvalidationTime());
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_1, 12, 12, false);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_2, 15, 15, false);
-    MaterializationsInvalidationCache.get().notifyTableModification(
-        DB_NAME, TBL_NAME_2, 7, 7, false);
-    invalidationInfos =
-        MaterializationsInvalidationCache.get().getMaterializationInvalidationInfo(
-            DB_NAME, ImmutableList.of(MV_NAME_1, MV_NAME_2));
-    Assert.assertEquals(7L, invalidationInfos.get(MV_NAME_1).getInvalidationTime());
-    Assert.assertEquals(12L, invalidationInfos.get(MV_NAME_2).getInvalidationTime());
-    // Cleanup (current = 24, duration = 4) -> Removes txn9, txn14, txn15, txn16, txn17, txn18
-    removed = MaterializationsInvalidationCache.get().cleanup(20L);
-    Assert.assertEquals(6L, removed);
-    invalidationInfos =
-        MaterializationsInvalidationCache.get().getMaterializationInvalidationInfo(
-            DB_NAME, ImmutableList.of(MV_NAME_1, MV_NAME_2));
-    Assert.assertEquals(7L, invalidationInfos.get(MV_NAME_1).getInvalidationTime());
-    Assert.assertEquals(12L, invalidationInfos.get(MV_NAME_2).getInvalidationTime());
-    // Cleanup (current = 28, duration = 4) -> Removes txn9
-    removed = MaterializationsInvalidationCache.get().cleanup(24L);
-    Assert.assertEquals(0L, removed);
-  }
-
-  private static BasicTxnInfo createTxnInfo(String dbName, String tableName, int i) {
-    BasicTxnInfo r = new BasicTxnInfo();
-    r.setDbname(dbName);
-    r.setTablename(tableName);
-    r.setTxnid(i);
-    r.setTime(i);
-    return r;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java b/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
index efa3e7c..816a735 100644
--- a/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
+++ b/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
@@ -1211,6 +1211,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
     // Update the metadata for the materialized view
     CreationMetadata cm = client.getTable(catName, dbName, tableNames[3]).getCreationMetadata();
     cm.addToTablesUsed(dbName + "." + tableNames[1]);
+    cm.unsetMaterializationTime();
     client.updateCreationMetadata(catName, dbName, tableNames[3], cm);
 
     List<String> partNames = new ArrayList<>();


[07/12] hive git commit: HIVE-20006: Make materializations invalidation cache work with multiple active remote metastores (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index ec129ef..24ffadb 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -128,7 +128,7 @@ import org.slf4j.LoggerFactory;
 
     public GetTablesResult get_table_objects_by_name_req(GetTablesRequest req) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException;
 
-    public Map<String,Materialization> get_materialization_invalidation_info(String dbname, List<String> tbl_names) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException;
+    public Materialization get_materialization_invalidation_info(CreationMetadata creation_metadata, String validTxnList) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException;
 
     public void update_creation_metadata(String catName, String dbname, String tbl_name, CreationMetadata creation_metadata) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException;
 
@@ -548,7 +548,7 @@ import org.slf4j.LoggerFactory;
 
     public void get_table_objects_by_name_req(GetTablesRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
-    public void get_materialization_invalidation_info(String dbname, List<String> tbl_names, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+    public void get_materialization_invalidation_info(CreationMetadata creation_metadata, String validTxnList, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void update_creation_metadata(String catName, String dbname, String tbl_name, CreationMetadata creation_metadata, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
@@ -2125,21 +2125,21 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_table_objects_by_name_req failed: unknown result");
     }
 
-    public Map<String,Materialization> get_materialization_invalidation_info(String dbname, List<String> tbl_names) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException
+    public Materialization get_materialization_invalidation_info(CreationMetadata creation_metadata, String validTxnList) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException
     {
-      send_get_materialization_invalidation_info(dbname, tbl_names);
+      send_get_materialization_invalidation_info(creation_metadata, validTxnList);
       return recv_get_materialization_invalidation_info();
     }
 
-    public void send_get_materialization_invalidation_info(String dbname, List<String> tbl_names) throws org.apache.thrift.TException
+    public void send_get_materialization_invalidation_info(CreationMetadata creation_metadata, String validTxnList) throws org.apache.thrift.TException
     {
       get_materialization_invalidation_info_args args = new get_materialization_invalidation_info_args();
-      args.setDbname(dbname);
-      args.setTbl_names(tbl_names);
+      args.setCreation_metadata(creation_metadata);
+      args.setValidTxnList(validTxnList);
       sendBase("get_materialization_invalidation_info", args);
     }
 
-    public Map<String,Materialization> recv_get_materialization_invalidation_info() throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException
+    public Materialization recv_get_materialization_invalidation_info() throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException
     {
       get_materialization_invalidation_info_result result = new get_materialization_invalidation_info_result();
       receiveBase(result, "get_materialization_invalidation_info");
@@ -8346,32 +8346,32 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void get_materialization_invalidation_info(String dbname, List<String> tbl_names, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void get_materialization_invalidation_info(CreationMetadata creation_metadata, String validTxnList, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      get_materialization_invalidation_info_call method_call = new get_materialization_invalidation_info_call(dbname, tbl_names, resultHandler, this, ___protocolFactory, ___transport);
+      get_materialization_invalidation_info_call method_call = new get_materialization_invalidation_info_call(creation_metadata, validTxnList, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_materialization_invalidation_info_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String dbname;
-      private List<String> tbl_names;
-      public get_materialization_invalidation_info_call(String dbname, List<String> tbl_names, 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 {
+      private CreationMetadata creation_metadata;
+      private String validTxnList;
+      public get_materialization_invalidation_info_call(CreationMetadata creation_metadata, String validTxnList, 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.dbname = dbname;
-        this.tbl_names = tbl_names;
+        this.creation_metadata = creation_metadata;
+        this.validTxnList = validTxnList;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
         prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_materialization_invalidation_info", org.apache.thrift.protocol.TMessageType.CALL, 0));
         get_materialization_invalidation_info_args args = new get_materialization_invalidation_info_args();
-        args.setDbname(dbname);
-        args.setTbl_names(tbl_names);
+        args.setCreation_metadata(creation_metadata);
+        args.setValidTxnList(validTxnList);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public Map<String,Materialization> getResult() throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException {
+      public Materialization getResult() throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -15395,7 +15395,7 @@ import org.slf4j.LoggerFactory;
       public get_materialization_invalidation_info_result getResult(I iface, get_materialization_invalidation_info_args args) throws org.apache.thrift.TException {
         get_materialization_invalidation_info_result result = new get_materialization_invalidation_info_result();
         try {
-          result.success = iface.get_materialization_invalidation_info(args.dbname, args.tbl_names);
+          result.success = iface.get_materialization_invalidation_info(args.creation_metadata, args.validTxnList);
         } catch (MetaException o1) {
           result.o1 = o1;
         } catch (InvalidOperationException o2) {
@@ -22471,7 +22471,7 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_materialization_invalidation_info<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_materialization_invalidation_info_args, Map<String,Materialization>> {
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_materialization_invalidation_info<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_materialization_invalidation_info_args, Materialization> {
       public get_materialization_invalidation_info() {
         super("get_materialization_invalidation_info");
       }
@@ -22480,10 +22480,10 @@ import org.slf4j.LoggerFactory;
         return new get_materialization_invalidation_info_args();
       }
 
-      public AsyncMethodCallback<Map<String,Materialization>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<Materialization> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Map<String,Materialization>>() { 
-          public void onComplete(Map<String,Materialization> o) {
+        return new AsyncMethodCallback<Materialization>() { 
+          public void onComplete(Materialization o) {
             get_materialization_invalidation_info_result result = new get_materialization_invalidation_info_result();
             result.success = o;
             try {
@@ -22533,8 +22533,8 @@ import org.slf4j.LoggerFactory;
         return false;
       }
 
-      public void start(I iface, get_materialization_invalidation_info_args args, org.apache.thrift.async.AsyncMethodCallback<Map<String,Materialization>> resultHandler) throws TException {
-        iface.get_materialization_invalidation_info(args.dbname, args.tbl_names,resultHandler);
+      public void start(I iface, get_materialization_invalidation_info_args args, org.apache.thrift.async.AsyncMethodCallback<Materialization> resultHandler) throws TException {
+        iface.get_materialization_invalidation_info(args.creation_metadata, args.validTxnList,resultHandler);
       }
     }
 
@@ -42384,13 +42384,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list952 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list952.size);
-                  String _elem953;
-                  for (int _i954 = 0; _i954 < _list952.size; ++_i954)
+                  org.apache.thrift.protocol.TList _list944 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list944.size);
+                  String _elem945;
+                  for (int _i946 = 0; _i946 < _list944.size; ++_i946)
                   {
-                    _elem953 = iprot.readString();
-                    struct.success.add(_elem953);
+                    _elem945 = iprot.readString();
+                    struct.success.add(_elem945);
                   }
                   iprot.readListEnd();
                 }
@@ -42425,9 +42425,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter955 : struct.success)
+            for (String _iter947 : struct.success)
             {
-              oprot.writeString(_iter955);
+              oprot.writeString(_iter947);
             }
             oprot.writeListEnd();
           }
@@ -42466,9 +42466,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter956 : struct.success)
+            for (String _iter948 : struct.success)
             {
-              oprot.writeString(_iter956);
+              oprot.writeString(_iter948);
             }
           }
         }
@@ -42483,13 +42483,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list957 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list957.size);
-            String _elem958;
-            for (int _i959 = 0; _i959 < _list957.size; ++_i959)
+            org.apache.thrift.protocol.TList _list949 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list949.size);
+            String _elem950;
+            for (int _i951 = 0; _i951 < _list949.size; ++_i951)
             {
-              _elem958 = iprot.readString();
-              struct.success.add(_elem958);
+              _elem950 = iprot.readString();
+              struct.success.add(_elem950);
             }
           }
           struct.setSuccessIsSet(true);
@@ -43143,13 +43143,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list960 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list960.size);
-                  String _elem961;
-                  for (int _i962 = 0; _i962 < _list960.size; ++_i962)
+                  org.apache.thrift.protocol.TList _list952 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list952.size);
+                  String _elem953;
+                  for (int _i954 = 0; _i954 < _list952.size; ++_i954)
                   {
-                    _elem961 = iprot.readString();
-                    struct.success.add(_elem961);
+                    _elem953 = iprot.readString();
+                    struct.success.add(_elem953);
                   }
                   iprot.readListEnd();
                 }
@@ -43184,9 +43184,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter963 : struct.success)
+            for (String _iter955 : struct.success)
             {
-              oprot.writeString(_iter963);
+              oprot.writeString(_iter955);
             }
             oprot.writeListEnd();
           }
@@ -43225,9 +43225,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter964 : struct.success)
+            for (String _iter956 : struct.success)
             {
-              oprot.writeString(_iter964);
+              oprot.writeString(_iter956);
             }
           }
         }
@@ -43242,13 +43242,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list965 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list965.size);
-            String _elem966;
-            for (int _i967 = 0; _i967 < _list965.size; ++_i967)
+            org.apache.thrift.protocol.TList _list957 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list957.size);
+            String _elem958;
+            for (int _i959 = 0; _i959 < _list957.size; ++_i959)
             {
-              _elem966 = iprot.readString();
-              struct.success.add(_elem966);
+              _elem958 = iprot.readString();
+              struct.success.add(_elem958);
             }
           }
           struct.setSuccessIsSet(true);
@@ -47855,16 +47855,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map968 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map968.size);
-                  String _key969;
-                  Type _val970;
-                  for (int _i971 = 0; _i971 < _map968.size; ++_i971)
+                  org.apache.thrift.protocol.TMap _map960 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map960.size);
+                  String _key961;
+                  Type _val962;
+                  for (int _i963 = 0; _i963 < _map960.size; ++_i963)
                   {
-                    _key969 = iprot.readString();
-                    _val970 = new Type();
-                    _val970.read(iprot);
-                    struct.success.put(_key969, _val970);
+                    _key961 = iprot.readString();
+                    _val962 = new Type();
+                    _val962.read(iprot);
+                    struct.success.put(_key961, _val962);
                   }
                   iprot.readMapEnd();
                 }
@@ -47899,10 +47899,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Map.Entry<String, Type> _iter972 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter964 : struct.success.entrySet())
             {
-              oprot.writeString(_iter972.getKey());
-              _iter972.getValue().write(oprot);
+              oprot.writeString(_iter964.getKey());
+              _iter964.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -47941,10 +47941,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter973 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter965 : struct.success.entrySet())
             {
-              oprot.writeString(_iter973.getKey());
-              _iter973.getValue().write(oprot);
+              oprot.writeString(_iter965.getKey());
+              _iter965.getValue().write(oprot);
             }
           }
         }
@@ -47959,16 +47959,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map974 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new HashMap<String,Type>(2*_map974.size);
-            String _key975;
-            Type _val976;
-            for (int _i977 = 0; _i977 < _map974.size; ++_i977)
+            org.apache.thrift.protocol.TMap _map966 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new HashMap<String,Type>(2*_map966.size);
+            String _key967;
+            Type _val968;
+            for (int _i969 = 0; _i969 < _map966.size; ++_i969)
             {
-              _key975 = iprot.readString();
-              _val976 = new Type();
-              _val976.read(iprot);
-              struct.success.put(_key975, _val976);
+              _key967 = iprot.readString();
+              _val968 = new Type();
+              _val968.read(iprot);
+              struct.success.put(_key967, _val968);
             }
           }
           struct.setSuccessIsSet(true);
@@ -49003,14 +49003,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list978 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list978.size);
-                  FieldSchema _elem979;
-                  for (int _i980 = 0; _i980 < _list978.size; ++_i980)
+                  org.apache.thrift.protocol.TList _list970 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list970.size);
+                  FieldSchema _elem971;
+                  for (int _i972 = 0; _i972 < _list970.size; ++_i972)
                   {
-                    _elem979 = new FieldSchema();
-                    _elem979.read(iprot);
-                    struct.success.add(_elem979);
+                    _elem971 = new FieldSchema();
+                    _elem971.read(iprot);
+                    struct.success.add(_elem971);
                   }
                   iprot.readListEnd();
                 }
@@ -49063,9 +49063,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter981 : struct.success)
+            for (FieldSchema _iter973 : struct.success)
             {
-              _iter981.write(oprot);
+              _iter973.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -49120,9 +49120,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter982 : struct.success)
+            for (FieldSchema _iter974 : struct.success)
             {
-              _iter982.write(oprot);
+              _iter974.write(oprot);
             }
           }
         }
@@ -49143,14 +49143,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list983 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list983.size);
-            FieldSchema _elem984;
-            for (int _i985 = 0; _i985 < _list983.size; ++_i985)
+            org.apache.thrift.protocol.TList _list975 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list975.size);
+            FieldSchema _elem976;
+            for (int _i977 = 0; _i977 < _list975.size; ++_i977)
             {
-              _elem984 = new FieldSchema();
-              _elem984.read(iprot);
-              struct.success.add(_elem984);
+              _elem976 = new FieldSchema();
+              _elem976.read(iprot);
+              struct.success.add(_elem976);
             }
           }
           struct.setSuccessIsSet(true);
@@ -50304,14 +50304,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list986 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list986.size);
-                  FieldSchema _elem987;
-                  for (int _i988 = 0; _i988 < _list986.size; ++_i988)
+                  org.apache.thrift.protocol.TList _list978 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list978.size);
+                  FieldSchema _elem979;
+                  for (int _i980 = 0; _i980 < _list978.size; ++_i980)
                   {
-                    _elem987 = new FieldSchema();
-                    _elem987.read(iprot);
-                    struct.success.add(_elem987);
+                    _elem979 = new FieldSchema();
+                    _elem979.read(iprot);
+                    struct.success.add(_elem979);
                   }
                   iprot.readListEnd();
                 }
@@ -50364,9 +50364,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter989 : struct.success)
+            for (FieldSchema _iter981 : struct.success)
             {
-              _iter989.write(oprot);
+              _iter981.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -50421,9 +50421,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter990 : struct.success)
+            for (FieldSchema _iter982 : struct.success)
             {
-              _iter990.write(oprot);
+              _iter982.write(oprot);
             }
           }
         }
@@ -50444,14 +50444,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list991 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list991.size);
-            FieldSchema _elem992;
-            for (int _i993 = 0; _i993 < _list991.size; ++_i993)
+            org.apache.thrift.protocol.TList _list983 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list983.size);
+            FieldSchema _elem984;
+            for (int _i985 = 0; _i985 < _list983.size; ++_i985)
             {
-              _elem992 = new FieldSchema();
-              _elem992.read(iprot);
-              struct.success.add(_elem992);
+              _elem984 = new FieldSchema();
+              _elem984.read(iprot);
+              struct.success.add(_elem984);
             }
           }
           struct.setSuccessIsSet(true);
@@ -51496,14 +51496,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list994 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list994.size);
-                  FieldSchema _elem995;
-                  for (int _i996 = 0; _i996 < _list994.size; ++_i996)
+                  org.apache.thrift.protocol.TList _list986 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list986.size);
+                  FieldSchema _elem987;
+                  for (int _i988 = 0; _i988 < _list986.size; ++_i988)
                   {
-                    _elem995 = new FieldSchema();
-                    _elem995.read(iprot);
-                    struct.success.add(_elem995);
+                    _elem987 = new FieldSchema();
+                    _elem987.read(iprot);
+                    struct.success.add(_elem987);
                   }
                   iprot.readListEnd();
                 }
@@ -51556,9 +51556,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter997 : struct.success)
+            for (FieldSchema _iter989 : struct.success)
             {
-              _iter997.write(oprot);
+              _iter989.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -51613,9 +51613,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter998 : struct.success)
+            for (FieldSchema _iter990 : struct.success)
             {
-              _iter998.write(oprot);
+              _iter990.write(oprot);
             }
           }
         }
@@ -51636,14 +51636,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list999 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list999.size);
-            FieldSchema _elem1000;
-            for (int _i1001 = 0; _i1001 < _list999.size; ++_i1001)
+            org.apache.thrift.protocol.TList _list991 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list991.size);
+            FieldSchema _elem992;
+            for (int _i993 = 0; _i993 < _list991.size; ++_i993)
             {
-              _elem1000 = new FieldSchema();
-              _elem1000.read(iprot);
-              struct.success.add(_elem1000);
+              _elem992 = new FieldSchema();
+              _elem992.read(iprot);
+              struct.success.add(_elem992);
             }
           }
           struct.setSuccessIsSet(true);
@@ -52797,14 +52797,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1002 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1002.size);
-                  FieldSchema _elem1003;
-                  for (int _i1004 = 0; _i1004 < _list1002.size; ++_i1004)
+                  org.apache.thrift.protocol.TList _list994 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list994.size);
+                  FieldSchema _elem995;
+                  for (int _i996 = 0; _i996 < _list994.size; ++_i996)
                   {
-                    _elem1003 = new FieldSchema();
-                    _elem1003.read(iprot);
-                    struct.success.add(_elem1003);
+                    _elem995 = new FieldSchema();
+                    _elem995.read(iprot);
+                    struct.success.add(_elem995);
                   }
                   iprot.readListEnd();
                 }
@@ -52857,9 +52857,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter1005 : struct.success)
+            for (FieldSchema _iter997 : struct.success)
             {
-              _iter1005.write(oprot);
+              _iter997.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -52914,9 +52914,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1006 : struct.success)
+            for (FieldSchema _iter998 : struct.success)
             {
-              _iter1006.write(oprot);
+              _iter998.write(oprot);
             }
           }
         }
@@ -52937,14 +52937,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1007 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1007.size);
-            FieldSchema _elem1008;
-            for (int _i1009 = 0; _i1009 < _list1007.size; ++_i1009)
+            org.apache.thrift.protocol.TList _list999 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list999.size);
+            FieldSchema _elem1000;
+            for (int _i1001 = 0; _i1001 < _list999.size; ++_i1001)
             {
-              _elem1008 = new FieldSchema();
-              _elem1008.read(iprot);
-              struct.success.add(_elem1008);
+              _elem1000 = new FieldSchema();
+              _elem1000.read(iprot);
+              struct.success.add(_elem1000);
             }
           }
           struct.setSuccessIsSet(true);
@@ -56073,14 +56073,14 @@ import org.slf4j.LoggerFactory;
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1010 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1010.size);
-                  SQLPrimaryKey _elem1011;
-                  for (int _i1012 = 0; _i1012 < _list1010.size; ++_i1012)
+                  org.apache.thrift.protocol.TList _list1002 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1002.size);
+                  SQLPrimaryKey _elem1003;
+                  for (int _i1004 = 0; _i1004 < _list1002.size; ++_i1004)
                   {
-                    _elem1011 = new SQLPrimaryKey();
-                    _elem1011.read(iprot);
-                    struct.primaryKeys.add(_elem1011);
+                    _elem1003 = new SQLPrimaryKey();
+                    _elem1003.read(iprot);
+                    struct.primaryKeys.add(_elem1003);
                   }
                   iprot.readListEnd();
                 }
@@ -56092,14 +56092,14 @@ import org.slf4j.LoggerFactory;
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1013 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1013.size);
-                  SQLForeignKey _elem1014;
-                  for (int _i1015 = 0; _i1015 < _list1013.size; ++_i1015)
+                  org.apache.thrift.protocol.TList _list1005 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1005.size);
+                  SQLForeignKey _elem1006;
+                  for (int _i1007 = 0; _i1007 < _list1005.size; ++_i1007)
                   {
-                    _elem1014 = new SQLForeignKey();
-                    _elem1014.read(iprot);
-                    struct.foreignKeys.add(_elem1014);
+                    _elem1006 = new SQLForeignKey();
+                    _elem1006.read(iprot);
+                    struct.foreignKeys.add(_elem1006);
                   }
                   iprot.readListEnd();
                 }
@@ -56111,14 +56111,14 @@ import org.slf4j.LoggerFactory;
             case 4: // UNIQUE_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1016 = iprot.readListBegin();
-                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1016.size);
-                  SQLUniqueConstraint _elem1017;
-                  for (int _i1018 = 0; _i1018 < _list1016.size; ++_i1018)
+                  org.apache.thrift.protocol.TList _list1008 = iprot.readListBegin();
+                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1008.size);
+                  SQLUniqueConstraint _elem1009;
+                  for (int _i1010 = 0; _i1010 < _list1008.size; ++_i1010)
                   {
-                    _elem1017 = new SQLUniqueConstraint();
-                    _elem1017.read(iprot);
-                    struct.uniqueConstraints.add(_elem1017);
+                    _elem1009 = new SQLUniqueConstraint();
+                    _elem1009.read(iprot);
+                    struct.uniqueConstraints.add(_elem1009);
                   }
                   iprot.readListEnd();
                 }
@@ -56130,14 +56130,14 @@ import org.slf4j.LoggerFactory;
             case 5: // NOT_NULL_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1019 = iprot.readListBegin();
-                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1019.size);
-                  SQLNotNullConstraint _elem1020;
-                  for (int _i1021 = 0; _i1021 < _list1019.size; ++_i1021)
+                  org.apache.thrift.protocol.TList _list1011 = iprot.readListBegin();
+                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1011.size);
+                  SQLNotNullConstraint _elem1012;
+                  for (int _i1013 = 0; _i1013 < _list1011.size; ++_i1013)
                   {
-                    _elem1020 = new SQLNotNullConstraint();
-                    _elem1020.read(iprot);
-                    struct.notNullConstraints.add(_elem1020);
+                    _elem1012 = new SQLNotNullConstraint();
+                    _elem1012.read(iprot);
+                    struct.notNullConstraints.add(_elem1012);
                   }
                   iprot.readListEnd();
                 }
@@ -56149,14 +56149,14 @@ import org.slf4j.LoggerFactory;
             case 6: // DEFAULT_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1022 = iprot.readListBegin();
-                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1022.size);
-                  SQLDefaultConstraint _elem1023;
-                  for (int _i1024 = 0; _i1024 < _list1022.size; ++_i1024)
+                  org.apache.thrift.protocol.TList _list1014 = iprot.readListBegin();
+                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1014.size);
+                  SQLDefaultConstraint _elem1015;
+                  for (int _i1016 = 0; _i1016 < _list1014.size; ++_i1016)
                   {
-                    _elem1023 = new SQLDefaultConstraint();
-                    _elem1023.read(iprot);
-                    struct.defaultConstraints.add(_elem1023);
+                    _elem1015 = new SQLDefaultConstraint();
+                    _elem1015.read(iprot);
+                    struct.defaultConstraints.add(_elem1015);
                   }
                   iprot.readListEnd();
                 }
@@ -56168,14 +56168,14 @@ import org.slf4j.LoggerFactory;
             case 7: // CHECK_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1025 = iprot.readListBegin();
-                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1025.size);
-                  SQLCheckConstraint _elem1026;
-                  for (int _i1027 = 0; _i1027 < _list1025.size; ++_i1027)
+                  org.apache.thrift.protocol.TList _list1017 = iprot.readListBegin();
+                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1017.size);
+                  SQLCheckConstraint _elem1018;
+                  for (int _i1019 = 0; _i1019 < _list1017.size; ++_i1019)
                   {
-                    _elem1026 = new SQLCheckConstraint();
-                    _elem1026.read(iprot);
-                    struct.checkConstraints.add(_elem1026);
+                    _elem1018 = new SQLCheckConstraint();
+                    _elem1018.read(iprot);
+                    struct.checkConstraints.add(_elem1018);
                   }
                   iprot.readListEnd();
                 }
@@ -56206,9 +56206,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size()));
-            for (SQLPrimaryKey _iter1028 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1020 : struct.primaryKeys)
             {
-              _iter1028.write(oprot);
+              _iter1020.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56218,9 +56218,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size()));
-            for (SQLForeignKey _iter1029 : struct.foreignKeys)
+            for (SQLForeignKey _iter1021 : struct.foreignKeys)
             {
-              _iter1029.write(oprot);
+              _iter1021.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56230,9 +56230,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(UNIQUE_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraints.size()));
-            for (SQLUniqueConstraint _iter1030 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1022 : struct.uniqueConstraints)
             {
-              _iter1030.write(oprot);
+              _iter1022.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56242,9 +56242,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraints.size()));
-            for (SQLNotNullConstraint _iter1031 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1023 : struct.notNullConstraints)
             {
-              _iter1031.write(oprot);
+              _iter1023.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56254,9 +56254,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(DEFAULT_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraints.size()));
-            for (SQLDefaultConstraint _iter1032 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1024 : struct.defaultConstraints)
             {
-              _iter1032.write(oprot);
+              _iter1024.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56266,9 +56266,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(CHECK_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.checkConstraints.size()));
-            for (SQLCheckConstraint _iter1033 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1025 : struct.checkConstraints)
             {
-              _iter1033.write(oprot);
+              _iter1025.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56320,54 +56320,54 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter1034 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1026 : struct.primaryKeys)
             {
-              _iter1034.write(oprot);
+              _iter1026.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter1035 : struct.foreignKeys)
+            for (SQLForeignKey _iter1027 : struct.foreignKeys)
             {
-              _iter1035.write(oprot);
+              _iter1027.write(oprot);
             }
           }
         }
         if (struct.isSetUniqueConstraints()) {
           {
             oprot.writeI32(struct.uniqueConstraints.size());
-            for (SQLUniqueConstraint _iter1036 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1028 : struct.uniqueConstraints)
             {
-              _iter1036.write(oprot);
+              _iter1028.write(oprot);
             }
           }
         }
         if (struct.isSetNotNullConstraints()) {
           {
             oprot.writeI32(struct.notNullConstraints.size());
-            for (SQLNotNullConstraint _iter1037 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1029 : struct.notNullConstraints)
             {
-              _iter1037.write(oprot);
+              _iter1029.write(oprot);
             }
           }
         }
         if (struct.isSetDefaultConstraints()) {
           {
             oprot.writeI32(struct.defaultConstraints.size());
-            for (SQLDefaultConstraint _iter1038 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1030 : struct.defaultConstraints)
             {
-              _iter1038.write(oprot);
+              _iter1030.write(oprot);
             }
           }
         }
         if (struct.isSetCheckConstraints()) {
           {
             oprot.writeI32(struct.checkConstraints.size());
-            for (SQLCheckConstraint _iter1039 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1031 : struct.checkConstraints)
             {
-              _iter1039.write(oprot);
+              _iter1031.write(oprot);
             }
           }
         }
@@ -56384,84 +56384,84 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1040 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1040.size);
-            SQLPrimaryKey _elem1041;
-            for (int _i1042 = 0; _i1042 < _list1040.size; ++_i1042)
+            org.apache.thrift.protocol.TList _list1032 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1032.size);
+            SQLPrimaryKey _elem1033;
+            for (int _i1034 = 0; _i1034 < _list1032.size; ++_i1034)
             {
-              _elem1041 = new SQLPrimaryKey();
-              _elem1041.read(iprot);
-              struct.primaryKeys.add(_elem1041);
+              _elem1033 = new SQLPrimaryKey();
+              _elem1033.read(iprot);
+              struct.primaryKeys.add(_elem1033);
             }
           }
           struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1043 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1043.size);
-            SQLForeignKey _elem1044;
-            for (int _i1045 = 0; _i1045 < _list1043.size; ++_i1045)
+            org.apache.thrift.protocol.TList _list1035 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1035.size);
+            SQLForeignKey _elem1036;
+            for (int _i1037 = 0; _i1037 < _list1035.size; ++_i1037)
             {
-              _elem1044 = new SQLForeignKey();
-              _elem1044.read(iprot);
-              struct.foreignKeys.add(_elem1044);
+              _elem1036 = new SQLForeignKey();
+              _elem1036.read(iprot);
+              struct.foreignKeys.add(_elem1036);
             }
           }
           struct.setForeignKeysIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list1046 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1046.size);
-            SQLUniqueConstraint _elem1047;
-            for (int _i1048 = 0; _i1048 < _list1046.size; ++_i1048)
+            org.apache.thrift.protocol.TList _list1038 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1038.size);
+            SQLUniqueConstraint _elem1039;
+            for (int _i1040 = 0; _i1040 < _list1038.size; ++_i1040)
             {
-              _elem1047 = new SQLUniqueConstraint();
-              _elem1047.read(iprot);
-              struct.uniqueConstraints.add(_elem1047);
+              _elem1039 = new SQLUniqueConstraint();
+              _elem1039.read(iprot);
+              struct.uniqueConstraints.add(_elem1039);
             }
           }
           struct.setUniqueConstraintsIsSet(true);
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1049 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1049.size);
-            SQLNotNullConstraint _elem1050;
-            for (int _i1051 = 0; _i1051 < _list1049.size; ++_i1051)
+            org.apache.thrift.protocol.TList _list1041 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1041.size);
+            SQLNotNullConstraint _elem1042;
+            for (int _i1043 = 0; _i1043 < _list1041.size; ++_i1043)
             {
-              _elem1050 = new SQLNotNullConstraint();
-              _elem1050.read(iprot);
-              struct.notNullConstraints.add(_elem1050);
+              _elem1042 = new SQLNotNullConstraint();
+              _elem1042.read(iprot);
+              struct.notNullConstraints.add(_elem1042);
             }
           }
           struct.setNotNullConstraintsIsSet(true);
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TList _list1052 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1052.size);
-            SQLDefaultConstraint _elem1053;
-            for (int _i1054 = 0; _i1054 < _list1052.size; ++_i1054)
+            org.apache.thrift.protocol.TList _list1044 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1044.size);
+            SQLDefaultConstraint _elem1045;
+            for (int _i1046 = 0; _i1046 < _list1044.size; ++_i1046)
             {
-              _elem1053 = new SQLDefaultConstraint();
-              _elem1053.read(iprot);
-              struct.defaultConstraints.add(_elem1053);
+              _elem1045 = new SQLDefaultConstraint();
+              _elem1045.read(iprot);
+              struct.defaultConstraints.add(_elem1045);
             }
           }
           struct.setDefaultConstraintsIsSet(true);
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list1055 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1055.size);
-            SQLCheckConstraint _elem1056;
-            for (int _i1057 = 0; _i1057 < _list1055.size; ++_i1057)
+            org.apache.thrift.protocol.TList _list1047 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1047.size);
+            SQLCheckConstraint _elem1048;
+            for (int _i1049 = 0; _i1049 < _list1047.size; ++_i1049)
             {
-              _elem1056 = new SQLCheckConstraint();
-              _elem1056.read(iprot);
-              struct.checkConstraints.add(_elem1056);
+              _elem1048 = new SQLCheckConstraint();
+              _elem1048.read(iprot);
+              struct.checkConstraints.add(_elem1048);
             }
           }
           struct.setCheckConstraintsIsSet(true);
@@ -65611,13 +65611,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin();
-                  struct.partNames = new ArrayList<String>(_list1058.size);
-                  String _elem1059;
-                  for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060)
+                  org.apache.thrift.protocol.TList _list1050 = iprot.readListBegin();
+                  struct.partNames = new ArrayList<String>(_list1050.size);
+                  String _elem1051;
+                  for (int _i1052 = 0; _i1052 < _list1050.size; ++_i1052)
                   {
-                    _elem1059 = iprot.readString();
-                    struct.partNames.add(_elem1059);
+                    _elem1051 = iprot.readString();
+                    struct.partNames.add(_elem1051);
                   }
                   iprot.readListEnd();
                 }
@@ -65653,9 +65653,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-            for (String _iter1061 : struct.partNames)
+            for (String _iter1053 : struct.partNames)
             {
-              oprot.writeString(_iter1061);
+              oprot.writeString(_iter1053);
             }
             oprot.writeListEnd();
           }
@@ -65698,9 +65698,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPartNames()) {
           {
             oprot.writeI32(struct.partNames.size());
-            for (String _iter1062 : struct.partNames)
+            for (String _iter1054 : struct.partNames)
             {
-              oprot.writeString(_iter1062);
+              oprot.writeString(_iter1054);
             }
           }
         }
@@ -65720,13 +65720,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1063 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partNames = new ArrayList<String>(_list1063.size);
-            String _elem1064;
-            for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065)
+            org.apache.thrift.protocol.TList _list1055 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partNames = new ArrayList<String>(_list1055.size);
+            String _elem1056;
+            for (int _i1057 = 0; _i1057 < _list1055.size; ++_i1057)
             {
-              _elem1064 = iprot.readString();
-              struct.partNames.add(_elem1064);
+              _elem1056 = iprot.readString();
+              struct.partNames.add(_elem1056);
             }
           }
           struct.setPartNamesIsSet(true);
@@ -66951,13 +66951,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1066 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1066.size);
-                  String _elem1067;
-                  for (int _i1068 = 0; _i1068 < _list1066.size; ++_i1068)
+                  org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1058.size);
+                  String _elem1059;
+                  for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060)
                   {
-                    _elem1067 = iprot.readString();
-                    struct.success.add(_elem1067);
+                    _elem1059 = iprot.readString();
+                    struct.success.add(_elem1059);
                   }
                   iprot.readListEnd();
                 }
@@ -66992,9 +66992,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1069 : struct.success)
+            for (String _iter1061 : struct.success)
             {
-              oprot.writeString(_iter1069);
+              oprot.writeString(_iter1061);
             }
             oprot.writeListEnd();
           }
@@ -67033,9 +67033,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1070 : struct.success)
+            for (String _iter1062 : struct.success)
             {
-              oprot.writeString(_iter1070);
+              oprot.writeString(_iter1062);
             }
           }
         }
@@ -67050,13 +67050,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1071 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1071.size);
-            String _elem1072;
-            for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073)
+            org.apache.thrift.protocol.TList _list1063 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1063.size);
+            String _elem1064;
+            for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065)
             {
-              _elem1072 = iprot.readString();
-              struct.success.add(_elem1072);
+              _elem1064 = iprot.readString();
+              struct.success.add(_elem1064);
             }
           }
           struct.setSuccessIsSet(true);
@@ -68030,13 +68030,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1074.size);
-                  String _elem1075;
-                  for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076)
+                  org.apache.thrift.protocol.TList _list1066 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1066.size);
+                  String _elem1067;
+                  for (int _i1068 = 0; _i1068 < _list1066.size; ++_i1068)
                   {
-                    _elem1075 = iprot.readString();
-                    struct.success.add(_elem1075);
+                    _elem1067 = iprot.readString();
+                    struct.success.add(_elem1067);
                   }
                   iprot.readListEnd();
                 }
@@ -68071,9 +68071,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1077 : struct.success)
+            for (String _iter1069 : struct.success)
             {
-              oprot.writeString(_iter1077);
+              oprot.writeString(_iter1069);
             }
             oprot.writeListEnd();
           }
@@ -68112,9 +68112,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1078 : struct.success)
+            for (String _iter1070 : struct.success)
             {
-              oprot.writeString(_iter1078);
+              oprot.writeString(_iter1070);
             }
           }
         }
@@ -68129,13 +68129,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1079 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1079.size);
-            String _elem1080;
-            for (int _i1081 = 0; _i1081 < _list1079.size; ++_i1081)
+            org.apache.thrift.protocol.TList _list1071 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1071.size);
+            String _elem1072;
+            for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073)
             {
-              _elem1080 = iprot.readString();
-              struct.success.add(_elem1080);
+              _elem1072 = iprot.readString();
+              struct.success.add(_elem1072);
             }
           }
           struct.setSuccessIsSet(true);
@@ -68901,13 +68901,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1082 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1082.size);
-                  String _elem1083;
-                  for (int _i1084 = 0; _i1084 < _list1082.size; ++_i1084)
+                  org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1074.size);
+                  String _elem1075;
+                  for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076)
                   {
-                    _elem1083 = iprot.readString();
-                    struct.success.add(_elem1083);
+                    _elem1075 = iprot.readString();
+                    struct.success.add(_elem1075);
                   }
                   iprot.readListEnd();
                 }
@@ -68942,9 +68942,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1085 : struct.success)
+            for (String _iter1077 : struct.success)
             {
-              oprot.writeString(_iter1085);
+              oprot.writeString(_iter1077);
             }
             oprot.writeListEnd();
           }
@@ -68983,9 +68983,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1086 : struct.success)
+            for (String _iter1078 : struct.success)
             {
-              oprot.writeString(_iter1086);
+              oprot.writeString(_iter1078);
             }
           }
         }
@@ -69000,13 +69000,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1087 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1087.size);
-            String _elem1088;
-            for (int _i1089 = 0; _i1089 < _list1087.size; ++_i1089)
+            org.apache.thrift.protocol.TList _list1079 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1079.size);
+            String _elem1080;
+            for (int _i1081 = 0; _i1081 < _list1079.size; ++_i1081)
             {
-              _elem1088 = iprot.readString();
-              struct.success.add(_elem1088);
+              _elem1080 = iprot.readString();
+              struct.success.add(_elem1080);
             }
           }
           struct.setSuccessIsSet(true);
@@ -69511,13 +69511,13 @@ import org.slf4j.LoggerFactory;
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1090 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list1090.size);
-                  String _elem1091;
-                  for (int _i1092 = 0; _i1092 < _list1090.size; ++_i1092)
+                  org.apache.thrift.protocol.TList _list1082 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list1082.size);
+                  String _elem1083;
+                  for (int _i1084 = 0; _i1084 < _list1082.size; ++_i1084)
                   {
-                    _elem1091 = iprot.readString();
-                    struct.tbl_types.add(_elem1091);
+                    _elem1083 = iprot.readString();
+                    struct.tbl_types.add(_elem1083);
                   }
                   iprot.readListEnd();
                 }
@@ -69553,9 +69553,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_TYPES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_types.size()));
-            for (String _iter1093 : struct.tbl_types)
+            for (String _iter1085 : struct.tbl_types)
             {
-              oprot.writeString(_iter1093);
+              oprot.writeString(_iter1085);
             }
             oprot.writeListEnd();
           }
@@ -69598,9 +69598,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter1094 : struct.tbl_types)
+            for (String _iter1086 : struct.tbl_types)
             {
-              oprot.writeString(_iter1094);
+              oprot.writeString(_iter1086);
             }
           }
         }
@@ -69620,13 +69620,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1095 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list1095.size);
-            String _elem1096;
-            for (int _i1097 = 0; _i1097 < _list1095.size; ++_i1097)
+            org.apache.thrift.protocol.TList _list1087 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list1087.size);
+            String _elem1088;
+            for (int _i1089 = 0; _i1089 < _list1087.size; ++_i1089)
             {
-              _elem1096 = iprot.readString();
-              struct.tbl_types.add(_elem1096);
+              _elem1088 = iprot.readString();
+              struct.tbl_types.add(_elem1088);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -70032,14 +70032,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1098 = iprot.readListBegin();
-                  struct.success = new ArrayList<TableMeta>(_list1098.size);
-                  TableMeta _elem1099;
-                  for (int _i1100 = 0; _i1100 < _list1098.size; ++_i1100)
+                  org.apache.thrift.protocol.TList _list1090 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list1090.size);
+                  TableMeta _elem1091;
+                  for (int _i1092 = 0; _i1092 < _list1090.size; ++_i1092)
                   {
-                    _elem1099 = new TableMeta();
-                    _elem1099.read(iprot);
-                    struct.success.add(_elem1099);
+                    _elem1091 = new TableMeta();
+                    _elem1091.read(iprot);
+                    struct.success.add(_elem1091);
                   }
                   iprot.readListEnd();
                 }
@@ -70074,9 +70074,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TableMeta _iter1101 : struct.success)
+            for (TableMeta _iter1093 : struct.success)
             {
-              _iter1101.write(oprot);
+              _iter1093.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -70115,9 +70115,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter1102 : struct.success)
+            for (TableMeta _iter1094 : struct.success)
             {
-              _iter1102.write(oprot);
+              _iter1094.write(oprot);
             }
           }
         }
@@ -70132,14 +70132,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1103 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TableMeta>(_list1103.size);
-            TableMeta _elem1104;
-            for (int _i1105 = 0; _i1105 < _list1103.size; ++_i1105)
+            org.apache.thrift.protocol.TList _list1095 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list1095.size);
+            TableMeta _elem1096;
+            for (int _i1097 = 0; _i1097 < _list1095.size; ++_i1097)
             {
-              _elem1104 = new TableMeta();
-              _elem1104.read(iprot);
-              struct.success.add(_elem1104);
+              _elem1096 = new TableMeta();
+              _elem1096.read(iprot);
+              struct.success.add(_elem1096);
             }
           }
           struct.setSuccessIsSet(true);
@@ -70905,13 +70905,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1106 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1106.size);
-                  String _elem1107;
-                  for (int _i1108 = 0; _i1108 < _list1106.size; ++_i1108)
+                  org.apache.thrift.protocol.TList _list1098 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1098.size);
+                  String _elem1099;
+                  for (int _i1100 = 0; _i1100 < _list1098.size; ++_i1100)
                   {
-                    _elem1107 = iprot.readString();
-                    struct.success.add(_elem1107);
+                    _elem1099 = iprot.readString();
+                    struct.success.add(_elem1099);
                   }
                   iprot.readListEnd();
                 }
@@ -70946,9 +70946,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1109 : struct.success)
+            for (String _iter1101 : struct.success)
             {
-              oprot.writeString(_iter1109);
+              oprot.writeString(_iter1101);
             }
             oprot.writeListEnd();
           }
@@ -70987,9 +70987,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1110 : struct.success)
+            for (String _iter1102 : struct.success)
             {
-              oprot.writeString(_iter1110);
+              oprot.writeString(_iter1102);
             }
           }
         }
@@ -71004,13 +71004,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1111.size);
-            String _elem1112;
-            for (int _i1113 = 0; _i1113 < _list1111.size; ++_i1113)
+            org.apache.thrift.protocol.TList _list1103 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1103.size);
+            String _elem1104;
+            for (int _i1105 = 0; _i1105 < _list1103.size; ++_i1105)
             {
-              _elem1112 = iprot.readString();
-              struct.success.add(_elem1112);
+              _elem1104 = iprot.readString();
+              struct.success.add(_elem1104);
             }
           }
           struct.setSuccessIsSet(true);
@@ -72463,13 +72463,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1114 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list1114.size);
-                  String _elem1115;
-                  for (int _i1116 = 0; _i1116 < _list1114.size; ++_i1116)
+                  org.apache.thrift.protocol.TList _list1106 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list1106.size);
+                  String _elem1107;
+                  for (int _i1108 = 0; _i1108 < _list1106.size; ++_i1108)
                   {
-                    _elem1115 = iprot.readString();
-                    struct.tbl_names.add(_elem1115);
+                    _elem1107 = iprot.readString();
+                    struct.tbl_names.add(_elem1107);
                   }
                   iprot.readListEnd();
                 }
@@ -72500,9 +72500,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter1117 : struct.tbl_names)
+            for (String _iter1109 : struct.tbl_names)
             {
-              oprot.writeString(_iter1117);
+              oprot.writeString(_iter1109);
             }
             oprot.writeListEnd();
           }
@@ -72539,9 +72539,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter1118 : struct.tbl_names)
+            for (String _iter1110 : struct.tbl_names)
             {
-              oprot.writeString(_iter1118);
+              oprot.writeString(_iter1110);
             }
           }
         }
@@ -72557,13 +72557,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1119 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list1119.size);
-            String _elem1120;
-            for (int _i1121 = 0; _i1121 < _list1119.size; ++_i1121)
+            org.apache.thrift.protocol.TList _list1111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list1111.size);
+            String _elem1112;
+            for (int _i1113 = 0; _i1113 < _list1111.size; ++_i1113)
             {
-              _elem1120 = iprot.readString();
-              struct.tbl_names.add(_elem1120);
+              _elem1112 = iprot.readString();
+              struct.tbl_names.add(_elem1112);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -72888,14 +72888,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1122 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list1122.size);
-                  Table _elem1123;
-                  for (int _i1124 = 0; _i1124 < _list1122.size; ++_i1124)
+                  org.apache.thrift.protocol.TList _list1114 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list1114.size);
+                  Table _elem1115;
+                  for (int _i1116 = 0; _i1116 < _list1114.size; ++_i1116)
                   {
-                    _elem1123 = new Table();
-                    _elem1123.read(iprot);
-                    struct.success.add(_elem1123);
+                    _elem1115 = new Table();
+                    _elem1115.read(iprot);
+                    struct.success.add(_elem1115);
                   }
                   iprot.readListEnd();
                 }
@@ -72921,9 +72921,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter1125 : struct.success)
+            for (Table _iter1117 : struct.success)
             {
-              _iter1125.write(oprot);
+              _iter1117.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -72954,9 +72954,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter1126 : struct.success)
+            for (Table _iter1118 : struct.success)
             {
-              _iter1126.write(oprot);
+              _iter1118.write(oprot);
             }
           }
         }
@@ -72968,14 +72968,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1127 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list1127.size);
-            Table _elem1128;
-            for (int _i1129 = 0; _i1129 < _list1127.size; ++_i1129)
+            org.apache.thrift.protocol.TList _list1119 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list1119.size);
+            Table _elem1120;
+            for (int _i1121 = 0; _i1121 < _list1119.size; ++_i1121)
             {
-              _elem1128 = new Table();
-              _elem1128.read(iprot);
-              struct.success.add(_elem1128);
+              _elem1120 = new Table();
+              _elem1120.read(iprot);
+              struct.success.add(_elem1120);
             }
           }
           struct.setSuccessIsSet(true);
@@ -74970,8 +74970,8 @@ import org.slf4j.LoggerFactory;
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_materialization_invalidation_info_args implements org.apache.thrift.TBase<get_materialization_invalidation_info_args, get_materialization_invalidation_info_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_materialization_invalidation_info_args>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_materialization_invalidation_info_args");
 
-    private static final org.apache.thrift.protocol.TField DBNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbname", org.apache.thrift.protocol.TType.STRING, (short)1);
-    private static final org.apache.thrift.protocol.TField TBL_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_names", org.apache.thrift.protocol.TType.LIST, (short)2);
+    private static final org.apache.thrift.protocol.TField CREATION_METADATA_FIELD_DESC = new org.apache.thrift.protocol.TField("creation_metadata", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField VALID_TXN_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validTxnList", org.apache.thrift.protocol.TType.STRING, (short)2);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
@@ -74979,13 +74979,13 @@ import org.slf4j.LoggerFactory;
       schemes.put(TupleScheme.class, new get_materialization_invalidation_info_argsTupleSchemeFactory());
     }
 
-    private String dbname; // required
-    private List<String> tbl_names; // required
+    private CreationMetadata creation_metadata; // required
+    private String validTxnList; // 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 {
-      DBNAME((short)1, "dbname"),
-      TBL_NAMES((short)2, "tbl_names");
+      CREATION_METADATA((short)1, "creation_metadata"),
+      VALID_TXN_LIST((short)2, "validTxnList");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -75000,10 +75000,10 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // DBNAME
-            return DBNAME;
-          case 2: // TBL_NAMES
-            return TBL_NAMES;
+          case 1: // CREATION_METADATA
+            return CREATION_METADATA;
+          case 2: // VALID_TXN_LIST
+            return VALID_TXN_LIST;
           default:
             return null;
         }
@@ -75047,11 +75047,10 @@ import org.slf4j.LoggerFactory;
     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.DBNAME, new org.apache.thrift.meta_data.FieldMetaData("dbname", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+      tmpMap.put(_Fields.CREATION_METADATA, new org.apache.thrift.meta_data.FieldMetaData("creation_metadata", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CreationMetadata.class)));
+      tmpMap.put(_Fields.VALID_TXN_LIST, new org.apache.thrift.meta_data.FieldMetaData("validTxnList", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.TBL_NAMES, new org.apache.thrift.meta_data.FieldMetaData("tbl_names", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_materialization_invalidation_info_args.class, metaDataMap);
     }
@@ -75060,24 +75059,23 @@ import org.slf4j.LoggerFactory;
     }
 
     public get_materialization_invalidation_info_args(
-      String dbname,
-      List<String> tbl_names)
+      CreationMetadata creation_metadata,
+      String validTxnList)
     {
       this();
-      this.dbname = dbname;
-      this.tbl_names = tbl_names;
+      this.creation_metadata = creation_metadata;
+      this.validTxnList = validTxnList;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
     public get_materialization_invalidation_info_args(get_materialization_invalidation_info_args other) {
-      if (other.isSetDbname()) {
-        this.dbname = other.dbname;
+      if (other.isSetCreation_metadata()) {
+        this.creation_metadata = new CreationMetadata(other.creation_metadata);
       }
-      if (other.isSetTbl_names()) {
-        List<String> __this__tbl_names = new ArrayList<String>(other.tbl_names);
-        this.tbl_names = __this__tbl_names;
+      if (other.isSetValidTxnList()) {
+        this.validTxnList = other.validTxnList;
       }
     }
 
@@ -75087,86 +75085,71 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public void clear() {
-      this.dbname = null;
-      this.tbl_names = null;
+      this.creation_metadata = null;
+      this.validTxnList = null;
     }
 
-    public String getDbname() {
-      return this.dbname;
+    public CreationMetadata getCreation_metadata() {
+      return this.creation_metadata;
     }
 
-    public void setDbname(String dbname) {
-      this.dbname = dbname;
+    public void setCreation_metadata(CreationMetadata creation_metadata) {
+      this.creation_metadata = creation_metadata;
     }
 
-    public void unsetDbname() {
-      this.dbname = null;
+    public void unsetCreation_metadata() {
+      this.creation_metadata = null;
     }
 
-    /** Returns true if field dbname is set (has been assigned a value) and false otherwise */
-    public boolean isSetDbname() {
-      return this.dbname != null;
+    /** Returns true if field creation_metadata is set (has been assigned a value) and false otherwise */
+    public boolean isSetCreation_metadata() {
+      return this.creation_metadata != null;
     }
 
-    public void setDbnameIsSet(boolean value) {
+    public void setCreation_metadataIsSet(boolean value) {
       if (!value) {
-        this.dbname = null;
+        this.creation_metadata = null;
       }
     }
 
-    public int getTbl_namesSize() {
-      return (this.tbl_names == null) ? 0 : this.tbl_names.size();
-    }
-
-    public java.util.Iterator<String> getTbl_namesIterator() {
-      return (this.tbl_names == null) ? null : this.tbl_names.iterator();
-    }
-
-    public void addToTbl_names(String elem) {
-      if (this.tbl_names == null) {
-        this.tbl_names = new ArrayList<String>();
-      }
-      this.tbl_names.add(elem);
+    public String getValidTxnList() {
+      return this.validTxnList;
     }
 
-    public List<String> getTbl_names() {
-      return this.tbl_names;
+    public void

<TRUNCATED>

[02/12] hive git commit: HIVE-20006: Make materializations invalidation cache work with multiple active remote metastores (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 031e72b..1285c08 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -14681,6 +14681,7 @@ class CreationMetadata:
    - tblName
    - tablesUsed
    - validTxnList
+   - materializationTime
   """
 
   thrift_spec = (
@@ -14690,14 +14691,16 @@ class CreationMetadata:
     (3, TType.STRING, 'tblName', None, None, ), # 3
     (4, TType.SET, 'tablesUsed', (TType.STRING,None), None, ), # 4
     (5, TType.STRING, 'validTxnList', None, None, ), # 5
+    (6, TType.I64, 'materializationTime', None, None, ), # 6
   )
 
-  def __init__(self, catName=None, dbName=None, tblName=None, tablesUsed=None, validTxnList=None,):
+  def __init__(self, catName=None, dbName=None, tblName=None, tablesUsed=None, validTxnList=None, materializationTime=None,):
     self.catName = catName
     self.dbName = dbName
     self.tblName = tblName
     self.tablesUsed = tablesUsed
     self.validTxnList = validTxnList
+    self.materializationTime = materializationTime
 
   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:
@@ -14738,6 +14741,11 @@ class CreationMetadata:
           self.validTxnList = iprot.readString()
         else:
           iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.I64:
+          self.materializationTime = iprot.readI64()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -14771,6 +14779,10 @@ class CreationMetadata:
       oprot.writeFieldBegin('validTxnList', TType.STRING, 5)
       oprot.writeString(self.validTxnList)
       oprot.writeFieldEnd()
+    if self.materializationTime is not None:
+      oprot.writeFieldBegin('materializationTime', TType.I64, 6)
+      oprot.writeI64(self.materializationTime)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -14793,6 +14805,7 @@ class CreationMetadata:
     value = (value * 31) ^ hash(self.tblName)
     value = (value * 31) ^ hash(self.tablesUsed)
     value = (value * 31) ^ hash(self.validTxnList)
+    value = (value * 31) ^ hash(self.materializationTime)
     return value
 
   def __repr__(self):
@@ -17613,24 +17626,15 @@ class TableMeta:
 class Materialization:
   """
   Attributes:
-   - tablesUsed
-   - validTxnList
-   - invalidationTime
    - sourceTablesUpdateDeleteModified
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.SET, 'tablesUsed', (TType.STRING,None), None, ), # 1
-    (2, TType.STRING, 'validTxnList', None, None, ), # 2
-    (3, TType.I64, 'invalidationTime', None, None, ), # 3
-    (4, TType.BOOL, 'sourceTablesUpdateDeleteModified', None, None, ), # 4
+    (1, TType.BOOL, 'sourceTablesUpdateDeleteModified', None, None, ), # 1
   )
 
-  def __init__(self, tablesUsed=None, validTxnList=None, invalidationTime=None, sourceTablesUpdateDeleteModified=None,):
-    self.tablesUsed = tablesUsed
-    self.validTxnList = validTxnList
-    self.invalidationTime = invalidationTime
+  def __init__(self, sourceTablesUpdateDeleteModified=None,):
     self.sourceTablesUpdateDeleteModified = sourceTablesUpdateDeleteModified
 
   def read(self, iprot):
@@ -17643,26 +17647,6 @@ class Materialization:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.SET:
-          self.tablesUsed = set()
-          (_etype763, _size760) = iprot.readSetBegin()
-          for _i764 in xrange(_size760):
-            _elem765 = iprot.readString()
-            self.tablesUsed.add(_elem765)
-          iprot.readSetEnd()
-        else:
-          iprot.skip(ftype)
-      elif fid == 2:
-        if ftype == TType.STRING:
-          self.validTxnList = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 3:
-        if ftype == TType.I64:
-          self.invalidationTime = iprot.readI64()
-        else:
-          iprot.skip(ftype)
-      elif fid == 4:
         if ftype == TType.BOOL:
           self.sourceTablesUpdateDeleteModified = iprot.readBool()
         else:
@@ -17677,39 +17661,21 @@ class Materialization:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
     oprot.writeStructBegin('Materialization')
-    if self.tablesUsed is not None:
-      oprot.writeFieldBegin('tablesUsed', TType.SET, 1)
-      oprot.writeSetBegin(TType.STRING, len(self.tablesUsed))
-      for iter766 in self.tablesUsed:
-        oprot.writeString(iter766)
-      oprot.writeSetEnd()
-      oprot.writeFieldEnd()
-    if self.validTxnList is not None:
-      oprot.writeFieldBegin('validTxnList', TType.STRING, 2)
-      oprot.writeString(self.validTxnList)
-      oprot.writeFieldEnd()
-    if self.invalidationTime is not None:
-      oprot.writeFieldBegin('invalidationTime', TType.I64, 3)
-      oprot.writeI64(self.invalidationTime)
-      oprot.writeFieldEnd()
     if self.sourceTablesUpdateDeleteModified is not None:
-      oprot.writeFieldBegin('sourceTablesUpdateDeleteModified', TType.BOOL, 4)
+      oprot.writeFieldBegin('sourceTablesUpdateDeleteModified', TType.BOOL, 1)
       oprot.writeBool(self.sourceTablesUpdateDeleteModified)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.tablesUsed is None:
-      raise TProtocol.TProtocolException(message='Required field tablesUsed is unset!')
+    if self.sourceTablesUpdateDeleteModified is None:
+      raise TProtocol.TProtocolException(message='Required field sourceTablesUpdateDeleteModified is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.tablesUsed)
-    value = (value * 31) ^ hash(self.validTxnList)
-    value = (value * 31) ^ hash(self.invalidationTime)
     value = (value * 31) ^ hash(self.sourceTablesUpdateDeleteModified)
     return value
 
@@ -18586,44 +18552,44 @@ class WMFullResourcePlan:
       elif fid == 2:
         if ftype == TType.LIST:
           self.pools = []
-          (_etype770, _size767) = iprot.readListBegin()
-          for _i771 in xrange(_size767):
-            _elem772 = WMPool()
-            _elem772.read(iprot)
-            self.pools.append(_elem772)
+          (_etype763, _size760) = iprot.readListBegin()
+          for _i764 in xrange(_size760):
+            _elem765 = WMPool()
+            _elem765.read(iprot)
+            self.pools.append(_elem765)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.mappings = []
-          (_etype776, _size773) = iprot.readListBegin()
-          for _i777 in xrange(_size773):
-            _elem778 = WMMapping()
-            _elem778.read(iprot)
-            self.mappings.append(_elem778)
+          (_etype769, _size766) = iprot.readListBegin()
+          for _i770 in xrange(_size766):
+            _elem771 = WMMapping()
+            _elem771.read(iprot)
+            self.mappings.append(_elem771)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype782, _size779) = iprot.readListBegin()
-          for _i783 in xrange(_size779):
-            _elem784 = WMTrigger()
-            _elem784.read(iprot)
-            self.triggers.append(_elem784)
+          (_etype775, _size772) = iprot.readListBegin()
+          for _i776 in xrange(_size772):
+            _elem777 = WMTrigger()
+            _elem777.read(iprot)
+            self.triggers.append(_elem777)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.poolTriggers = []
-          (_etype788, _size785) = iprot.readListBegin()
-          for _i789 in xrange(_size785):
-            _elem790 = WMPoolTrigger()
-            _elem790.read(iprot)
-            self.poolTriggers.append(_elem790)
+          (_etype781, _size778) = iprot.readListBegin()
+          for _i782 in xrange(_size778):
+            _elem783 = WMPoolTrigger()
+            _elem783.read(iprot)
+            self.poolTriggers.append(_elem783)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18644,29 +18610,29 @@ class WMFullResourcePlan:
     if self.pools is not None:
       oprot.writeFieldBegin('pools', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.pools))
-      for iter791 in self.pools:
-        iter791.write(oprot)
+      for iter784 in self.pools:
+        iter784.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.mappings is not None:
       oprot.writeFieldBegin('mappings', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.mappings))
-      for iter792 in self.mappings:
-        iter792.write(oprot)
+      for iter785 in self.mappings:
+        iter785.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.triggers is not None:
       oprot.writeFieldBegin('triggers', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.triggers))
-      for iter793 in self.triggers:
-        iter793.write(oprot)
+      for iter786 in self.triggers:
+        iter786.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.poolTriggers is not None:
       oprot.writeFieldBegin('poolTriggers', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.poolTriggers))
-      for iter794 in self.poolTriggers:
-        iter794.write(oprot)
+      for iter787 in self.poolTriggers:
+        iter787.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19140,11 +19106,11 @@ class WMGetAllResourcePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.resourcePlans = []
-          (_etype798, _size795) = iprot.readListBegin()
-          for _i799 in xrange(_size795):
-            _elem800 = WMResourcePlan()
-            _elem800.read(iprot)
-            self.resourcePlans.append(_elem800)
+          (_etype791, _size788) = iprot.readListBegin()
+          for _i792 in xrange(_size788):
+            _elem793 = WMResourcePlan()
+            _elem793.read(iprot)
+            self.resourcePlans.append(_elem793)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19161,8 +19127,8 @@ class WMGetAllResourcePlanResponse:
     if self.resourcePlans is not None:
       oprot.writeFieldBegin('resourcePlans', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.resourcePlans))
-      for iter801 in self.resourcePlans:
-        iter801.write(oprot)
+      for iter794 in self.resourcePlans:
+        iter794.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19466,20 +19432,20 @@ class WMValidateResourcePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.errors = []
-          (_etype805, _size802) = iprot.readListBegin()
-          for _i806 in xrange(_size802):
-            _elem807 = iprot.readString()
-            self.errors.append(_elem807)
+          (_etype798, _size795) = iprot.readListBegin()
+          for _i799 in xrange(_size795):
+            _elem800 = iprot.readString()
+            self.errors.append(_elem800)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.warnings = []
-          (_etype811, _size808) = iprot.readListBegin()
-          for _i812 in xrange(_size808):
-            _elem813 = iprot.readString()
-            self.warnings.append(_elem813)
+          (_etype804, _size801) = iprot.readListBegin()
+          for _i805 in xrange(_size801):
+            _elem806 = iprot.readString()
+            self.warnings.append(_elem806)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19496,15 +19462,15 @@ class WMValidateResourcePlanResponse:
     if self.errors is not None:
       oprot.writeFieldBegin('errors', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.errors))
-      for iter814 in self.errors:
-        oprot.writeString(iter814)
+      for iter807 in self.errors:
+        oprot.writeString(iter807)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.warnings is not None:
       oprot.writeFieldBegin('warnings', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.warnings))
-      for iter815 in self.warnings:
-        oprot.writeString(iter815)
+      for iter808 in self.warnings:
+        oprot.writeString(iter808)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20081,11 +20047,11 @@ class WMGetTriggersForResourePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype819, _size816) = iprot.readListBegin()
-          for _i820 in xrange(_size816):
-            _elem821 = WMTrigger()
-            _elem821.read(iprot)
-            self.triggers.append(_elem821)
+          (_etype812, _size809) = iprot.readListBegin()
+          for _i813 in xrange(_size809):
+            _elem814 = WMTrigger()
+            _elem814.read(iprot)
+            self.triggers.append(_elem814)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20102,8 +20068,8 @@ class WMGetTriggersForResourePlanResponse:
     if self.triggers is not None:
       oprot.writeFieldBegin('triggers', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.triggers))
-      for iter822 in self.triggers:
-        iter822.write(oprot)
+      for iter815 in self.triggers:
+        iter815.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21287,11 +21253,11 @@ class SchemaVersion:
       elif fid == 4:
         if ftype == TType.LIST:
           self.cols = []
-          (_etype826, _size823) = iprot.readListBegin()
-          for _i827 in xrange(_size823):
-            _elem828 = FieldSchema()
-            _elem828.read(iprot)
-            self.cols.append(_elem828)
+          (_etype819, _size816) = iprot.readListBegin()
+          for _i820 in xrange(_size816):
+            _elem821 = FieldSchema()
+            _elem821.read(iprot)
+            self.cols.append(_elem821)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21351,8 +21317,8 @@ class SchemaVersion:
     if self.cols is not None:
       oprot.writeFieldBegin('cols', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.cols))
-      for iter829 in self.cols:
-        iter829.write(oprot)
+      for iter822 in self.cols:
+        iter822.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.state is not None:
@@ -21607,11 +21573,11 @@ class FindSchemasByColsResp:
       if fid == 1:
         if ftype == TType.LIST:
           self.schemaVersions = []
-          (_etype833, _size830) = iprot.readListBegin()
-          for _i834 in xrange(_size830):
-            _elem835 = SchemaVersionDescriptor()
-            _elem835.read(iprot)
-            self.schemaVersions.append(_elem835)
+          (_etype826, _size823) = iprot.readListBegin()
+          for _i827 in xrange(_size823):
+            _elem828 = SchemaVersionDescriptor()
+            _elem828.read(iprot)
+            self.schemaVersions.append(_elem828)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21628,8 +21594,8 @@ class FindSchemasByColsResp:
     if self.schemaVersions is not None:
       oprot.writeFieldBegin('schemaVersions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.schemaVersions))
-      for iter836 in self.schemaVersions:
-        iter836.write(oprot)
+      for iter829 in self.schemaVersions:
+        iter829.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 0348ff2..a0fabfe 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -3277,13 +3277,15 @@ class CreationMetadata
   TBLNAME = 3
   TABLESUSED = 4
   VALIDTXNLIST = 5
+  MATERIALIZATIONTIME = 6
 
   FIELDS = {
     CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName'},
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     TBLNAME => {:type => ::Thrift::Types::STRING, :name => 'tblName'},
     TABLESUSED => {:type => ::Thrift::Types::SET, :name => 'tablesUsed', :element => {:type => ::Thrift::Types::STRING}},
-    VALIDTXNLIST => {:type => ::Thrift::Types::STRING, :name => 'validTxnList', :optional => true}
+    VALIDTXNLIST => {:type => ::Thrift::Types::STRING, :name => 'validTxnList', :optional => true},
+    MATERIALIZATIONTIME => {:type => ::Thrift::Types::I64, :name => 'materializationTime', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -3952,22 +3954,16 @@ end
 
 class Materialization
   include ::Thrift::Struct, ::Thrift::Struct_Union
-  TABLESUSED = 1
-  VALIDTXNLIST = 2
-  INVALIDATIONTIME = 3
-  SOURCETABLESUPDATEDELETEMODIFIED = 4
+  SOURCETABLESUPDATEDELETEMODIFIED = 1
 
   FIELDS = {
-    TABLESUSED => {:type => ::Thrift::Types::SET, :name => 'tablesUsed', :element => {:type => ::Thrift::Types::STRING}},
-    VALIDTXNLIST => {:type => ::Thrift::Types::STRING, :name => 'validTxnList', :optional => true},
-    INVALIDATIONTIME => {:type => ::Thrift::Types::I64, :name => 'invalidationTime', :optional => true},
-    SOURCETABLESUPDATEDELETEMODIFIED => {:type => ::Thrift::Types::BOOL, :name => 'sourceTablesUpdateDeleteModified', :optional => true}
+    SOURCETABLESUPDATEDELETEMODIFIED => {:type => ::Thrift::Types::BOOL, :name => 'sourceTablesUpdateDeleteModified'}
   }
 
   def struct_fields; FIELDS; end
 
   def validate
-    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tablesUsed is unset!') unless @tablesUsed
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field sourceTablesUpdateDeleteModified is unset!') if @sourceTablesUpdateDeleteModified.nil?
   end
 
   ::Thrift::Struct.generate_accessors self

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index 2bd958e..5ecfbed 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -726,13 +726,13 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_table_objects_by_name_req failed: unknown result')
     end
 
-    def get_materialization_invalidation_info(dbname, tbl_names)
-      send_get_materialization_invalidation_info(dbname, tbl_names)
+    def get_materialization_invalidation_info(creation_metadata, validTxnList)
+      send_get_materialization_invalidation_info(creation_metadata, validTxnList)
       return recv_get_materialization_invalidation_info()
     end
 
-    def send_get_materialization_invalidation_info(dbname, tbl_names)
-      send_message('get_materialization_invalidation_info', Get_materialization_invalidation_info_args, :dbname => dbname, :tbl_names => tbl_names)
+    def send_get_materialization_invalidation_info(creation_metadata, validTxnList)
+      send_message('get_materialization_invalidation_info', Get_materialization_invalidation_info_args, :creation_metadata => creation_metadata, :validTxnList => validTxnList)
     end
 
     def recv_get_materialization_invalidation_info()
@@ -4043,7 +4043,7 @@ module ThriftHiveMetastore
       args = read_args(iprot, Get_materialization_invalidation_info_args)
       result = Get_materialization_invalidation_info_result.new()
       begin
-        result.success = @handler.get_materialization_invalidation_info(args.dbname, args.tbl_names)
+        result.success = @handler.get_materialization_invalidation_info(args.creation_metadata, args.validTxnList)
       rescue ::MetaException => o1
         result.o1 = o1
       rescue ::InvalidOperationException => o2
@@ -7654,12 +7654,12 @@ module ThriftHiveMetastore
 
   class Get_materialization_invalidation_info_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
-    DBNAME = 1
-    TBL_NAMES = 2
+    CREATION_METADATA = 1
+    VALIDTXNLIST = 2
 
     FIELDS = {
-      DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbname'},
-      TBL_NAMES => {:type => ::Thrift::Types::LIST, :name => 'tbl_names', :element => {:type => ::Thrift::Types::STRING}}
+      CREATION_METADATA => {:type => ::Thrift::Types::STRUCT, :name => 'creation_metadata', :class => ::CreationMetadata},
+      VALIDTXNLIST => {:type => ::Thrift::Types::STRING, :name => 'validTxnList'}
     }
 
     def struct_fields; FIELDS; end
@@ -7678,7 +7678,7 @@ module ThriftHiveMetastore
     O3 = 3
 
     FIELDS = {
-      SUCCESS => {:type => ::Thrift::Types::MAP, :name => 'success', :key => {:type => ::Thrift::Types::STRING}, :value => {:type => ::Thrift::Types::STRUCT, :class => ::Materialization}},
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::Materialization},
       O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException},
       O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::InvalidOperationException},
       O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::UnknownDBException}

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 8d88749..e6f7333 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -3009,8 +3009,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     @Override
-    public Map<String, Materialization> get_materialization_invalidation_info(final String dbName, final List<String> tableNames) {
-      return MaterializationsInvalidationCache.get().getMaterializationInvalidationInfo(dbName, tableNames);
+    public Materialization get_materialization_invalidation_info(final CreationMetadata cm, final String validTxnList) throws MetaException {
+      return getTxnHandler().getMaterializationInvalidationInfo(cm, validTxnList);
     }
 
     @Override
@@ -8670,13 +8670,13 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     @Override
     public LockResponse get_lock_materialization_rebuild(String dbName, String tableName, long txnId)
         throws TException {
-      return MaterializationsRebuildLockHandler.get().lockResource(dbName, tableName, txnId);
+      return getTxnHandler().lockMaterializationRebuild(dbName, tableName, txnId);
     }
 
     @Override
     public boolean heartbeat_lock_materialization_rebuild(String dbName, String tableName, long txnId)
         throws TException {
-      return MaterializationsRebuildLockHandler.get().refreshLockResource(dbName, tableName, txnId);
+      return getTxnHandler().heartbeatLockMaterializationRebuild(dbName, tableName, txnId);
     }
 
     @Override
@@ -8992,8 +8992,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           false);
       IHMSHandler handler = newRetryingHMSHandler(baseHandler, conf);
 
-      // Initialize materializations invalidation cache
-      MaterializationsInvalidationCache.get().init(conf, handler);
       TServerSocket serverSocket;
 
       if (useSasl) {

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index bfd7141..acdb73b 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -167,8 +167,6 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
       // instantiate the metastore server handler directly instead of connecting
       // through the network
       client = HiveMetaStore.newRetryingHMSHandler("hive client", this.conf, true);
-      // Initialize materializations invalidation cache (only for local metastore)
-      MaterializationsInvalidationCache.get().init(conf, (IHMSHandler) client);
       isConnected = true;
       snapshotActiveConf();
       return;
@@ -1610,10 +1608,9 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   @Override
-  public Map<String, Materialization> getMaterializationsInvalidationInfo(String dbName, List<String> viewNames)
+  public Materialization getMaterializationInvalidationInfo(CreationMetadata cm, String validTxnList)
       throws MetaException, InvalidOperationException, UnknownDBException, TException {
-    return client.get_materialization_invalidation_info(
-        dbName, filterHook.filterTableNames(getDefaultCatalog(conf), dbName, viewNames));
+    return client.get_materialization_invalidation_info(cm, validTxnList);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index b5d147b..9661beb 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -775,7 +775,7 @@ public interface IMetaStoreClient {
   /**
    * Returns the invalidation information for the materialized views given as input.
    */
-  Map<String, Materialization> getMaterializationsInvalidationInfo(String dbName, List<String> viewNames)
+  Materialization getMaterializationInvalidationInfo(CreationMetadata cm, String validTxnList)
       throws MetaException, InvalidOperationException, UnknownDBException, TException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsCacheCleanerTask.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsCacheCleanerTask.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsCacheCleanerTask.java
deleted file mode 100644
index cc168a9..0000000
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsCacheCleanerTask.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.metastore;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * Task responsible for cleaning the transactions that are not useful from the
- * materializations cache.
- */
-public class MaterializationsCacheCleanerTask implements MetastoreTaskThread {
-  private static final Logger LOG = LoggerFactory.getLogger(MaterializationsCacheCleanerTask.class);
-
-  private Configuration conf;
-
-  @Override
-  public long runFrequency(TimeUnit unit) {
-    return MetastoreConf.getTimeVar(conf,
-        MetastoreConf.ConfVars.MATERIALIZATIONS_INVALIDATION_CACHE_CLEAN_FREQUENCY, unit);
-  }
-
-  @Override
-  public void setConf(Configuration configuration) {
-    conf = configuration;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public void run() {
-    long removedCnt = MaterializationsInvalidationCache.get().cleanup(System.currentTimeMillis() -
-        MetastoreConf.getTimeVar(conf,
-            MetastoreConf.ConfVars.MATERIALIZATIONS_INVALIDATION_CACHE_EXPIRY_DURATION, TimeUnit.MILLISECONDS));
-    if (removedCnt > 0) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Number of transaction entries deleted from materializations cache: " + removedCnt);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsInvalidationCache.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsInvalidationCache.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsInvalidationCache.java
deleted file mode 100644
index fc644f0..0000000
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsInvalidationCache.java
+++ /dev/null
@@ -1,543 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.metastore;
-
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.ConcurrentSkipListSet;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-import org.apache.hadoop.conf.Configuration;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
-import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
-import org.apache.hadoop.hive.common.ValidWriteIdList;
-import org.apache.hadoop.hive.metastore.api.BasicTxnInfo;
-import org.apache.hadoop.hive.metastore.api.LockResponse;
-import org.apache.hadoop.hive.metastore.api.Materialization;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-
-/**
- * This cache keeps information in memory about the table modifications so materialized views
- * can verify their invalidation time, i.e., the moment after materialization on which the
- * first transaction to the tables they used happened. This information is kept in memory
- * to check the invalidation quickly. However, we store enough information in the metastore
- * to bring this cache up if the metastore is restarted or would crashed. This cache lives
- * in the metastore server.
- */
-public final class MaterializationsInvalidationCache {
-
-  private static final Logger LOG = LoggerFactory.getLogger(MaterializationsInvalidationCache.class);
-
-  /* Singleton */
-  private static final MaterializationsInvalidationCache SINGLETON = new MaterializationsInvalidationCache();
-
-  /* If this boolean is true, this class has no functionality. Only for debugging purposes. */
-  private boolean disable;
-
-  /* Key is the database name. Each value is a map from the unique view qualified name to
-   * the materialization invalidation info. This invalidation object contains information
-   * such as the tables used by the materialized view, whether there was any update or
-   * delete in the source tables since the materialized view was created or rebuilt,
-   * or the invalidation time, i.e., first modification of the tables used by materialized
-   * view after the view was created. */
-  private final ConcurrentMap<String, ConcurrentMap<String, Materialization>> materializations =
-      new ConcurrentHashMap<>();
-
-  /*
-   * Key is a qualified table name. The value is a (sorted) tree map (supporting concurrent
-   * modifications) that will keep the modifications for a given table in the order of their
-   * transaction id. This is useful to quickly check the invalidation time for a given
-   * materialization.
-   */
-  private final ConcurrentMap<String, ConcurrentSkipListMap<Long, Long>> tableModifications =
-      new ConcurrentHashMap<>();
-
-  private final ConcurrentMap<String, ConcurrentSkipListSet<Long>> updateDeleteTableModifications =
-      new ConcurrentHashMap<>();
-
-  /* Whether the cache has been initialized or not. */
-  private boolean initialized;
-  /* Configuration for cache. */
-  private Configuration conf;
-  /* Handler to connect to metastore. */
-  private IHMSHandler handler;
-
-  private MaterializationsInvalidationCache() {
-  }
-
-  /**
-   * Get instance of MaterializationsInvalidationCache.
-   *
-   * @return the singleton
-   */
-  public static MaterializationsInvalidationCache get() {
-    return SINGLETON;
-  }
-
-  /**
-   * Initialize the invalidation cache.
-   *
-   * The method is synchronized because we want to avoid initializing the invalidation cache
-   * multiple times in embedded mode. This will not happen when we run the metastore remotely
-   * as the method is called only once.
-   */
-  public synchronized void init(Configuration conf, IHMSHandler handler) {
-    this.conf = conf;
-    this.handler = handler;
-
-    // This will only be true for debugging purposes
-    this.disable = MetastoreConf.getVar(conf,
-        MetastoreConf.ConfVars.MATERIALIZATIONS_INVALIDATION_CACHE_IMPL).equals("DISABLE");
-    if (disable) {
-      // Nothing to do
-      return;
-    }
-
-    if (!initialized) {
-      this.initialized = true;
-      ExecutorService pool = Executors.newCachedThreadPool();
-      pool.submit(new Loader());
-      pool.shutdown();
-    }
-  }
-
-  private class Loader implements Runnable {
-    @Override
-    public void run() {
-      try {
-        RawStore store = handler.getMS();
-        for (String catName : store.getCatalogs()) {
-          for (String dbName : store.getAllDatabases(catName)) {
-            for (Table mv : store.getTableObjectsByName(catName, dbName,
-                store.getTables(catName, dbName, null, TableType.MATERIALIZED_VIEW))) {
-              addMaterializedView(mv.getDbName(), mv.getTableName(), ImmutableSet.copyOf(mv.getCreationMetadata().getTablesUsed()),
-                  mv.getCreationMetadata().getValidTxnList(), OpType.LOAD);
-            }
-          }
-        }
-        LOG.info("Initialized materializations invalidation cache");
-      } catch (Exception e) {
-        LOG.error("Problem connecting to the metastore when initializing the view registry");
-      }
-    }
-  }
-
-  /**
-   * Adds a newly created materialized view to the cache.
-   *
-   * @param dbName
-   * @param tableName
-   * @param tablesUsed tables used by the materialized view
-   * @param validTxnList
-   */
-  public void createMaterializedView(String dbName, String tableName, Set<String> tablesUsed,
-      String validTxnList) {
-    addMaterializedView(dbName, tableName, tablesUsed, validTxnList, OpType.CREATE);
-  }
-
-  /**
-   * Method to call when materialized view is modified.
-   *
-   * @param dbName
-   * @param tableName
-   * @param tablesUsed tables used by the materialized view
-   * @param validTxnList
-   */
-  public void alterMaterializedView(String dbName, String tableName, Set<String> tablesUsed,
-      String validTxnList) {
-    addMaterializedView(dbName, tableName, tablesUsed, validTxnList, OpType.ALTER);
-  }
-
-  /**
-   * Adds the materialized view to the cache.
-   *
-   * @param dbName
-   * @param tableName
-   * @param tablesUsed tables used by the materialized view
-   * @param validTxnList
-   * @param opType
-   */
-  private void addMaterializedView(String dbName, String tableName, Set<String> tablesUsed,
-      String validTxnList, OpType opType) {
-    if (disable) {
-      // Nothing to do
-      return;
-    }
-    // We are going to create the map for each view in the given database
-    ConcurrentMap<String, Materialization> cq =
-        new ConcurrentHashMap<String, Materialization>();
-    final ConcurrentMap<String, Materialization> prevCq = materializations.putIfAbsent(
-        dbName, cq);
-    if (prevCq != null) {
-      cq = prevCq;
-    }
-    // Start the process to add materialization to the cache
-    // Before loading the materialization in the cache, we need to update some
-    // important information in the registry to account for rewriting invalidation
-    if (validTxnList == null) {
-      // This can happen when the materialized view was created on non-transactional tables
-      return;
-    }
-    if (opType == OpType.CREATE || opType == OpType.ALTER) {
-      // You store the materialized view
-      Materialization materialization = new Materialization(tablesUsed);
-      materialization.setValidTxnList(validTxnList);
-      cq.put(tableName, materialization);
-    } else {
-      ValidTxnWriteIdList txnList = new ValidTxnWriteIdList(validTxnList);
-      for (String qNameTableUsed : tablesUsed) {
-        ValidWriteIdList tableTxnList = txnList.getTableValidWriteIdList(qNameTableUsed);
-        // First we insert a new tree set to keep table modifications, unless it already exists
-        ConcurrentSkipListMap<Long, Long> modificationsTree = new ConcurrentSkipListMap<>();
-        final ConcurrentSkipListMap<Long, Long> prevModificationsTree = tableModifications.putIfAbsent(
-                qNameTableUsed, modificationsTree);
-        if (prevModificationsTree != null) {
-          modificationsTree = prevModificationsTree;
-        }
-        // If we are not creating the MV at this instant, but instead it was created previously
-        // and we are loading it into the cache, we need to go through the transaction entries and
-        // check if the MV is still valid.
-        try {
-          String[] names =  qNameTableUsed.split("\\.");
-          BasicTxnInfo e = handler.getTxnHandler().getFirstCompletedTransactionForTableAfterCommit(
-                  names[0], names[1], tableTxnList);
-          if (!e.isIsnull()) {
-            modificationsTree.put(e.getTxnid(), e.getTime());
-            // We do not need to do anything more for current table, as we detected
-            // a modification event that was in the metastore.
-            continue;
-          }
-        } catch (MetaException ex) {
-          LOG.debug("Materialized view " + Warehouse.getQualifiedName(dbName, tableName) +
-                  " ignored; error loading view into invalidation cache", ex);
-          return;
-        }
-      }
-      // For LOAD, you only add it if it does exist as you might be loading an outdated MV
-      Materialization materialization = new Materialization(tablesUsed);
-      materialization.setValidTxnList(validTxnList);
-      cq.putIfAbsent(tableName, materialization);
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Cached materialized view for rewriting in invalidation cache: " +
-          Warehouse.getQualifiedName(dbName, tableName));
-    }
-  }
-
-  /**
-   * This method is called when a table is modified. That way we can keep track of the
-   * invalidation for the MVs that use that table.
-   */
-  public void notifyTableModification(String dbName, String tableName,
-      long txnId, long newModificationTime, boolean isUpdateDelete) {
-    if (disable) {
-      // Nothing to do
-      return;
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Notification for table {} in database {} received -> id: {}, time: {}",
-          tableName, dbName, txnId, newModificationTime);
-    }
-    if (isUpdateDelete) {
-      // We update first the update/delete modifications record
-      ConcurrentSkipListSet<Long> modificationsSet = new ConcurrentSkipListSet<>();
-      final ConcurrentSkipListSet<Long> prevModificationsSet =
-          updateDeleteTableModifications.putIfAbsent(Warehouse.getQualifiedName(dbName, tableName),
-              modificationsSet);
-      if (prevModificationsSet != null) {
-        modificationsSet = prevModificationsSet;
-      }
-      modificationsSet.add(txnId);
-    }
-    ConcurrentSkipListMap<Long, Long> modificationsTree = new ConcurrentSkipListMap<>();
-    final ConcurrentSkipListMap<Long, Long> prevModificationsTree =
-        tableModifications.putIfAbsent(Warehouse.getQualifiedName(dbName, tableName), modificationsTree);
-    if (prevModificationsTree != null) {
-      modificationsTree = prevModificationsTree;
-    }
-    modificationsTree.put(txnId, newModificationTime);
-  }
-
-  /**
-   * Removes the materialized view from the cache.
-   *
-   * @param dbName
-   * @param tableName
-   */
-  public void dropMaterializedView(String dbName, String tableName) {
-    if (disable) {
-      // Nothing to do
-      return;
-    }
-    materializations.get(dbName).remove(tableName);
-  }
-
-  /**
-   * Returns the materialized views in the cache for the given database.
-   *
-   * @param dbName the database
-   * @return the collection of materialized views, or the empty collection if none
-   */
-  public Map<String, Materialization> getMaterializationInvalidationInfo(
-      String dbName, List<String> materializationNames) {
-    if (materializations.get(dbName) != null) {
-      ImmutableMap.Builder<String, Materialization> m = ImmutableMap.builder();
-      for (String materializationName : materializationNames) {
-        Materialization materialization =
-            materializations.get(dbName).get(materializationName);
-        if (materialization == null) {
-          LOG.debug("Materialization {} skipped as there is no information "
-              + "in the invalidation cache about it", materializationName);
-          continue;
-        }
-        // We create a deep copy of the materialization, as we need to set the time
-        // and whether any update/delete operation happen on the tables that it uses
-        // since it was created.
-        Materialization materializationCopy = new Materialization(
-            materialization.getTablesUsed());
-        materializationCopy.setValidTxnList(materialization.getValidTxnList());
-        enrichWithInvalidationInfo(materializationCopy);
-        m.put(materializationName, materializationCopy);
-      }
-      Map<String, Materialization> result = m.build();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Retrieved the following materializations from the invalidation cache: {}", result);
-      }
-      return result;
-    }
-    return ImmutableMap.of();
-  }
-
-  private void enrichWithInvalidationInfo(Materialization materialization) {
-    String materializationTxnListString = materialization.getValidTxnList();
-    if (materializationTxnListString == null) {
-      // This can happen when the materialization was created on non-transactional tables
-      materialization.setInvalidationTime(Long.MIN_VALUE);
-      return;
-    }
-
-    // We will obtain the modification time as follows.
-    // First, we obtain the first element after high watermark (if any)
-    // Then, we iterate through the elements from min open txn till high
-    // watermark, updating the modification time after creation if needed
-    ValidTxnWriteIdList materializationTxnList = new ValidTxnWriteIdList(materializationTxnListString);
-    long firstModificationTimeAfterCreation = 0L;
-    boolean containsUpdateDelete = false;
-    for (String qNameTableUsed : materialization.getTablesUsed()) {
-      final ValidWriteIdList tableMaterializationTxnList =
-          materializationTxnList.getTableValidWriteIdList(qNameTableUsed);
-
-      final ConcurrentSkipListMap<Long, Long> usedTableModifications =
-          tableModifications.get(qNameTableUsed);
-      if (usedTableModifications == null) {
-        // This is not necessarily an error, since the table may be empty. To be safe,
-        // instead of including this materialized view, we just log the information and
-        // skip it (if table is really empty, it will not matter for performance anyway).
-        LOG.warn("No information found in invalidation cache for table {}, possible tables are: {}",
-            qNameTableUsed, tableModifications.keySet());
-        materialization.setInvalidationTime(Long.MIN_VALUE);
-        return;
-      }
-      final ConcurrentSkipListSet<Long> usedUDTableModifications =
-          updateDeleteTableModifications.get(qNameTableUsed);
-      final Entry<Long, Long> tn = usedTableModifications.higherEntry(tableMaterializationTxnList.getHighWatermark());
-      if (tn != null) {
-        if (firstModificationTimeAfterCreation == 0L ||
-            tn.getValue() < firstModificationTimeAfterCreation) {
-          firstModificationTimeAfterCreation = tn.getValue();
-        }
-        // Check if there was any update/delete after creation
-        containsUpdateDelete = usedUDTableModifications != null &&
-            !usedUDTableModifications.tailSet(tableMaterializationTxnList.getHighWatermark(), false).isEmpty();
-      }
-      // Min open txn might be null if there were no open transactions
-      // when this transaction was being executed
-      if (tableMaterializationTxnList.getMinOpenWriteId() != null) {
-        // Invalid transaction list is sorted
-        int pos = 0;
-        for (Map.Entry<Long, Long> t : usedTableModifications
-                .subMap(tableMaterializationTxnList.getMinOpenWriteId(), tableMaterializationTxnList.getHighWatermark()).entrySet()) {
-          while (pos < tableMaterializationTxnList.getInvalidWriteIds().length &&
-              tableMaterializationTxnList.getInvalidWriteIds()[pos] != t.getKey()) {
-            pos++;
-          }
-          if (pos >= tableMaterializationTxnList.getInvalidWriteIds().length) {
-            break;
-          }
-          if (firstModificationTimeAfterCreation == 0L ||
-              t.getValue() < firstModificationTimeAfterCreation) {
-            firstModificationTimeAfterCreation = t.getValue();
-          }
-          containsUpdateDelete = containsUpdateDelete ||
-              (usedUDTableModifications != null && usedUDTableModifications.contains(t.getKey()));
-        }
-      }
-    }
-
-    materialization.setInvalidationTime(firstModificationTimeAfterCreation);
-    materialization.setSourceTablesUpdateDeleteModified(containsUpdateDelete);
-  }
-
-  private enum OpType {
-    CREATE,
-    LOAD,
-    ALTER
-  }
-
-  /**
-   * Removes transaction events that are not relevant anymore.
-   * @param minTime events generated before this time (ms) can be deleted from the cache
-   * @return number of events that were deleted from the cache
-   */
-  public long cleanup(long minTime) {
-    // To remove, mv should meet two conditions:
-    // 1) Current time - time of transaction > config parameter, and
-    // 2) Transaction should not be associated with invalidation of a MV
-    if (disable || !initialized) {
-      // Bail out
-      return 0L;
-    }
-    // We execute the cleanup in two steps
-    // First we gather all the transactions that need to be kept
-    final Multimap<String, Long> keepTxnInfos = HashMultimap.create();
-    for (Map.Entry<String, ConcurrentMap<String, Materialization>> e : materializations.entrySet()) {
-      for (Materialization m : e.getValue().values()) {
-        ValidTxnWriteIdList txnList = new ValidTxnWriteIdList(m.getValidTxnList());
-        boolean canBeDeleted = false;
-        String currentTableForInvalidatingTxn = null;
-        long currentInvalidatingTxnId = 0L;
-        long currentInvalidatingTxnTime = 0L;
-        for (String qNameTableUsed : m.getTablesUsed()) {
-          ValidWriteIdList tableTxnList = txnList.getTableValidWriteIdList(qNameTableUsed);
-          final Entry<Long, Long> tn = tableModifications.get(qNameTableUsed)
-              .higherEntry(tableTxnList.getHighWatermark());
-          if (tn != null) {
-            if (currentInvalidatingTxnTime == 0L ||
-                tn.getValue() < currentInvalidatingTxnTime) {
-              // This transaction 1) is the first one examined for this materialization, or
-              // 2) it is the invalidating transaction. Hence we add it to the transactions to keep.
-              // 1.- We remove the previous invalidating transaction from the transactions
-              // to be kept (if needed).
-              if (canBeDeleted && currentInvalidatingTxnTime < minTime) {
-                keepTxnInfos.remove(currentTableForInvalidatingTxn, currentInvalidatingTxnId);
-              }
-              // 2.- We add this transaction to the transactions that should be kept.
-              canBeDeleted = !keepTxnInfos.get(qNameTableUsed).contains(tn.getKey());
-              keepTxnInfos.put(qNameTableUsed, tn.getKey());
-              // 3.- We record this transaction as the current invalidating transaction.
-              currentTableForInvalidatingTxn = qNameTableUsed;
-              currentInvalidatingTxnId = tn.getKey();
-              currentInvalidatingTxnTime = tn.getValue();
-            }
-          }
-          if (tableTxnList.getMinOpenWriteId() != null) {
-            // Invalid transaction list is sorted
-            int pos = 0;
-            for (Entry<Long, Long> t : tableModifications.get(qNameTableUsed)
-                .subMap(tableTxnList.getMinOpenWriteId(), tableTxnList.getHighWatermark()).entrySet()) {
-              while (pos < tableTxnList.getInvalidWriteIds().length &&
-                  tableTxnList.getInvalidWriteIds()[pos] != t.getKey()) {
-                pos++;
-              }
-              if (pos >= tableTxnList.getInvalidWriteIds().length) {
-                break;
-              }
-              if (currentInvalidatingTxnTime == 0L ||
-                  t.getValue() < currentInvalidatingTxnTime) {
-                // This transaction 1) is the first one examined for this materialization, or
-                // 2) it is the invalidating transaction. Hence we add it to the transactions to keep.
-                // 1.- We remove the previous invalidating transaction from the transactions
-                // to be kept (if needed).
-                if (canBeDeleted && currentInvalidatingTxnTime < minTime) {
-                  keepTxnInfos.remove(currentTableForInvalidatingTxn, currentInvalidatingTxnId);
-                }
-                // 2.- We add this transaction to the transactions that should be kept.
-                canBeDeleted = !keepTxnInfos.get(qNameTableUsed).contains(t.getKey());
-                keepTxnInfos.put(qNameTableUsed, t.getKey());
-                // 3.- We record this transaction as the current invalidating transaction.
-                currentTableForInvalidatingTxn = qNameTableUsed;
-                currentInvalidatingTxnId = t.getKey();
-                currentInvalidatingTxnTime = t.getValue();
-              }
-            }
-          }
-        }
-      }
-    }
-    // Second, we remove the transactions
-    long removed = 0L;
-    for (Entry<String, ConcurrentSkipListMap<Long, Long>> e : tableModifications.entrySet()) {
-      Collection<Long> c = keepTxnInfos.get(e.getKey());
-      ConcurrentSkipListSet<Long> updateDeleteForTable = updateDeleteTableModifications.get(e.getKey());
-      for (Iterator<Entry<Long, Long>> it = e.getValue().entrySet().iterator(); it.hasNext();) {
-        Entry<Long, Long> v = it.next();
-        // We need to check again the time because some of the transactions might not be explored
-        // above, e.g., transactions above the highest transaction mark for all the materialized
-        // views.
-        if (v.getValue() < minTime && (c.isEmpty() || !c.contains(v.getKey()))) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Transaction removed from cache for table {} -> id: {}, time: {}",
-                e.getKey(), v.getKey(), v.getValue());
-          }
-          if (updateDeleteForTable != null) {
-            updateDeleteForTable.remove(v.getKey());
-          }
-          it.remove();
-          removed++;
-        }
-      }
-    }
-    return removed;
-  }
-
-  /**
-   * Checks whether the given materialization exists in the invalidation cache.
-   * @param dbName the database name for the materialization
-   * @param tblName the table name for the materialization
-   * @return true if we have information about the materialization in the cache,
-   * false otherwise
-   */
-  public boolean containsMaterialization(String dbName, String tblName) {
-    if (disable || dbName == null || tblName == null) {
-      return false;
-    }
-    ConcurrentMap<String, Materialization> dbMaterializations = materializations.get(dbName);
-    if (dbMaterializations == null || dbMaterializations.get(tblName) == null) {
-      // This is a table
-      return false;
-    }
-    return true;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsRebuildLockCleanerTask.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsRebuildLockCleanerTask.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsRebuildLockCleanerTask.java
index 8ca9ede..9ce7d6d 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsRebuildLockCleanerTask.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsRebuildLockCleanerTask.java
@@ -18,7 +18,10 @@
 package org.apache.hadoop.hive.metastore;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -32,6 +35,7 @@ public class MaterializationsRebuildLockCleanerTask implements MetastoreTaskThre
   private static final Logger LOG = LoggerFactory.getLogger(MaterializationsRebuildLockCleanerTask.class);
 
   private Configuration conf;
+  private TxnStore txnHandler;
 
   @Override
   public long runFrequency(TimeUnit unit) {
@@ -41,6 +45,7 @@ public class MaterializationsRebuildLockCleanerTask implements MetastoreTaskThre
   @Override
   public void setConf(Configuration configuration) {
     conf = configuration;
+    txnHandler = TxnUtils.getTxnStore(conf);
   }
 
   @Override
@@ -50,11 +55,26 @@ public class MaterializationsRebuildLockCleanerTask implements MetastoreTaskThre
 
   @Override
   public void run() {
-    long removedCnt = MaterializationsRebuildLockHandler.get().cleanupResourceLocks(
-        MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS));
-    if (removedCnt > 0) {
-      if (LOG.isDebugEnabled()) {
-        LOG.info("Number of materialization locks deleted: " + removedCnt);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Cleaning up materialization rebuild locks");
+    }
+
+    TxnStore.MutexAPI.LockHandle handle = null;
+    try {
+      handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.MaterializationRebuild.name());
+      ValidTxnList validTxnList = TxnUtils.createValidReadTxnList(txnHandler.getOpenTxns(), 0);
+      long removedCnt = txnHandler.cleanupMaterializationRebuildLocks(validTxnList,
+          MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS));
+      if (removedCnt > 0) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Number of materialization locks deleted: " + removedCnt);
+        }
+      }
+    } catch(Throwable t) {
+      LOG.error("Serious error in {}", Thread.currentThread().getName(), ": {}" + t.getMessage(), t);
+    } finally {
+      if(handle != null) {
+        handle.releaseLocks();
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 8721022..bdcbf41 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -1335,13 +1335,6 @@ public class ObjectStore implements RawStore, Configurable {
     } finally {
       if (!commited) {
         rollbackTransaction();
-      } else {
-        if (MetaStoreUtils.isMaterializedViewTable(tbl)) {
-          // Add to the invalidation cache
-          MaterializationsInvalidationCache.get().createMaterializedView(
-              tbl.getDbName(), tbl.getTableName(), tbl.getCreationMetadata().getTablesUsed(),
-              tbl.getCreationMetadata().getValidTxnList());
-        }
       }
     }
   }
@@ -1439,10 +1432,6 @@ public class ObjectStore implements RawStore, Configurable {
     } finally {
       if (!success) {
         rollbackTransaction();
-      } else {
-        if (materializedView) {
-          MaterializationsInvalidationCache.get().dropMaterializedView(dbName, tableName);
-        }
       }
     }
     return success;
@@ -2285,13 +2274,14 @@ public class ObjectStore implements RawStore, Configurable {
     if (m == null) {
       return null;
     }
+    assert !m.isSetMaterializationTime();
     Set<MTable> tablesUsed = new HashSet<>();
     for (String fullyQualifiedName : m.getTablesUsed()) {
       String[] names =  fullyQualifiedName.split("\\.");
       tablesUsed.add(getMTable(m.getCatName(), names[0], names[1], false).mtbl);
     }
     return new MCreationMetadata(m.getCatName(), m.getDbName(), m.getTblName(),
-        tablesUsed, m.getValidTxnList());
+        tablesUsed, m.getValidTxnList(), System.currentTimeMillis());
   }
 
   private CreationMetadata convertToCreationMetadata(
@@ -2307,6 +2297,7 @@ public class ObjectStore implements RawStore, Configurable {
     }
     CreationMetadata r = new CreationMetadata(s.getCatalogName(),
         s.getDbName(), s.getTblName(), tablesUsed);
+    r.setMaterializationTime(s.getMaterializationTime());
     if (s.getTxnList() != null) {
       r.setValidTxnList(s.getTxnList());
     }
@@ -4210,16 +4201,13 @@ public class ObjectStore implements RawStore, Configurable {
       MCreationMetadata newMcm = convertToMCreationMetadata(cm);
       MCreationMetadata mcm = getCreationMetadata(catName, dbname, tablename);
       mcm.setTables(newMcm.getTables());
+      mcm.setMaterializationTime(newMcm.getMaterializationTime());
       mcm.setTxnList(newMcm.getTxnList());
       // commit the changes
       success = commitTransaction();
     } finally {
       if (!success) {
         rollbackTransaction();
-      } else {
-        // Add to the invalidation cache if the creation signature has changed
-        MaterializationsInvalidationCache.get().alterMaterializedView(
-            dbname, tablename, cm.getTablesUsed(), cm.getValidTxnList());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
index 74a301f..c2bbba5 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
@@ -21,7 +21,6 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.DefaultStorageSchemaReader;
 import org.apache.hadoop.hive.metastore.HiveAlterHandler;
-import org.apache.hadoop.hive.metastore.MaterializationsCacheCleanerTask;
 import org.apache.hadoop.hive.metastore.MaterializationsRebuildLockCleanerTask;
 import org.apache.hadoop.hive.metastore.MetastoreTaskThread;
 import org.apache.hadoop.hive.metastore.RuntimeStatsCleanerTask;
@@ -762,8 +761,6 @@ public class MetastoreConf {
     TASK_THREADS_ALWAYS("metastore.task.threads.always", "metastore.task.threads.always",
         EventCleanerTask.class.getName() + "," + RuntimeStatsCleanerTask.class.getName() + "," +
         "org.apache.hadoop.hive.metastore.repl.DumpDirCleanerTask" + "," +
-        MaterializationsCacheCleanerTask.class.getName() + "," +
-            MaterializationsRebuildLockCleanerTask.class.getName() + "," + RuntimeStatsCleanerTask.class.getName() + "," +
             "org.apache.hadoop.hive.metastore.HiveProtoEventsCleanerTask",
         "Comma separated list of tasks that will be started in separate threads.  These will " +
             "always be started, regardless of whether the metastore is running in embedded mode " +
@@ -772,7 +769,8 @@ public class MetastoreConf {
         AcidHouseKeeperService.class.getName() + "," +
             AcidOpenTxnsCounterService.class.getName() + "," +
             AcidCompactionHistoryService.class.getName() + "," +
-            AcidWriteSetService.class.getName(),
+            AcidWriteSetService.class.getName() + "," +
+            MaterializationsRebuildLockCleanerTask.class.getName(),
         "Command separated list of tasks that will be started in separate threads.  These will be" +
             " started only when the metastore is running as a separate service.  They must " +
             "implement " + MetastoreTaskThread.class.getName()),

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
index 66b5d48..2d65126 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
@@ -22,8 +22,8 @@ import java.util.Set;
 /**
  * Represents the creation metadata of a materialization.
  * It includes the database and table name for the materialization,
- * the set of tables that it uses, and the valid transaction list
- * when it was created.
+ * the set of tables that it uses, the valid transaction list
+ * when it was created, and the creation/rebuild time.
  */
 public class MCreationMetadata {
 
@@ -32,17 +32,19 @@ public class MCreationMetadata {
   private String tblName;
   private Set<MTable> tables;
   private String txnList;
+  private long materializationTime;
 
   public MCreationMetadata() {
   }
 
   public MCreationMetadata(String catName, String dbName, String tblName,
-      Set<MTable> tables, String txnList) {
+      Set<MTable> tables, String txnList, long materializationTime) {
     this.catalogName = catName;
     this.dbName = dbName;
     this.tblName = tblName;
     this.tables = tables;
     this.txnList = txnList;
+    this.materializationTime = materializationTime;
   }
 
   public Set<MTable> getTables() {
@@ -84,4 +86,12 @@ public class MCreationMetadata {
   public void setTblName(String tblName) {
     this.tblName = tblName;
   }
+
+  public long getMaterializationTime() {
+    return materializationTime;
+  }
+
+  public void setMaterializationTime(long materializationTime) {
+    this.materializationTime = materializationTime;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
index f8c2ca2..2bae133 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
@@ -94,9 +94,9 @@ public final class TxnDbUtil {
           "  CTC_DATABASE varchar(128) NOT NULL," +
           "  CTC_TABLE varchar(128)," +
           "  CTC_PARTITION varchar(767)," +
-          "  CTC_ID bigint GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1) NOT NULL," +
           "  CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL," +
-          "  CTC_WRITEID bigint)");
+          "  CTC_WRITEID bigint," +
+          "  CTC_UPDATE_DELETE char(1) NOT NULL)");
       stmt.execute("CREATE TABLE NEXT_TXN_ID (" + "  NTXN_NEXT bigint NOT NULL)");
       stmt.execute("INSERT INTO NEXT_TXN_ID VALUES(1)");
 
@@ -194,6 +194,14 @@ public final class TxnDbUtil {
           " PRIMARY KEY (RTM_REPL_POLICY, RTM_SRC_TXN_ID))"
       );
 
+      stmt.execute("CREATE TABLE MATERIALIZATION_REBUILD_LOCKS (" +
+          "  MRL_TXN_ID BIGINT NOT NULL, " +
+          "  MRL_DB_NAME VARCHAR(128) NOT NULL, " +
+          "  MRL_TBL_NAME VARCHAR(256) NOT NULL, " +
+          "  MRL_LAST_HEARTBEAT BIGINT NOT NULL, " +
+          "  PRIMARY KEY(MRL_TXN_ID))"
+      );
+
       try {
         stmt.execute("CREATE TABLE \"APP\".\"SEQUENCE_TABLE\" (\"SEQUENCE_NAME\" VARCHAR(256) NOT " +
 
@@ -336,6 +344,7 @@ public final class TxnDbUtil {
         success &= dropTable(stmt, "AUX_TABLE", retryCount);
         success &= dropTable(stmt, "WRITE_SET", retryCount);
         success &= dropTable(stmt, "REPL_TXN_MAP", retryCount);
+        success &= dropTable(stmt, "MATERIALIZATION_REBUILD_LOCKS", retryCount);
         /*
          * Don't drop NOTIFICATION_LOG, SEQUENCE_TABLE and NOTIFICATION_SEQUENCE as its used by other
          * table which are not txn related to generate primary key. So if these tables are dropped


[11/12] hive git commit: HIVE-20006: Make materializations invalidation cache work with multiple active remote metastores (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index a816ae7..9d57d4c 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -2334,14 +2334,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1226;
-            ::apache::thrift::protocol::TType _etype1229;
-            xfer += iprot->readListBegin(_etype1229, _size1226);
-            this->success.resize(_size1226);
-            uint32_t _i1230;
-            for (_i1230 = 0; _i1230 < _size1226; ++_i1230)
+            uint32_t _size1219;
+            ::apache::thrift::protocol::TType _etype1222;
+            xfer += iprot->readListBegin(_etype1222, _size1219);
+            this->success.resize(_size1219);
+            uint32_t _i1223;
+            for (_i1223 = 0; _i1223 < _size1219; ++_i1223)
             {
-              xfer += iprot->readString(this->success[_i1230]);
+              xfer += iprot->readString(this->success[_i1223]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2380,10 +2380,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 _iter1231;
-      for (_iter1231 = this->success.begin(); _iter1231 != this->success.end(); ++_iter1231)
+      std::vector<std::string> ::const_iterator _iter1224;
+      for (_iter1224 = this->success.begin(); _iter1224 != this->success.end(); ++_iter1224)
       {
-        xfer += oprot->writeString((*_iter1231));
+        xfer += oprot->writeString((*_iter1224));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2428,14 +2428,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1232;
-            ::apache::thrift::protocol::TType _etype1235;
-            xfer += iprot->readListBegin(_etype1235, _size1232);
-            (*(this->success)).resize(_size1232);
-            uint32_t _i1236;
-            for (_i1236 = 0; _i1236 < _size1232; ++_i1236)
+            uint32_t _size1225;
+            ::apache::thrift::protocol::TType _etype1228;
+            xfer += iprot->readListBegin(_etype1228, _size1225);
+            (*(this->success)).resize(_size1225);
+            uint32_t _i1229;
+            for (_i1229 = 0; _i1229 < _size1225; ++_i1229)
             {
-              xfer += iprot->readString((*(this->success))[_i1236]);
+              xfer += iprot->readString((*(this->success))[_i1229]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2552,14 +2552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1237;
-            ::apache::thrift::protocol::TType _etype1240;
-            xfer += iprot->readListBegin(_etype1240, _size1237);
-            this->success.resize(_size1237);
-            uint32_t _i1241;
-            for (_i1241 = 0; _i1241 < _size1237; ++_i1241)
+            uint32_t _size1230;
+            ::apache::thrift::protocol::TType _etype1233;
+            xfer += iprot->readListBegin(_etype1233, _size1230);
+            this->success.resize(_size1230);
+            uint32_t _i1234;
+            for (_i1234 = 0; _i1234 < _size1230; ++_i1234)
             {
-              xfer += iprot->readString(this->success[_i1241]);
+              xfer += iprot->readString(this->success[_i1234]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2598,10 +2598,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 _iter1242;
-      for (_iter1242 = this->success.begin(); _iter1242 != this->success.end(); ++_iter1242)
+      std::vector<std::string> ::const_iterator _iter1235;
+      for (_iter1235 = this->success.begin(); _iter1235 != this->success.end(); ++_iter1235)
       {
-        xfer += oprot->writeString((*_iter1242));
+        xfer += oprot->writeString((*_iter1235));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2646,14 +2646,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1243;
-            ::apache::thrift::protocol::TType _etype1246;
-            xfer += iprot->readListBegin(_etype1246, _size1243);
-            (*(this->success)).resize(_size1243);
-            uint32_t _i1247;
-            for (_i1247 = 0; _i1247 < _size1243; ++_i1247)
+            uint32_t _size1236;
+            ::apache::thrift::protocol::TType _etype1239;
+            xfer += iprot->readListBegin(_etype1239, _size1236);
+            (*(this->success)).resize(_size1236);
+            uint32_t _i1240;
+            for (_i1240 = 0; _i1240 < _size1236; ++_i1240)
             {
-              xfer += iprot->readString((*(this->success))[_i1247]);
+              xfer += iprot->readString((*(this->success))[_i1240]);
             }
             xfer += iprot->readListEnd();
           }
@@ -3715,17 +3715,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1248;
-            ::apache::thrift::protocol::TType _ktype1249;
-            ::apache::thrift::protocol::TType _vtype1250;
-            xfer += iprot->readMapBegin(_ktype1249, _vtype1250, _size1248);
-            uint32_t _i1252;
-            for (_i1252 = 0; _i1252 < _size1248; ++_i1252)
+            uint32_t _size1241;
+            ::apache::thrift::protocol::TType _ktype1242;
+            ::apache::thrift::protocol::TType _vtype1243;
+            xfer += iprot->readMapBegin(_ktype1242, _vtype1243, _size1241);
+            uint32_t _i1245;
+            for (_i1245 = 0; _i1245 < _size1241; ++_i1245)
             {
-              std::string _key1253;
-              xfer += iprot->readString(_key1253);
-              Type& _val1254 = this->success[_key1253];
-              xfer += _val1254.read(iprot);
+              std::string _key1246;
+              xfer += iprot->readString(_key1246);
+              Type& _val1247 = this->success[_key1246];
+              xfer += _val1247.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -3764,11 +3764,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 _iter1255;
-      for (_iter1255 = this->success.begin(); _iter1255 != this->success.end(); ++_iter1255)
+      std::map<std::string, Type> ::const_iterator _iter1248;
+      for (_iter1248 = this->success.begin(); _iter1248 != this->success.end(); ++_iter1248)
       {
-        xfer += oprot->writeString(_iter1255->first);
-        xfer += _iter1255->second.write(oprot);
+        xfer += oprot->writeString(_iter1248->first);
+        xfer += _iter1248->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -3813,17 +3813,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1256;
-            ::apache::thrift::protocol::TType _ktype1257;
-            ::apache::thrift::protocol::TType _vtype1258;
-            xfer += iprot->readMapBegin(_ktype1257, _vtype1258, _size1256);
-            uint32_t _i1260;
-            for (_i1260 = 0; _i1260 < _size1256; ++_i1260)
+            uint32_t _size1249;
+            ::apache::thrift::protocol::TType _ktype1250;
+            ::apache::thrift::protocol::TType _vtype1251;
+            xfer += iprot->readMapBegin(_ktype1250, _vtype1251, _size1249);
+            uint32_t _i1253;
+            for (_i1253 = 0; _i1253 < _size1249; ++_i1253)
             {
-              std::string _key1261;
-              xfer += iprot->readString(_key1261);
-              Type& _val1262 = (*(this->success))[_key1261];
-              xfer += _val1262.read(iprot);
+              std::string _key1254;
+              xfer += iprot->readString(_key1254);
+              Type& _val1255 = (*(this->success))[_key1254];
+              xfer += _val1255.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -3977,14 +3977,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1263;
-            ::apache::thrift::protocol::TType _etype1266;
-            xfer += iprot->readListBegin(_etype1266, _size1263);
-            this->success.resize(_size1263);
-            uint32_t _i1267;
-            for (_i1267 = 0; _i1267 < _size1263; ++_i1267)
+            uint32_t _size1256;
+            ::apache::thrift::protocol::TType _etype1259;
+            xfer += iprot->readListBegin(_etype1259, _size1256);
+            this->success.resize(_size1256);
+            uint32_t _i1260;
+            for (_i1260 = 0; _i1260 < _size1256; ++_i1260)
             {
-              xfer += this->success[_i1267].read(iprot);
+              xfer += this->success[_i1260].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4039,10 +4039,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 _iter1268;
-      for (_iter1268 = this->success.begin(); _iter1268 != this->success.end(); ++_iter1268)
+      std::vector<FieldSchema> ::const_iterator _iter1261;
+      for (_iter1261 = this->success.begin(); _iter1261 != this->success.end(); ++_iter1261)
       {
-        xfer += (*_iter1268).write(oprot);
+        xfer += (*_iter1261).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4095,14 +4095,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1269;
-            ::apache::thrift::protocol::TType _etype1272;
-            xfer += iprot->readListBegin(_etype1272, _size1269);
-            (*(this->success)).resize(_size1269);
-            uint32_t _i1273;
-            for (_i1273 = 0; _i1273 < _size1269; ++_i1273)
+            uint32_t _size1262;
+            ::apache::thrift::protocol::TType _etype1265;
+            xfer += iprot->readListBegin(_etype1265, _size1262);
+            (*(this->success)).resize(_size1262);
+            uint32_t _i1266;
+            for (_i1266 = 0; _i1266 < _size1262; ++_i1266)
             {
-              xfer += (*(this->success))[_i1273].read(iprot);
+              xfer += (*(this->success))[_i1266].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4288,14 +4288,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1274;
-            ::apache::thrift::protocol::TType _etype1277;
-            xfer += iprot->readListBegin(_etype1277, _size1274);
-            this->success.resize(_size1274);
-            uint32_t _i1278;
-            for (_i1278 = 0; _i1278 < _size1274; ++_i1278)
+            uint32_t _size1267;
+            ::apache::thrift::protocol::TType _etype1270;
+            xfer += iprot->readListBegin(_etype1270, _size1267);
+            this->success.resize(_size1267);
+            uint32_t _i1271;
+            for (_i1271 = 0; _i1271 < _size1267; ++_i1271)
             {
-              xfer += this->success[_i1278].read(iprot);
+              xfer += this->success[_i1271].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4350,10 +4350,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 _iter1279;
-      for (_iter1279 = this->success.begin(); _iter1279 != this->success.end(); ++_iter1279)
+      std::vector<FieldSchema> ::const_iterator _iter1272;
+      for (_iter1272 = this->success.begin(); _iter1272 != this->success.end(); ++_iter1272)
       {
-        xfer += (*_iter1279).write(oprot);
+        xfer += (*_iter1272).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4406,14 +4406,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1280;
-            ::apache::thrift::protocol::TType _etype1283;
-            xfer += iprot->readListBegin(_etype1283, _size1280);
-            (*(this->success)).resize(_size1280);
-            uint32_t _i1284;
-            for (_i1284 = 0; _i1284 < _size1280; ++_i1284)
+            uint32_t _size1273;
+            ::apache::thrift::protocol::TType _etype1276;
+            xfer += iprot->readListBegin(_etype1276, _size1273);
+            (*(this->success)).resize(_size1273);
+            uint32_t _i1277;
+            for (_i1277 = 0; _i1277 < _size1273; ++_i1277)
             {
-              xfer += (*(this->success))[_i1284].read(iprot);
+              xfer += (*(this->success))[_i1277].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4583,14 +4583,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1285;
-            ::apache::thrift::protocol::TType _etype1288;
-            xfer += iprot->readListBegin(_etype1288, _size1285);
-            this->success.resize(_size1285);
-            uint32_t _i1289;
-            for (_i1289 = 0; _i1289 < _size1285; ++_i1289)
+            uint32_t _size1278;
+            ::apache::thrift::protocol::TType _etype1281;
+            xfer += iprot->readListBegin(_etype1281, _size1278);
+            this->success.resize(_size1278);
+            uint32_t _i1282;
+            for (_i1282 = 0; _i1282 < _size1278; ++_i1282)
             {
-              xfer += this->success[_i1289].read(iprot);
+              xfer += this->success[_i1282].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4645,10 +4645,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 _iter1290;
-      for (_iter1290 = this->success.begin(); _iter1290 != this->success.end(); ++_iter1290)
+      std::vector<FieldSchema> ::const_iterator _iter1283;
+      for (_iter1283 = this->success.begin(); _iter1283 != this->success.end(); ++_iter1283)
       {
-        xfer += (*_iter1290).write(oprot);
+        xfer += (*_iter1283).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4701,14 +4701,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1291;
-            ::apache::thrift::protocol::TType _etype1294;
-            xfer += iprot->readListBegin(_etype1294, _size1291);
-            (*(this->success)).resize(_size1291);
-            uint32_t _i1295;
-            for (_i1295 = 0; _i1295 < _size1291; ++_i1295)
+            uint32_t _size1284;
+            ::apache::thrift::protocol::TType _etype1287;
+            xfer += iprot->readListBegin(_etype1287, _size1284);
+            (*(this->success)).resize(_size1284);
+            uint32_t _i1288;
+            for (_i1288 = 0; _i1288 < _size1284; ++_i1288)
             {
-              xfer += (*(this->success))[_i1295].read(iprot);
+              xfer += (*(this->success))[_i1288].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4894,14 +4894,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1296;
-            ::apache::thrift::protocol::TType _etype1299;
-            xfer += iprot->readListBegin(_etype1299, _size1296);
-            this->success.resize(_size1296);
-            uint32_t _i1300;
-            for (_i1300 = 0; _i1300 < _size1296; ++_i1300)
+            uint32_t _size1289;
+            ::apache::thrift::protocol::TType _etype1292;
+            xfer += iprot->readListBegin(_etype1292, _size1289);
+            this->success.resize(_size1289);
+            uint32_t _i1293;
+            for (_i1293 = 0; _i1293 < _size1289; ++_i1293)
             {
-              xfer += this->success[_i1300].read(iprot);
+              xfer += this->success[_i1293].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4956,10 +4956,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 _iter1301;
-      for (_iter1301 = this->success.begin(); _iter1301 != this->success.end(); ++_iter1301)
+      std::vector<FieldSchema> ::const_iterator _iter1294;
+      for (_iter1294 = this->success.begin(); _iter1294 != this->success.end(); ++_iter1294)
       {
-        xfer += (*_iter1301).write(oprot);
+        xfer += (*_iter1294).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5012,14 +5012,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1302;
-            ::apache::thrift::protocol::TType _etype1305;
-            xfer += iprot->readListBegin(_etype1305, _size1302);
-            (*(this->success)).resize(_size1302);
-            uint32_t _i1306;
-            for (_i1306 = 0; _i1306 < _size1302; ++_i1306)
+            uint32_t _size1295;
+            ::apache::thrift::protocol::TType _etype1298;
+            xfer += iprot->readListBegin(_etype1298, _size1295);
+            (*(this->success)).resize(_size1295);
+            uint32_t _i1299;
+            for (_i1299 = 0; _i1299 < _size1295; ++_i1299)
             {
-              xfer += (*(this->success))[_i1306].read(iprot);
+              xfer += (*(this->success))[_i1299].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5612,14 +5612,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size1307;
-            ::apache::thrift::protocol::TType _etype1310;
-            xfer += iprot->readListBegin(_etype1310, _size1307);
-            this->primaryKeys.resize(_size1307);
-            uint32_t _i1311;
-            for (_i1311 = 0; _i1311 < _size1307; ++_i1311)
+            uint32_t _size1300;
+            ::apache::thrift::protocol::TType _etype1303;
+            xfer += iprot->readListBegin(_etype1303, _size1300);
+            this->primaryKeys.resize(_size1300);
+            uint32_t _i1304;
+            for (_i1304 = 0; _i1304 < _size1300; ++_i1304)
             {
-              xfer += this->primaryKeys[_i1311].read(iprot);
+              xfer += this->primaryKeys[_i1304].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5632,14 +5632,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size1312;
-            ::apache::thrift::protocol::TType _etype1315;
-            xfer += iprot->readListBegin(_etype1315, _size1312);
-            this->foreignKeys.resize(_size1312);
-            uint32_t _i1316;
-            for (_i1316 = 0; _i1316 < _size1312; ++_i1316)
+            uint32_t _size1305;
+            ::apache::thrift::protocol::TType _etype1308;
+            xfer += iprot->readListBegin(_etype1308, _size1305);
+            this->foreignKeys.resize(_size1305);
+            uint32_t _i1309;
+            for (_i1309 = 0; _i1309 < _size1305; ++_i1309)
             {
-              xfer += this->foreignKeys[_i1316].read(iprot);
+              xfer += this->foreignKeys[_i1309].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5652,14 +5652,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size1317;
-            ::apache::thrift::protocol::TType _etype1320;
-            xfer += iprot->readListBegin(_etype1320, _size1317);
-            this->uniqueConstraints.resize(_size1317);
-            uint32_t _i1321;
-            for (_i1321 = 0; _i1321 < _size1317; ++_i1321)
+            uint32_t _size1310;
+            ::apache::thrift::protocol::TType _etype1313;
+            xfer += iprot->readListBegin(_etype1313, _size1310);
+            this->uniqueConstraints.resize(_size1310);
+            uint32_t _i1314;
+            for (_i1314 = 0; _i1314 < _size1310; ++_i1314)
             {
-              xfer += this->uniqueConstraints[_i1321].read(iprot);
+              xfer += this->uniqueConstraints[_i1314].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5672,14 +5672,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraints.clear();
-            uint32_t _size1322;
-            ::apache::thrift::protocol::TType _etype1325;
-            xfer += iprot->readListBegin(_etype1325, _size1322);
-            this->notNullConstraints.resize(_size1322);
-            uint32_t _i1326;
-            for (_i1326 = 0; _i1326 < _size1322; ++_i1326)
+            uint32_t _size1315;
+            ::apache::thrift::protocol::TType _etype1318;
+            xfer += iprot->readListBegin(_etype1318, _size1315);
+            this->notNullConstraints.resize(_size1315);
+            uint32_t _i1319;
+            for (_i1319 = 0; _i1319 < _size1315; ++_i1319)
             {
-              xfer += this->notNullConstraints[_i1326].read(iprot);
+              xfer += this->notNullConstraints[_i1319].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5692,14 +5692,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->defaultConstraints.clear();
-            uint32_t _size1327;
-            ::apache::thrift::protocol::TType _etype1330;
-            xfer += iprot->readListBegin(_etype1330, _size1327);
-            this->defaultConstraints.resize(_size1327);
-            uint32_t _i1331;
-            for (_i1331 = 0; _i1331 < _size1327; ++_i1331)
+            uint32_t _size1320;
+            ::apache::thrift::protocol::TType _etype1323;
+            xfer += iprot->readListBegin(_etype1323, _size1320);
+            this->defaultConstraints.resize(_size1320);
+            uint32_t _i1324;
+            for (_i1324 = 0; _i1324 < _size1320; ++_i1324)
             {
-              xfer += this->defaultConstraints[_i1331].read(iprot);
+              xfer += this->defaultConstraints[_i1324].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5712,14 +5712,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->checkConstraints.clear();
-            uint32_t _size1332;
-            ::apache::thrift::protocol::TType _etype1335;
-            xfer += iprot->readListBegin(_etype1335, _size1332);
-            this->checkConstraints.resize(_size1332);
-            uint32_t _i1336;
-            for (_i1336 = 0; _i1336 < _size1332; ++_i1336)
+            uint32_t _size1325;
+            ::apache::thrift::protocol::TType _etype1328;
+            xfer += iprot->readListBegin(_etype1328, _size1325);
+            this->checkConstraints.resize(_size1325);
+            uint32_t _i1329;
+            for (_i1329 = 0; _i1329 < _size1325; ++_i1329)
             {
-              xfer += this->checkConstraints[_i1336].read(iprot);
+              xfer += this->checkConstraints[_i1329].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5752,10 +5752,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 _iter1337;
-    for (_iter1337 = this->primaryKeys.begin(); _iter1337 != this->primaryKeys.end(); ++_iter1337)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1330;
+    for (_iter1330 = this->primaryKeys.begin(); _iter1330 != this->primaryKeys.end(); ++_iter1330)
     {
-      xfer += (*_iter1337).write(oprot);
+      xfer += (*_iter1330).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5764,10 +5764,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 _iter1338;
-    for (_iter1338 = this->foreignKeys.begin(); _iter1338 != this->foreignKeys.end(); ++_iter1338)
+    std::vector<SQLForeignKey> ::const_iterator _iter1331;
+    for (_iter1331 = this->foreignKeys.begin(); _iter1331 != this->foreignKeys.end(); ++_iter1331)
     {
-      xfer += (*_iter1338).write(oprot);
+      xfer += (*_iter1331).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5776,10 +5776,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->uniqueConstraints.size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1339;
-    for (_iter1339 = this->uniqueConstraints.begin(); _iter1339 != this->uniqueConstraints.end(); ++_iter1339)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1332;
+    for (_iter1332 = this->uniqueConstraints.begin(); _iter1332 != this->uniqueConstraints.end(); ++_iter1332)
     {
-      xfer += (*_iter1339).write(oprot);
+      xfer += (*_iter1332).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5788,10 +5788,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->notNullConstraints.size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1340;
-    for (_iter1340 = this->notNullConstraints.begin(); _iter1340 != this->notNullConstraints.end(); ++_iter1340)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1333;
+    for (_iter1333 = this->notNullConstraints.begin(); _iter1333 != this->notNullConstraints.end(); ++_iter1333)
     {
-      xfer += (*_iter1340).write(oprot);
+      xfer += (*_iter1333).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5800,10 +5800,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->defaultConstraints.size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1341;
-    for (_iter1341 = this->defaultConstraints.begin(); _iter1341 != this->defaultConstraints.end(); ++_iter1341)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1334;
+    for (_iter1334 = this->defaultConstraints.begin(); _iter1334 != this->defaultConstraints.end(); ++_iter1334)
     {
-      xfer += (*_iter1341).write(oprot);
+      xfer += (*_iter1334).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5812,10 +5812,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->checkConstraints.size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter1342;
-    for (_iter1342 = this->checkConstraints.begin(); _iter1342 != this->checkConstraints.end(); ++_iter1342)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1335;
+    for (_iter1335 = this->checkConstraints.begin(); _iter1335 != this->checkConstraints.end(); ++_iter1335)
     {
-      xfer += (*_iter1342).write(oprot);
+      xfer += (*_iter1335).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5843,10 +5843,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 _iter1343;
-    for (_iter1343 = (*(this->primaryKeys)).begin(); _iter1343 != (*(this->primaryKeys)).end(); ++_iter1343)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1336;
+    for (_iter1336 = (*(this->primaryKeys)).begin(); _iter1336 != (*(this->primaryKeys)).end(); ++_iter1336)
     {
-      xfer += (*_iter1343).write(oprot);
+      xfer += (*_iter1336).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5855,10 +5855,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 _iter1344;
-    for (_iter1344 = (*(this->foreignKeys)).begin(); _iter1344 != (*(this->foreignKeys)).end(); ++_iter1344)
+    std::vector<SQLForeignKey> ::const_iterator _iter1337;
+    for (_iter1337 = (*(this->foreignKeys)).begin(); _iter1337 != (*(this->foreignKeys)).end(); ++_iter1337)
     {
-      xfer += (*_iter1344).write(oprot);
+      xfer += (*_iter1337).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5867,10 +5867,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->uniqueConstraints)).size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1345;
-    for (_iter1345 = (*(this->uniqueConstraints)).begin(); _iter1345 != (*(this->uniqueConstraints)).end(); ++_iter1345)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1338;
+    for (_iter1338 = (*(this->uniqueConstraints)).begin(); _iter1338 != (*(this->uniqueConstraints)).end(); ++_iter1338)
     {
-      xfer += (*_iter1345).write(oprot);
+      xfer += (*_iter1338).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5879,10 +5879,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->notNullConstraints)).size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1346;
-    for (_iter1346 = (*(this->notNullConstraints)).begin(); _iter1346 != (*(this->notNullConstraints)).end(); ++_iter1346)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1339;
+    for (_iter1339 = (*(this->notNullConstraints)).begin(); _iter1339 != (*(this->notNullConstraints)).end(); ++_iter1339)
     {
-      xfer += (*_iter1346).write(oprot);
+      xfer += (*_iter1339).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5891,10 +5891,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->defaultConstraints)).size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1347;
-    for (_iter1347 = (*(this->defaultConstraints)).begin(); _iter1347 != (*(this->defaultConstraints)).end(); ++_iter1347)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1340;
+    for (_iter1340 = (*(this->defaultConstraints)).begin(); _iter1340 != (*(this->defaultConstraints)).end(); ++_iter1340)
     {
-      xfer += (*_iter1347).write(oprot);
+      xfer += (*_iter1340).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5903,10 +5903,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->checkConstraints)).size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter1348;
-    for (_iter1348 = (*(this->checkConstraints)).begin(); _iter1348 != (*(this->checkConstraints)).end(); ++_iter1348)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1341;
+    for (_iter1341 = (*(this->checkConstraints)).begin(); _iter1341 != (*(this->checkConstraints)).end(); ++_iter1341)
     {
-      xfer += (*_iter1348).write(oprot);
+      xfer += (*_iter1341).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8074,14 +8074,14 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size1349;
-            ::apache::thrift::protocol::TType _etype1352;
-            xfer += iprot->readListBegin(_etype1352, _size1349);
-            this->partNames.resize(_size1349);
-            uint32_t _i1353;
-            for (_i1353 = 0; _i1353 < _size1349; ++_i1353)
+            uint32_t _size1342;
+            ::apache::thrift::protocol::TType _etype1345;
+            xfer += iprot->readListBegin(_etype1345, _size1342);
+            this->partNames.resize(_size1342);
+            uint32_t _i1346;
+            for (_i1346 = 0; _i1346 < _size1342; ++_i1346)
             {
-              xfer += iprot->readString(this->partNames[_i1353]);
+              xfer += iprot->readString(this->partNames[_i1346]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8118,10 +8118,10 @@ uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-    std::vector<std::string> ::const_iterator _iter1354;
-    for (_iter1354 = this->partNames.begin(); _iter1354 != this->partNames.end(); ++_iter1354)
+    std::vector<std::string> ::const_iterator _iter1347;
+    for (_iter1347 = this->partNames.begin(); _iter1347 != this->partNames.end(); ++_iter1347)
     {
-      xfer += oprot->writeString((*_iter1354));
+      xfer += oprot->writeString((*_iter1347));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8153,10 +8153,10 @@ uint32_t ThriftHiveMetastore_truncate_table_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partNames)).size()));
-    std::vector<std::string> ::const_iterator _iter1355;
-    for (_iter1355 = (*(this->partNames)).begin(); _iter1355 != (*(this->partNames)).end(); ++_iter1355)
+    std::vector<std::string> ::const_iterator _iter1348;
+    for (_iter1348 = (*(this->partNames)).begin(); _iter1348 != (*(this->partNames)).end(); ++_iter1348)
     {
-      xfer += oprot->writeString((*_iter1355));
+      xfer += oprot->writeString((*_iter1348));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8400,14 +8400,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1356;
-            ::apache::thrift::protocol::TType _etype1359;
-            xfer += iprot->readListBegin(_etype1359, _size1356);
-            this->success.resize(_size1356);
-            uint32_t _i1360;
-            for (_i1360 = 0; _i1360 < _size1356; ++_i1360)
+            uint32_t _size1349;
+            ::apache::thrift::protocol::TType _etype1352;
+            xfer += iprot->readListBegin(_etype1352, _size1349);
+            this->success.resize(_size1349);
+            uint32_t _i1353;
+            for (_i1353 = 0; _i1353 < _size1349; ++_i1353)
             {
-              xfer += iprot->readString(this->success[_i1360]);
+              xfer += iprot->readString(this->success[_i1353]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8446,10 +8446,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 _iter1361;
-      for (_iter1361 = this->success.begin(); _iter1361 != this->success.end(); ++_iter1361)
+      std::vector<std::string> ::const_iterator _iter1354;
+      for (_iter1354 = this->success.begin(); _iter1354 != this->success.end(); ++_iter1354)
       {
-        xfer += oprot->writeString((*_iter1361));
+        xfer += oprot->writeString((*_iter1354));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8494,14 +8494,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1362;
-            ::apache::thrift::protocol::TType _etype1365;
-            xfer += iprot->readListBegin(_etype1365, _size1362);
-            (*(this->success)).resize(_size1362);
-            uint32_t _i1366;
-            for (_i1366 = 0; _i1366 < _size1362; ++_i1366)
+            uint32_t _size1355;
+            ::apache::thrift::protocol::TType _etype1358;
+            xfer += iprot->readListBegin(_etype1358, _size1355);
+            (*(this->success)).resize(_size1355);
+            uint32_t _i1359;
+            for (_i1359 = 0; _i1359 < _size1355; ++_i1359)
             {
-              xfer += iprot->readString((*(this->success))[_i1366]);
+              xfer += iprot->readString((*(this->success))[_i1359]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8671,14 +8671,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 _size1367;
-            ::apache::thrift::protocol::TType _etype1370;
-            xfer += iprot->readListBegin(_etype1370, _size1367);
-            this->success.resize(_size1367);
-            uint32_t _i1371;
-            for (_i1371 = 0; _i1371 < _size1367; ++_i1371)
+            uint32_t _size1360;
+            ::apache::thrift::protocol::TType _etype1363;
+            xfer += iprot->readListBegin(_etype1363, _size1360);
+            this->success.resize(_size1360);
+            uint32_t _i1364;
+            for (_i1364 = 0; _i1364 < _size1360; ++_i1364)
             {
-              xfer += iprot->readString(this->success[_i1371]);
+              xfer += iprot->readString(this->success[_i1364]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8717,10 +8717,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 _iter1372;
-      for (_iter1372 = this->success.begin(); _iter1372 != this->success.end(); ++_iter1372)
+      std::vector<std::string> ::const_iterator _iter1365;
+      for (_iter1365 = this->success.begin(); _iter1365 != this->success.end(); ++_iter1365)
       {
-        xfer += oprot->writeString((*_iter1372));
+        xfer += oprot->writeString((*_iter1365));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8765,14 +8765,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1373;
-            ::apache::thrift::protocol::TType _etype1376;
-            xfer += iprot->readListBegin(_etype1376, _size1373);
-            (*(this->success)).resize(_size1373);
-            uint32_t _i1377;
-            for (_i1377 = 0; _i1377 < _size1373; ++_i1377)
+            uint32_t _size1366;
+            ::apache::thrift::protocol::TType _etype1369;
+            xfer += iprot->readListBegin(_etype1369, _size1366);
+            (*(this->success)).resize(_size1366);
+            uint32_t _i1370;
+            for (_i1370 = 0; _i1370 < _size1366; ++_i1370)
             {
-              xfer += iprot->readString((*(this->success))[_i1377]);
+              xfer += iprot->readString((*(this->success))[_i1370]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8910,14 +8910,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1378;
-            ::apache::thrift::protocol::TType _etype1381;
-            xfer += iprot->readListBegin(_etype1381, _size1378);
-            this->success.resize(_size1378);
-            uint32_t _i1382;
-            for (_i1382 = 0; _i1382 < _size1378; ++_i1382)
+            uint32_t _size1371;
+            ::apache::thrift::protocol::TType _etype1374;
+            xfer += iprot->readListBegin(_etype1374, _size1371);
+            this->success.resize(_size1371);
+            uint32_t _i1375;
+            for (_i1375 = 0; _i1375 < _size1371; ++_i1375)
             {
-              xfer += iprot->readString(this->success[_i1382]);
+              xfer += iprot->readString(this->success[_i1375]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8956,10 +8956,10 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::write(
     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 _iter1383;
-      for (_iter1383 = this->success.begin(); _iter1383 != this->success.end(); ++_iter1383)
+      std::vector<std::string> ::const_iterator _iter1376;
+      for (_iter1376 = this->success.begin(); _iter1376 != this->success.end(); ++_iter1376)
       {
-        xfer += oprot->writeString((*_iter1383));
+        xfer += oprot->writeString((*_iter1376));
       }
       xfer += oprot->writeListEnd();
     }
@@ -9004,14 +9004,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_presult::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1384;
-            ::apache::thrift::protocol::TType _etype1387;
-            xfer += iprot->readListBegin(_etype1387, _size1384);
-            (*(this->success)).resize(_size1384);
-            uint32_t _i1388;
-            for (_i1388 = 0; _i1388 < _size1384; ++_i1388)
+            uint32_t _size1377;
+            ::apache::thrift::protocol::TType _etype1380;
+            xfer += iprot->readListBegin(_etype1380, _size1377);
+            (*(this->success)).resize(_size1377);
+            uint32_t _i1381;
+            for (_i1381 = 0; _i1381 < _size1377; ++_i1381)
             {
-              xfer += iprot->readString((*(this->success))[_i1388]);
+              xfer += iprot->readString((*(this->success))[_i1381]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9086,14 +9086,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 _size1389;
-            ::apache::thrift::protocol::TType _etype1392;
-            xfer += iprot->readListBegin(_etype1392, _size1389);
-            this->tbl_types.resize(_size1389);
-            uint32_t _i1393;
-            for (_i1393 = 0; _i1393 < _size1389; ++_i1393)
+            uint32_t _size1382;
+            ::apache::thrift::protocol::TType _etype1385;
+            xfer += iprot->readListBegin(_etype1385, _size1382);
+            this->tbl_types.resize(_size1382);
+            uint32_t _i1386;
+            for (_i1386 = 0; _i1386 < _size1382; ++_i1386)
             {
-              xfer += iprot->readString(this->tbl_types[_i1393]);
+              xfer += iprot->readString(this->tbl_types[_i1386]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9130,10 +9130,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 _iter1394;
-    for (_iter1394 = this->tbl_types.begin(); _iter1394 != this->tbl_types.end(); ++_iter1394)
+    std::vector<std::string> ::const_iterator _iter1387;
+    for (_iter1387 = this->tbl_types.begin(); _iter1387 != this->tbl_types.end(); ++_iter1387)
     {
-      xfer += oprot->writeString((*_iter1394));
+      xfer += oprot->writeString((*_iter1387));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9165,10 +9165,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 _iter1395;
-    for (_iter1395 = (*(this->tbl_types)).begin(); _iter1395 != (*(this->tbl_types)).end(); ++_iter1395)
+    std::vector<std::string> ::const_iterator _iter1388;
+    for (_iter1388 = (*(this->tbl_types)).begin(); _iter1388 != (*(this->tbl_types)).end(); ++_iter1388)
     {
-      xfer += oprot->writeString((*_iter1395));
+      xfer += oprot->writeString((*_iter1388));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9209,14 +9209,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1396;
-            ::apache::thrift::protocol::TType _etype1399;
-            xfer += iprot->readListBegin(_etype1399, _size1396);
-            this->success.resize(_size1396);
-            uint32_t _i1400;
-            for (_i1400 = 0; _i1400 < _size1396; ++_i1400)
+            uint32_t _size1389;
+            ::apache::thrift::protocol::TType _etype1392;
+            xfer += iprot->readListBegin(_etype1392, _size1389);
+            this->success.resize(_size1389);
+            uint32_t _i1393;
+            for (_i1393 = 0; _i1393 < _size1389; ++_i1393)
             {
-              xfer += this->success[_i1400].read(iprot);
+              xfer += this->success[_i1393].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9255,10 +9255,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 _iter1401;
-      for (_iter1401 = this->success.begin(); _iter1401 != this->success.end(); ++_iter1401)
+      std::vector<TableMeta> ::const_iterator _iter1394;
+      for (_iter1394 = this->success.begin(); _iter1394 != this->success.end(); ++_iter1394)
       {
-        xfer += (*_iter1401).write(oprot);
+        xfer += (*_iter1394).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -9303,14 +9303,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1402;
-            ::apache::thrift::protocol::TType _etype1405;
-            xfer += iprot->readListBegin(_etype1405, _size1402);
-            (*(this->success)).resize(_size1402);
-            uint32_t _i1406;
-            for (_i1406 = 0; _i1406 < _size1402; ++_i1406)
+            uint32_t _size1395;
+            ::apache::thrift::protocol::TType _etype1398;
+            xfer += iprot->readListBegin(_etype1398, _size1395);
+            (*(this->success)).resize(_size1395);
+            uint32_t _i1399;
+            for (_i1399 = 0; _i1399 < _size1395; ++_i1399)
             {
-              xfer += (*(this->success))[_i1406].read(iprot);
+              xfer += (*(this->success))[_i1399].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9448,14 +9448,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1407;
-            ::apache::thrift::protocol::TType _etype1410;
-            xfer += iprot->readListBegin(_etype1410, _size1407);
-            this->success.resize(_size1407);
-            uint32_t _i1411;
-            for (_i1411 = 0; _i1411 < _size1407; ++_i1411)
+            uint32_t _size1400;
+            ::apache::thrift::protocol::TType _etype1403;
+            xfer += iprot->readListBegin(_etype1403, _size1400);
+            this->success.resize(_size1400);
+            uint32_t _i1404;
+            for (_i1404 = 0; _i1404 < _size1400; ++_i1404)
             {
-              xfer += iprot->readString(this->success[_i1411]);
+              xfer += iprot->readString(this->success[_i1404]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9494,10 +9494,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 _iter1412;
-      for (_iter1412 = this->success.begin(); _iter1412 != this->success.end(); ++_iter1412)
+      std::vector<std::string> ::const_iterator _iter1405;
+      for (_iter1405 = this->success.begin(); _iter1405 != this->success.end(); ++_iter1405)
       {
-        xfer += oprot->writeString((*_iter1412));
+        xfer += oprot->writeString((*_iter1405));
       }
       xfer += oprot->writeListEnd();
     }
@@ -9542,14 +9542,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1413;
-            ::apache::thrift::protocol::TType _etype1416;
-            xfer += iprot->readListBegin(_etype1416, _size1413);
-            (*(this->success)).resize(_size1413);
-            uint32_t _i1417;
-            for (_i1417 = 0; _i1417 < _size1413; ++_i1417)
+            uint32_t _size1406;
+            ::apache::thrift::protocol::TType _etype1409;
+            xfer += iprot->readListBegin(_etype1409, _size1406);
+            (*(this->success)).resize(_size1406);
+            uint32_t _i1410;
+            for (_i1410 = 0; _i1410 < _size1406; ++_i1410)
             {
-              xfer += iprot->readString((*(this->success))[_i1417]);
+              xfer += iprot->readString((*(this->success))[_i1410]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9859,14 +9859,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 _size1418;
-            ::apache::thrift::protocol::TType _etype1421;
-            xfer += iprot->readListBegin(_etype1421, _size1418);
-            this->tbl_names.resize(_size1418);
-            uint32_t _i1422;
-            for (_i1422 = 0; _i1422 < _size1418; ++_i1422)
+            uint32_t _size1411;
+            ::apache::thrift::protocol::TType _etype1414;
+            xfer += iprot->readListBegin(_etype1414, _size1411);
+            this->tbl_names.resize(_size1411);
+            uint32_t _i1415;
+            for (_i1415 = 0; _i1415 < _size1411; ++_i1415)
             {
-              xfer += iprot->readString(this->tbl_names[_i1422]);
+              xfer += iprot->readString(this->tbl_names[_i1415]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9899,10 +9899,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 _iter1423;
-    for (_iter1423 = this->tbl_names.begin(); _iter1423 != this->tbl_names.end(); ++_iter1423)
+    std::vector<std::string> ::const_iterator _iter1416;
+    for (_iter1416 = this->tbl_names.begin(); _iter1416 != this->tbl_names.end(); ++_iter1416)
     {
-      xfer += oprot->writeString((*_iter1423));
+      xfer += oprot->writeString((*_iter1416));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9930,10 +9930,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 _iter1424;
-    for (_iter1424 = (*(this->tbl_names)).begin(); _iter1424 != (*(this->tbl_names)).end(); ++_iter1424)
+    std::vector<std::string> ::const_iterator _iter1417;
+    for (_iter1417 = (*(this->tbl_names)).begin(); _iter1417 != (*(this->tbl_names)).end(); ++_iter1417)
     {
-      xfer += oprot->writeString((*_iter1424));
+      xfer += oprot->writeString((*_iter1417));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9974,14 +9974,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 _size1425;
-            ::apache::thrift::protocol::TType _etype1428;
-            xfer += iprot->readListBegin(_etype1428, _size1425);
-            this->success.resize(_size1425);
-            uint32_t _i1429;
-            for (_i1429 = 0; _i1429 < _size1425; ++_i1429)
+            uint32_t _size1418;
+            ::apache::thrift::protocol::TType _etype1421;
+            xfer += iprot->readListBegin(_etype1421, _size1418);
+            this->success.resize(_size1418);
+            uint32_t _i1422;
+            for (_i1422 = 0; _i1422 < _size1418; ++_i1422)
             {
-              xfer += this->success[_i1429].read(iprot);
+              xfer += this->success[_i1422].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10012,10 +10012,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 _iter1430;
-      for (_iter1430 = this->success.begin(); _iter1430 != this->success.end(); ++_iter1430)
+      std::vector<Table> ::const_iterator _iter1423;
+      for (_iter1423 = this->success.begin(); _iter1423 != this->success.end(); ++_iter1423)
       {
-        xfer += (*_iter1430).write(oprot);
+        xfer += (*_iter1423).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -10056,14 +10056,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 _size1431;
-            ::apache::thrift::protocol::TType _etype1434;
-            xfer += iprot->readListBegin(_etype1434, _size1431);
-            (*(this->success)).resize(_size1431);
-            uint32_t _i1435;
-            for (_i1435 = 0; _i1435 < _size1431; ++_i1435)
+            uint32_t _size1424;
+            ::apache::thrift::protocol::TType _etype1427;
+            xfer += iprot->readListBegin(_etype1427, _size1424);
+            (*(this->success)).resize(_size1424);
+            uint32_t _i1428;
+            for (_i1428 = 0; _i1428 < _size1424; ++_i1428)
             {
-              xfer += (*(this->success))[_i1435].read(iprot);
+              xfer += (*(this->success))[_i1428].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10585,29 +10585,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::read(::
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->dbname);
-          this->__isset.dbname = true;
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->creation_metadata.read(iprot);
+          this->__isset.creation_metadata = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->tbl_names.clear();
-            uint32_t _size1436;
-            ::apache::thrift::protocol::TType _etype1439;
-            xfer += iprot->readListBegin(_etype1439, _size1436);
-            this->tbl_names.resize(_size1436);
-            uint32_t _i1440;
-            for (_i1440 = 0; _i1440 < _size1436; ++_i1440)
-            {
-              xfer += iprot->readString(this->tbl_names[_i1440]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.tbl_names = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->validTxnList);
+          this->__isset.validTxnList = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -10629,20 +10617,12 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::write(:
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
   xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_materialization_invalidation_info_args");
 
-  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->dbname);
+  xfer += oprot->writeFieldBegin("creation_metadata", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->creation_metadata.write(oprot);
   xfer += oprot->writeFieldEnd();
 
-  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 _iter1441;
-    for (_iter1441 = this->tbl_names.begin(); _iter1441 != this->tbl_names.end(); ++_iter1441)
-    {
-      xfer += oprot->writeString((*_iter1441));
-    }
-    xfer += oprot->writeListEnd();
-  }
+  xfer += oprot->writeFieldBegin("validTxnList", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->validTxnList);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -10660,20 +10640,12 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_pargs::write(
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
   xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_materialization_invalidation_info_pargs");
 
-  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->dbname)));
+  xfer += oprot->writeFieldBegin("creation_metadata", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->creation_metadata)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
-  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 _iter1442;
-    for (_iter1442 = (*(this->tbl_names)).begin(); _iter1442 != (*(this->tbl_names)).end(); ++_iter1442)
-    {
-      xfer += oprot->writeString((*_iter1442));
-    }
-    xfer += oprot->writeListEnd();
-  }
+  xfer += oprot->writeFieldBegin("validTxnList", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString((*(this->validTxnList)));
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -10708,23 +10680,8 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::read(
     switch (fid)
     {
       case 0:
-        if (ftype == ::apache::thrift::protocol::T_MAP) {
-          {
-            this->success.clear();
-            uint32_t _size1443;
-            ::apache::thrift::protocol::TType _ktype1444;
-            ::apache::thrift::protocol::TType _vtype1445;
-            xfer += iprot->readMapBegin(_ktype1444, _vtype1445, _size1443);
-            uint32_t _i1447;
-            for (_i1447 = 0; _i1447 < _size1443; ++_i1447)
-            {
-              std::string _key1448;
-              xfer += iprot->readString(_key1448);
-              Materialization& _val1449 = this->success[_key1448];
-              xfer += _val1449.read(iprot);
-            }
-            xfer += iprot->readMapEnd();
-          }
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->success.read(iprot);
           this->__isset.success = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -10773,17 +10730,8 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::write
   xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_materialization_invalidation_info_result");
 
   if (this->__isset.success) {
-    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, Materialization> ::const_iterator _iter1450;
-      for (_iter1450 = this->success.begin(); _iter1450 != this->success.end(); ++_iter1450)
-      {
-        xfer += oprot->writeString(_iter1450->first);
-        xfer += _iter1450->second.write(oprot);
-      }
-      xfer += oprot->writeMapEnd();
-    }
+    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
+    xfer += this->success.write(oprot);
     xfer += oprot->writeFieldEnd();
   } else if (this->__isset.o1) {
     xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
@@ -10830,23 +10778,8 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_presult::read
     switch (fid)
     {
       case 0:
-        if (ftype == ::apache::thrift::protocol::T_MAP) {
-          {
-            (*(this->success)).clear();
-            uint32_t _size1451;
-            ::apache::thrift::protocol::TType _ktype1452;
-            ::apache::thrift::protocol::TType _vtype1453;
-            xfer += iprot->readMapBegin(_ktype1452, _vtype1453, _size1451);
-            uint32_t _i1455;
-            for (_i1455 = 0; _i1455 < _size1451; ++_i1455)
-            {
-              std::string _key1456;
-              xfer += iprot->readString(_key1456);
-              Materialization& _val1457 = (*(this->success))[_key1456];
-              xfer += _val1457.read(iprot);
-            }
-            xfer += iprot->readMapEnd();
-          }
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += (*(this->success)).read(iprot);
           this->__isset.success = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -11304,14 +11237,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 _size1458;
-            ::apache::thrift::protocol::TType _etype1461;
-            xfer += iprot->readListBegin(_etype1461, _size1458);
-            this->success.resize(_size1458);
-            uint32_t _i1462;
-            for (_i1462 = 0; _i1462 < _size1458; ++_i1462)
+            uint32_t _size1429;
+            ::apache::thrift::protocol::TType _etype1432;
+            xfer += iprot->readListBegin(_etype1432, _size1429);
+            this->success.resize(_size1429);
+            uint32_t _i1433;
+            for (_i1433 = 0; _i1433 < _size1429; ++_i1433)
             {
-              xfer += iprot->readString(this->success[_i1462]);
+              xfer += iprot->readString(this->success[_i1433]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11366,10 +11299,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 _iter1463;
-      for (_iter1463 = this->success.begin(); _iter1463 != this->success.end(); ++_iter1463)
+      std::vector<std::string> ::const_iterator _iter1434;
+      for (_iter1434 = this->success.begin(); _iter1434 != this->success.end(); ++_iter1434)
       {
-        xfer += oprot->writeString((*_iter1463));
+        xfer += oprot->writeString((*_iter1434));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11422,14 +11355,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 _size1464;
-            ::apache::thrift::protocol::TType _etype1467;
-            xfer += iprot->readListBegin(_etype1467, _size1464);
-            (*(this->success)).resize(_size1464);
-            uint32_t _i1468;
-            for (_i1468 = 0; _i1468 < _size1464; ++_i1468)
+            uint32_t _size1435;
+            ::apache::thrift::protocol::TType _etype1438;
+            xfer += iprot->readListBegin(_etype1438, _size1435);
+            (*(this->success)).resize(_size1435);
+            uint32_t _i1439;
+            for (_i1439 = 0; _i1439 < _size1435; ++_i1439)
             {
-              xfer += iprot->readString((*(this->success))[_i1468]);
+              xfer += iprot->readString((*(this->success))[_i1439]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12763,14 +12696,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size1469;
-            ::apache::thrift::protocol::TType _etype1472;
-            xfer += iprot->readListBegin(_etype1472, _size1469);
-            this->new_parts.resize(_size1469);
-            uint32_t _i1473;
-            for (_i1473 = 0; _i1473 < _size1469; ++_i1473)
+            uint32_t _size1440;
+            ::apache::thrift::protocol::TType _etype1443;
+            xfer += iprot->readListBegin(_etype1443, _size1440);
+            this->new_parts.resize(_size1440);
+            uint32_t _i1444;
+            for (_i1444 = 0; _i1444 < _size1440; ++_i1444)
             {
-              xfer += this->new_parts[_i1473].read(iprot);
+              xfer += this->new_parts[_i1444].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12799,10 +12732,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 _iter1474;
-    for (_iter1474 = this->new_parts.begin(); _iter1474 != this->new_parts.end(); ++_iter1474)
+    std::vector<Partition> ::const_iterator _iter1445;
+    for (_iter1445 = this->new_parts.begin(); _iter1445 != this->new_parts.end(); ++_iter1445)
     {
-      xfer += (*_iter1474).write(oprot);
+      xfer += (*_iter1445).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -12826,10 +12759,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 _iter1475;
-    for (_iter1475 = (*(this->new_parts)).begin(); _iter1475 != (*(this->new_parts)).end(); ++_iter1475)
+    std::vector<Partition> ::const_iterator _iter1446;
+    for (_iter1446 = (*(this->new_parts)).begin(); _iter1446 != (*(this->new_parts)).end(); ++_iter1446)
     {
-      xfer += (*_iter1475).write(oprot);
+      xfer += (*_iter1446).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13038,14 +12971,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 _size1476;
-            ::apache::thrift::protocol::TType _etype1479;
-            xfer += iprot->readListBegin(_etype1479, _size1476);
-            this->new_parts.resize(_size1476);
-            uint32_t _i1480;
-            for (_i1480 = 0; _i1480 < _size1476; ++_i1480)
+            uint32_t _size1447;
+            ::apache::thrift::protocol::TType _etype1450;
+            xfer += iprot->readListBegin(_etype1450, _size1447);
+            this->new_parts.resize(_size1447);
+            uint32_t _i1451;
+            for (_i1451 = 0; _i1451 < _size1447; ++_i1451)
             {
-              xfer += this->new_parts[_i1480].read(iprot);
+              xfer += this->new_parts[_i1451].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13074,10 +13007,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 _iter1481;
-    for (_iter1481 = this->new_parts.begin(); _iter1481 != this->new_parts.end(); ++_iter1481)
+    std::vector<PartitionSpec> ::const_iterator _iter1452;
+    for (_iter1452 = this->new_parts.begin(); _iter1452 != this->new_parts.end(); ++_iter1452)
     {
-      xfer += (*_iter1481).write(oprot);
+      xfer += (*_iter1452).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13101,10 +13034,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 _iter1482;
-    for (_iter1482 = (*(this->new_parts)).begin(); _iter1482 != (*(this->new_parts)).end(); ++_iter1482)
+    std::vector<PartitionSpec> ::const_iterator _iter1453;
+    for (_iter1453 = (*(this->new_parts)).begin(); _iter1453 != (*(this->new_parts)).end(); ++_iter1453)
     {
-      xfer += (*_iter1482).write(oprot);
+      xfer += (*_iter1453).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13329,14 +13262,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1483;
-            ::apache::thrift::protocol::TType _etype1486;
-            xfer += iprot->readListBegin(_etype1486, _size1483);
-            this->part_vals.resize(_size1483);
-            uint32_t _i1487;
-            for (_i1487 = 0; _i1487 < _size1483; ++_i1487)
+            uint32_t _size1454;
+            ::apache::thrift::protocol::TType _etype1457;
+            xfer += iprot->readListBegin(_etype1457, _size1454);
+            this->part_vals.resize(_size1454);
+            uint32_t _i1458;
+            for (_i1458 = 0; _i1458 < _size1454; ++_i1458)
             {
-              xfer += iprot->readString(this->part_vals[_i1487]);
+              xfer += iprot->readString(this->part_vals[_i1458]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13373,10 +13306,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 _iter1488;
-    for (_iter1488 = this->part_vals.begin(); _iter1488 != this->part_vals.end(); ++_iter1488)
+    std::vector<std::string> ::const_iterator _iter1459;
+    for (_iter1459 = this->part_vals.begin(); _iter1459 != this->part_vals.end(); ++_iter1459)
     {
-      xfer += oprot->writeString((*_iter1488));
+      xfer += oprot->writeString((*_iter1459));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13408,10 +13341,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 _iter1489;
-    for (_iter1489 = (*(this->part_vals)).begin(); _iter1489 != (*(this->part_vals)).end(); ++_iter1489)
+    std::vector<std::string> ::const_iterator _iter1460;
+    for (_iter1460 = (*(this->part_vals)).begin(); _iter1460 != (*(this->part_vals)).end(); ++_iter1460)
     {
-      xfer += oprot->writeString((*_iter1489));
+      xfer += oprot->writeString((*_iter1460));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13883,14 +13816,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1490;
-            ::apache::thrift::protocol::TType _etype1493;
-            xfer += iprot->readListBegin(_etype1493, _size1490);
-            this->part_vals.resize(_size1490);
-            uint32_t _i1494;
-            for (_i1494 = 0; _i1494 < _size1490; ++_i1494)
+            uint32_t _size1461;
+            ::apache::thrift::protocol::TType _etype1464;
+            xfer += iprot->readListBegin(_etype1464, _size1461);
+            this->part_vals.resize(_size1461);
+            uint32_t _i1465;
+            for (_i1465 = 0; _i1465 < _size1461; ++_i1465)
             {
-              xfer += iprot->readString(this->part_vals[_i1494]);
+              xfer += iprot->readString(this->part_vals[_i1465]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13935,10 +13868,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 _iter1495;
-    for (_iter1495 = this->part_vals.begin(); _iter1495 != this->part_vals.end(); ++_iter1495)
+    std::vector<std::string> ::const_iterator _iter1466;
+    for (_iter1466 = this->part_vals.begin(); _iter1466 != this->part_vals.end(); ++_iter1466)
     {
-      xfer += oprot->writeString((*_iter1495));
+      xfer += oprot->writeString((*_iter1466));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13974,10 +13907,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 _iter1496;
-    for (_iter1496 = (*(this->part_vals)).begin(); _iter1496 != (*(this->part_vals)).end(); ++_iter1496)
+    std::vector<std::string> ::const_iterator _iter1467;
+    for (_iter1467 = (*(this->part_vals)).begin(); _iter1467 != (*(this->part_vals)).end(); ++_iter1467)
     {
-      xfer += oprot->writeString((*_iter1496));
+      xfer += oprot->writeString((*_iter1467));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14780,14 +14713,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1497;
-            ::apache::thrift::protocol::TType _etype1500;
-            xfer += iprot->readListBegin(_etype1500, _size1497);
-            this->part_vals.resize(_size1497);
-            uint32_t _i1501;
-            for (_i1501 = 0; _i1501 < _size1497; ++_i1501)
+            uint32_t _size1468;
+            ::apache::thrift::protocol::TType _etype1471;
+            xfer += iprot->readListBegin(_etype1471, _size1468);
+            this->part_vals.resize(_size1468);
+            uint32_t _i1472;
+            for (_i1472 = 0; _i1472 < _size1468; ++_i1472)
             {
-              xfer += iprot->readString(this->part_vals[_i1501]);
+              xfer += iprot->readString(this->part_vals[_i1472]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14832,10 +14765,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 _iter1502;
-    for (_iter1502 = this->part_vals.begin(); _iter1502 != this->part_vals.end(); ++_iter1502)
+    std::vector<std::string> ::const_iterator _iter1473;
+    for (_iter1473 = this->part_vals.begin(); _iter1473 != this->part_vals.end(); ++_iter1473)
     {
-      xfer += oprot->writeString((*_iter1502));
+      xfer += oprot->writeString((*_iter1473));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14871,10 +14804,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 _iter1503;
-    for (_iter1503 = (*(this->part_vals)).begin(); _iter1503 != (*(this->part_vals)).end(); ++_iter1503)
+    std::vector<std::string> ::const_iterator _iter1474;
+    for (_iter1474 = (*(this->part_vals)).begin(); _iter1474 != (*(this->part_vals)).end(); ++_iter1474)
     {
-      xfer += oprot->writeString((*_iter1503));
+      xfer += oprot->writeString((*_iter1474));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15083,14 +15016,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1504;
-            ::apache::thrift::protocol::TType _etype1507;
-            xfer += iprot->readListBegin(_etype1507, _size1504);
-            this->part_vals.resize(_size1504);
-            uint32_t _i1508;
-            for (_i1508 = 0; _i1508 < _size1504; ++_i1508)
+            uint32_t _size1475;
+            ::apache::thrift::protocol::TType _etype1478;
+            xfer += iprot->readListBegin(_etype1478, _size1475);
+            this->part_vals.resize(_size1475);
+            uint32_t _i1479;
+            for (_i1479 = 0; _i1479 < _size1475; ++_i1479)
             {
-              xfer += iprot->readString(this->part_vals[_i1508]);
+              xfer += iprot->readString(this->part_vals[_i1479]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15143,10 +15076,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 _iter1509;
-    for (_iter1509 = this->part_vals.begin(); _iter1509 != this->part_vals.end(); ++_iter1509)
+    std::vector<std::string> ::const_iterator _iter1480;
+    for (_iter1480 = this->part_vals.begin(); _iter1480 != this->part_vals.end(); ++_iter1480)
     {
-      xfer += oprot->writeString((*_iter1509));
+      xfer += oprot->writeString((*_iter1480));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15186,10 +15119,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 _iter1510;
-    for (_iter1510 = (*(this->part_vals)).begin(); _iter1510 != (*(this->part_vals)).end(); ++_iter1510)
+    std::vector<std::string> ::const_iterator _iter1481;
+    for (_iter1481 = (*(this->part_vals)).begin(); _iter1481 != (*(this->part_vals)).end(); ++_iter1481)
     {
-      xfer += oprot->writeString((*_iter1510));
+      xfer += oprot->writeString((*_iter1481));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16195,14 +16128,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1511;
-            ::apache::thrift::protocol::TType _etype1514;
-            xfer += iprot->readListBegin(_etype1514, _size1511);
-            this->part_vals.resize(_size1511);
-            uint32_t _i1515;
-            for (_i1515 = 0; _i1515 < _size1511; ++_i1515)
+            uint32_t _size1482;
+            ::apache::thrift::protocol::TType _etype1485;
+            xfer += iprot->readListBegin(_etype1485, _size1482);
+            this->part_vals.resize(_size1482);
+            uint32_t _i1486;
+            for (_i1486 = 0; _i1486 < _size1482; ++_i1486)
             {
-              xfer += iprot->readString(this->part_vals[_i1515]);
+              xfer += iprot->readString(this->part_vals[_i1486]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16239,10 +16172,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 _iter1516;
-    for (_iter1516 = this->part_vals.begin(); _iter1516 != this->part_vals.end(); ++_iter1516)
+    std::vector<std::string> ::const_it

<TRUNCATED>

[06/12] hive git commit: HIVE-20006: Make materializations invalidation cache work with multiple active remote metastores (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
index 4467479..f0c308d 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
@@ -755,14 +755,14 @@ import org.slf4j.LoggerFactory;
           case 2: // POOLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list872 = iprot.readListBegin();
-                struct.pools = new ArrayList<WMPool>(_list872.size);
-                WMPool _elem873;
-                for (int _i874 = 0; _i874 < _list872.size; ++_i874)
+                org.apache.thrift.protocol.TList _list864 = iprot.readListBegin();
+                struct.pools = new ArrayList<WMPool>(_list864.size);
+                WMPool _elem865;
+                for (int _i866 = 0; _i866 < _list864.size; ++_i866)
                 {
-                  _elem873 = new WMPool();
-                  _elem873.read(iprot);
-                  struct.pools.add(_elem873);
+                  _elem865 = new WMPool();
+                  _elem865.read(iprot);
+                  struct.pools.add(_elem865);
                 }
                 iprot.readListEnd();
               }
@@ -774,14 +774,14 @@ import org.slf4j.LoggerFactory;
           case 3: // MAPPINGS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list875 = iprot.readListBegin();
-                struct.mappings = new ArrayList<WMMapping>(_list875.size);
-                WMMapping _elem876;
-                for (int _i877 = 0; _i877 < _list875.size; ++_i877)
+                org.apache.thrift.protocol.TList _list867 = iprot.readListBegin();
+                struct.mappings = new ArrayList<WMMapping>(_list867.size);
+                WMMapping _elem868;
+                for (int _i869 = 0; _i869 < _list867.size; ++_i869)
                 {
-                  _elem876 = new WMMapping();
-                  _elem876.read(iprot);
-                  struct.mappings.add(_elem876);
+                  _elem868 = new WMMapping();
+                  _elem868.read(iprot);
+                  struct.mappings.add(_elem868);
                 }
                 iprot.readListEnd();
               }
@@ -793,14 +793,14 @@ import org.slf4j.LoggerFactory;
           case 4: // TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list878 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list878.size);
-                WMTrigger _elem879;
-                for (int _i880 = 0; _i880 < _list878.size; ++_i880)
+                org.apache.thrift.protocol.TList _list870 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list870.size);
+                WMTrigger _elem871;
+                for (int _i872 = 0; _i872 < _list870.size; ++_i872)
                 {
-                  _elem879 = new WMTrigger();
-                  _elem879.read(iprot);
-                  struct.triggers.add(_elem879);
+                  _elem871 = new WMTrigger();
+                  _elem871.read(iprot);
+                  struct.triggers.add(_elem871);
                 }
                 iprot.readListEnd();
               }
@@ -812,14 +812,14 @@ import org.slf4j.LoggerFactory;
           case 5: // POOL_TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list881 = iprot.readListBegin();
-                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list881.size);
-                WMPoolTrigger _elem882;
-                for (int _i883 = 0; _i883 < _list881.size; ++_i883)
+                org.apache.thrift.protocol.TList _list873 = iprot.readListBegin();
+                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list873.size);
+                WMPoolTrigger _elem874;
+                for (int _i875 = 0; _i875 < _list873.size; ++_i875)
                 {
-                  _elem882 = new WMPoolTrigger();
-                  _elem882.read(iprot);
-                  struct.poolTriggers.add(_elem882);
+                  _elem874 = new WMPoolTrigger();
+                  _elem874.read(iprot);
+                  struct.poolTriggers.add(_elem874);
                 }
                 iprot.readListEnd();
               }
@@ -850,9 +850,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(POOLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.pools.size()));
-          for (WMPool _iter884 : struct.pools)
+          for (WMPool _iter876 : struct.pools)
           {
-            _iter884.write(oprot);
+            _iter876.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -863,9 +863,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(MAPPINGS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mappings.size()));
-            for (WMMapping _iter885 : struct.mappings)
+            for (WMMapping _iter877 : struct.mappings)
             {
-              _iter885.write(oprot);
+              _iter877.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -877,9 +877,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size()));
-            for (WMTrigger _iter886 : struct.triggers)
+            for (WMTrigger _iter878 : struct.triggers)
             {
-              _iter886.write(oprot);
+              _iter878.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -891,9 +891,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(POOL_TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.poolTriggers.size()));
-            for (WMPoolTrigger _iter887 : struct.poolTriggers)
+            for (WMPoolTrigger _iter879 : struct.poolTriggers)
             {
-              _iter887.write(oprot);
+              _iter879.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -920,9 +920,9 @@ import org.slf4j.LoggerFactory;
       struct.plan.write(oprot);
       {
         oprot.writeI32(struct.pools.size());
-        for (WMPool _iter888 : struct.pools)
+        for (WMPool _iter880 : struct.pools)
         {
-          _iter888.write(oprot);
+          _iter880.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -939,27 +939,27 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetMappings()) {
         {
           oprot.writeI32(struct.mappings.size());
-          for (WMMapping _iter889 : struct.mappings)
+          for (WMMapping _iter881 : struct.mappings)
           {
-            _iter889.write(oprot);
+            _iter881.write(oprot);
           }
         }
       }
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter890 : struct.triggers)
+          for (WMTrigger _iter882 : struct.triggers)
           {
-            _iter890.write(oprot);
+            _iter882.write(oprot);
           }
         }
       }
       if (struct.isSetPoolTriggers()) {
         {
           oprot.writeI32(struct.poolTriggers.size());
-          for (WMPoolTrigger _iter891 : struct.poolTriggers)
+          for (WMPoolTrigger _iter883 : struct.poolTriggers)
           {
-            _iter891.write(oprot);
+            _iter883.write(oprot);
           }
         }
       }
@@ -972,56 +972,56 @@ import org.slf4j.LoggerFactory;
       struct.plan.read(iprot);
       struct.setPlanIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list892 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.pools = new ArrayList<WMPool>(_list892.size);
-        WMPool _elem893;
-        for (int _i894 = 0; _i894 < _list892.size; ++_i894)
+        org.apache.thrift.protocol.TList _list884 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.pools = new ArrayList<WMPool>(_list884.size);
+        WMPool _elem885;
+        for (int _i886 = 0; _i886 < _list884.size; ++_i886)
         {
-          _elem893 = new WMPool();
-          _elem893.read(iprot);
-          struct.pools.add(_elem893);
+          _elem885 = new WMPool();
+          _elem885.read(iprot);
+          struct.pools.add(_elem885);
         }
       }
       struct.setPoolsIsSet(true);
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list895 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.mappings = new ArrayList<WMMapping>(_list895.size);
-          WMMapping _elem896;
-          for (int _i897 = 0; _i897 < _list895.size; ++_i897)
+          org.apache.thrift.protocol.TList _list887 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.mappings = new ArrayList<WMMapping>(_list887.size);
+          WMMapping _elem888;
+          for (int _i889 = 0; _i889 < _list887.size; ++_i889)
           {
-            _elem896 = new WMMapping();
-            _elem896.read(iprot);
-            struct.mappings.add(_elem896);
+            _elem888 = new WMMapping();
+            _elem888.read(iprot);
+            struct.mappings.add(_elem888);
           }
         }
         struct.setMappingsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list898 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list898.size);
-          WMTrigger _elem899;
-          for (int _i900 = 0; _i900 < _list898.size; ++_i900)
+          org.apache.thrift.protocol.TList _list890 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list890.size);
+          WMTrigger _elem891;
+          for (int _i892 = 0; _i892 < _list890.size; ++_i892)
           {
-            _elem899 = new WMTrigger();
-            _elem899.read(iprot);
-            struct.triggers.add(_elem899);
+            _elem891 = new WMTrigger();
+            _elem891.read(iprot);
+            struct.triggers.add(_elem891);
           }
         }
         struct.setTriggersIsSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list901 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list901.size);
-          WMPoolTrigger _elem902;
-          for (int _i903 = 0; _i903 < _list901.size; ++_i903)
+          org.apache.thrift.protocol.TList _list893 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list893.size);
+          WMPoolTrigger _elem894;
+          for (int _i895 = 0; _i895 < _list893.size; ++_i895)
           {
-            _elem902 = new WMPoolTrigger();
-            _elem902.read(iprot);
-            struct.poolTriggers.add(_elem902);
+            _elem894 = new WMPoolTrigger();
+            _elem894.read(iprot);
+            struct.poolTriggers.add(_elem894);
           }
         }
         struct.setPoolTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
index c6cb845..6eed84b 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // RESOURCE_PLANS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list904 = iprot.readListBegin();
-                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list904.size);
-                WMResourcePlan _elem905;
-                for (int _i906 = 0; _i906 < _list904.size; ++_i906)
+                org.apache.thrift.protocol.TList _list896 = iprot.readListBegin();
+                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list896.size);
+                WMResourcePlan _elem897;
+                for (int _i898 = 0; _i898 < _list896.size; ++_i898)
                 {
-                  _elem905 = new WMResourcePlan();
-                  _elem905.read(iprot);
-                  struct.resourcePlans.add(_elem905);
+                  _elem897 = new WMResourcePlan();
+                  _elem897.read(iprot);
+                  struct.resourcePlans.add(_elem897);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(RESOURCE_PLANS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.resourcePlans.size()));
-            for (WMResourcePlan _iter907 : struct.resourcePlans)
+            for (WMResourcePlan _iter899 : struct.resourcePlans)
             {
-              _iter907.write(oprot);
+              _iter899.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetResourcePlans()) {
         {
           oprot.writeI32(struct.resourcePlans.size());
-          for (WMResourcePlan _iter908 : struct.resourcePlans)
+          for (WMResourcePlan _iter900 : struct.resourcePlans)
           {
-            _iter908.write(oprot);
+            _iter900.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list909 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list909.size);
-          WMResourcePlan _elem910;
-          for (int _i911 = 0; _i911 < _list909.size; ++_i911)
+          org.apache.thrift.protocol.TList _list901 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list901.size);
+          WMResourcePlan _elem902;
+          for (int _i903 = 0; _i903 < _list901.size; ++_i903)
           {
-            _elem910 = new WMResourcePlan();
-            _elem910.read(iprot);
-            struct.resourcePlans.add(_elem910);
+            _elem902 = new WMResourcePlan();
+            _elem902.read(iprot);
+            struct.resourcePlans.add(_elem902);
           }
         }
         struct.setResourcePlansIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
index 9eed335..53ea5d5 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list928 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list928.size);
-                WMTrigger _elem929;
-                for (int _i930 = 0; _i930 < _list928.size; ++_i930)
+                org.apache.thrift.protocol.TList _list920 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list920.size);
+                WMTrigger _elem921;
+                for (int _i922 = 0; _i922 < _list920.size; ++_i922)
                 {
-                  _elem929 = new WMTrigger();
-                  _elem929.read(iprot);
-                  struct.triggers.add(_elem929);
+                  _elem921 = new WMTrigger();
+                  _elem921.read(iprot);
+                  struct.triggers.add(_elem921);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size()));
-            for (WMTrigger _iter931 : struct.triggers)
+            for (WMTrigger _iter923 : struct.triggers)
             {
-              _iter931.write(oprot);
+              _iter923.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter932 : struct.triggers)
+          for (WMTrigger _iter924 : struct.triggers)
           {
-            _iter932.write(oprot);
+            _iter924.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list933 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list933.size);
-          WMTrigger _elem934;
-          for (int _i935 = 0; _i935 < _list933.size; ++_i935)
+          org.apache.thrift.protocol.TList _list925 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list925.size);
+          WMTrigger _elem926;
+          for (int _i927 = 0; _i927 < _list925.size; ++_i927)
           {
-            _elem934 = new WMTrigger();
-            _elem934.read(iprot);
-            struct.triggers.add(_elem934);
+            _elem926 = new WMTrigger();
+            _elem926.read(iprot);
+            struct.triggers.add(_elem926);
           }
         }
         struct.setTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
index ee9251c..0dd8a5e 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
@@ -441,13 +441,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ERRORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list912 = iprot.readListBegin();
-                struct.errors = new ArrayList<String>(_list912.size);
-                String _elem913;
-                for (int _i914 = 0; _i914 < _list912.size; ++_i914)
+                org.apache.thrift.protocol.TList _list904 = iprot.readListBegin();
+                struct.errors = new ArrayList<String>(_list904.size);
+                String _elem905;
+                for (int _i906 = 0; _i906 < _list904.size; ++_i906)
                 {
-                  _elem913 = iprot.readString();
-                  struct.errors.add(_elem913);
+                  _elem905 = iprot.readString();
+                  struct.errors.add(_elem905);
                 }
                 iprot.readListEnd();
               }
@@ -459,13 +459,13 @@ import org.slf4j.LoggerFactory;
           case 2: // WARNINGS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list915 = iprot.readListBegin();
-                struct.warnings = new ArrayList<String>(_list915.size);
-                String _elem916;
-                for (int _i917 = 0; _i917 < _list915.size; ++_i917)
+                org.apache.thrift.protocol.TList _list907 = iprot.readListBegin();
+                struct.warnings = new ArrayList<String>(_list907.size);
+                String _elem908;
+                for (int _i909 = 0; _i909 < _list907.size; ++_i909)
                 {
-                  _elem916 = iprot.readString();
-                  struct.warnings.add(_elem916);
+                  _elem908 = iprot.readString();
+                  struct.warnings.add(_elem908);
                 }
                 iprot.readListEnd();
               }
@@ -492,9 +492,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(ERRORS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.errors.size()));
-            for (String _iter918 : struct.errors)
+            for (String _iter910 : struct.errors)
             {
-              oprot.writeString(_iter918);
+              oprot.writeString(_iter910);
             }
             oprot.writeListEnd();
           }
@@ -506,9 +506,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(WARNINGS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.warnings.size()));
-            for (String _iter919 : struct.warnings)
+            for (String _iter911 : struct.warnings)
             {
-              oprot.writeString(_iter919);
+              oprot.writeString(_iter911);
             }
             oprot.writeListEnd();
           }
@@ -543,18 +543,18 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetErrors()) {
         {
           oprot.writeI32(struct.errors.size());
-          for (String _iter920 : struct.errors)
+          for (String _iter912 : struct.errors)
           {
-            oprot.writeString(_iter920);
+            oprot.writeString(_iter912);
           }
         }
       }
       if (struct.isSetWarnings()) {
         {
           oprot.writeI32(struct.warnings.size());
-          for (String _iter921 : struct.warnings)
+          for (String _iter913 : struct.warnings)
           {
-            oprot.writeString(_iter921);
+            oprot.writeString(_iter913);
           }
         }
       }
@@ -566,26 +566,26 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list922 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.errors = new ArrayList<String>(_list922.size);
-          String _elem923;
-          for (int _i924 = 0; _i924 < _list922.size; ++_i924)
+          org.apache.thrift.protocol.TList _list914 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.errors = new ArrayList<String>(_list914.size);
+          String _elem915;
+          for (int _i916 = 0; _i916 < _list914.size; ++_i916)
           {
-            _elem923 = iprot.readString();
-            struct.errors.add(_elem923);
+            _elem915 = iprot.readString();
+            struct.errors.add(_elem915);
           }
         }
         struct.setErrorsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list925 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.warnings = new ArrayList<String>(_list925.size);
-          String _elem926;
-          for (int _i927 = 0; _i927 < _list925.size; ++_i927)
+          org.apache.thrift.protocol.TList _list917 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.warnings = new ArrayList<String>(_list917.size);
+          String _elem918;
+          for (int _i919 = 0; _i919 < _list917.size; ++_i919)
           {
-            _elem926 = iprot.readString();
-            struct.warnings.add(_elem926);
+            _elem918 = iprot.readString();
+            struct.warnings.add(_elem918);
           }
         }
         struct.setWarningsIsSet(true);


[03/12] hive git commit: HIVE-20006: Make materializations invalidation cache work with multiple active remote metastores (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index a5bcc10..2ae6d9a 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -347,11 +347,11 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
-  def get_materialization_invalidation_info(self, dbname, tbl_names):
+  def get_materialization_invalidation_info(self, creation_metadata, validTxnList):
     """
     Parameters:
-     - dbname
-     - tbl_names
+     - creation_metadata
+     - validTxnList
     """
     pass
 
@@ -3141,20 +3141,20 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o3
     raise TApplicationException(TApplicationException.MISSING_RESULT, "get_table_objects_by_name_req failed: unknown result")
 
-  def get_materialization_invalidation_info(self, dbname, tbl_names):
+  def get_materialization_invalidation_info(self, creation_metadata, validTxnList):
     """
     Parameters:
-     - dbname
-     - tbl_names
+     - creation_metadata
+     - validTxnList
     """
-    self.send_get_materialization_invalidation_info(dbname, tbl_names)
+    self.send_get_materialization_invalidation_info(creation_metadata, validTxnList)
     return self.recv_get_materialization_invalidation_info()
 
-  def send_get_materialization_invalidation_info(self, dbname, tbl_names):
+  def send_get_materialization_invalidation_info(self, creation_metadata, validTxnList):
     self._oprot.writeMessageBegin('get_materialization_invalidation_info', TMessageType.CALL, self._seqid)
     args = get_materialization_invalidation_info_args()
-    args.dbname = dbname
-    args.tbl_names = tbl_names
+    args.creation_metadata = creation_metadata
+    args.validTxnList = validTxnList
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
@@ -10357,7 +10357,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     iprot.readMessageEnd()
     result = get_materialization_invalidation_info_result()
     try:
-      result.success = self._handler.get_materialization_invalidation_info(args.dbname, args.tbl_names)
+      result.success = self._handler.get_materialization_invalidation_info(args.creation_metadata, args.validTxnList)
       msg_type = TMessageType.REPLY
     except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
       raise
@@ -16045,10 +16045,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype840, _size837) = iprot.readListBegin()
-          for _i841 in xrange(_size837):
-            _elem842 = iprot.readString()
-            self.success.append(_elem842)
+          (_etype833, _size830) = iprot.readListBegin()
+          for _i834 in xrange(_size830):
+            _elem835 = iprot.readString()
+            self.success.append(_elem835)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16071,8 +16071,8 @@ class get_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter843 in self.success:
-        oprot.writeString(iter843)
+      for iter836 in self.success:
+        oprot.writeString(iter836)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16177,10 +16177,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype847, _size844) = iprot.readListBegin()
-          for _i848 in xrange(_size844):
-            _elem849 = iprot.readString()
-            self.success.append(_elem849)
+          (_etype840, _size837) = iprot.readListBegin()
+          for _i841 in xrange(_size837):
+            _elem842 = iprot.readString()
+            self.success.append(_elem842)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16203,8 +16203,8 @@ class get_all_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter850 in self.success:
-        oprot.writeString(iter850)
+      for iter843 in self.success:
+        oprot.writeString(iter843)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16974,12 +16974,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype852, _vtype853, _size851 ) = iprot.readMapBegin()
-          for _i855 in xrange(_size851):
-            _key856 = iprot.readString()
-            _val857 = Type()
-            _val857.read(iprot)
-            self.success[_key856] = _val857
+          (_ktype845, _vtype846, _size844 ) = iprot.readMapBegin()
+          for _i848 in xrange(_size844):
+            _key849 = iprot.readString()
+            _val850 = Type()
+            _val850.read(iprot)
+            self.success[_key849] = _val850
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -17002,9 +17002,9 @@ class get_type_all_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter858,viter859 in self.success.items():
-        oprot.writeString(kiter858)
-        viter859.write(oprot)
+      for kiter851,viter852 in self.success.items():
+        oprot.writeString(kiter851)
+        viter852.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -17147,11 +17147,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype863, _size860) = iprot.readListBegin()
-          for _i864 in xrange(_size860):
-            _elem865 = FieldSchema()
-            _elem865.read(iprot)
-            self.success.append(_elem865)
+          (_etype856, _size853) = iprot.readListBegin()
+          for _i857 in xrange(_size853):
+            _elem858 = FieldSchema()
+            _elem858.read(iprot)
+            self.success.append(_elem858)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17186,8 +17186,8 @@ class get_fields_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter866 in self.success:
-        iter866.write(oprot)
+      for iter859 in self.success:
+        iter859.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17354,11 +17354,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype870, _size867) = iprot.readListBegin()
-          for _i871 in xrange(_size867):
-            _elem872 = FieldSchema()
-            _elem872.read(iprot)
-            self.success.append(_elem872)
+          (_etype863, _size860) = iprot.readListBegin()
+          for _i864 in xrange(_size860):
+            _elem865 = FieldSchema()
+            _elem865.read(iprot)
+            self.success.append(_elem865)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17393,8 +17393,8 @@ class get_fields_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter873 in self.success:
-        iter873.write(oprot)
+      for iter866 in self.success:
+        iter866.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17547,11 +17547,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype877, _size874) = iprot.readListBegin()
-          for _i878 in xrange(_size874):
-            _elem879 = FieldSchema()
-            _elem879.read(iprot)
-            self.success.append(_elem879)
+          (_etype870, _size867) = iprot.readListBegin()
+          for _i871 in xrange(_size867):
+            _elem872 = FieldSchema()
+            _elem872.read(iprot)
+            self.success.append(_elem872)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17586,8 +17586,8 @@ class get_schema_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter880 in self.success:
-        iter880.write(oprot)
+      for iter873 in self.success:
+        iter873.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17754,11 +17754,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype884, _size881) = iprot.readListBegin()
-          for _i885 in xrange(_size881):
-            _elem886 = FieldSchema()
-            _elem886.read(iprot)
-            self.success.append(_elem886)
+          (_etype877, _size874) = iprot.readListBegin()
+          for _i878 in xrange(_size874):
+            _elem879 = FieldSchema()
+            _elem879.read(iprot)
+            self.success.append(_elem879)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17793,8 +17793,8 @@ class get_schema_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter887 in self.success:
-        iter887.write(oprot)
+      for iter880 in self.success:
+        iter880.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18247,66 +18247,66 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype891, _size888) = iprot.readListBegin()
-          for _i892 in xrange(_size888):
-            _elem893 = SQLPrimaryKey()
-            _elem893.read(iprot)
-            self.primaryKeys.append(_elem893)
+          (_etype884, _size881) = iprot.readListBegin()
+          for _i885 in xrange(_size881):
+            _elem886 = SQLPrimaryKey()
+            _elem886.read(iprot)
+            self.primaryKeys.append(_elem886)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype897, _size894) = iprot.readListBegin()
-          for _i898 in xrange(_size894):
-            _elem899 = SQLForeignKey()
-            _elem899.read(iprot)
-            self.foreignKeys.append(_elem899)
+          (_etype890, _size887) = iprot.readListBegin()
+          for _i891 in xrange(_size887):
+            _elem892 = SQLForeignKey()
+            _elem892.read(iprot)
+            self.foreignKeys.append(_elem892)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.uniqueConstraints = []
-          (_etype903, _size900) = iprot.readListBegin()
-          for _i904 in xrange(_size900):
-            _elem905 = SQLUniqueConstraint()
-            _elem905.read(iprot)
-            self.uniqueConstraints.append(_elem905)
+          (_etype896, _size893) = iprot.readListBegin()
+          for _i897 in xrange(_size893):
+            _elem898 = SQLUniqueConstraint()
+            _elem898.read(iprot)
+            self.uniqueConstraints.append(_elem898)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.notNullConstraints = []
-          (_etype909, _size906) = iprot.readListBegin()
-          for _i910 in xrange(_size906):
-            _elem911 = SQLNotNullConstraint()
-            _elem911.read(iprot)
-            self.notNullConstraints.append(_elem911)
+          (_etype902, _size899) = iprot.readListBegin()
+          for _i903 in xrange(_size899):
+            _elem904 = SQLNotNullConstraint()
+            _elem904.read(iprot)
+            self.notNullConstraints.append(_elem904)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.LIST:
           self.defaultConstraints = []
-          (_etype915, _size912) = iprot.readListBegin()
-          for _i916 in xrange(_size912):
-            _elem917 = SQLDefaultConstraint()
-            _elem917.read(iprot)
-            self.defaultConstraints.append(_elem917)
+          (_etype908, _size905) = iprot.readListBegin()
+          for _i909 in xrange(_size905):
+            _elem910 = SQLDefaultConstraint()
+            _elem910.read(iprot)
+            self.defaultConstraints.append(_elem910)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 7:
         if ftype == TType.LIST:
           self.checkConstraints = []
-          (_etype921, _size918) = iprot.readListBegin()
-          for _i922 in xrange(_size918):
-            _elem923 = SQLCheckConstraint()
-            _elem923.read(iprot)
-            self.checkConstraints.append(_elem923)
+          (_etype914, _size911) = iprot.readListBegin()
+          for _i915 in xrange(_size911):
+            _elem916 = SQLCheckConstraint()
+            _elem916.read(iprot)
+            self.checkConstraints.append(_elem916)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18327,43 +18327,43 @@ class create_table_with_constraints_args:
     if self.primaryKeys is not None:
       oprot.writeFieldBegin('primaryKeys', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
-      for iter924 in self.primaryKeys:
-        iter924.write(oprot)
+      for iter917 in self.primaryKeys:
+        iter917.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.foreignKeys is not None:
       oprot.writeFieldBegin('foreignKeys', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
-      for iter925 in self.foreignKeys:
-        iter925.write(oprot)
+      for iter918 in self.foreignKeys:
+        iter918.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.uniqueConstraints is not None:
       oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints))
-      for iter926 in self.uniqueConstraints:
-        iter926.write(oprot)
+      for iter919 in self.uniqueConstraints:
+        iter919.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.notNullConstraints is not None:
       oprot.writeFieldBegin('notNullConstraints', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints))
-      for iter927 in self.notNullConstraints:
-        iter927.write(oprot)
+      for iter920 in self.notNullConstraints:
+        iter920.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.defaultConstraints is not None:
       oprot.writeFieldBegin('defaultConstraints', TType.LIST, 6)
       oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraints))
-      for iter928 in self.defaultConstraints:
-        iter928.write(oprot)
+      for iter921 in self.defaultConstraints:
+        iter921.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.checkConstraints is not None:
       oprot.writeFieldBegin('checkConstraints', TType.LIST, 7)
       oprot.writeListBegin(TType.STRUCT, len(self.checkConstraints))
-      for iter929 in self.checkConstraints:
-        iter929.write(oprot)
+      for iter922 in self.checkConstraints:
+        iter922.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19923,10 +19923,10 @@ class truncate_table_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype933, _size930) = iprot.readListBegin()
-          for _i934 in xrange(_size930):
-            _elem935 = iprot.readString()
-            self.partNames.append(_elem935)
+          (_etype926, _size923) = iprot.readListBegin()
+          for _i927 in xrange(_size923):
+            _elem928 = iprot.readString()
+            self.partNames.append(_elem928)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19951,8 +19951,8 @@ class truncate_table_args:
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter936 in self.partNames:
-        oprot.writeString(iter936)
+      for iter929 in self.partNames:
+        oprot.writeString(iter929)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20152,10 +20152,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype940, _size937) = iprot.readListBegin()
-          for _i941 in xrange(_size937):
-            _elem942 = iprot.readString()
-            self.success.append(_elem942)
+          (_etype933, _size930) = iprot.readListBegin()
+          for _i934 in xrange(_size930):
+            _elem935 = iprot.readString()
+            self.success.append(_elem935)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20178,8 +20178,8 @@ class get_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter943 in self.success:
-        oprot.writeString(iter943)
+      for iter936 in self.success:
+        oprot.writeString(iter936)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20329,10 +20329,10 @@ class get_tables_by_type_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype947, _size944) = iprot.readListBegin()
-          for _i948 in xrange(_size944):
-            _elem949 = iprot.readString()
-            self.success.append(_elem949)
+          (_etype940, _size937) = iprot.readListBegin()
+          for _i941 in xrange(_size937):
+            _elem942 = iprot.readString()
+            self.success.append(_elem942)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20355,8 +20355,8 @@ class get_tables_by_type_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter950 in self.success:
-        oprot.writeString(iter950)
+      for iter943 in self.success:
+        oprot.writeString(iter943)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20480,10 +20480,10 @@ class get_materialized_views_for_rewriting_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype954, _size951) = iprot.readListBegin()
-          for _i955 in xrange(_size951):
-            _elem956 = iprot.readString()
-            self.success.append(_elem956)
+          (_etype947, _size944) = iprot.readListBegin()
+          for _i948 in xrange(_size944):
+            _elem949 = iprot.readString()
+            self.success.append(_elem949)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20506,8 +20506,8 @@ class get_materialized_views_for_rewriting_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter957 in self.success:
-        oprot.writeString(iter957)
+      for iter950 in self.success:
+        oprot.writeString(iter950)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20580,10 +20580,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype961, _size958) = iprot.readListBegin()
-          for _i962 in xrange(_size958):
-            _elem963 = iprot.readString()
-            self.tbl_types.append(_elem963)
+          (_etype954, _size951) = iprot.readListBegin()
+          for _i955 in xrange(_size951):
+            _elem956 = iprot.readString()
+            self.tbl_types.append(_elem956)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20608,8 +20608,8 @@ class get_table_meta_args:
     if self.tbl_types is not None:
       oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.tbl_types))
-      for iter964 in self.tbl_types:
-        oprot.writeString(iter964)
+      for iter957 in self.tbl_types:
+        oprot.writeString(iter957)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20665,11 +20665,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype968, _size965) = iprot.readListBegin()
-          for _i969 in xrange(_size965):
-            _elem970 = TableMeta()
-            _elem970.read(iprot)
-            self.success.append(_elem970)
+          (_etype961, _size958) = iprot.readListBegin()
+          for _i962 in xrange(_size958):
+            _elem963 = TableMeta()
+            _elem963.read(iprot)
+            self.success.append(_elem963)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20692,8 +20692,8 @@ class get_table_meta_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter971 in self.success:
-        iter971.write(oprot)
+      for iter964 in self.success:
+        iter964.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20817,10 +20817,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype975, _size972) = iprot.readListBegin()
-          for _i976 in xrange(_size972):
-            _elem977 = iprot.readString()
-            self.success.append(_elem977)
+          (_etype968, _size965) = iprot.readListBegin()
+          for _i969 in xrange(_size965):
+            _elem970 = iprot.readString()
+            self.success.append(_elem970)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20843,8 +20843,8 @@ class get_all_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter978 in self.success:
-        oprot.writeString(iter978)
+      for iter971 in self.success:
+        oprot.writeString(iter971)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21080,10 +21080,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype982, _size979) = iprot.readListBegin()
-          for _i983 in xrange(_size979):
-            _elem984 = iprot.readString()
-            self.tbl_names.append(_elem984)
+          (_etype975, _size972) = iprot.readListBegin()
+          for _i976 in xrange(_size972):
+            _elem977 = iprot.readString()
+            self.tbl_names.append(_elem977)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21104,8 +21104,8 @@ class get_table_objects_by_name_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter985 in self.tbl_names:
-        oprot.writeString(iter985)
+      for iter978 in self.tbl_names:
+        oprot.writeString(iter978)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21157,11 +21157,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype989, _size986) = iprot.readListBegin()
-          for _i990 in xrange(_size986):
-            _elem991 = Table()
-            _elem991.read(iprot)
-            self.success.append(_elem991)
+          (_etype982, _size979) = iprot.readListBegin()
+          for _i983 in xrange(_size979):
+            _elem984 = Table()
+            _elem984.read(iprot)
+            self.success.append(_elem984)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21178,8 +21178,8 @@ class get_table_objects_by_name_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter992 in self.success:
-        iter992.write(oprot)
+      for iter985 in self.success:
+        iter985.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21540,19 +21540,19 @@ class get_table_objects_by_name_req_result:
 class get_materialization_invalidation_info_args:
   """
   Attributes:
-   - dbname
-   - tbl_names
+   - creation_metadata
+   - validTxnList
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRING, 'dbname', None, None, ), # 1
-    (2, TType.LIST, 'tbl_names', (TType.STRING,None), None, ), # 2
+    (1, TType.STRUCT, 'creation_metadata', (CreationMetadata, CreationMetadata.thrift_spec), None, ), # 1
+    (2, TType.STRING, 'validTxnList', None, None, ), # 2
   )
 
-  def __init__(self, dbname=None, tbl_names=None,):
-    self.dbname = dbname
-    self.tbl_names = tbl_names
+  def __init__(self, creation_metadata=None, validTxnList=None,):
+    self.creation_metadata = creation_metadata
+    self.validTxnList = validTxnList
 
   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:
@@ -21564,18 +21564,14 @@ class get_materialization_invalidation_info_args:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.STRING:
-          self.dbname = iprot.readString()
+        if ftype == TType.STRUCT:
+          self.creation_metadata = CreationMetadata()
+          self.creation_metadata.read(iprot)
         else:
           iprot.skip(ftype)
       elif fid == 2:
-        if ftype == TType.LIST:
-          self.tbl_names = []
-          (_etype996, _size993) = iprot.readListBegin()
-          for _i997 in xrange(_size993):
-            _elem998 = iprot.readString()
-            self.tbl_names.append(_elem998)
-          iprot.readListEnd()
+        if ftype == TType.STRING:
+          self.validTxnList = iprot.readString()
         else:
           iprot.skip(ftype)
       else:
@@ -21588,16 +21584,13 @@ class get_materialization_invalidation_info_args:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
     oprot.writeStructBegin('get_materialization_invalidation_info_args')
-    if self.dbname is not None:
-      oprot.writeFieldBegin('dbname', TType.STRING, 1)
-      oprot.writeString(self.dbname)
+    if self.creation_metadata is not None:
+      oprot.writeFieldBegin('creation_metadata', TType.STRUCT, 1)
+      self.creation_metadata.write(oprot)
       oprot.writeFieldEnd()
-    if self.tbl_names is not None:
-      oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
-      oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter999 in self.tbl_names:
-        oprot.writeString(iter999)
-      oprot.writeListEnd()
+    if self.validTxnList is not None:
+      oprot.writeFieldBegin('validTxnList', TType.STRING, 2)
+      oprot.writeString(self.validTxnList)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
@@ -21608,8 +21601,8 @@ class get_materialization_invalidation_info_args:
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.dbname)
-    value = (value * 31) ^ hash(self.tbl_names)
+    value = (value * 31) ^ hash(self.creation_metadata)
+    value = (value * 31) ^ hash(self.validTxnList)
     return value
 
   def __repr__(self):
@@ -21633,7 +21626,7 @@ class get_materialization_invalidation_info_result:
   """
 
   thrift_spec = (
-    (0, TType.MAP, 'success', (TType.STRING,None,TType.STRUCT,(Materialization, Materialization.thrift_spec)), None, ), # 0
+    (0, TType.STRUCT, 'success', (Materialization, Materialization.thrift_spec), None, ), # 0
     (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
     (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2
     (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3
@@ -21655,15 +21648,9 @@ class get_materialization_invalidation_info_result:
       if ftype == TType.STOP:
         break
       if fid == 0:
-        if ftype == TType.MAP:
-          self.success = {}
-          (_ktype1001, _vtype1002, _size1000 ) = iprot.readMapBegin()
-          for _i1004 in xrange(_size1000):
-            _key1005 = iprot.readString()
-            _val1006 = Materialization()
-            _val1006.read(iprot)
-            self.success[_key1005] = _val1006
-          iprot.readMapEnd()
+        if ftype == TType.STRUCT:
+          self.success = Materialization()
+          self.success.read(iprot)
         else:
           iprot.skip(ftype)
       elif fid == 1:
@@ -21695,12 +21682,8 @@ class get_materialization_invalidation_info_result:
       return
     oprot.writeStructBegin('get_materialization_invalidation_info_result')
     if self.success is not None:
-      oprot.writeFieldBegin('success', TType.MAP, 0)
-      oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter1007,viter1008 in self.success.items():
-        oprot.writeString(kiter1007)
-        viter1008.write(oprot)
-      oprot.writeMapEnd()
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
       oprot.writeFieldEnd()
     if self.o1 is not None:
       oprot.writeFieldBegin('o1', TType.STRUCT, 1)
@@ -22064,10 +22047,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1012, _size1009) = iprot.readListBegin()
-          for _i1013 in xrange(_size1009):
-            _elem1014 = iprot.readString()
-            self.success.append(_elem1014)
+          (_etype989, _size986) = iprot.readListBegin()
+          for _i990 in xrange(_size986):
+            _elem991 = iprot.readString()
+            self.success.append(_elem991)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22102,8 +22085,8 @@ class get_table_names_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1015 in self.success:
-        oprot.writeString(iter1015)
+      for iter992 in self.success:
+        oprot.writeString(iter992)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23073,11 +23056,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1019, _size1016) = iprot.readListBegin()
-          for _i1020 in xrange(_size1016):
-            _elem1021 = Partition()
-            _elem1021.read(iprot)
-            self.new_parts.append(_elem1021)
+          (_etype996, _size993) = iprot.readListBegin()
+          for _i997 in xrange(_size993):
+            _elem998 = Partition()
+            _elem998.read(iprot)
+            self.new_parts.append(_elem998)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23094,8 +23077,8 @@ class add_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1022 in self.new_parts:
-        iter1022.write(oprot)
+      for iter999 in self.new_parts:
+        iter999.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23253,11 +23236,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1026, _size1023) = iprot.readListBegin()
-          for _i1027 in xrange(_size1023):
-            _elem1028 = PartitionSpec()
-            _elem1028.read(iprot)
-            self.new_parts.append(_elem1028)
+          (_etype1003, _size1000) = iprot.readListBegin()
+          for _i1004 in xrange(_size1000):
+            _elem1005 = PartitionSpec()
+            _elem1005.read(iprot)
+            self.new_parts.append(_elem1005)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23274,8 +23257,8 @@ class add_partitions_pspec_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1029 in self.new_parts:
-        iter1029.write(oprot)
+      for iter1006 in self.new_parts:
+        iter1006.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23449,10 +23432,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1033, _size1030) = iprot.readListBegin()
-          for _i1034 in xrange(_size1030):
-            _elem1035 = iprot.readString()
-            self.part_vals.append(_elem1035)
+          (_etype1010, _size1007) = iprot.readListBegin()
+          for _i1011 in xrange(_size1007):
+            _elem1012 = iprot.readString()
+            self.part_vals.append(_elem1012)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23477,8 +23460,8 @@ class append_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1036 in self.part_vals:
-        oprot.writeString(iter1036)
+      for iter1013 in self.part_vals:
+        oprot.writeString(iter1013)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23831,10 +23814,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1040, _size1037) = iprot.readListBegin()
-          for _i1041 in xrange(_size1037):
-            _elem1042 = iprot.readString()
-            self.part_vals.append(_elem1042)
+          (_etype1017, _size1014) = iprot.readListBegin()
+          for _i1018 in xrange(_size1014):
+            _elem1019 = iprot.readString()
+            self.part_vals.append(_elem1019)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23865,8 +23848,8 @@ class append_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1043 in self.part_vals:
-        oprot.writeString(iter1043)
+      for iter1020 in self.part_vals:
+        oprot.writeString(iter1020)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -24461,10 +24444,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1047, _size1044) = iprot.readListBegin()
-          for _i1048 in xrange(_size1044):
-            _elem1049 = iprot.readString()
-            self.part_vals.append(_elem1049)
+          (_etype1024, _size1021) = iprot.readListBegin()
+          for _i1025 in xrange(_size1021):
+            _elem1026 = iprot.readString()
+            self.part_vals.append(_elem1026)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24494,8 +24477,8 @@ class drop_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1050 in self.part_vals:
-        oprot.writeString(iter1050)
+      for iter1027 in self.part_vals:
+        oprot.writeString(iter1027)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -24668,10 +24651,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1054, _size1051) = iprot.readListBegin()
-          for _i1055 in xrange(_size1051):
-            _elem1056 = iprot.readString()
-            self.part_vals.append(_elem1056)
+          (_etype1031, _size1028) = iprot.readListBegin()
+          for _i1032 in xrange(_size1028):
+            _elem1033 = iprot.readString()
+            self.part_vals.append(_elem1033)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24707,8 +24690,8 @@ class drop_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1057 in self.part_vals:
-        oprot.writeString(iter1057)
+      for iter1034 in self.part_vals:
+        oprot.writeString(iter1034)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -25445,10 +25428,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1061, _size1058) = iprot.readListBegin()
-          for _i1062 in xrange(_size1058):
-            _elem1063 = iprot.readString()
-            self.part_vals.append(_elem1063)
+          (_etype1038, _size1035) = iprot.readListBegin()
+          for _i1039 in xrange(_size1035):
+            _elem1040 = iprot.readString()
+            self.part_vals.append(_elem1040)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25473,8 +25456,8 @@ class get_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1064 in self.part_vals:
-        oprot.writeString(iter1064)
+      for iter1041 in self.part_vals:
+        oprot.writeString(iter1041)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -25633,11 +25616,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1066, _vtype1067, _size1065 ) = iprot.readMapBegin()
-          for _i1069 in xrange(_size1065):
-            _key1070 = iprot.readString()
-            _val1071 = iprot.readString()
-            self.partitionSpecs[_key1070] = _val1071
+          (_ktype1043, _vtype1044, _size1042 ) = iprot.readMapBegin()
+          for _i1046 in xrange(_size1042):
+            _key1047 = iprot.readString()
+            _val1048 = iprot.readString()
+            self.partitionSpecs[_key1047] = _val1048
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -25674,9 +25657,9 @@ class exchange_partition_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter1072,viter1073 in self.partitionSpecs.items():
-        oprot.writeString(kiter1072)
-        oprot.writeString(viter1073)
+      for kiter1049,viter1050 in self.partitionSpecs.items():
+        oprot.writeString(kiter1049)
+        oprot.writeString(viter1050)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -25881,11 +25864,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1075, _vtype1076, _size1074 ) = iprot.readMapBegin()
-          for _i1078 in xrange(_size1074):
-            _key1079 = iprot.readString()
-            _val1080 = iprot.readString()
-            self.partitionSpecs[_key1079] = _val1080
+          (_ktype1052, _vtype1053, _size1051 ) = iprot.readMapBegin()
+          for _i1055 in xrange(_size1051):
+            _key1056 = iprot.readString()
+            _val1057 = iprot.readString()
+            self.partitionSpecs[_key1056] = _val1057
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -25922,9 +25905,9 @@ class exchange_partitions_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter1081,viter1082 in self.partitionSpecs.items():
-        oprot.writeString(kiter1081)
-        oprot.writeString(viter1082)
+      for kiter1058,viter1059 in self.partitionSpecs.items():
+        oprot.writeString(kiter1058)
+        oprot.writeString(viter1059)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -26007,11 +25990,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1086, _size1083) = iprot.readListBegin()
-          for _i1087 in xrange(_size1083):
-            _elem1088 = Partition()
-            _elem1088.read(iprot)
-            self.success.append(_elem1088)
+          (_etype1063, _size1060) = iprot.readListBegin()
+          for _i1064 in xrange(_size1060):
+            _elem1065 = Partition()
+            _elem1065.read(iprot)
+            self.success.append(_elem1065)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26052,8 +26035,8 @@ class exchange_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1089 in self.success:
-        iter1089.write(oprot)
+      for iter1066 in self.success:
+        iter1066.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26147,10 +26130,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1093, _size1090) = iprot.readListBegin()
-          for _i1094 in xrange(_size1090):
-            _elem1095 = iprot.readString()
-            self.part_vals.append(_elem1095)
+          (_etype1070, _size1067) = iprot.readListBegin()
+          for _i1071 in xrange(_size1067):
+            _elem1072 = iprot.readString()
+            self.part_vals.append(_elem1072)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26162,10 +26145,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1099, _size1096) = iprot.readListBegin()
-          for _i1100 in xrange(_size1096):
-            _elem1101 = iprot.readString()
-            self.group_names.append(_elem1101)
+          (_etype1076, _size1073) = iprot.readListBegin()
+          for _i1077 in xrange(_size1073):
+            _elem1078 = iprot.readString()
+            self.group_names.append(_elem1078)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26190,8 +26173,8 @@ class get_partition_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1102 in self.part_vals:
-        oprot.writeString(iter1102)
+      for iter1079 in self.part_vals:
+        oprot.writeString(iter1079)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -26201,8 +26184,8 @@ class get_partition_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1103 in self.group_names:
-        oprot.writeString(iter1103)
+      for iter1080 in self.group_names:
+        oprot.writeString(iter1080)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26631,11 +26614,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1107, _size1104) = iprot.readListBegin()
-          for _i1108 in xrange(_size1104):
-            _elem1109 = Partition()
-            _elem1109.read(iprot)
-            self.success.append(_elem1109)
+          (_etype1084, _size1081) = iprot.readListBegin()
+          for _i1085 in xrange(_size1081):
+            _elem1086 = Partition()
+            _elem1086.read(iprot)
+            self.success.append(_elem1086)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26664,8 +26647,8 @@ class get_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1110 in self.success:
-        iter1110.write(oprot)
+      for iter1087 in self.success:
+        iter1087.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26759,10 +26742,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1114, _size1111) = iprot.readListBegin()
-          for _i1115 in xrange(_size1111):
-            _elem1116 = iprot.readString()
-            self.group_names.append(_elem1116)
+          (_etype1091, _size1088) = iprot.readListBegin()
+          for _i1092 in xrange(_size1088):
+            _elem1093 = iprot.readString()
+            self.group_names.append(_elem1093)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26795,8 +26778,8 @@ class get_partitions_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1117 in self.group_names:
-        oprot.writeString(iter1117)
+      for iter1094 in self.group_names:
+        oprot.writeString(iter1094)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26857,11 +26840,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1121, _size1118) = iprot.readListBegin()
-          for _i1122 in xrange(_size1118):
-            _elem1123 = Partition()
-            _elem1123.read(iprot)
-            self.success.append(_elem1123)
+          (_etype1098, _size1095) = iprot.readListBegin()
+          for _i1099 in xrange(_size1095):
+            _elem1100 = Partition()
+            _elem1100.read(iprot)
+            self.success.append(_elem1100)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26890,8 +26873,8 @@ class get_partitions_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1124 in self.success:
-        iter1124.write(oprot)
+      for iter1101 in self.success:
+        iter1101.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27049,11 +27032,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1128, _size1125) = iprot.readListBegin()
-          for _i1129 in xrange(_size1125):
-            _elem1130 = PartitionSpec()
-            _elem1130.read(iprot)
-            self.success.append(_elem1130)
+          (_etype1105, _size1102) = iprot.readListBegin()
+          for _i1106 in xrange(_size1102):
+            _elem1107 = PartitionSpec()
+            _elem1107.read(iprot)
+            self.success.append(_elem1107)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27082,8 +27065,8 @@ class get_partitions_pspec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1131 in self.success:
-        iter1131.write(oprot)
+      for iter1108 in self.success:
+        iter1108.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27241,10 +27224,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1135, _size1132) = iprot.readListBegin()
-          for _i1136 in xrange(_size1132):
-            _elem1137 = iprot.readString()
-            self.success.append(_elem1137)
+          (_etype1112, _size1109) = iprot.readListBegin()
+          for _i1113 in xrange(_size1109):
+            _elem1114 = iprot.readString()
+            self.success.append(_elem1114)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27273,8 +27256,8 @@ class get_partition_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1138 in self.success:
-        oprot.writeString(iter1138)
+      for iter1115 in self.success:
+        oprot.writeString(iter1115)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27514,10 +27497,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1142, _size1139) = iprot.readListBegin()
-          for _i1143 in xrange(_size1139):
-            _elem1144 = iprot.readString()
-            self.part_vals.append(_elem1144)
+          (_etype1119, _size1116) = iprot.readListBegin()
+          for _i1120 in xrange(_size1116):
+            _elem1121 = iprot.readString()
+            self.part_vals.append(_elem1121)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27547,8 +27530,8 @@ class get_partitions_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1145 in self.part_vals:
-        oprot.writeString(iter1145)
+      for iter1122 in self.part_vals:
+        oprot.writeString(iter1122)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -27612,11 +27595,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1149, _size1146) = iprot.readListBegin()
-          for _i1150 in xrange(_size1146):
-            _elem1151 = Partition()
-            _elem1151.read(iprot)
-            self.success.append(_elem1151)
+          (_etype1126, _size1123) = iprot.readListBegin()
+          for _i1127 in xrange(_size1123):
+            _elem1128 = Partition()
+            _elem1128.read(iprot)
+            self.success.append(_elem1128)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27645,8 +27628,8 @@ class get_partitions_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1152 in self.success:
-        iter1152.write(oprot)
+      for iter1129 in self.success:
+        iter1129.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27733,10 +27716,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1156, _size1153) = iprot.readListBegin()
-          for _i1157 in xrange(_size1153):
-            _elem1158 = iprot.readString()
-            self.part_vals.append(_elem1158)
+          (_etype1133, _size1130) = iprot.readListBegin()
+          for _i1134 in xrange(_size1130):
+            _elem1135 = iprot.readString()
+            self.part_vals.append(_elem1135)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27753,10 +27736,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1162, _size1159) = iprot.readListBegin()
-          for _i1163 in xrange(_size1159):
-            _elem1164 = iprot.readString()
-            self.group_names.append(_elem1164)
+          (_etype1139, _size1136) = iprot.readListBegin()
+          for _i1140 in xrange(_size1136):
+            _elem1141 = iprot.readString()
+            self.group_names.append(_elem1141)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27781,8 +27764,8 @@ class get_partitions_ps_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1165 in self.part_vals:
-        oprot.writeString(iter1165)
+      for iter1142 in self.part_vals:
+        oprot.writeString(iter1142)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -27796,8 +27779,8 @@ class get_partitions_ps_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1166 in self.group_names:
-        oprot.writeString(iter1166)
+      for iter1143 in self.group_names:
+        oprot.writeString(iter1143)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -27859,11 +27842,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1170, _size1167) = iprot.readListBegin()
-          for _i1171 in xrange(_size1167):
-            _elem1172 = Partition()
-            _elem1172.read(iprot)
-            self.success.append(_elem1172)
+          (_etype1147, _size1144) = iprot.readListBegin()
+          for _i1148 in xrange(_size1144):
+            _elem1149 = Partition()
+            _elem1149.read(iprot)
+            self.success.append(_elem1149)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27892,8 +27875,8 @@ class get_partitions_ps_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1173 in self.success:
-        iter1173.write(oprot)
+      for iter1150 in self.success:
+        iter1150.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27974,10 +27957,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1177, _size1174) = iprot.readListBegin()
-          for _i1178 in xrange(_size1174):
-            _elem1179 = iprot.readString()
-            self.part_vals.append(_elem1179)
+          (_etype1154, _size1151) = iprot.readListBegin()
+          for _i1155 in xrange(_size1151):
+            _elem1156 = iprot.readString()
+            self.part_vals.append(_elem1156)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28007,8 +27990,8 @@ class get_partition_names_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1180 in self.part_vals:
-        oprot.writeString(iter1180)
+      for iter1157 in self.part_vals:
+        oprot.writeString(iter1157)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -28072,10 +28055,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1184, _size1181) = iprot.readListBegin()
-          for _i1185 in xrange(_size1181):
-            _elem1186 = iprot.readString()
-            self.success.append(_elem1186)
+          (_etype1161, _size1158) = iprot.readListBegin()
+          for _i1162 in xrange(_size1158):
+            _elem1163 = iprot.readString()
+            self.success.append(_elem1163)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28104,8 +28087,8 @@ class get_partition_names_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1187 in self.success:
-        oprot.writeString(iter1187)
+      for iter1164 in self.success:
+        oprot.writeString(iter1164)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28276,11 +28259,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1191, _size1188) = iprot.readListBegin()
-          for _i1192 in xrange(_size1188):
-            _elem1193 = Partition()
-            _elem1193.read(iprot)
-            self.success.append(_elem1193)
+          (_etype1168, _size1165) = iprot.readListBegin()
+          for _i1169 in xrange(_size1165):
+            _elem1170 = Partition()
+            _elem1170.read(iprot)
+            self.success.append(_elem1170)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28309,8 +28292,8 @@ class get_partitions_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1194 in self.success:
-        iter1194.write(oprot)
+      for iter1171 in self.success:
+        iter1171.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28481,11 +28464,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1198, _size1195) = iprot.readListBegin()
-          for _i1199 in xrange(_size1195):
-            _elem1200 = PartitionSpec()
-            _elem1200.read(iprot)
-            self.success.append(_elem1200)
+          (_etype1175, _size1172) = iprot.readListBegin()
+          for _i1176 in xrange(_size1172):
+            _elem1177 = PartitionSpec()
+            _elem1177.read(iprot)
+            self.success.append(_elem1177)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28514,8 +28497,8 @@ class get_part_specs_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1201 in self.success:
-        iter1201.write(oprot)
+      for iter1178 in self.success:
+        iter1178.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28935,10 +28918,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype1205, _size1202) = iprot.readListBegin()
-          for _i1206 in xrange(_size1202):
-            _elem1207 = iprot.readString()
-            self.names.append(_elem1207)
+          (_etype1182, _size1179) = iprot.readListBegin()
+          for _i1183 in xrange(_size1179):
+            _elem1184 = iprot.readString()
+            self.names.append(_elem1184)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28963,8 +28946,8 @@ class get_partitions_by_names_args:
     if self.names is not None:
       oprot.writeFieldBegin('names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.names))
-      for iter1208 in self.names:
-        oprot.writeString(iter1208)
+      for iter1185 in self.names:
+        oprot.writeString(iter1185)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -29023,11 +29006,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1212, _size1209) = iprot.readListBegin()
-          for _i1213 in xrange(_size1209):
-            _elem1214 = Partition()
-            _elem1214.read(iprot)
-            self.success.append(_elem1214)
+          (_etype1189, _size1186) = iprot.readListBegin()
+          for _i1190 in xrange(_size1186):
+            _elem1191 = Partition()
+            _elem1191.read(iprot)
+            self.success.append(_elem1191)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29056,8 +29039,8 @@ class get_partitions_by_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1215 in self.success:
-        iter1215.write(oprot)
+      for iter1192 in self.success:
+        iter1192.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29307,11 +29290,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1219, _size1216) = iprot.readListBegin()
-          for _i1220 in xrange(_size1216):
-            _elem1221 = Partition()
-            _elem1221.read(iprot)
-            self.new_parts.append(_elem1221)
+          (_etype1196, _size1193) = iprot.readListBegin()
+          for _i1197 in xrange(_size1193):
+            _elem1198 = Partition()
+            _elem1198.read(iprot)
+            self.new_parts.append(_elem1198)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29336,8 +29319,8 @@ class alter_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1222 in self.new_parts:
-        iter1222.write(oprot)
+      for iter1199 in self.new_parts:
+        iter1199.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -29490,11 +29473,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1226, _size1223) = iprot.readListBegin()
-          for _i1227 in xrange(_size1223):
-            _elem1228 = Partition()
-            _elem1228.read(iprot)
-            self.new_parts.append(_elem1228)
+          (_etype1203, _size1200) = iprot.readListBegin()
+          for _i1204 in xrange(_size1200):
+            _elem1205 = Partition()
+            _elem1205.read(iprot)
+            self.new_parts.append(_elem1205)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29525,8 +29508,8 @@ class alter_partitions_with_environment_context_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1229 in self.new_parts:
-        iter1229.write(oprot)
+      for iter1206 in self.new_parts:
+        iter1206.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -29870,10 +29853,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1233, _size1230) = iprot.readListBegin()
-          for _i1234 in xrange(_size1230):
-            _elem1235 = iprot.readString()
-            self.part_vals.append(_elem1235)
+          (_etype1210, _size1207) = iprot.readListBegin()
+          for _i1211 in xrange(_size1207):
+            _elem1212 = iprot.readString()
+            self.part_vals.append(_elem1212)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29904,8 +29887,8 @@ class rename_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1236 in self.part_vals:
-        oprot.writeString(iter1236)
+      for iter1213 in self.part_vals:
+        oprot.writeString(iter1213)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -30047,10 +30030,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1240, _size1237) = iprot.readListBegin()
-          for _i1241 in xrange(_size1237):
-            _elem1242 = iprot.readString()
-            self.part_vals.append(_elem1242)
+          (_etype1217, _size1214) = iprot.readListBegin()
+          for _i1218 in xrange(_size1214):
+            _elem1219 = iprot.readString()
+            self.part_vals.append(_elem1219)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30072,8 +30055,8 @@ class partition_name_has_valid_characters_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1243 in self.part_vals:
-        oprot.writeString(iter1243)
+      for iter1220 in self.part_vals:
+        oprot.writeString(iter1220)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -30431,10 +30414,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1247, _size1244) = iprot.readListBegin()
-          for _i1248 in xrange(_size1244):
-            _elem1249 = iprot.readString()
-            self.success.append(_elem1249)
+          (_etype1224, _size1221) = iprot.readListBegin()
+          for _i1225 in xrange(_size1221):
+            _elem1226 = iprot.readString()
+            self.success.append(_elem1226)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30457,8 +30440,8 @@ class partition_name_to_vals_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1250 in self.success:
-        oprot.writeString(iter1250)
+      for iter1227 in self.success:
+        oprot.writeString(iter1227)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30582,11 +30565,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype1252, _vtype1253, _size1251 ) = iprot.readMapBegin()
-          for _i1255 in xrange(_size1251):
-            _key1256 = iprot.readString()
-            _val1257 = iprot.readString()
-            self.success[_key1256] = _val1257
+          (_ktype1229, _vtype1230, _size1228 ) = iprot.readMapBegin()
+          for _i1232 in xrange(_size1228):
+            _key1233 = iprot.readString()
+            _val1234 = iprot.readString()
+            self.success[_key1233] = _val1234
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -30609,9 +30592,9 @@ class partition_name_to_spec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success))
-      for kiter1258,viter1259 in self.success.items():
-        oprot.writeString(kiter1258)
-        oprot.writeString(viter1259)
+      for kiter1235,viter1236 in self.success.items():
+        oprot.writeString(kiter1235)
+        oprot.writeString(viter1236)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30687,11 +30670,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1261, _vtype1262, _size1260 ) = iprot.readMapBegin()
-          for _i1264 in xrange(_size1260):
-            _key1265 = iprot.readString()
-            _val1266 = iprot.readString()
-            self.part_vals[_key1265] = _val1266
+          (_ktype1238, _vtype1239, _size1237 ) = iprot.readMapBegin()
+          for _i1241 in xrange(_size1237):
+            _key1242 = iprot.readString()
+            _val1243 = iprot.readString()
+            self.part_vals[_key1242] = _val1243
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -30721,9 +30704,9 @@ class markPartitionForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter1267,viter1268 in self.part_vals.items():
-        oprot.writeString(kiter1267)
-        oprot.writeString(viter1268)
+      for kiter1244,viter1245 in self.part_vals.items():
+        oprot.writeString(kiter1244)
+        oprot.writeString(viter1245)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -30937,11 +30920,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1270, _vtype1271, _size1269 ) = iprot.readMapBegin()
-          for _i1273 in xrange(_size1269):
-            _key1274 = iprot.readString()
-            _val1275 = iprot.readString()
-            self.part_vals[_key1274] = _val1275
+          (_ktype1247, _vtype1248, _size1246 ) = iprot.readMapBegin()
+          for _i1250 in xrange(_size1246):
+            _key1251 = iprot.readString()
+            _val1252 = iprot.readString()
+            self.part_vals[_key1251] = _val1252
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -30971,9 +30954,9 @@ class isPartitionMarkedForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter1276,viter1277 in self.part_vals.items():
-        oprot.writeString(kiter1276)
-        oprot.writeString(viter1277)
+      for kiter1253,viter1254 in self.part_vals.items():
+        oprot.writeString(kiter1253)
+        oprot.writeString(viter1254)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -34625,10 +34608,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1281, _size1278) = iprot.readListBegin()
-          for _i1282 in xrange(_size1278):
-            _elem1283 = iprot.readString()
-            self.success.append(_elem1283)
+          (_etype1258, _size1255) = iprot.readListBegin()
+          for _i1259 in xrange(_size1255):
+            _elem1260 = iprot.readString()
+            self.success.append(_elem1260)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -34651,8 +34634,8 @@ class get_functions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1284 in self.success:
-        oprot.writeString(iter1284)
+      for iter1261 in self.success:
+        oprot.writeString(iter1261)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -35340,10 +35323,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1288, _size1285) = iprot.readListBegin()
-          for _i1289 in xrange(_size1285):
-            _elem1290 = iprot.readString()
-            self.success.append(_elem1290)
+          (_etype1265, _size1262) = iprot.readListBegin()
+          for _i1266 in xrange(_size1262):
+            _elem1267 = iprot.readString()
+            self.success.append(_elem1267)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -35366,8 +35349,8 @@ class get_role_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1291 in self.success:
-        oprot.writeString(iter1291)
+      for iter1268 in self.success:
+        oprot.writeString(iter1268)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -35881,11 +35864,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1295, _size1292) = iprot.readListBegin()
-          for _i1296 in xrange(_size1292):
-            _elem1297 = Role()
-            _elem1297.read(iprot)
-            self.success.append(_elem1297)
+          (_etype1272, _size1269) = iprot.readListBegin()
+          for _i1273 in xrange(_size1269):
+            _elem1274 = Role()
+            _elem1274.read(iprot)
+            self.success.append(_elem1274)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -35908,8 +35891,8 @@ class list_roles_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1298 in self.success:
-        iter1298.write(oprot)
+      for iter1275 in self.success:
+        iter1275.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -36418,10 +36401,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1302, _size1299) = iprot.readListBegin()
-          for _i1303 in xrange(_size1299):
-            _elem1304 = iprot.readString()
-            self.group_names.append(_elem1304)
+          (_etype1279, _size1276) = iprot.readListBegin()
+          for _i1280 in xrange(_size1276):
+            _elem1281 = iprot.readString()
+            self.group_names.append(_elem1281)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36446,8 +36429,8 @@ class get_privilege_set_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1305 in self.group_names:
-        oprot.writeString(iter1305)
+      for iter1282 in self.group_names:
+        oprot.writeString(iter1282)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -36674,11 +36657,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1309, _size1306) = iprot.readListBegin()
-          for _i1310 in xrange(_size1306):
-            _elem1311 = HiveObjectPrivilege()
-            _elem1311.read(iprot)
-            self.success.append(_elem1311)
+          (_etype1286, _size1283) = iprot.readListBegin()
+          for _i1287 in xrange(_size1283):
+            _elem1288 = HiveObjectPrivilege()
+            _elem1288.read(iprot)
+            self.success.append(_elem1288)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36701,8 +36684,8 @@ class list_privileges_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1312 in self.success:
-        iter1312.write(oprot)
+      for iter1289 in self.success:
+        iter1289.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -37372,10 +37355,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1316, _size1313) = iprot.readListBegin()
-          for _i1317 in xrange(_size1313):
-            _elem1318 = iprot.readString()
-            self.group_names.append(_elem1318)
+          (_etype1293, _size1290) = iprot.readListBegin()
+          for _i1294 in xrange(_size1290):
+            _elem1295 = iprot.readString()
+            self.group_names.append(_elem1295)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -37396,8 +37379,8 @@ class set_ugi_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1319 in self.group_names:
-        oprot.writeString(iter1319)
+      for iter1296 in self.group_names:
+        oprot.writeString(iter1296)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -37452,10 +37435,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1323, _size1320) = iprot.readListBegin()
-          for _i1324 in xrange(_size1320):
-            _elem1325 = iprot.readString()
-            self.success.append(_elem1325)
+          (_etype1300, _size1297) = iprot.readListBegin()
+          for _i1301 in xrange(_size1297):
+            _elem1302 = iprot.readString()
+            self.success.append(_elem1302)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -37478,8 +37461,8 @@ class set_ugi_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1326 in self.success:
-        oprot.writeString(iter1326)
+      for iter1303 in self.success:
+        oprot.writeString(iter1303)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -38411,10 +38394,10 @@ class get_all_token_identifiers_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1330, _size1327) = iprot.readListBegin()
-          for _i1331 in xrange(_size1327):
-            _elem1332 = iprot.readString()
-            self.success.append(_elem1332)
+          (_etype1307, _size1304) = iprot.readListBegin()
+          for _i1308 in xrange(_size1304):
+            _elem1309 = iprot.readString()
+            self.success.append(_elem1309)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -38431,8 +38414,8 @@ class get_all_token_identifiers_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1333 in self.success:
-        oprot.writeString(iter1333)
+      for iter1310 in self.success:
+        oprot.writeString(iter1310)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -38959,10 +38942,10 @@ class get_master_keys_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1337, _size1334) = iprot.readListBegin()
-          for _i1338 in xrange(_size1334):
-            _elem1339 = iprot.readString()
-            self.success.append(_elem1339)
+          (_etype1314, _size1311) = iprot.readListBegin()
+          for _i1315 in xrange(_size1311):
+            _elem1316 = iprot.readString()
+            self.success.append(_elem1316)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -38979,8 +38962,8 @@ class get_master_keys_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1340 in self.success:
-        oprot.writeString(iter1340)
+      for iter1317 in self.success:
+        oprot.writeString(iter1317)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -47387,11 +47370,11 @@ class get_schema_all_versions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1344, _size1341) = iprot.readListBegin()
-          for _i1345 in xrange(_size1341):
-            _elem1346 = SchemaVersion()
-            _elem1346.read(iprot)
-            self.success.append(_elem1346)
+          (_etype1321, _size1318) = iprot.readListBegin()
+          for _i1322 in xrange(_size1318):
+            _elem1323 = SchemaVersion()
+            _elem1323.read(iprot)
+            self.success.append(_elem1323)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -47420,8 +47403,8 @@ class get_schema_all_versions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1347 in self.success:
-        iter1347.write(oprot)
+      for iter1324 in self.success:
+        iter1324.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -48896,11 +48879,11 @@ class get_runtime_stats_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1351, _size1348) = iprot.readListBegin()
-          for _i1352 in xrange(_size1348):
-            _elem1353 = RuntimeStat()
-            _elem1353.read(iprot)
-            self.success.append(_elem1353)
+          (_etype1328, _size1325) = iprot.readListBegin()
+          for _i1329 in xrange(_size1325):
+            _elem1330 = RuntimeStat()
+            _elem1330.read(iprot)
+            self.success.append(_elem1330)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -48923,8 +48906,8 @@ class get_runtime_stats_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1354 in self.success:
-        iter1354.write(oprot)
+      for iter1331 in self.success:
+        iter1331.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:


[09/12] hive git commit: HIVE-20006: Make materializations invalidation cache work with multiple active remote metastores (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 7f06b3b..26420dd 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -21220,6 +21220,11 @@ void CreationMetadata::__set_validTxnList(const std::string& val) {
 __isset.validTxnList = true;
 }
 
+void CreationMetadata::__set_materializationTime(const int64_t val) {
+  this->materializationTime = val;
+__isset.materializationTime = true;
+}
+
 uint32_t CreationMetadata::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -21298,6 +21303,14 @@ uint32_t CreationMetadata::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->materializationTime);
+          this->__isset.materializationTime = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -21352,6 +21365,11 @@ uint32_t CreationMetadata::write(::apache::thrift::protocol::TProtocol* oprot) c
     xfer += oprot->writeString(this->validTxnList);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.materializationTime) {
+    xfer += oprot->writeFieldBegin("materializationTime", ::apache::thrift::protocol::T_I64, 6);
+    xfer += oprot->writeI64(this->materializationTime);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -21364,6 +21382,7 @@ void swap(CreationMetadata &a, CreationMetadata &b) {
   swap(a.tblName, b.tblName);
   swap(a.tablesUsed, b.tablesUsed);
   swap(a.validTxnList, b.validTxnList);
+  swap(a.materializationTime, b.materializationTime);
   swap(a.__isset, b.__isset);
 }
 
@@ -21373,6 +21392,7 @@ CreationMetadata::CreationMetadata(const CreationMetadata& other837) {
   tblName = other837.tblName;
   tablesUsed = other837.tablesUsed;
   validTxnList = other837.validTxnList;
+  materializationTime = other837.materializationTime;
   __isset = other837.__isset;
 }
 CreationMetadata& CreationMetadata::operator=(const CreationMetadata& other838) {
@@ -21381,6 +21401,7 @@ CreationMetadata& CreationMetadata::operator=(const CreationMetadata& other838)
   tblName = other838.tblName;
   tablesUsed = other838.tablesUsed;
   validTxnList = other838.validTxnList;
+  materializationTime = other838.materializationTime;
   __isset = other838.__isset;
   return *this;
 }
@@ -21392,6 +21413,7 @@ void CreationMetadata::printTo(std::ostream& out) const {
   out << ", " << "tblName=" << to_string(tblName);
   out << ", " << "tablesUsed=" << to_string(tablesUsed);
   out << ", " << "validTxnList="; (__isset.validTxnList ? (out << to_string(validTxnList)) : (out << "<null>"));
+  out << ", " << "materializationTime="; (__isset.materializationTime ? (out << to_string(materializationTime)) : (out << "<null>"));
   out << ")";
 }
 
@@ -25434,23 +25456,8 @@ Materialization::~Materialization() throw() {
 }
 
 
-void Materialization::__set_tablesUsed(const std::set<std::string> & val) {
-  this->tablesUsed = val;
-}
-
-void Materialization::__set_validTxnList(const std::string& val) {
-  this->validTxnList = val;
-__isset.validTxnList = true;
-}
-
-void Materialization::__set_invalidationTime(const int64_t val) {
-  this->invalidationTime = val;
-__isset.invalidationTime = true;
-}
-
 void Materialization::__set_sourceTablesUpdateDeleteModified(const bool val) {
   this->sourceTablesUpdateDeleteModified = val;
-__isset.sourceTablesUpdateDeleteModified = true;
 }
 
 uint32_t Materialization::read(::apache::thrift::protocol::TProtocol* iprot) {
@@ -25465,7 +25472,7 @@ uint32_t Materialization::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   using ::apache::thrift::protocol::TProtocolException;
 
-  bool isset_tablesUsed = false;
+  bool isset_sourceTablesUpdateDeleteModified = false;
 
   while (true)
   {
@@ -25476,46 +25483,9 @@ uint32_t Materialization::read(::apache::thrift::protocol::TProtocol* iprot) {
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_SET) {
-          {
-            this->tablesUsed.clear();
-            uint32_t _size1012;
-            ::apache::thrift::protocol::TType _etype1015;
-            xfer += iprot->readSetBegin(_etype1015, _size1012);
-            uint32_t _i1016;
-            for (_i1016 = 0; _i1016 < _size1012; ++_i1016)
-            {
-              std::string _elem1017;
-              xfer += iprot->readString(_elem1017);
-              this->tablesUsed.insert(_elem1017);
-            }
-            xfer += iprot->readSetEnd();
-          }
-          isset_tablesUsed = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->validTxnList);
-          this->__isset.validTxnList = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_I64) {
-          xfer += iprot->readI64(this->invalidationTime);
-          this->__isset.invalidationTime = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 4:
         if (ftype == ::apache::thrift::protocol::T_BOOL) {
           xfer += iprot->readBool(this->sourceTablesUpdateDeleteModified);
-          this->__isset.sourceTablesUpdateDeleteModified = true;
+          isset_sourceTablesUpdateDeleteModified = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -25529,7 +25499,7 @@ uint32_t Materialization::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   xfer += iprot->readStructEnd();
 
-  if (!isset_tablesUsed)
+  if (!isset_sourceTablesUpdateDeleteModified)
     throw TProtocolException(TProtocolException::INVALID_DATA);
   return xfer;
 }
@@ -25539,33 +25509,10 @@ uint32_t Materialization::write(::apache::thrift::protocol::TProtocol* oprot) co
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
   xfer += oprot->writeStructBegin("Materialization");
 
-  xfer += oprot->writeFieldBegin("tablesUsed", ::apache::thrift::protocol::T_SET, 1);
-  {
-    xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tablesUsed.size()));
-    std::set<std::string> ::const_iterator _iter1018;
-    for (_iter1018 = this->tablesUsed.begin(); _iter1018 != this->tablesUsed.end(); ++_iter1018)
-    {
-      xfer += oprot->writeString((*_iter1018));
-    }
-    xfer += oprot->writeSetEnd();
-  }
+  xfer += oprot->writeFieldBegin("sourceTablesUpdateDeleteModified", ::apache::thrift::protocol::T_BOOL, 1);
+  xfer += oprot->writeBool(this->sourceTablesUpdateDeleteModified);
   xfer += oprot->writeFieldEnd();
 
-  if (this->__isset.validTxnList) {
-    xfer += oprot->writeFieldBegin("validTxnList", ::apache::thrift::protocol::T_STRING, 2);
-    xfer += oprot->writeString(this->validTxnList);
-    xfer += oprot->writeFieldEnd();
-  }
-  if (this->__isset.invalidationTime) {
-    xfer += oprot->writeFieldBegin("invalidationTime", ::apache::thrift::protocol::T_I64, 3);
-    xfer += oprot->writeI64(this->invalidationTime);
-    xfer += oprot->writeFieldEnd();
-  }
-  if (this->__isset.sourceTablesUpdateDeleteModified) {
-    xfer += oprot->writeFieldBegin("sourceTablesUpdateDeleteModified", ::apache::thrift::protocol::T_BOOL, 4);
-    xfer += oprot->writeBool(this->sourceTablesUpdateDeleteModified);
-    xfer += oprot->writeFieldEnd();
-  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -25573,35 +25520,20 @@ uint32_t Materialization::write(::apache::thrift::protocol::TProtocol* oprot) co
 
 void swap(Materialization &a, Materialization &b) {
   using ::std::swap;
-  swap(a.tablesUsed, b.tablesUsed);
-  swap(a.validTxnList, b.validTxnList);
-  swap(a.invalidationTime, b.invalidationTime);
   swap(a.sourceTablesUpdateDeleteModified, b.sourceTablesUpdateDeleteModified);
-  swap(a.__isset, b.__isset);
 }
 
-Materialization::Materialization(const Materialization& other1019) {
-  tablesUsed = other1019.tablesUsed;
-  validTxnList = other1019.validTxnList;
-  invalidationTime = other1019.invalidationTime;
-  sourceTablesUpdateDeleteModified = other1019.sourceTablesUpdateDeleteModified;
-  __isset = other1019.__isset;
+Materialization::Materialization(const Materialization& other1012) {
+  sourceTablesUpdateDeleteModified = other1012.sourceTablesUpdateDeleteModified;
 }
-Materialization& Materialization::operator=(const Materialization& other1020) {
-  tablesUsed = other1020.tablesUsed;
-  validTxnList = other1020.validTxnList;
-  invalidationTime = other1020.invalidationTime;
-  sourceTablesUpdateDeleteModified = other1020.sourceTablesUpdateDeleteModified;
-  __isset = other1020.__isset;
+Materialization& Materialization::operator=(const Materialization& other1013) {
+  sourceTablesUpdateDeleteModified = other1013.sourceTablesUpdateDeleteModified;
   return *this;
 }
 void Materialization::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
   out << "Materialization(";
-  out << "tablesUsed=" << to_string(tablesUsed);
-  out << ", " << "validTxnList="; (__isset.validTxnList ? (out << to_string(validTxnList)) : (out << "<null>"));
-  out << ", " << "invalidationTime="; (__isset.invalidationTime ? (out << to_string(invalidationTime)) : (out << "<null>"));
-  out << ", " << "sourceTablesUpdateDeleteModified="; (__isset.sourceTablesUpdateDeleteModified ? (out << to_string(sourceTablesUpdateDeleteModified)) : (out << "<null>"));
+  out << "sourceTablesUpdateDeleteModified=" << to_string(sourceTablesUpdateDeleteModified);
   out << ")";
 }
 
@@ -25661,9 +25593,9 @@ uint32_t WMResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast1021;
-          xfer += iprot->readI32(ecast1021);
-          this->status = (WMResourcePlanStatus::type)ecast1021;
+          int32_t ecast1014;
+          xfer += iprot->readI32(ecast1014);
+          this->status = (WMResourcePlanStatus::type)ecast1014;
           this->__isset.status = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -25737,19 +25669,19 @@ void swap(WMResourcePlan &a, WMResourcePlan &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMResourcePlan::WMResourcePlan(const WMResourcePlan& other1022) {
-  name = other1022.name;
-  status = other1022.status;
-  queryParallelism = other1022.queryParallelism;
-  defaultPoolPath = other1022.defaultPoolPath;
-  __isset = other1022.__isset;
+WMResourcePlan::WMResourcePlan(const WMResourcePlan& other1015) {
+  name = other1015.name;
+  status = other1015.status;
+  queryParallelism = other1015.queryParallelism;
+  defaultPoolPath = other1015.defaultPoolPath;
+  __isset = other1015.__isset;
 }
-WMResourcePlan& WMResourcePlan::operator=(const WMResourcePlan& other1023) {
-  name = other1023.name;
-  status = other1023.status;
-  queryParallelism = other1023.queryParallelism;
-  defaultPoolPath = other1023.defaultPoolPath;
-  __isset = other1023.__isset;
+WMResourcePlan& WMResourcePlan::operator=(const WMResourcePlan& other1016) {
+  name = other1016.name;
+  status = other1016.status;
+  queryParallelism = other1016.queryParallelism;
+  defaultPoolPath = other1016.defaultPoolPath;
+  __isset = other1016.__isset;
   return *this;
 }
 void WMResourcePlan::printTo(std::ostream& out) const {
@@ -25828,9 +25760,9 @@ uint32_t WMNullableResourcePlan::read(::apache::thrift::protocol::TProtocol* ipr
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast1024;
-          xfer += iprot->readI32(ecast1024);
-          this->status = (WMResourcePlanStatus::type)ecast1024;
+          int32_t ecast1017;
+          xfer += iprot->readI32(ecast1017);
+          this->status = (WMResourcePlanStatus::type)ecast1017;
           this->__isset.status = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -25931,23 +25863,23 @@ void swap(WMNullableResourcePlan &a, WMNullableResourcePlan &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMNullableResourcePlan::WMNullableResourcePlan(const WMNullableResourcePlan& other1025) {
-  name = other1025.name;
-  status = other1025.status;
-  queryParallelism = other1025.queryParallelism;
-  isSetQueryParallelism = other1025.isSetQueryParallelism;
-  defaultPoolPath = other1025.defaultPoolPath;
-  isSetDefaultPoolPath = other1025.isSetDefaultPoolPath;
-  __isset = other1025.__isset;
-}
-WMNullableResourcePlan& WMNullableResourcePlan::operator=(const WMNullableResourcePlan& other1026) {
-  name = other1026.name;
-  status = other1026.status;
-  queryParallelism = other1026.queryParallelism;
-  isSetQueryParallelism = other1026.isSetQueryParallelism;
-  defaultPoolPath = other1026.defaultPoolPath;
-  isSetDefaultPoolPath = other1026.isSetDefaultPoolPath;
-  __isset = other1026.__isset;
+WMNullableResourcePlan::WMNullableResourcePlan(const WMNullableResourcePlan& other1018) {
+  name = other1018.name;
+  status = other1018.status;
+  queryParallelism = other1018.queryParallelism;
+  isSetQueryParallelism = other1018.isSetQueryParallelism;
+  defaultPoolPath = other1018.defaultPoolPath;
+  isSetDefaultPoolPath = other1018.isSetDefaultPoolPath;
+  __isset = other1018.__isset;
+}
+WMNullableResourcePlan& WMNullableResourcePlan::operator=(const WMNullableResourcePlan& other1019) {
+  name = other1019.name;
+  status = other1019.status;
+  queryParallelism = other1019.queryParallelism;
+  isSetQueryParallelism = other1019.isSetQueryParallelism;
+  defaultPoolPath = other1019.defaultPoolPath;
+  isSetDefaultPoolPath = other1019.isSetDefaultPoolPath;
+  __isset = other1019.__isset;
   return *this;
 }
 void WMNullableResourcePlan::printTo(std::ostream& out) const {
@@ -26112,21 +26044,21 @@ void swap(WMPool &a, WMPool &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMPool::WMPool(const WMPool& other1027) {
-  resourcePlanName = other1027.resourcePlanName;
-  poolPath = other1027.poolPath;
-  allocFraction = other1027.allocFraction;
-  queryParallelism = other1027.queryParallelism;
-  schedulingPolicy = other1027.schedulingPolicy;
-  __isset = other1027.__isset;
+WMPool::WMPool(const WMPool& other1020) {
+  resourcePlanName = other1020.resourcePlanName;
+  poolPath = other1020.poolPath;
+  allocFraction = other1020.allocFraction;
+  queryParallelism = other1020.queryParallelism;
+  schedulingPolicy = other1020.schedulingPolicy;
+  __isset = other1020.__isset;
 }
-WMPool& WMPool::operator=(const WMPool& other1028) {
-  resourcePlanName = other1028.resourcePlanName;
-  poolPath = other1028.poolPath;
-  allocFraction = other1028.allocFraction;
-  queryParallelism = other1028.queryParallelism;
-  schedulingPolicy = other1028.schedulingPolicy;
-  __isset = other1028.__isset;
+WMPool& WMPool::operator=(const WMPool& other1021) {
+  resourcePlanName = other1021.resourcePlanName;
+  poolPath = other1021.poolPath;
+  allocFraction = other1021.allocFraction;
+  queryParallelism = other1021.queryParallelism;
+  schedulingPolicy = other1021.schedulingPolicy;
+  __isset = other1021.__isset;
   return *this;
 }
 void WMPool::printTo(std::ostream& out) const {
@@ -26309,23 +26241,23 @@ void swap(WMNullablePool &a, WMNullablePool &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMNullablePool::WMNullablePool(const WMNullablePool& other1029) {
-  resourcePlanName = other1029.resourcePlanName;
-  poolPath = other1029.poolPath;
-  allocFraction = other1029.allocFraction;
-  queryParallelism = other1029.queryParallelism;
-  schedulingPolicy = other1029.schedulingPolicy;
-  isSetSchedulingPolicy = other1029.isSetSchedulingPolicy;
-  __isset = other1029.__isset;
-}
-WMNullablePool& WMNullablePool::operator=(const WMNullablePool& other1030) {
-  resourcePlanName = other1030.resourcePlanName;
-  poolPath = other1030.poolPath;
-  allocFraction = other1030.allocFraction;
-  queryParallelism = other1030.queryParallelism;
-  schedulingPolicy = other1030.schedulingPolicy;
-  isSetSchedulingPolicy = other1030.isSetSchedulingPolicy;
-  __isset = other1030.__isset;
+WMNullablePool::WMNullablePool(const WMNullablePool& other1022) {
+  resourcePlanName = other1022.resourcePlanName;
+  poolPath = other1022.poolPath;
+  allocFraction = other1022.allocFraction;
+  queryParallelism = other1022.queryParallelism;
+  schedulingPolicy = other1022.schedulingPolicy;
+  isSetSchedulingPolicy = other1022.isSetSchedulingPolicy;
+  __isset = other1022.__isset;
+}
+WMNullablePool& WMNullablePool::operator=(const WMNullablePool& other1023) {
+  resourcePlanName = other1023.resourcePlanName;
+  poolPath = other1023.poolPath;
+  allocFraction = other1023.allocFraction;
+  queryParallelism = other1023.queryParallelism;
+  schedulingPolicy = other1023.schedulingPolicy;
+  isSetSchedulingPolicy = other1023.isSetSchedulingPolicy;
+  __isset = other1023.__isset;
   return *this;
 }
 void WMNullablePool::printTo(std::ostream& out) const {
@@ -26490,21 +26422,21 @@ void swap(WMTrigger &a, WMTrigger &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMTrigger::WMTrigger(const WMTrigger& other1031) {
-  resourcePlanName = other1031.resourcePlanName;
-  triggerName = other1031.triggerName;
-  triggerExpression = other1031.triggerExpression;
-  actionExpression = other1031.actionExpression;
-  isInUnmanaged = other1031.isInUnmanaged;
-  __isset = other1031.__isset;
-}
-WMTrigger& WMTrigger::operator=(const WMTrigger& other1032) {
-  resourcePlanName = other1032.resourcePlanName;
-  triggerName = other1032.triggerName;
-  triggerExpression = other1032.triggerExpression;
-  actionExpression = other1032.actionExpression;
-  isInUnmanaged = other1032.isInUnmanaged;
-  __isset = other1032.__isset;
+WMTrigger::WMTrigger(const WMTrigger& other1024) {
+  resourcePlanName = other1024.resourcePlanName;
+  triggerName = other1024.triggerName;
+  triggerExpression = other1024.triggerExpression;
+  actionExpression = other1024.actionExpression;
+  isInUnmanaged = other1024.isInUnmanaged;
+  __isset = other1024.__isset;
+}
+WMTrigger& WMTrigger::operator=(const WMTrigger& other1025) {
+  resourcePlanName = other1025.resourcePlanName;
+  triggerName = other1025.triggerName;
+  triggerExpression = other1025.triggerExpression;
+  actionExpression = other1025.actionExpression;
+  isInUnmanaged = other1025.isInUnmanaged;
+  __isset = other1025.__isset;
   return *this;
 }
 void WMTrigger::printTo(std::ostream& out) const {
@@ -26669,21 +26601,21 @@ void swap(WMMapping &a, WMMapping &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMMapping::WMMapping(const WMMapping& other1033) {
-  resourcePlanName = other1033.resourcePlanName;
-  entityType = other1033.entityType;
-  entityName = other1033.entityName;
-  poolPath = other1033.poolPath;
-  ordering = other1033.ordering;
-  __isset = other1033.__isset;
-}
-WMMapping& WMMapping::operator=(const WMMapping& other1034) {
-  resourcePlanName = other1034.resourcePlanName;
-  entityType = other1034.entityType;
-  entityName = other1034.entityName;
-  poolPath = other1034.poolPath;
-  ordering = other1034.ordering;
-  __isset = other1034.__isset;
+WMMapping::WMMapping(const WMMapping& other1026) {
+  resourcePlanName = other1026.resourcePlanName;
+  entityType = other1026.entityType;
+  entityName = other1026.entityName;
+  poolPath = other1026.poolPath;
+  ordering = other1026.ordering;
+  __isset = other1026.__isset;
+}
+WMMapping& WMMapping::operator=(const WMMapping& other1027) {
+  resourcePlanName = other1027.resourcePlanName;
+  entityType = other1027.entityType;
+  entityName = other1027.entityName;
+  poolPath = other1027.poolPath;
+  ordering = other1027.ordering;
+  __isset = other1027.__isset;
   return *this;
 }
 void WMMapping::printTo(std::ostream& out) const {
@@ -26789,13 +26721,13 @@ void swap(WMPoolTrigger &a, WMPoolTrigger &b) {
   swap(a.trigger, b.trigger);
 }
 
-WMPoolTrigger::WMPoolTrigger(const WMPoolTrigger& other1035) {
-  pool = other1035.pool;
-  trigger = other1035.trigger;
+WMPoolTrigger::WMPoolTrigger(const WMPoolTrigger& other1028) {
+  pool = other1028.pool;
+  trigger = other1028.trigger;
 }
-WMPoolTrigger& WMPoolTrigger::operator=(const WMPoolTrigger& other1036) {
-  pool = other1036.pool;
-  trigger = other1036.trigger;
+WMPoolTrigger& WMPoolTrigger::operator=(const WMPoolTrigger& other1029) {
+  pool = other1029.pool;
+  trigger = other1029.trigger;
   return *this;
 }
 void WMPoolTrigger::printTo(std::ostream& out) const {
@@ -26869,14 +26801,14 @@ uint32_t WMFullResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->pools.clear();
-            uint32_t _size1037;
-            ::apache::thrift::protocol::TType _etype1040;
-            xfer += iprot->readListBegin(_etype1040, _size1037);
-            this->pools.resize(_size1037);
-            uint32_t _i1041;
-            for (_i1041 = 0; _i1041 < _size1037; ++_i1041)
+            uint32_t _size1030;
+            ::apache::thrift::protocol::TType _etype1033;
+            xfer += iprot->readListBegin(_etype1033, _size1030);
+            this->pools.resize(_size1030);
+            uint32_t _i1034;
+            for (_i1034 = 0; _i1034 < _size1030; ++_i1034)
             {
-              xfer += this->pools[_i1041].read(iprot);
+              xfer += this->pools[_i1034].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -26889,14 +26821,14 @@ uint32_t WMFullResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->mappings.clear();
-            uint32_t _size1042;
-            ::apache::thrift::protocol::TType _etype1045;
-            xfer += iprot->readListBegin(_etype1045, _size1042);
-            this->mappings.resize(_size1042);
-            uint32_t _i1046;
-            for (_i1046 = 0; _i1046 < _size1042; ++_i1046)
+            uint32_t _size1035;
+            ::apache::thrift::protocol::TType _etype1038;
+            xfer += iprot->readListBegin(_etype1038, _size1035);
+            this->mappings.resize(_size1035);
+            uint32_t _i1039;
+            for (_i1039 = 0; _i1039 < _size1035; ++_i1039)
             {
-              xfer += this->mappings[_i1046].read(iprot);
+              xfer += this->mappings[_i1039].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -26909,14 +26841,14 @@ uint32_t WMFullResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->triggers.clear();
-            uint32_t _size1047;
-            ::apache::thrift::protocol::TType _etype1050;
-            xfer += iprot->readListBegin(_etype1050, _size1047);
-            this->triggers.resize(_size1047);
-            uint32_t _i1051;
-            for (_i1051 = 0; _i1051 < _size1047; ++_i1051)
+            uint32_t _size1040;
+            ::apache::thrift::protocol::TType _etype1043;
+            xfer += iprot->readListBegin(_etype1043, _size1040);
+            this->triggers.resize(_size1040);
+            uint32_t _i1044;
+            for (_i1044 = 0; _i1044 < _size1040; ++_i1044)
             {
-              xfer += this->triggers[_i1051].read(iprot);
+              xfer += this->triggers[_i1044].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -26929,14 +26861,14 @@ uint32_t WMFullResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->poolTriggers.clear();
-            uint32_t _size1052;
-            ::apache::thrift::protocol::TType _etype1055;
-            xfer += iprot->readListBegin(_etype1055, _size1052);
-            this->poolTriggers.resize(_size1052);
-            uint32_t _i1056;
-            for (_i1056 = 0; _i1056 < _size1052; ++_i1056)
+            uint32_t _size1045;
+            ::apache::thrift::protocol::TType _etype1048;
+            xfer += iprot->readListBegin(_etype1048, _size1045);
+            this->poolTriggers.resize(_size1045);
+            uint32_t _i1049;
+            for (_i1049 = 0; _i1049 < _size1045; ++_i1049)
             {
-              xfer += this->poolTriggers[_i1056].read(iprot);
+              xfer += this->poolTriggers[_i1049].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -26973,10 +26905,10 @@ uint32_t WMFullResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("pools", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->pools.size()));
-    std::vector<WMPool> ::const_iterator _iter1057;
-    for (_iter1057 = this->pools.begin(); _iter1057 != this->pools.end(); ++_iter1057)
+    std::vector<WMPool> ::const_iterator _iter1050;
+    for (_iter1050 = this->pools.begin(); _iter1050 != this->pools.end(); ++_iter1050)
     {
-      xfer += (*_iter1057).write(oprot);
+      xfer += (*_iter1050).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -26986,10 +26918,10 @@ uint32_t WMFullResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot)
     xfer += oprot->writeFieldBegin("mappings", ::apache::thrift::protocol::T_LIST, 3);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->mappings.size()));
-      std::vector<WMMapping> ::const_iterator _iter1058;
-      for (_iter1058 = this->mappings.begin(); _iter1058 != this->mappings.end(); ++_iter1058)
+      std::vector<WMMapping> ::const_iterator _iter1051;
+      for (_iter1051 = this->mappings.begin(); _iter1051 != this->mappings.end(); ++_iter1051)
       {
-        xfer += (*_iter1058).write(oprot);
+        xfer += (*_iter1051).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -26999,10 +26931,10 @@ uint32_t WMFullResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot)
     xfer += oprot->writeFieldBegin("triggers", ::apache::thrift::protocol::T_LIST, 4);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->triggers.size()));
-      std::vector<WMTrigger> ::const_iterator _iter1059;
-      for (_iter1059 = this->triggers.begin(); _iter1059 != this->triggers.end(); ++_iter1059)
+      std::vector<WMTrigger> ::const_iterator _iter1052;
+      for (_iter1052 = this->triggers.begin(); _iter1052 != this->triggers.end(); ++_iter1052)
       {
-        xfer += (*_iter1059).write(oprot);
+        xfer += (*_iter1052).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -27012,10 +26944,10 @@ uint32_t WMFullResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot)
     xfer += oprot->writeFieldBegin("poolTriggers", ::apache::thrift::protocol::T_LIST, 5);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->poolTriggers.size()));
-      std::vector<WMPoolTrigger> ::const_iterator _iter1060;
-      for (_iter1060 = this->poolTriggers.begin(); _iter1060 != this->poolTriggers.end(); ++_iter1060)
+      std::vector<WMPoolTrigger> ::const_iterator _iter1053;
+      for (_iter1053 = this->poolTriggers.begin(); _iter1053 != this->poolTriggers.end(); ++_iter1053)
       {
-        xfer += (*_iter1060).write(oprot);
+        xfer += (*_iter1053).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -27036,21 +26968,21 @@ void swap(WMFullResourcePlan &a, WMFullResourcePlan &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMFullResourcePlan::WMFullResourcePlan(const WMFullResourcePlan& other1061) {
-  plan = other1061.plan;
-  pools = other1061.pools;
-  mappings = other1061.mappings;
-  triggers = other1061.triggers;
-  poolTriggers = other1061.poolTriggers;
-  __isset = other1061.__isset;
-}
-WMFullResourcePlan& WMFullResourcePlan::operator=(const WMFullResourcePlan& other1062) {
-  plan = other1062.plan;
-  pools = other1062.pools;
-  mappings = other1062.mappings;
-  triggers = other1062.triggers;
-  poolTriggers = other1062.poolTriggers;
-  __isset = other1062.__isset;
+WMFullResourcePlan::WMFullResourcePlan(const WMFullResourcePlan& other1054) {
+  plan = other1054.plan;
+  pools = other1054.pools;
+  mappings = other1054.mappings;
+  triggers = other1054.triggers;
+  poolTriggers = other1054.poolTriggers;
+  __isset = other1054.__isset;
+}
+WMFullResourcePlan& WMFullResourcePlan::operator=(const WMFullResourcePlan& other1055) {
+  plan = other1055.plan;
+  pools = other1055.pools;
+  mappings = other1055.mappings;
+  triggers = other1055.triggers;
+  poolTriggers = other1055.poolTriggers;
+  __isset = other1055.__isset;
   return *this;
 }
 void WMFullResourcePlan::printTo(std::ostream& out) const {
@@ -27155,15 +27087,15 @@ void swap(WMCreateResourcePlanRequest &a, WMCreateResourcePlanRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMCreateResourcePlanRequest::WMCreateResourcePlanRequest(const WMCreateResourcePlanRequest& other1063) {
-  resourcePlan = other1063.resourcePlan;
-  copyFrom = other1063.copyFrom;
-  __isset = other1063.__isset;
+WMCreateResourcePlanRequest::WMCreateResourcePlanRequest(const WMCreateResourcePlanRequest& other1056) {
+  resourcePlan = other1056.resourcePlan;
+  copyFrom = other1056.copyFrom;
+  __isset = other1056.__isset;
 }
-WMCreateResourcePlanRequest& WMCreateResourcePlanRequest::operator=(const WMCreateResourcePlanRequest& other1064) {
-  resourcePlan = other1064.resourcePlan;
-  copyFrom = other1064.copyFrom;
-  __isset = other1064.__isset;
+WMCreateResourcePlanRequest& WMCreateResourcePlanRequest::operator=(const WMCreateResourcePlanRequest& other1057) {
+  resourcePlan = other1057.resourcePlan;
+  copyFrom = other1057.copyFrom;
+  __isset = other1057.__isset;
   return *this;
 }
 void WMCreateResourcePlanRequest::printTo(std::ostream& out) const {
@@ -27223,11 +27155,11 @@ void swap(WMCreateResourcePlanResponse &a, WMCreateResourcePlanResponse &b) {
   (void) b;
 }
 
-WMCreateResourcePlanResponse::WMCreateResourcePlanResponse(const WMCreateResourcePlanResponse& other1065) {
-  (void) other1065;
+WMCreateResourcePlanResponse::WMCreateResourcePlanResponse(const WMCreateResourcePlanResponse& other1058) {
+  (void) other1058;
 }
-WMCreateResourcePlanResponse& WMCreateResourcePlanResponse::operator=(const WMCreateResourcePlanResponse& other1066) {
-  (void) other1066;
+WMCreateResourcePlanResponse& WMCreateResourcePlanResponse::operator=(const WMCreateResourcePlanResponse& other1059) {
+  (void) other1059;
   return *this;
 }
 void WMCreateResourcePlanResponse::printTo(std::ostream& out) const {
@@ -27285,11 +27217,11 @@ void swap(WMGetActiveResourcePlanRequest &a, WMGetActiveResourcePlanRequest &b)
   (void) b;
 }
 
-WMGetActiveResourcePlanRequest::WMGetActiveResourcePlanRequest(const WMGetActiveResourcePlanRequest& other1067) {
-  (void) other1067;
+WMGetActiveResourcePlanRequest::WMGetActiveResourcePlanRequest(const WMGetActiveResourcePlanRequest& other1060) {
+  (void) other1060;
 }
-WMGetActiveResourcePlanRequest& WMGetActiveResourcePlanRequest::operator=(const WMGetActiveResourcePlanRequest& other1068) {
-  (void) other1068;
+WMGetActiveResourcePlanRequest& WMGetActiveResourcePlanRequest::operator=(const WMGetActiveResourcePlanRequest& other1061) {
+  (void) other1061;
   return *this;
 }
 void WMGetActiveResourcePlanRequest::printTo(std::ostream& out) const {
@@ -27370,13 +27302,13 @@ void swap(WMGetActiveResourcePlanResponse &a, WMGetActiveResourcePlanResponse &b
   swap(a.__isset, b.__isset);
 }
 
-WMGetActiveResourcePlanResponse::WMGetActiveResourcePlanResponse(const WMGetActiveResourcePlanResponse& other1069) {
-  resourcePlan = other1069.resourcePlan;
-  __isset = other1069.__isset;
+WMGetActiveResourcePlanResponse::WMGetActiveResourcePlanResponse(const WMGetActiveResourcePlanResponse& other1062) {
+  resourcePlan = other1062.resourcePlan;
+  __isset = other1062.__isset;
 }
-WMGetActiveResourcePlanResponse& WMGetActiveResourcePlanResponse::operator=(const WMGetActiveResourcePlanResponse& other1070) {
-  resourcePlan = other1070.resourcePlan;
-  __isset = other1070.__isset;
+WMGetActiveResourcePlanResponse& WMGetActiveResourcePlanResponse::operator=(const WMGetActiveResourcePlanResponse& other1063) {
+  resourcePlan = other1063.resourcePlan;
+  __isset = other1063.__isset;
   return *this;
 }
 void WMGetActiveResourcePlanResponse::printTo(std::ostream& out) const {
@@ -27458,13 +27390,13 @@ void swap(WMGetResourcePlanRequest &a, WMGetResourcePlanRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMGetResourcePlanRequest::WMGetResourcePlanRequest(const WMGetResourcePlanRequest& other1071) {
-  resourcePlanName = other1071.resourcePlanName;
-  __isset = other1071.__isset;
+WMGetResourcePlanRequest::WMGetResourcePlanRequest(const WMGetResourcePlanRequest& other1064) {
+  resourcePlanName = other1064.resourcePlanName;
+  __isset = other1064.__isset;
 }
-WMGetResourcePlanRequest& WMGetResourcePlanRequest::operator=(const WMGetResourcePlanRequest& other1072) {
-  resourcePlanName = other1072.resourcePlanName;
-  __isset = other1072.__isset;
+WMGetResourcePlanRequest& WMGetResourcePlanRequest::operator=(const WMGetResourcePlanRequest& other1065) {
+  resourcePlanName = other1065.resourcePlanName;
+  __isset = other1065.__isset;
   return *this;
 }
 void WMGetResourcePlanRequest::printTo(std::ostream& out) const {
@@ -27546,13 +27478,13 @@ void swap(WMGetResourcePlanResponse &a, WMGetResourcePlanResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMGetResourcePlanResponse::WMGetResourcePlanResponse(const WMGetResourcePlanResponse& other1073) {
-  resourcePlan = other1073.resourcePlan;
-  __isset = other1073.__isset;
+WMGetResourcePlanResponse::WMGetResourcePlanResponse(const WMGetResourcePlanResponse& other1066) {
+  resourcePlan = other1066.resourcePlan;
+  __isset = other1066.__isset;
 }
-WMGetResourcePlanResponse& WMGetResourcePlanResponse::operator=(const WMGetResourcePlanResponse& other1074) {
-  resourcePlan = other1074.resourcePlan;
-  __isset = other1074.__isset;
+WMGetResourcePlanResponse& WMGetResourcePlanResponse::operator=(const WMGetResourcePlanResponse& other1067) {
+  resourcePlan = other1067.resourcePlan;
+  __isset = other1067.__isset;
   return *this;
 }
 void WMGetResourcePlanResponse::printTo(std::ostream& out) const {
@@ -27611,11 +27543,11 @@ void swap(WMGetAllResourcePlanRequest &a, WMGetAllResourcePlanRequest &b) {
   (void) b;
 }
 
-WMGetAllResourcePlanRequest::WMGetAllResourcePlanRequest(const WMGetAllResourcePlanRequest& other1075) {
-  (void) other1075;
+WMGetAllResourcePlanRequest::WMGetAllResourcePlanRequest(const WMGetAllResourcePlanRequest& other1068) {
+  (void) other1068;
 }
-WMGetAllResourcePlanRequest& WMGetAllResourcePlanRequest::operator=(const WMGetAllResourcePlanRequest& other1076) {
-  (void) other1076;
+WMGetAllResourcePlanRequest& WMGetAllResourcePlanRequest::operator=(const WMGetAllResourcePlanRequest& other1069) {
+  (void) other1069;
   return *this;
 }
 void WMGetAllResourcePlanRequest::printTo(std::ostream& out) const {
@@ -27659,14 +27591,14 @@ uint32_t WMGetAllResourcePlanResponse::read(::apache::thrift::protocol::TProtoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->resourcePlans.clear();
-            uint32_t _size1077;
-            ::apache::thrift::protocol::TType _etype1080;
-            xfer += iprot->readListBegin(_etype1080, _size1077);
-            this->resourcePlans.resize(_size1077);
-            uint32_t _i1081;
-            for (_i1081 = 0; _i1081 < _size1077; ++_i1081)
+            uint32_t _size1070;
+            ::apache::thrift::protocol::TType _etype1073;
+            xfer += iprot->readListBegin(_etype1073, _size1070);
+            this->resourcePlans.resize(_size1070);
+            uint32_t _i1074;
+            for (_i1074 = 0; _i1074 < _size1070; ++_i1074)
             {
-              xfer += this->resourcePlans[_i1081].read(iprot);
+              xfer += this->resourcePlans[_i1074].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -27696,10 +27628,10 @@ uint32_t WMGetAllResourcePlanResponse::write(::apache::thrift::protocol::TProtoc
     xfer += oprot->writeFieldBegin("resourcePlans", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->resourcePlans.size()));
-      std::vector<WMResourcePlan> ::const_iterator _iter1082;
-      for (_iter1082 = this->resourcePlans.begin(); _iter1082 != this->resourcePlans.end(); ++_iter1082)
+      std::vector<WMResourcePlan> ::const_iterator _iter1075;
+      for (_iter1075 = this->resourcePlans.begin(); _iter1075 != this->resourcePlans.end(); ++_iter1075)
       {
-        xfer += (*_iter1082).write(oprot);
+        xfer += (*_iter1075).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -27716,13 +27648,13 @@ void swap(WMGetAllResourcePlanResponse &a, WMGetAllResourcePlanResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMGetAllResourcePlanResponse::WMGetAllResourcePlanResponse(const WMGetAllResourcePlanResponse& other1083) {
-  resourcePlans = other1083.resourcePlans;
-  __isset = other1083.__isset;
+WMGetAllResourcePlanResponse::WMGetAllResourcePlanResponse(const WMGetAllResourcePlanResponse& other1076) {
+  resourcePlans = other1076.resourcePlans;
+  __isset = other1076.__isset;
 }
-WMGetAllResourcePlanResponse& WMGetAllResourcePlanResponse::operator=(const WMGetAllResourcePlanResponse& other1084) {
-  resourcePlans = other1084.resourcePlans;
-  __isset = other1084.__isset;
+WMGetAllResourcePlanResponse& WMGetAllResourcePlanResponse::operator=(const WMGetAllResourcePlanResponse& other1077) {
+  resourcePlans = other1077.resourcePlans;
+  __isset = other1077.__isset;
   return *this;
 }
 void WMGetAllResourcePlanResponse::printTo(std::ostream& out) const {
@@ -27880,21 +27812,21 @@ void swap(WMAlterResourcePlanRequest &a, WMAlterResourcePlanRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMAlterResourcePlanRequest::WMAlterResourcePlanRequest(const WMAlterResourcePlanRequest& other1085) {
-  resourcePlanName = other1085.resourcePlanName;
-  resourcePlan = other1085.resourcePlan;
-  isEnableAndActivate = other1085.isEnableAndActivate;
-  isForceDeactivate = other1085.isForceDeactivate;
-  isReplace = other1085.isReplace;
-  __isset = other1085.__isset;
-}
-WMAlterResourcePlanRequest& WMAlterResourcePlanRequest::operator=(const WMAlterResourcePlanRequest& other1086) {
-  resourcePlanName = other1086.resourcePlanName;
-  resourcePlan = other1086.resourcePlan;
-  isEnableAndActivate = other1086.isEnableAndActivate;
-  isForceDeactivate = other1086.isForceDeactivate;
-  isReplace = other1086.isReplace;
-  __isset = other1086.__isset;
+WMAlterResourcePlanRequest::WMAlterResourcePlanRequest(const WMAlterResourcePlanRequest& other1078) {
+  resourcePlanName = other1078.resourcePlanName;
+  resourcePlan = other1078.resourcePlan;
+  isEnableAndActivate = other1078.isEnableAndActivate;
+  isForceDeactivate = other1078.isForceDeactivate;
+  isReplace = other1078.isReplace;
+  __isset = other1078.__isset;
+}
+WMAlterResourcePlanRequest& WMAlterResourcePlanRequest::operator=(const WMAlterResourcePlanRequest& other1079) {
+  resourcePlanName = other1079.resourcePlanName;
+  resourcePlan = other1079.resourcePlan;
+  isEnableAndActivate = other1079.isEnableAndActivate;
+  isForceDeactivate = other1079.isForceDeactivate;
+  isReplace = other1079.isReplace;
+  __isset = other1079.__isset;
   return *this;
 }
 void WMAlterResourcePlanRequest::printTo(std::ostream& out) const {
@@ -27980,13 +27912,13 @@ void swap(WMAlterResourcePlanResponse &a, WMAlterResourcePlanResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMAlterResourcePlanResponse::WMAlterResourcePlanResponse(const WMAlterResourcePlanResponse& other1087) {
-  fullResourcePlan = other1087.fullResourcePlan;
-  __isset = other1087.__isset;
+WMAlterResourcePlanResponse::WMAlterResourcePlanResponse(const WMAlterResourcePlanResponse& other1080) {
+  fullResourcePlan = other1080.fullResourcePlan;
+  __isset = other1080.__isset;
 }
-WMAlterResourcePlanResponse& WMAlterResourcePlanResponse::operator=(const WMAlterResourcePlanResponse& other1088) {
-  fullResourcePlan = other1088.fullResourcePlan;
-  __isset = other1088.__isset;
+WMAlterResourcePlanResponse& WMAlterResourcePlanResponse::operator=(const WMAlterResourcePlanResponse& other1081) {
+  fullResourcePlan = other1081.fullResourcePlan;
+  __isset = other1081.__isset;
   return *this;
 }
 void WMAlterResourcePlanResponse::printTo(std::ostream& out) const {
@@ -28068,13 +28000,13 @@ void swap(WMValidateResourcePlanRequest &a, WMValidateResourcePlanRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMValidateResourcePlanRequest::WMValidateResourcePlanRequest(const WMValidateResourcePlanRequest& other1089) {
-  resourcePlanName = other1089.resourcePlanName;
-  __isset = other1089.__isset;
+WMValidateResourcePlanRequest::WMValidateResourcePlanRequest(const WMValidateResourcePlanRequest& other1082) {
+  resourcePlanName = other1082.resourcePlanName;
+  __isset = other1082.__isset;
 }
-WMValidateResourcePlanRequest& WMValidateResourcePlanRequest::operator=(const WMValidateResourcePlanRequest& other1090) {
-  resourcePlanName = other1090.resourcePlanName;
-  __isset = other1090.__isset;
+WMValidateResourcePlanRequest& WMValidateResourcePlanRequest::operator=(const WMValidateResourcePlanRequest& other1083) {
+  resourcePlanName = other1083.resourcePlanName;
+  __isset = other1083.__isset;
   return *this;
 }
 void WMValidateResourcePlanRequest::printTo(std::ostream& out) const {
@@ -28124,14 +28056,14 @@ uint32_t WMValidateResourcePlanResponse::read(::apache::thrift::protocol::TProto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->errors.clear();
-            uint32_t _size1091;
-            ::apache::thrift::protocol::TType _etype1094;
-            xfer += iprot->readListBegin(_etype1094, _size1091);
-            this->errors.resize(_size1091);
-            uint32_t _i1095;
-            for (_i1095 = 0; _i1095 < _size1091; ++_i1095)
+            uint32_t _size1084;
+            ::apache::thrift::protocol::TType _etype1087;
+            xfer += iprot->readListBegin(_etype1087, _size1084);
+            this->errors.resize(_size1084);
+            uint32_t _i1088;
+            for (_i1088 = 0; _i1088 < _size1084; ++_i1088)
             {
-              xfer += iprot->readString(this->errors[_i1095]);
+              xfer += iprot->readString(this->errors[_i1088]);
             }
             xfer += iprot->readListEnd();
           }
@@ -28144,14 +28076,14 @@ uint32_t WMValidateResourcePlanResponse::read(::apache::thrift::protocol::TProto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->warnings.clear();
-            uint32_t _size1096;
-            ::apache::thrift::protocol::TType _etype1099;
-            xfer += iprot->readListBegin(_etype1099, _size1096);
-            this->warnings.resize(_size1096);
-            uint32_t _i1100;
-            for (_i1100 = 0; _i1100 < _size1096; ++_i1100)
+            uint32_t _size1089;
+            ::apache::thrift::protocol::TType _etype1092;
+            xfer += iprot->readListBegin(_etype1092, _size1089);
+            this->warnings.resize(_size1089);
+            uint32_t _i1093;
+            for (_i1093 = 0; _i1093 < _size1089; ++_i1093)
             {
-              xfer += iprot->readString(this->warnings[_i1100]);
+              xfer += iprot->readString(this->warnings[_i1093]);
             }
             xfer += iprot->readListEnd();
           }
@@ -28181,10 +28113,10 @@ uint32_t WMValidateResourcePlanResponse::write(::apache::thrift::protocol::TProt
     xfer += oprot->writeFieldBegin("errors", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->errors.size()));
-      std::vector<std::string> ::const_iterator _iter1101;
-      for (_iter1101 = this->errors.begin(); _iter1101 != this->errors.end(); ++_iter1101)
+      std::vector<std::string> ::const_iterator _iter1094;
+      for (_iter1094 = this->errors.begin(); _iter1094 != this->errors.end(); ++_iter1094)
       {
-        xfer += oprot->writeString((*_iter1101));
+        xfer += oprot->writeString((*_iter1094));
       }
       xfer += oprot->writeListEnd();
     }
@@ -28194,10 +28126,10 @@ uint32_t WMValidateResourcePlanResponse::write(::apache::thrift::protocol::TProt
     xfer += oprot->writeFieldBegin("warnings", ::apache::thrift::protocol::T_LIST, 2);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->warnings.size()));
-      std::vector<std::string> ::const_iterator _iter1102;
-      for (_iter1102 = this->warnings.begin(); _iter1102 != this->warnings.end(); ++_iter1102)
+      std::vector<std::string> ::const_iterator _iter1095;
+      for (_iter1095 = this->warnings.begin(); _iter1095 != this->warnings.end(); ++_iter1095)
       {
-        xfer += oprot->writeString((*_iter1102));
+        xfer += oprot->writeString((*_iter1095));
       }
       xfer += oprot->writeListEnd();
     }
@@ -28215,15 +28147,15 @@ void swap(WMValidateResourcePlanResponse &a, WMValidateResourcePlanResponse &b)
   swap(a.__isset, b.__isset);
 }
 
-WMValidateResourcePlanResponse::WMValidateResourcePlanResponse(const WMValidateResourcePlanResponse& other1103) {
-  errors = other1103.errors;
-  warnings = other1103.warnings;
-  __isset = other1103.__isset;
+WMValidateResourcePlanResponse::WMValidateResourcePlanResponse(const WMValidateResourcePlanResponse& other1096) {
+  errors = other1096.errors;
+  warnings = other1096.warnings;
+  __isset = other1096.__isset;
 }
-WMValidateResourcePlanResponse& WMValidateResourcePlanResponse::operator=(const WMValidateResourcePlanResponse& other1104) {
-  errors = other1104.errors;
-  warnings = other1104.warnings;
-  __isset = other1104.__isset;
+WMValidateResourcePlanResponse& WMValidateResourcePlanResponse::operator=(const WMValidateResourcePlanResponse& other1097) {
+  errors = other1097.errors;
+  warnings = other1097.warnings;
+  __isset = other1097.__isset;
   return *this;
 }
 void WMValidateResourcePlanResponse::printTo(std::ostream& out) const {
@@ -28306,13 +28238,13 @@ void swap(WMDropResourcePlanRequest &a, WMDropResourcePlanRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMDropResourcePlanRequest::WMDropResourcePlanRequest(const WMDropResourcePlanRequest& other1105) {
-  resourcePlanName = other1105.resourcePlanName;
-  __isset = other1105.__isset;
+WMDropResourcePlanRequest::WMDropResourcePlanRequest(const WMDropResourcePlanRequest& other1098) {
+  resourcePlanName = other1098.resourcePlanName;
+  __isset = other1098.__isset;
 }
-WMDropResourcePlanRequest& WMDropResourcePlanRequest::operator=(const WMDropResourcePlanRequest& other1106) {
-  resourcePlanName = other1106.resourcePlanName;
-  __isset = other1106.__isset;
+WMDropResourcePlanRequest& WMDropResourcePlanRequest::operator=(const WMDropResourcePlanRequest& other1099) {
+  resourcePlanName = other1099.resourcePlanName;
+  __isset = other1099.__isset;
   return *this;
 }
 void WMDropResourcePlanRequest::printTo(std::ostream& out) const {
@@ -28371,11 +28303,11 @@ void swap(WMDropResourcePlanResponse &a, WMDropResourcePlanResponse &b) {
   (void) b;
 }
 
-WMDropResourcePlanResponse::WMDropResourcePlanResponse(const WMDropResourcePlanResponse& other1107) {
-  (void) other1107;
+WMDropResourcePlanResponse::WMDropResourcePlanResponse(const WMDropResourcePlanResponse& other1100) {
+  (void) other1100;
 }
-WMDropResourcePlanResponse& WMDropResourcePlanResponse::operator=(const WMDropResourcePlanResponse& other1108) {
-  (void) other1108;
+WMDropResourcePlanResponse& WMDropResourcePlanResponse::operator=(const WMDropResourcePlanResponse& other1101) {
+  (void) other1101;
   return *this;
 }
 void WMDropResourcePlanResponse::printTo(std::ostream& out) const {
@@ -28456,13 +28388,13 @@ void swap(WMCreateTriggerRequest &a, WMCreateTriggerRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMCreateTriggerRequest::WMCreateTriggerRequest(const WMCreateTriggerRequest& other1109) {
-  trigger = other1109.trigger;
-  __isset = other1109.__isset;
+WMCreateTriggerRequest::WMCreateTriggerRequest(const WMCreateTriggerRequest& other1102) {
+  trigger = other1102.trigger;
+  __isset = other1102.__isset;
 }
-WMCreateTriggerRequest& WMCreateTriggerRequest::operator=(const WMCreateTriggerRequest& other1110) {
-  trigger = other1110.trigger;
-  __isset = other1110.__isset;
+WMCreateTriggerRequest& WMCreateTriggerRequest::operator=(const WMCreateTriggerRequest& other1103) {
+  trigger = other1103.trigger;
+  __isset = other1103.__isset;
   return *this;
 }
 void WMCreateTriggerRequest::printTo(std::ostream& out) const {
@@ -28521,11 +28453,11 @@ void swap(WMCreateTriggerResponse &a, WMCreateTriggerResponse &b) {
   (void) b;
 }
 
-WMCreateTriggerResponse::WMCreateTriggerResponse(const WMCreateTriggerResponse& other1111) {
-  (void) other1111;
+WMCreateTriggerResponse::WMCreateTriggerResponse(const WMCreateTriggerResponse& other1104) {
+  (void) other1104;
 }
-WMCreateTriggerResponse& WMCreateTriggerResponse::operator=(const WMCreateTriggerResponse& other1112) {
-  (void) other1112;
+WMCreateTriggerResponse& WMCreateTriggerResponse::operator=(const WMCreateTriggerResponse& other1105) {
+  (void) other1105;
   return *this;
 }
 void WMCreateTriggerResponse::printTo(std::ostream& out) const {
@@ -28606,13 +28538,13 @@ void swap(WMAlterTriggerRequest &a, WMAlterTriggerRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMAlterTriggerRequest::WMAlterTriggerRequest(const WMAlterTriggerRequest& other1113) {
-  trigger = other1113.trigger;
-  __isset = other1113.__isset;
+WMAlterTriggerRequest::WMAlterTriggerRequest(const WMAlterTriggerRequest& other1106) {
+  trigger = other1106.trigger;
+  __isset = other1106.__isset;
 }
-WMAlterTriggerRequest& WMAlterTriggerRequest::operator=(const WMAlterTriggerRequest& other1114) {
-  trigger = other1114.trigger;
-  __isset = other1114.__isset;
+WMAlterTriggerRequest& WMAlterTriggerRequest::operator=(const WMAlterTriggerRequest& other1107) {
+  trigger = other1107.trigger;
+  __isset = other1107.__isset;
   return *this;
 }
 void WMAlterTriggerRequest::printTo(std::ostream& out) const {
@@ -28671,11 +28603,11 @@ void swap(WMAlterTriggerResponse &a, WMAlterTriggerResponse &b) {
   (void) b;
 }
 
-WMAlterTriggerResponse::WMAlterTriggerResponse(const WMAlterTriggerResponse& other1115) {
-  (void) other1115;
+WMAlterTriggerResponse::WMAlterTriggerResponse(const WMAlterTriggerResponse& other1108) {
+  (void) other1108;
 }
-WMAlterTriggerResponse& WMAlterTriggerResponse::operator=(const WMAlterTriggerResponse& other1116) {
-  (void) other1116;
+WMAlterTriggerResponse& WMAlterTriggerResponse::operator=(const WMAlterTriggerResponse& other1109) {
+  (void) other1109;
   return *this;
 }
 void WMAlterTriggerResponse::printTo(std::ostream& out) const {
@@ -28775,15 +28707,15 @@ void swap(WMDropTriggerRequest &a, WMDropTriggerRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMDropTriggerRequest::WMDropTriggerRequest(const WMDropTriggerRequest& other1117) {
-  resourcePlanName = other1117.resourcePlanName;
-  triggerName = other1117.triggerName;
-  __isset = other1117.__isset;
+WMDropTriggerRequest::WMDropTriggerRequest(const WMDropTriggerRequest& other1110) {
+  resourcePlanName = other1110.resourcePlanName;
+  triggerName = other1110.triggerName;
+  __isset = other1110.__isset;
 }
-WMDropTriggerRequest& WMDropTriggerRequest::operator=(const WMDropTriggerRequest& other1118) {
-  resourcePlanName = other1118.resourcePlanName;
-  triggerName = other1118.triggerName;
-  __isset = other1118.__isset;
+WMDropTriggerRequest& WMDropTriggerRequest::operator=(const WMDropTriggerRequest& other1111) {
+  resourcePlanName = other1111.resourcePlanName;
+  triggerName = other1111.triggerName;
+  __isset = other1111.__isset;
   return *this;
 }
 void WMDropTriggerRequest::printTo(std::ostream& out) const {
@@ -28843,11 +28775,11 @@ void swap(WMDropTriggerResponse &a, WMDropTriggerResponse &b) {
   (void) b;
 }
 
-WMDropTriggerResponse::WMDropTriggerResponse(const WMDropTriggerResponse& other1119) {
-  (void) other1119;
+WMDropTriggerResponse::WMDropTriggerResponse(const WMDropTriggerResponse& other1112) {
+  (void) other1112;
 }
-WMDropTriggerResponse& WMDropTriggerResponse::operator=(const WMDropTriggerResponse& other1120) {
-  (void) other1120;
+WMDropTriggerResponse& WMDropTriggerResponse::operator=(const WMDropTriggerResponse& other1113) {
+  (void) other1113;
   return *this;
 }
 void WMDropTriggerResponse::printTo(std::ostream& out) const {
@@ -28928,13 +28860,13 @@ void swap(WMGetTriggersForResourePlanRequest &a, WMGetTriggersForResourePlanRequ
   swap(a.__isset, b.__isset);
 }
 
-WMGetTriggersForResourePlanRequest::WMGetTriggersForResourePlanRequest(const WMGetTriggersForResourePlanRequest& other1121) {
-  resourcePlanName = other1121.resourcePlanName;
-  __isset = other1121.__isset;
+WMGetTriggersForResourePlanRequest::WMGetTriggersForResourePlanRequest(const WMGetTriggersForResourePlanRequest& other1114) {
+  resourcePlanName = other1114.resourcePlanName;
+  __isset = other1114.__isset;
 }
-WMGetTriggersForResourePlanRequest& WMGetTriggersForResourePlanRequest::operator=(const WMGetTriggersForResourePlanRequest& other1122) {
-  resourcePlanName = other1122.resourcePlanName;
-  __isset = other1122.__isset;
+WMGetTriggersForResourePlanRequest& WMGetTriggersForResourePlanRequest::operator=(const WMGetTriggersForResourePlanRequest& other1115) {
+  resourcePlanName = other1115.resourcePlanName;
+  __isset = other1115.__isset;
   return *this;
 }
 void WMGetTriggersForResourePlanRequest::printTo(std::ostream& out) const {
@@ -28979,14 +28911,14 @@ uint32_t WMGetTriggersForResourePlanResponse::read(::apache::thrift::protocol::T
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->triggers.clear();
-            uint32_t _size1123;
-            ::apache::thrift::protocol::TType _etype1126;
-            xfer += iprot->readListBegin(_etype1126, _size1123);
-            this->triggers.resize(_size1123);
-            uint32_t _i1127;
-            for (_i1127 = 0; _i1127 < _size1123; ++_i1127)
+            uint32_t _size1116;
+            ::apache::thrift::protocol::TType _etype1119;
+            xfer += iprot->readListBegin(_etype1119, _size1116);
+            this->triggers.resize(_size1116);
+            uint32_t _i1120;
+            for (_i1120 = 0; _i1120 < _size1116; ++_i1120)
             {
-              xfer += this->triggers[_i1127].read(iprot);
+              xfer += this->triggers[_i1120].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -29016,10 +28948,10 @@ uint32_t WMGetTriggersForResourePlanResponse::write(::apache::thrift::protocol::
     xfer += oprot->writeFieldBegin("triggers", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->triggers.size()));
-      std::vector<WMTrigger> ::const_iterator _iter1128;
-      for (_iter1128 = this->triggers.begin(); _iter1128 != this->triggers.end(); ++_iter1128)
+      std::vector<WMTrigger> ::const_iterator _iter1121;
+      for (_iter1121 = this->triggers.begin(); _iter1121 != this->triggers.end(); ++_iter1121)
       {
-        xfer += (*_iter1128).write(oprot);
+        xfer += (*_iter1121).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -29036,13 +28968,13 @@ void swap(WMGetTriggersForResourePlanResponse &a, WMGetTriggersForResourePlanRes
   swap(a.__isset, b.__isset);
 }
 
-WMGetTriggersForResourePlanResponse::WMGetTriggersForResourePlanResponse(const WMGetTriggersForResourePlanResponse& other1129) {
-  triggers = other1129.triggers;
-  __isset = other1129.__isset;
+WMGetTriggersForResourePlanResponse::WMGetTriggersForResourePlanResponse(const WMGetTriggersForResourePlanResponse& other1122) {
+  triggers = other1122.triggers;
+  __isset = other1122.__isset;
 }
-WMGetTriggersForResourePlanResponse& WMGetTriggersForResourePlanResponse::operator=(const WMGetTriggersForResourePlanResponse& other1130) {
-  triggers = other1130.triggers;
-  __isset = other1130.__isset;
+WMGetTriggersForResourePlanResponse& WMGetTriggersForResourePlanResponse::operator=(const WMGetTriggersForResourePlanResponse& other1123) {
+  triggers = other1123.triggers;
+  __isset = other1123.__isset;
   return *this;
 }
 void WMGetTriggersForResourePlanResponse::printTo(std::ostream& out) const {
@@ -29124,13 +29056,13 @@ void swap(WMCreatePoolRequest &a, WMCreatePoolRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMCreatePoolRequest::WMCreatePoolRequest(const WMCreatePoolRequest& other1131) {
-  pool = other1131.pool;
-  __isset = other1131.__isset;
+WMCreatePoolRequest::WMCreatePoolRequest(const WMCreatePoolRequest& other1124) {
+  pool = other1124.pool;
+  __isset = other1124.__isset;
 }
-WMCreatePoolRequest& WMCreatePoolRequest::operator=(const WMCreatePoolRequest& other1132) {
-  pool = other1132.pool;
-  __isset = other1132.__isset;
+WMCreatePoolRequest& WMCreatePoolRequest::operator=(const WMCreatePoolRequest& other1125) {
+  pool = other1125.pool;
+  __isset = other1125.__isset;
   return *this;
 }
 void WMCreatePoolRequest::printTo(std::ostream& out) const {
@@ -29189,11 +29121,11 @@ void swap(WMCreatePoolResponse &a, WMCreatePoolResponse &b) {
   (void) b;
 }
 
-WMCreatePoolResponse::WMCreatePoolResponse(const WMCreatePoolResponse& other1133) {
-  (void) other1133;
+WMCreatePoolResponse::WMCreatePoolResponse(const WMCreatePoolResponse& other1126) {
+  (void) other1126;
 }
-WMCreatePoolResponse& WMCreatePoolResponse::operator=(const WMCreatePoolResponse& other1134) {
-  (void) other1134;
+WMCreatePoolResponse& WMCreatePoolResponse::operator=(const WMCreatePoolResponse& other1127) {
+  (void) other1127;
   return *this;
 }
 void WMCreatePoolResponse::printTo(std::ostream& out) const {
@@ -29293,15 +29225,15 @@ void swap(WMAlterPoolRequest &a, WMAlterPoolRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMAlterPoolRequest::WMAlterPoolRequest(const WMAlterPoolRequest& other1135) {
-  pool = other1135.pool;
-  poolPath = other1135.poolPath;
-  __isset = other1135.__isset;
+WMAlterPoolRequest::WMAlterPoolRequest(const WMAlterPoolRequest& other1128) {
+  pool = other1128.pool;
+  poolPath = other1128.poolPath;
+  __isset = other1128.__isset;
 }
-WMAlterPoolRequest& WMAlterPoolRequest::operator=(const WMAlterPoolRequest& other1136) {
-  pool = other1136.pool;
-  poolPath = other1136.poolPath;
-  __isset = other1136.__isset;
+WMAlterPoolRequest& WMAlterPoolRequest::operator=(const WMAlterPoolRequest& other1129) {
+  pool = other1129.pool;
+  poolPath = other1129.poolPath;
+  __isset = other1129.__isset;
   return *this;
 }
 void WMAlterPoolRequest::printTo(std::ostream& out) const {
@@ -29361,11 +29293,11 @@ void swap(WMAlterPoolResponse &a, WMAlterPoolResponse &b) {
   (void) b;
 }
 
-WMAlterPoolResponse::WMAlterPoolResponse(const WMAlterPoolResponse& other1137) {
-  (void) other1137;
+WMAlterPoolResponse::WMAlterPoolResponse(const WMAlterPoolResponse& other1130) {
+  (void) other1130;
 }
-WMAlterPoolResponse& WMAlterPoolResponse::operator=(const WMAlterPoolResponse& other1138) {
-  (void) other1138;
+WMAlterPoolResponse& WMAlterPoolResponse::operator=(const WMAlterPoolResponse& other1131) {
+  (void) other1131;
   return *this;
 }
 void WMAlterPoolResponse::printTo(std::ostream& out) const {
@@ -29465,15 +29397,15 @@ void swap(WMDropPoolRequest &a, WMDropPoolRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMDropPoolRequest::WMDropPoolRequest(const WMDropPoolRequest& other1139) {
-  resourcePlanName = other1139.resourcePlanName;
-  poolPath = other1139.poolPath;
-  __isset = other1139.__isset;
+WMDropPoolRequest::WMDropPoolRequest(const WMDropPoolRequest& other1132) {
+  resourcePlanName = other1132.resourcePlanName;
+  poolPath = other1132.poolPath;
+  __isset = other1132.__isset;
 }
-WMDropPoolRequest& WMDropPoolRequest::operator=(const WMDropPoolRequest& other1140) {
-  resourcePlanName = other1140.resourcePlanName;
-  poolPath = other1140.poolPath;
-  __isset = other1140.__isset;
+WMDropPoolRequest& WMDropPoolRequest::operator=(const WMDropPoolRequest& other1133) {
+  resourcePlanName = other1133.resourcePlanName;
+  poolPath = other1133.poolPath;
+  __isset = other1133.__isset;
   return *this;
 }
 void WMDropPoolRequest::printTo(std::ostream& out) const {
@@ -29533,11 +29465,11 @@ void swap(WMDropPoolResponse &a, WMDropPoolResponse &b) {
   (void) b;
 }
 
-WMDropPoolResponse::WMDropPoolResponse(const WMDropPoolResponse& other1141) {
-  (void) other1141;
+WMDropPoolResponse::WMDropPoolResponse(const WMDropPoolResponse& other1134) {
+  (void) other1134;
 }
-WMDropPoolResponse& WMDropPoolResponse::operator=(const WMDropPoolResponse& other1142) {
-  (void) other1142;
+WMDropPoolResponse& WMDropPoolResponse::operator=(const WMDropPoolResponse& other1135) {
+  (void) other1135;
   return *this;
 }
 void WMDropPoolResponse::printTo(std::ostream& out) const {
@@ -29637,15 +29569,15 @@ void swap(WMCreateOrUpdateMappingRequest &a, WMCreateOrUpdateMappingRequest &b)
   swap(a.__isset, b.__isset);
 }
 
-WMCreateOrUpdateMappingRequest::WMCreateOrUpdateMappingRequest(const WMCreateOrUpdateMappingRequest& other1143) {
-  mapping = other1143.mapping;
-  update = other1143.update;
-  __isset = other1143.__isset;
+WMCreateOrUpdateMappingRequest::WMCreateOrUpdateMappingRequest(const WMCreateOrUpdateMappingRequest& other1136) {
+  mapping = other1136.mapping;
+  update = other1136.update;
+  __isset = other1136.__isset;
 }
-WMCreateOrUpdateMappingRequest& WMCreateOrUpdateMappingRequest::operator=(const WMCreateOrUpdateMappingRequest& other1144) {
-  mapping = other1144.mapping;
-  update = other1144.update;
-  __isset = other1144.__isset;
+WMCreateOrUpdateMappingRequest& WMCreateOrUpdateMappingRequest::operator=(const WMCreateOrUpdateMappingRequest& other1137) {
+  mapping = other1137.mapping;
+  update = other1137.update;
+  __isset = other1137.__isset;
   return *this;
 }
 void WMCreateOrUpdateMappingRequest::printTo(std::ostream& out) const {
@@ -29705,11 +29637,11 @@ void swap(WMCreateOrUpdateMappingResponse &a, WMCreateOrUpdateMappingResponse &b
   (void) b;
 }
 
-WMCreateOrUpdateMappingResponse::WMCreateOrUpdateMappingResponse(const WMCreateOrUpdateMappingResponse& other1145) {
-  (void) other1145;
+WMCreateOrUpdateMappingResponse::WMCreateOrUpdateMappingResponse(const WMCreateOrUpdateMappingResponse& other1138) {
+  (void) other1138;
 }
-WMCreateOrUpdateMappingResponse& WMCreateOrUpdateMappingResponse::operator=(const WMCreateOrUpdateMappingResponse& other1146) {
-  (void) other1146;
+WMCreateOrUpdateMappingResponse& WMCreateOrUpdateMappingResponse::operator=(const WMCreateOrUpdateMappingResponse& other1139) {
+  (void) other1139;
   return *this;
 }
 void WMCreateOrUpdateMappingResponse::printTo(std::ostream& out) const {
@@ -29790,13 +29722,13 @@ void swap(WMDropMappingRequest &a, WMDropMappingRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMDropMappingRequest::WMDropMappingRequest(const WMDropMappingRequest& other1147) {
-  mapping = other1147.mapping;
-  __isset = other1147.__isset;
+WMDropMappingRequest::WMDropMappingRequest(const WMDropMappingRequest& other1140) {
+  mapping = other1140.mapping;
+  __isset = other1140.__isset;
 }
-WMDropMappingRequest& WMDropMappingRequest::operator=(const WMDropMappingRequest& other1148) {
-  mapping = other1148.mapping;
-  __isset = other1148.__isset;
+WMDropMappingRequest& WMDropMappingRequest::operator=(const WMDropMappingRequest& other1141) {
+  mapping = other1141.mapping;
+  __isset = other1141.__isset;
   return *this;
 }
 void WMDropMappingRequest::printTo(std::ostream& out) const {
@@ -29855,11 +29787,11 @@ void swap(WMDropMappingResponse &a, WMDropMappingResponse &b) {
   (void) b;
 }
 
-WMDropMappingResponse::WMDropMappingResponse(const WMDropMappingResponse& other1149) {
-  (void) other1149;
+WMDropMappingResponse::WMDropMappingResponse(const WMDropMappingResponse& other1142) {
+  (void) other1142;
 }
-WMDropMappingResponse& WMDropMappingResponse::operator=(const WMDropMappingResponse& other1150) {
-  (void) other1150;
+WMDropMappingResponse& WMDropMappingResponse::operator=(const WMDropMappingResponse& other1143) {
+  (void) other1143;
   return *this;
 }
 void WMDropMappingResponse::printTo(std::ostream& out) const {
@@ -29997,19 +29929,19 @@ void swap(WMCreateOrDropTriggerToPoolMappingRequest &a, WMCreateOrDropTriggerToP
   swap(a.__isset, b.__isset);
 }
 
-WMCreateOrDropTriggerToPoolMappingRequest::WMCreateOrDropTriggerToPoolMappingRequest(const WMCreateOrDropTriggerToPoolMappingRequest& other1151) {
-  resourcePlanName = other1151.resourcePlanName;
-  triggerName = other1151.triggerName;
-  poolPath = other1151.poolPath;
-  drop = other1151.drop;
-  __isset = other1151.__isset;
+WMCreateOrDropTriggerToPoolMappingRequest::WMCreateOrDropTriggerToPoolMappingRequest(const WMCreateOrDropTriggerToPoolMappingRequest& other1144) {
+  resourcePlanName = other1144.resourcePlanName;
+  triggerName = other1144.triggerName;
+  poolPath = other1144.poolPath;
+  drop = other1144.drop;
+  __isset = other1144.__isset;
 }
-WMCreateOrDropTriggerToPoolMappingRequest& WMCreateOrDropTriggerToPoolMappingRequest::operator=(const WMCreateOrDropTriggerToPoolMappingRequest& other1152) {
-  resourcePlanName = other1152.resourcePlanName;
-  triggerName = other1152.triggerName;
-  poolPath = other1152.poolPath;
-  drop = other1152.drop;
-  __isset = other1152.__isset;
+WMCreateOrDropTriggerToPoolMappingRequest& WMCreateOrDropTriggerToPoolMappingRequest::operator=(const WMCreateOrDropTriggerToPoolMappingRequest& other1145) {
+  resourcePlanName = other1145.resourcePlanName;
+  triggerName = other1145.triggerName;
+  poolPath = other1145.poolPath;
+  drop = other1145.drop;
+  __isset = other1145.__isset;
   return *this;
 }
 void WMCreateOrDropTriggerToPoolMappingRequest::printTo(std::ostream& out) const {
@@ -30071,11 +30003,11 @@ void swap(WMCreateOrDropTriggerToPoolMappingResponse &a, WMCreateOrDropTriggerTo
   (void) b;
 }
 
-WMCreateOrDropTriggerToPoolMappingResponse::WMCreateOrDropTriggerToPoolMappingResponse(const WMCreateOrDropTriggerToPoolMappingResponse& other1153) {
-  (void) other1153;
+WMCreateOrDropTriggerToPoolMappingResponse::WMCreateOrDropTriggerToPoolMappingResponse(const WMCreateOrDropTriggerToPoolMappingResponse& other1146) {
+  (void) other1146;
 }
-WMCreateOrDropTriggerToPoolMappingResponse& WMCreateOrDropTriggerToPoolMappingResponse::operator=(const WMCreateOrDropTriggerToPoolMappingResponse& other1154) {
-  (void) other1154;
+WMCreateOrDropTriggerToPoolMappingResponse& WMCreateOrDropTriggerToPoolMappingResponse::operator=(const WMCreateOrDropTriggerToPoolMappingResponse& other1147) {
+  (void) other1147;
   return *this;
 }
 void WMCreateOrDropTriggerToPoolMappingResponse::printTo(std::ostream& out) const {
@@ -30150,9 +30082,9 @@ uint32_t ISchema::read(::apache::thrift::protocol::TProtocol* iprot) {
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast1155;
-          xfer += iprot->readI32(ecast1155);
-          this->schemaType = (SchemaType::type)ecast1155;
+          int32_t ecast1148;
+          xfer += iprot->readI32(ecast1148);
+          this->schemaType = (SchemaType::type)ecast1148;
           this->__isset.schemaType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -30184,9 +30116,9 @@ uint32_t ISchema::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast1156;
-          xfer += iprot->readI32(ecast1156);
-          this->compatibility = (SchemaCompatibility::type)ecast1156;
+          int32_t ecast1149;
+          xfer += iprot->readI32(ecast1149);
+          this->compatibility = (SchemaCompatibility::type)ecast1149;
           this->__isset.compatibility = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -30194,9 +30126,9 @@ uint32_t ISchema::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast1157;
-          xfer += iprot->readI32(ecast1157);
-          this->validationLevel = (SchemaValidation::type)ecast1157;
+          int32_t ecast1150;
+          xfer += iprot->readI32(ecast1150);
+          this->validationLevel = (SchemaValidation::type)ecast1150;
           this->__isset.validationLevel = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -30300,29 +30232,29 @@ void swap(ISchema &a, ISchema &b) {
   swap(a.__isset, b.__isset);
 }
 
-ISchema::ISchema(const ISchema& other1158) {
-  schemaType = other1158.schemaType;
-  name = other1158.name;
-  catName = other1158.catName;
-  dbName = other1158.dbName;
-  compatibility = other1158.compatibility;
-  validationLevel = other1158.validationLevel;
-  canEvolve = other1158.canEvolve;
-  schemaGroup = other1158.schemaGroup;
-  description = other1158.description;
-  __isset = other1158.__isset;
-}
-ISchema& ISchema::operator=(const ISchema& other1159) {
-  schemaType = other1159.schemaType;
-  name = other1159.name;
-  catName = other1159.catName;
-  dbName = other1159.dbName;
-  compatibility = other1159.compatibility;
-  validationLevel = other1159.validationLevel;
-  canEvolve = other1159.canEvolve;
-  schemaGroup = other1159.schemaGroup;
-  description = other1159.description;
-  __isset = other1159.__isset;
+ISchema::ISchema(const ISchema& other1151) {
+  schemaType = other1151.schemaType;
+  name = other1151.name;
+  catName = other1151.catName;
+  dbName = other1151.dbName;
+  compatibility = other1151.compatibility;
+  validationLevel = other1151.validationLevel;
+  canEvolve = other1151.canEvolve;
+  schemaGroup = other1151.schemaGroup;
+  description = other1151.description;
+  __isset = other1151.__isset;
+}
+ISchema& ISchema::operator=(const ISchema& other1152) {
+  schemaType = other1152.schemaType;
+  name = other1152.name;
+  catName = other1152.catName;
+  dbName = other1152.dbName;
+  compatibility = other1152.compatibility;
+  validationLevel = other1152.validationLevel;
+  canEvolve = other1152.canEvolve;
+  schemaGroup = other1152.schemaGroup;
+  description = other1152.description;
+  __isset = other1152.__isset;
   return *this;
 }
 void ISchema::printTo(std::ostream& out) const {
@@ -30444,17 +30376,17 @@ void swap(ISchemaName &a, ISchemaName &b) {
   swap(a.__isset, b.__isset);
 }
 
-ISchemaName::ISchemaName(const ISchemaName& other1160) {
-  catName = other1160.catName;
-  dbName = other1160.dbName;
-  schemaName = other1160.schemaName;
-  __isset = other1160.__isset;
+ISchemaName::ISchemaName(const ISchemaName& other1153) {
+  catName = other1153.catName;
+  dbName = other1153.dbName;
+  schemaName = other1153.schemaName;
+  __isset = other1153.__isset;
 }
-ISchemaName& ISchemaName::operator=(const ISchemaName& other1161) {
-  catName = other1161.catName;
-  dbName = other1161.dbName;
-  schemaName = other1161.schemaName;
-  __isset = other1161.__isset;
+ISchemaName& ISchemaName::operator=(const ISchemaName& other1154) {
+  catName = other1154.catName;
+  dbName = other1154.dbName;
+  schemaName = other1154.schemaName;
+  __isset = other1154.__isset;
   return *this;
 }
 void ISchemaName::printTo(std::ostream& out) const {
@@ -30553,15 +30485,15 @@ void swap(AlterISchemaRequest &a, AlterISchemaRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlterISchemaRequest::AlterISchemaRequest(const AlterISchemaRequest& other1162) {
-  name = other1162.name;
-  newSchema = other1162.newSchema;
-  __isset = other1162.__isset;
+AlterISchemaRequest::AlterISchemaRequest(const AlterISchemaRequest& other1155) {
+  name = other1155.name;
+  newSchema = other1155.newSchema;
+  __isset = other1155.__isset;
 }
-AlterISchemaRequest& AlterISchemaRequest::operator=(const AlterISchemaRequest& other1163) {
-  name = other1163.name;
-  newSchema = other1163.newSchema;
-  __isset = other1163.__isset;
+AlterISchemaRequest& AlterISchemaRequest::operator=(const AlterISchemaRequest& other1156) {
+  name = other1156.name;
+  newSchema = other1156.newSchema;
+  __isset = other1156.__isset;
   return *this;
 }
 void AlterISchemaRequest::printTo(std::ostream& out) const {
@@ -30672,14 +30604,14 @@ uint32_t SchemaVersion::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->cols.clear();
-            uint32_t _size1164;
-            ::apache::thrift::protocol::TType _etype1167;
-            xfer += iprot->readListBegin(_etype1167, _size1164);
-            this->cols.resize(_size1164);
-            uint32_t _i1168;
-            for (_i1168 = 0; _i1168 < _size1164; ++_i1168)
+            uint32_t _size1157;
+            ::apache::thrift::protocol::TType _etype1160;
+            xfer += iprot->readListBegin(_etype1160, _size1157);
+            this->cols.resize(_size1157);
+            uint32_t _i1161;
+            for (_i1161 = 0; _i1161 < _size1157; ++_i1161)
             {
-              xfer += this->cols[_i1168].read(iprot);
+              xfer += this->cols[_i1161].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -30690,9 +30622,9 @@ uint32_t SchemaVersion::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast1169;
-          xfer += iprot->readI32(ecast1169);
-          this->state = (SchemaVersionState::type)ecast1169;
+          int32_t ecast1162;
+          xfer += iprot->readI32(ecast1162);
+          this->state = (SchemaVersionState::type)ecast1162;
           this->__isset.state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -30770,10 +30702,10 @@ uint32_t SchemaVersion::write(::apache::thrift::protocol::TProtocol* oprot) cons
   xfer += oprot->writeFieldBegin("cols", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->cols.size()));
-    std::vector<FieldSchema> ::const_iterator _iter1170;
-    for (_iter1170 = this->cols.begin(); _iter1170 != this->cols.end(); ++_iter1170)
+    std::vector<FieldSchema> ::const_iterator _iter1163;
+    for (_iter1163 = this->cols.begin(); _iter1163 != this->cols.end(); ++_iter1163)
     {
-      xfer += (*_iter1170).write(oprot);
+      xfer += (*_iter1163).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -30829,31 +30761,31 @@ void swap(SchemaVersion &a, SchemaVersion &b) {
   swap(a.__isset, b.__isset);
 }
 
-SchemaVersion::SchemaVersion(const SchemaVersion& other1171) {
-  schema = other1171.schema;
-  version = other1171.version;
-  createdAt = other1171.createdAt;
-  cols = other1171.cols;
-  state = other1171.state;
-  description = other1171.description;
-  schemaText = other1171.schemaText;
-  fingerprint = other1171.fingerprint;
-  name = other1171.name;
-  serDe = other1171.serDe;
-  __isset = other1171.__isset;
-}
-SchemaVersion& SchemaVersion::operator=(const SchemaVersion& other1172) {
-  schema = other1172.schema;
-  version = other1172.version;
-  createdAt = other1172.createdAt;
-  cols = other1172.cols;
-  state = other1172.state;
-  description = other1172.description;
-  schemaText = other1172.schemaText;
-  fingerprint = other1172.fingerprint;
-  name = other1172.name;
-  serDe = other1172.serDe;
-  __isset = other1172.__isset;
+SchemaVersion::SchemaVersion(const SchemaVersion& other1164) {
+  schema = other1164.schema;
+  version = other1164.version;
+  createdAt = other1164.createdAt;
+  cols = other1164.cols;
+  state = other1164.state;
+  description = other1164.description;
+  schemaText = other1164.schemaText;
+  fingerprint = other1164.fingerprint;
+  name = other1164.name;
+  serDe = other1164.serDe;
+  __isset = other1164.__isset;
+}
+SchemaVersion& SchemaVersion::operator=(const SchemaVersion& other1165) {
+  schema = other1165.schema;
+  version = other1165.version;
+  createdAt = other1165.createdAt;
+  cols = other1165.cols;
+  state = other1165.state;
+  description = other1165.description;
+  schemaText = other1165.schemaText;
+  fingerprint = other1165.fingerprint;
+  name = other1165.name;
+  serDe = other1165.serDe;
+  __isset = other1165.__isset;
   return *this;
 }
 void SchemaVersion::printTo(std::ostream& out) const {
@@ -30959,15 +30891,15 @@ void swap(SchemaVersionDescriptor &a, SchemaVersionDescriptor &b) {
   swap(a.__isset, b.__isset);
 }
 
-SchemaVersionDescriptor::SchemaVersionDescriptor(const SchemaVersionDescriptor& other1173) {
-  schema = other1173.schema;
-  version = other1173.version;
-  __isset = other1173.__isset;
+SchemaVersionDescriptor::SchemaVersionDescriptor(const SchemaVersionDescriptor& other1166) {
+  schema = other1166.schema;
+  version = other1166.version;
+  __isset = other1166.__isset;
 }
-SchemaVersionDescriptor& SchemaVersionDescriptor::operator=(const SchemaVersionDescriptor& other1174) {
-  schema = other1174.schema;
-  version = other1174.version;
-  __isset = other1174.__isset;
+SchemaVersionDescriptor& SchemaVersionDescriptor::operator=(const SchemaVersionDescriptor& other1167) {
+  schema = other1167.schema;
+  version = other1167.version;
+  __isset = other1167.__isset;
   return *this;
 }
 void SchemaVersionDescriptor::printTo(std::ostream& out) const {
@@ -31088,17 +31020,17 @@ void swap(FindSchemasByColsRqst &a, FindSchemasByColsRqst &b) {
   swap(a.__isset, b.__isset);
 }
 
-FindSchemasByColsRqst::FindSchemasByColsRqst(const FindSchemasByColsRqst& other1175) {
-  colName = other1175.colName;
-  colNamespace = other1175.colNamespace;
-  type = other1175.type;
-  __isset = other1175.__isset;
+FindSchemasByColsRqst::FindSchemasByColsRqst(const FindSchemasByColsRqst& other1168) {
+  colName = other1168.colName;
+  colNamespace = other1168.colNamespace;
+  type = other1168.type;
+  __isset = other1168.__isset;
 }
-FindSchemasByColsRqst& FindSchemasByColsRqst::operator=(const FindSchemasByColsRqst& other1176) {
-  colName = other1176.colName;
-  colNamespace = other1176.colNamespace;
-  type = other1176.type;
-  __isset = other1176.__isset;
+FindSchemasByColsRqst& FindSchemasByColsRqst::operator=(const FindSchemasByColsRqst& other1169) {
+  colName = other1169.colName;
+  colNamespace = other1169.colNamespace;
+  type = other1169.type;
+  __isset = other1169.__isset;
   return *this;
 }
 void FindSchemasByColsRqst::printTo(std::ostream& out) const {
@@ -31144,14 +31076,14 @@ uint32_t FindSchemasByColsResp::read(::apache::thrift::protocol::TProtocol* ipro
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->schemaVersions.clear();
-            uint32_t _size1177;
-            ::apache::thrift::protocol::TType _etype1180;
-            xfer += iprot->readListBegin(_etype1180, _size1177);
-            this->schemaVersions.resize(_size1177);
-            uint32_t _i1181;
-            for (_i1181 = 0; _i1181 < _size1177; ++_i1181)
+            uint32_t _size1170;
+            ::apache::thrift::protocol::TType _etype1173;
+            xfer += iprot->readListBegin(_etype1173, _size1170);
+            this->schemaVersions.resize(_size1170);
+            uint32_t _i1174;
+            for (_i1174 = 0; _i1174 < _size1170; ++_i1174)
             {
-              xfer += this->schemaVersions[_i1181].read(iprot);
+              xfer += this->schemaVersions[_i1174].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -31180,10 +31112,10 @@ uint32_t FindSchemasByColsResp::write(::apache::thrift::protocol::TProtocol* opr
   xfer += oprot->writeFieldBegin("schemaVersions", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->schemaVersions.size()));
-    std::vector<SchemaVersionDescriptor> ::const_iterator _iter1182;
-    for (_iter1182 = this->schemaVersions.begin(); _iter1182 != this->schemaVersions.end(); ++_iter1182)
+    std::vector<SchemaVersionDescriptor> ::const_iterator _iter1175;
+    for (_iter1175 = this->schemaVersions.begin(); _iter1175 != this->schemaVersions.end(); ++_iter1175)
     {
-      xfer += (*_iter1182).write(oprot);
+      xfer += (*_iter1175).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -31200,13 +31132,13 @@ void swap(FindSchemasByColsResp &a, FindSchemasByColsResp &b) {
   swap(a.__isset, b.__isset);
 }
 
-FindSchemasByColsResp::FindSchemasByColsResp(const FindSchemasByColsResp& other1183) {
-  schemaVersions = other1183.schemaVersions;
-  __isset = other1183.__isset;
+FindSchemasByColsResp::FindSchemasByColsResp(const FindSchemasByColsResp& other1176) {
+  schemaVersions = other1176.schemaVersions;
+  __isset = other1176.__isset;
 }
-FindSchemasByColsResp& FindSchemasByColsResp::operator=(const FindSchemasByColsResp& other1184) {
-  schemaVersions = other1184.schemaVersions;
-  __isset = other1184.__isset;
+FindSchemasByColsResp& FindSchemasByColsResp::operator=(const FindSchemasByColsResp& other1177) {
+  schemaVersions = other1177.schemaVersions;
+  __isset = other1177.__isset;
   return *this;
 }
 void FindSchemasByColsResp::printTo(std::ostream& out) const {
@@ -31303,15 +31235,15 @@ void swap(MapSchemaVersionToSerdeRequest &a, MapSchemaVersionToSerdeRequest &b)
   swap(a.__isset, b.__isset);
 }
 
-MapSchemaVersionToSerdeRequest::MapSchemaVersionToSerdeRequest(const MapSchemaVersionToSerdeRequest& other1185) {
-  schemaVersion = other1185.schemaVersion;
-  serdeName = other1185.serdeName;
-  __isset = other1185.__isset;
+MapSchemaVersionToSerdeRequest::MapSchemaVersionToSerdeRequest(const MapSchemaVersionToSerdeRequest& other1178) {
+  schemaVersion = other1178.schemaVersion;
+  serdeName = other1178.serdeName;
+  __isset = other1178.__isset;
 }
-MapSchemaVersionToSerdeRequest& MapSchemaVersionToSerdeRequest::operator=(const MapSchemaVersionToSerdeRequest& other1186) {
-  schemaVersion = other1186.schemaVersion;
-  serdeName = other1186.serdeName;
-  __isset = other1186.__isset;
+MapSchemaVersionToSerdeRequest& MapSchemaVersionToSerdeRequest::operator=(const MapSchemaVersionToSerdeRequest& other1179) {
+  schemaVersion = other1179.schemaVersion;
+  serdeName = other1179.serdeName;
+  __isset = other1179.__isset;
   return *this;
 }
 void MapSchemaVersionToSerdeRequest::printTo(std::ostream& out) const {
@@ -31366,9 +31298,9 @@ uint32_t SetSchemaVersionStateRequest::read(::apache::thrift::protocol::TProtoco
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast1187;
-          xfer += iprot->readI32(ecast1187);
-          this->state = (SchemaVersionState::type)ecast1187;
+          int32_t ecast1180;
+          xfer += iprot->readI32(ecast1180);
+          this->state = (SchemaVersionState::type)ecast1180;
           this->__isset.state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -31411,15 +31343,15 @@ void swap(SetSchemaVersionStateRequest &a, SetSchemaVersionStateRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-SetSchemaVersionStateRequest::SetSchemaVersionStateRequest(const SetSchemaVersionStateRequest& other1188) {
-  schemaVersion = other1188.schemaVersion;
-  state = other1188.state;
-  __isset = other1188.__isset;
+SetSchemaVersionStateRequest::SetSchemaVersionStateRequest(const SetSchemaVersionStateRequest& other1181) {
+  schemaVersion = other1181.schemaVersion;
+  state = other1181.state;
+  __isset = other1181.__isset;
 }
-SetSchemaVersionStateRequest& SetSchemaVersionStateRequest::operator=(const SetSchemaVersionStateRequest& other1189) {
-  schemaVersion = other1189.schemaVersion;
-  state = other1189.state;
-  __isset = other1189.__isset;
+SetSchemaVersionStateRequest& SetSchemaVersionStateRequest::operator=(const SetSchemaVersionStateRequest& other1182) {
+  schemaVersion = other1182.schemaVersion;
+  state = other1182.state;
+  __isset = other1182.__isset;
   return *this;
 }
 void SetSchemaVersionStateRequest::printTo(std::ostream& out) const {
@@ -31500,13 +31432,13 @@ void swap(GetSerdeRequest &a, GetSerdeRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetSerdeRequest::GetSerdeRequest(const GetSerdeRequest& other1190) {
-  serdeName = other1190.serdeName;
-  __isset = other1190.__isset;
+GetSerdeRequest::GetSerdeRequest(const GetSerdeRequest& other1183) {
+  serdeName = other1183.serdeName;
+  __isset = other1183.__isset;
 }
-GetSerdeRequest& GetSerdeRequest::operator=(const GetSerdeRequest& other1191) {
-  serdeName = other1191.serdeName;
-  __isset = other1191.__isset;
+GetSerdeRequest& GetSerdeRequest::operator=(const GetSerdeRequest& other1184) {
+  serdeName = other1184.serdeName;
+  __isset = other1184.__isset;
   return *this;
 }
 void GetSerdeRequest::printTo(std::ostream& out) const {
@@ -31628,17 +31560,17 @@ void swap(RuntimeStat &a, RuntimeStat &b) {
   swap(a.__isset, b.__isset);
 }
 
-RuntimeStat::RuntimeStat(const RuntimeStat& other1192) {
-  createTime = other1192.createTime;
-  weight = other1192.weight;
-  payload = other1192.payload;
-  __isset = other1192.__isset;
+RuntimeStat::RuntimeStat(const RuntimeStat& other1185) {
+  createTime = other1185.createTime;
+  weight = other1185.weight;
+  payload = other1185.payload;
+  __isset = other1185.__isset;
 }
-RuntimeStat& RuntimeStat::operator=(const RuntimeStat& other1193) {
-  createTime = other1193.createTime;
-  weight = other1193.weight;
-  payload = other1193.payload;
-  __isset = other1193.__isset;
+RuntimeStat& RuntimeStat::operator=(const RuntimeStat& other1186) {
+  createTime = other1186.createTime;
+  weight = other1186.weight;
+  payload = other1186.payload;
+  __isset = other1186.__isset;
   return *this;
 }
 void RuntimeStat::printTo(std::ostream& out) const {
@@ -31742,13 +31674,13 @@ void swap(GetRuntimeStatsRequest &a, GetRuntimeStatsRequest &b) {
   swap(a.maxCreateTime, b.maxCreateTime);
 }
 
-GetRuntimeStatsRequest::GetRuntimeStatsRequest(const GetRuntimeStatsRequest& other1194) {
-  maxWeight = other1194.maxWeight;
-  maxCreateTime = other1194.maxCreateTime;
+GetRuntimeStatsRequest::GetRuntimeStatsRequest(const GetRuntimeStatsRequest& other1187) {
+  maxWeight = other1187.maxWeight;
+  maxCreateTime = other1187.maxCreateTime;
 }
-GetRuntimeStatsRequest& GetRuntimeStatsRequest::operator=(const GetRuntimeStatsRequest& other1195) {
-  maxWeight = other1195.maxWeight;
-  maxCreateTime = other1195.maxCreateTime;
+GetRuntimeStatsRequest& GetRuntimeStatsRequest::operator=(const GetRuntimeStatsRequest& other1188) {
+  maxWeight = other1188.maxWeight;
+  maxCreateTime = other1188.maxCreateTime;
   return *this;
 }
 void GetRuntimeStatsRequest::printTo(std::ostream& out) const {
@@ -31829,13 +31761,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other1196) : TException() {
-  message = other1196.message;
-  __isset = other1196.__isset;
+MetaException::MetaException(const MetaException& other1189) : TException() {
+  message = other1189.message;
+  __isset = other1189.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other1197) {
-  message = other1197.message;
-  __isset = other1197.__isset;
+MetaException& MetaException::operator=(const MetaException& other1190) {
+  message = other1190.message;
+  __isset = other1190.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -31926,13 +31858,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other1198) : TException() {
-  message = other1198.message;
-  __isset = other1198.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other1191) : TException() {
+

<TRUNCATED>

[05/12] hive git commit: HIVE-20006: Make materializations invalidation cache work with multiple active remote metastores (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index a83017b..29e787b 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -334,14 +334,14 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function get_table_objects_by_name_req(\metastore\GetTablesRequest $req);
   /**
-   * @param string $dbname
-   * @param string[] $tbl_names
-   * @return array
+   * @param \metastore\CreationMetadata $creation_metadata
+   * @param string $validTxnList
+   * @return \metastore\Materialization
    * @throws \metastore\MetaException
    * @throws \metastore\InvalidOperationException
    * @throws \metastore\UnknownDBException
    */
-  public function get_materialization_invalidation_info($dbname, array $tbl_names);
+  public function get_materialization_invalidation_info(\metastore\CreationMetadata $creation_metadata, $validTxnList);
   /**
    * @param string $catName
    * @param string $dbname
@@ -3999,17 +3999,17 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("get_table_objects_by_name_req failed: unknown result");
   }
 
-  public function get_materialization_invalidation_info($dbname, array $tbl_names)
+  public function get_materialization_invalidation_info(\metastore\CreationMetadata $creation_metadata, $validTxnList)
   {
-    $this->send_get_materialization_invalidation_info($dbname, $tbl_names);
+    $this->send_get_materialization_invalidation_info($creation_metadata, $validTxnList);
     return $this->recv_get_materialization_invalidation_info();
   }
 
-  public function send_get_materialization_invalidation_info($dbname, array $tbl_names)
+  public function send_get_materialization_invalidation_info(\metastore\CreationMetadata $creation_metadata, $validTxnList)
   {
     $args = new \metastore\ThriftHiveMetastore_get_materialization_invalidation_info_args();
-    $args->dbname = $dbname;
-    $args->tbl_names = $tbl_names;
+    $args->creation_metadata = $creation_metadata;
+    $args->validTxnList = $validTxnList;
     $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
     if ($bin_accel)
     {
@@ -15496,14 +15496,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size841 = 0;
-            $_etype844 = 0;
-            $xfer += $input->readListBegin($_etype844, $_size841);
-            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
+            $_size833 = 0;
+            $_etype836 = 0;
+            $xfer += $input->readListBegin($_etype836, $_size833);
+            for ($_i837 = 0; $_i837 < $_size833; ++$_i837)
             {
-              $elem846 = null;
-              $xfer += $input->readString($elem846);
-              $this->success []= $elem846;
+              $elem838 = null;
+              $xfer += $input->readString($elem838);
+              $this->success []= $elem838;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15539,9 +15539,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter847)
+          foreach ($this->success as $iter839)
           {
-            $xfer += $output->writeString($iter847);
+            $xfer += $output->writeString($iter839);
           }
         }
         $output->writeListEnd();
@@ -15672,14 +15672,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size848 = 0;
-            $_etype851 = 0;
-            $xfer += $input->readListBegin($_etype851, $_size848);
-            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
+            $_size840 = 0;
+            $_etype843 = 0;
+            $xfer += $input->readListBegin($_etype843, $_size840);
+            for ($_i844 = 0; $_i844 < $_size840; ++$_i844)
             {
-              $elem853 = null;
-              $xfer += $input->readString($elem853);
-              $this->success []= $elem853;
+              $elem845 = null;
+              $xfer += $input->readString($elem845);
+              $this->success []= $elem845;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15715,9 +15715,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter854)
+          foreach ($this->success as $iter846)
           {
-            $xfer += $output->writeString($iter854);
+            $xfer += $output->writeString($iter846);
           }
         }
         $output->writeListEnd();
@@ -16718,18 +16718,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size855 = 0;
-            $_ktype856 = 0;
-            $_vtype857 = 0;
-            $xfer += $input->readMapBegin($_ktype856, $_vtype857, $_size855);
-            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
+            $_size847 = 0;
+            $_ktype848 = 0;
+            $_vtype849 = 0;
+            $xfer += $input->readMapBegin($_ktype848, $_vtype849, $_size847);
+            for ($_i851 = 0; $_i851 < $_size847; ++$_i851)
             {
-              $key860 = '';
-              $val861 = new \metastore\Type();
-              $xfer += $input->readString($key860);
-              $val861 = new \metastore\Type();
-              $xfer += $val861->read($input);
-              $this->success[$key860] = $val861;
+              $key852 = '';
+              $val853 = new \metastore\Type();
+              $xfer += $input->readString($key852);
+              $val853 = new \metastore\Type();
+              $xfer += $val853->read($input);
+              $this->success[$key852] = $val853;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -16765,10 +16765,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter862 => $viter863)
+          foreach ($this->success as $kiter854 => $viter855)
           {
-            $xfer += $output->writeString($kiter862);
-            $xfer += $viter863->write($output);
+            $xfer += $output->writeString($kiter854);
+            $xfer += $viter855->write($output);
           }
         }
         $output->writeMapEnd();
@@ -16972,15 +16972,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size864 = 0;
-            $_etype867 = 0;
-            $xfer += $input->readListBegin($_etype867, $_size864);
-            for ($_i868 = 0; $_i868 < $_size864; ++$_i868)
+            $_size856 = 0;
+            $_etype859 = 0;
+            $xfer += $input->readListBegin($_etype859, $_size856);
+            for ($_i860 = 0; $_i860 < $_size856; ++$_i860)
             {
-              $elem869 = null;
-              $elem869 = new \metastore\FieldSchema();
-              $xfer += $elem869->read($input);
-              $this->success []= $elem869;
+              $elem861 = null;
+              $elem861 = new \metastore\FieldSchema();
+              $xfer += $elem861->read($input);
+              $this->success []= $elem861;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17032,9 +17032,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter870)
+          foreach ($this->success as $iter862)
           {
-            $xfer += $iter870->write($output);
+            $xfer += $iter862->write($output);
           }
         }
         $output->writeListEnd();
@@ -17276,15 +17276,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size871 = 0;
-            $_etype874 = 0;
-            $xfer += $input->readListBegin($_etype874, $_size871);
-            for ($_i875 = 0; $_i875 < $_size871; ++$_i875)
+            $_size863 = 0;
+            $_etype866 = 0;
+            $xfer += $input->readListBegin($_etype866, $_size863);
+            for ($_i867 = 0; $_i867 < $_size863; ++$_i867)
             {
-              $elem876 = null;
-              $elem876 = new \metastore\FieldSchema();
-              $xfer += $elem876->read($input);
-              $this->success []= $elem876;
+              $elem868 = null;
+              $elem868 = new \metastore\FieldSchema();
+              $xfer += $elem868->read($input);
+              $this->success []= $elem868;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17336,9 +17336,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter877)
+          foreach ($this->success as $iter869)
           {
-            $xfer += $iter877->write($output);
+            $xfer += $iter869->write($output);
           }
         }
         $output->writeListEnd();
@@ -17552,15 +17552,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size878 = 0;
-            $_etype881 = 0;
-            $xfer += $input->readListBegin($_etype881, $_size878);
-            for ($_i882 = 0; $_i882 < $_size878; ++$_i882)
+            $_size870 = 0;
+            $_etype873 = 0;
+            $xfer += $input->readListBegin($_etype873, $_size870);
+            for ($_i874 = 0; $_i874 < $_size870; ++$_i874)
             {
-              $elem883 = null;
-              $elem883 = new \metastore\FieldSchema();
-              $xfer += $elem883->read($input);
-              $this->success []= $elem883;
+              $elem875 = null;
+              $elem875 = new \metastore\FieldSchema();
+              $xfer += $elem875->read($input);
+              $this->success []= $elem875;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17612,9 +17612,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter884)
+          foreach ($this->success as $iter876)
           {
-            $xfer += $iter884->write($output);
+            $xfer += $iter876->write($output);
           }
         }
         $output->writeListEnd();
@@ -17856,15 +17856,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size885 = 0;
-            $_etype888 = 0;
-            $xfer += $input->readListBegin($_etype888, $_size885);
-            for ($_i889 = 0; $_i889 < $_size885; ++$_i889)
+            $_size877 = 0;
+            $_etype880 = 0;
+            $xfer += $input->readListBegin($_etype880, $_size877);
+            for ($_i881 = 0; $_i881 < $_size877; ++$_i881)
             {
-              $elem890 = null;
-              $elem890 = new \metastore\FieldSchema();
-              $xfer += $elem890->read($input);
-              $this->success []= $elem890;
+              $elem882 = null;
+              $elem882 = new \metastore\FieldSchema();
+              $xfer += $elem882->read($input);
+              $this->success []= $elem882;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17916,9 +17916,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter891)
+          foreach ($this->success as $iter883)
           {
-            $xfer += $iter891->write($output);
+            $xfer += $iter883->write($output);
           }
         }
         $output->writeListEnd();
@@ -18590,15 +18590,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size892 = 0;
-            $_etype895 = 0;
-            $xfer += $input->readListBegin($_etype895, $_size892);
-            for ($_i896 = 0; $_i896 < $_size892; ++$_i896)
+            $_size884 = 0;
+            $_etype887 = 0;
+            $xfer += $input->readListBegin($_etype887, $_size884);
+            for ($_i888 = 0; $_i888 < $_size884; ++$_i888)
             {
-              $elem897 = null;
-              $elem897 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem897->read($input);
-              $this->primaryKeys []= $elem897;
+              $elem889 = null;
+              $elem889 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem889->read($input);
+              $this->primaryKeys []= $elem889;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18608,15 +18608,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size898 = 0;
-            $_etype901 = 0;
-            $xfer += $input->readListBegin($_etype901, $_size898);
-            for ($_i902 = 0; $_i902 < $_size898; ++$_i902)
+            $_size890 = 0;
+            $_etype893 = 0;
+            $xfer += $input->readListBegin($_etype893, $_size890);
+            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
             {
-              $elem903 = null;
-              $elem903 = new \metastore\SQLForeignKey();
-              $xfer += $elem903->read($input);
-              $this->foreignKeys []= $elem903;
+              $elem895 = null;
+              $elem895 = new \metastore\SQLForeignKey();
+              $xfer += $elem895->read($input);
+              $this->foreignKeys []= $elem895;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18626,15 +18626,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 4:
           if ($ftype == TType::LST) {
             $this->uniqueConstraints = array();
-            $_size904 = 0;
-            $_etype907 = 0;
-            $xfer += $input->readListBegin($_etype907, $_size904);
-            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
+            $_size896 = 0;
+            $_etype899 = 0;
+            $xfer += $input->readListBegin($_etype899, $_size896);
+            for ($_i900 = 0; $_i900 < $_size896; ++$_i900)
             {
-              $elem909 = null;
-              $elem909 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem909->read($input);
-              $this->uniqueConstraints []= $elem909;
+              $elem901 = null;
+              $elem901 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem901->read($input);
+              $this->uniqueConstraints []= $elem901;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18644,15 +18644,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->notNullConstraints = array();
-            $_size910 = 0;
-            $_etype913 = 0;
-            $xfer += $input->readListBegin($_etype913, $_size910);
-            for ($_i914 = 0; $_i914 < $_size910; ++$_i914)
+            $_size902 = 0;
+            $_etype905 = 0;
+            $xfer += $input->readListBegin($_etype905, $_size902);
+            for ($_i906 = 0; $_i906 < $_size902; ++$_i906)
             {
-              $elem915 = null;
-              $elem915 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem915->read($input);
-              $this->notNullConstraints []= $elem915;
+              $elem907 = null;
+              $elem907 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem907->read($input);
+              $this->notNullConstraints []= $elem907;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18662,15 +18662,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->defaultConstraints = array();
-            $_size916 = 0;
-            $_etype919 = 0;
-            $xfer += $input->readListBegin($_etype919, $_size916);
-            for ($_i920 = 0; $_i920 < $_size916; ++$_i920)
+            $_size908 = 0;
+            $_etype911 = 0;
+            $xfer += $input->readListBegin($_etype911, $_size908);
+            for ($_i912 = 0; $_i912 < $_size908; ++$_i912)
             {
-              $elem921 = null;
-              $elem921 = new \metastore\SQLDefaultConstraint();
-              $xfer += $elem921->read($input);
-              $this->defaultConstraints []= $elem921;
+              $elem913 = null;
+              $elem913 = new \metastore\SQLDefaultConstraint();
+              $xfer += $elem913->read($input);
+              $this->defaultConstraints []= $elem913;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18680,15 +18680,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 7:
           if ($ftype == TType::LST) {
             $this->checkConstraints = array();
-            $_size922 = 0;
-            $_etype925 = 0;
-            $xfer += $input->readListBegin($_etype925, $_size922);
-            for ($_i926 = 0; $_i926 < $_size922; ++$_i926)
+            $_size914 = 0;
+            $_etype917 = 0;
+            $xfer += $input->readListBegin($_etype917, $_size914);
+            for ($_i918 = 0; $_i918 < $_size914; ++$_i918)
             {
-              $elem927 = null;
-              $elem927 = new \metastore\SQLCheckConstraint();
-              $xfer += $elem927->read($input);
-              $this->checkConstraints []= $elem927;
+              $elem919 = null;
+              $elem919 = new \metastore\SQLCheckConstraint();
+              $xfer += $elem919->read($input);
+              $this->checkConstraints []= $elem919;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18724,9 +18724,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter928)
+          foreach ($this->primaryKeys as $iter920)
           {
-            $xfer += $iter928->write($output);
+            $xfer += $iter920->write($output);
           }
         }
         $output->writeListEnd();
@@ -18741,9 +18741,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter929)
+          foreach ($this->foreignKeys as $iter921)
           {
-            $xfer += $iter929->write($output);
+            $xfer += $iter921->write($output);
           }
         }
         $output->writeListEnd();
@@ -18758,9 +18758,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
         {
-          foreach ($this->uniqueConstraints as $iter930)
+          foreach ($this->uniqueConstraints as $iter922)
           {
-            $xfer += $iter930->write($output);
+            $xfer += $iter922->write($output);
           }
         }
         $output->writeListEnd();
@@ -18775,9 +18775,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
         {
-          foreach ($this->notNullConstraints as $iter931)
+          foreach ($this->notNullConstraints as $iter923)
           {
-            $xfer += $iter931->write($output);
+            $xfer += $iter923->write($output);
           }
         }
         $output->writeListEnd();
@@ -18792,9 +18792,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints));
         {
-          foreach ($this->defaultConstraints as $iter932)
+          foreach ($this->defaultConstraints as $iter924)
           {
-            $xfer += $iter932->write($output);
+            $xfer += $iter924->write($output);
           }
         }
         $output->writeListEnd();
@@ -18809,9 +18809,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->checkConstraints));
         {
-          foreach ($this->checkConstraints as $iter933)
+          foreach ($this->checkConstraints as $iter925)
           {
-            $xfer += $iter933->write($output);
+            $xfer += $iter925->write($output);
           }
         }
         $output->writeListEnd();
@@ -20811,14 +20811,14 @@ class ThriftHiveMetastore_truncate_table_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size934 = 0;
-            $_etype937 = 0;
-            $xfer += $input->readListBegin($_etype937, $_size934);
-            for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
+            $_size926 = 0;
+            $_etype929 = 0;
+            $xfer += $input->readListBegin($_etype929, $_size926);
+            for ($_i930 = 0; $_i930 < $_size926; ++$_i930)
             {
-              $elem939 = null;
-              $xfer += $input->readString($elem939);
-              $this->partNames []= $elem939;
+              $elem931 = null;
+              $xfer += $input->readString($elem931);
+              $this->partNames []= $elem931;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20856,9 +20856,9 @@ class ThriftHiveMetastore_truncate_table_args {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter940)
+          foreach ($this->partNames as $iter932)
           {
-            $xfer += $output->writeString($iter940);
+            $xfer += $output->writeString($iter932);
           }
         }
         $output->writeListEnd();
@@ -21109,14 +21109,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size941 = 0;
-            $_etype944 = 0;
-            $xfer += $input->readListBegin($_etype944, $_size941);
-            for ($_i945 = 0; $_i945 < $_size941; ++$_i945)
+            $_size933 = 0;
+            $_etype936 = 0;
+            $xfer += $input->readListBegin($_etype936, $_size933);
+            for ($_i937 = 0; $_i937 < $_size933; ++$_i937)
             {
-              $elem946 = null;
-              $xfer += $input->readString($elem946);
-              $this->success []= $elem946;
+              $elem938 = null;
+              $xfer += $input->readString($elem938);
+              $this->success []= $elem938;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21152,9 +21152,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter947)
+          foreach ($this->success as $iter939)
           {
-            $xfer += $output->writeString($iter947);
+            $xfer += $output->writeString($iter939);
           }
         }
         $output->writeListEnd();
@@ -21356,14 +21356,14 @@ class ThriftHiveMetastore_get_tables_by_type_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size948 = 0;
-            $_etype951 = 0;
-            $xfer += $input->readListBegin($_etype951, $_size948);
-            for ($_i952 = 0; $_i952 < $_size948; ++$_i952)
+            $_size940 = 0;
+            $_etype943 = 0;
+            $xfer += $input->readListBegin($_etype943, $_size940);
+            for ($_i944 = 0; $_i944 < $_size940; ++$_i944)
             {
-              $elem953 = null;
-              $xfer += $input->readString($elem953);
-              $this->success []= $elem953;
+              $elem945 = null;
+              $xfer += $input->readString($elem945);
+              $this->success []= $elem945;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21399,9 +21399,9 @@ class ThriftHiveMetastore_get_tables_by_type_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter954)
+          foreach ($this->success as $iter946)
           {
-            $xfer += $output->writeString($iter954);
+            $xfer += $output->writeString($iter946);
           }
         }
         $output->writeListEnd();
@@ -21557,14 +21557,14 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size955 = 0;
-            $_etype958 = 0;
-            $xfer += $input->readListBegin($_etype958, $_size955);
-            for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
+            $_size947 = 0;
+            $_etype950 = 0;
+            $xfer += $input->readListBegin($_etype950, $_size947);
+            for ($_i951 = 0; $_i951 < $_size947; ++$_i951)
             {
-              $elem960 = null;
-              $xfer += $input->readString($elem960);
-              $this->success []= $elem960;
+              $elem952 = null;
+              $xfer += $input->readString($elem952);
+              $this->success []= $elem952;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21600,9 +21600,9 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter961)
+          foreach ($this->success as $iter953)
           {
-            $xfer += $output->writeString($iter961);
+            $xfer += $output->writeString($iter953);
           }
         }
         $output->writeListEnd();
@@ -21707,14 +21707,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size962 = 0;
-            $_etype965 = 0;
-            $xfer += $input->readListBegin($_etype965, $_size962);
-            for ($_i966 = 0; $_i966 < $_size962; ++$_i966)
+            $_size954 = 0;
+            $_etype957 = 0;
+            $xfer += $input->readListBegin($_etype957, $_size954);
+            for ($_i958 = 0; $_i958 < $_size954; ++$_i958)
             {
-              $elem967 = null;
-              $xfer += $input->readString($elem967);
-              $this->tbl_types []= $elem967;
+              $elem959 = null;
+              $xfer += $input->readString($elem959);
+              $this->tbl_types []= $elem959;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21752,9 +21752,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter968)
+          foreach ($this->tbl_types as $iter960)
           {
-            $xfer += $output->writeString($iter968);
+            $xfer += $output->writeString($iter960);
           }
         }
         $output->writeListEnd();
@@ -21831,15 +21831,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size969 = 0;
-            $_etype972 = 0;
-            $xfer += $input->readListBegin($_etype972, $_size969);
-            for ($_i973 = 0; $_i973 < $_size969; ++$_i973)
+            $_size961 = 0;
+            $_etype964 = 0;
+            $xfer += $input->readListBegin($_etype964, $_size961);
+            for ($_i965 = 0; $_i965 < $_size961; ++$_i965)
             {
-              $elem974 = null;
-              $elem974 = new \metastore\TableMeta();
-              $xfer += $elem974->read($input);
-              $this->success []= $elem974;
+              $elem966 = null;
+              $elem966 = new \metastore\TableMeta();
+              $xfer += $elem966->read($input);
+              $this->success []= $elem966;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21875,9 +21875,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter975)
+          foreach ($this->success as $iter967)
           {
-            $xfer += $iter975->write($output);
+            $xfer += $iter967->write($output);
           }
         }
         $output->writeListEnd();
@@ -22033,14 +22033,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size976 = 0;
-            $_etype979 = 0;
-            $xfer += $input->readListBegin($_etype979, $_size976);
-            for ($_i980 = 0; $_i980 < $_size976; ++$_i980)
+            $_size968 = 0;
+            $_etype971 = 0;
+            $xfer += $input->readListBegin($_etype971, $_size968);
+            for ($_i972 = 0; $_i972 < $_size968; ++$_i972)
             {
-              $elem981 = null;
-              $xfer += $input->readString($elem981);
-              $this->success []= $elem981;
+              $elem973 = null;
+              $xfer += $input->readString($elem973);
+              $this->success []= $elem973;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22076,9 +22076,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter982)
+          foreach ($this->success as $iter974)
           {
-            $xfer += $output->writeString($iter982);
+            $xfer += $output->writeString($iter974);
           }
         }
         $output->writeListEnd();
@@ -22393,14 +22393,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size983 = 0;
-            $_etype986 = 0;
-            $xfer += $input->readListBegin($_etype986, $_size983);
-            for ($_i987 = 0; $_i987 < $_size983; ++$_i987)
+            $_size975 = 0;
+            $_etype978 = 0;
+            $xfer += $input->readListBegin($_etype978, $_size975);
+            for ($_i979 = 0; $_i979 < $_size975; ++$_i979)
             {
-              $elem988 = null;
-              $xfer += $input->readString($elem988);
-              $this->tbl_names []= $elem988;
+              $elem980 = null;
+              $xfer += $input->readString($elem980);
+              $this->tbl_names []= $elem980;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22433,9 +22433,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter989)
+          foreach ($this->tbl_names as $iter981)
           {
-            $xfer += $output->writeString($iter989);
+            $xfer += $output->writeString($iter981);
           }
         }
         $output->writeListEnd();
@@ -22500,15 +22500,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size990 = 0;
-            $_etype993 = 0;
-            $xfer += $input->readListBegin($_etype993, $_size990);
-            for ($_i994 = 0; $_i994 < $_size990; ++$_i994)
+            $_size982 = 0;
+            $_etype985 = 0;
+            $xfer += $input->readListBegin($_etype985, $_size982);
+            for ($_i986 = 0; $_i986 < $_size982; ++$_i986)
             {
-              $elem995 = null;
-              $elem995 = new \metastore\Table();
-              $xfer += $elem995->read($input);
-              $this->success []= $elem995;
+              $elem987 = null;
+              $elem987 = new \metastore\Table();
+              $xfer += $elem987->read($input);
+              $this->success []= $elem987;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22536,9 +22536,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter996)
+          foreach ($this->success as $iter988)
           {
-            $xfer += $iter996->write($output);
+            $xfer += $iter988->write($output);
           }
         }
         $output->writeListEnd();
@@ -23001,37 +23001,34 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
   static $_TSPEC;
 
   /**
-   * @var string
+   * @var \metastore\CreationMetadata
    */
-  public $dbname = null;
+  public $creation_metadata = null;
   /**
-   * @var string[]
+   * @var string
    */
-  public $tbl_names = null;
+  public $validTxnList = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'dbname',
-          'type' => TType::STRING,
+          'var' => 'creation_metadata',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\CreationMetadata',
           ),
         2 => array(
-          'var' => 'tbl_names',
-          'type' => TType::LST,
-          'etype' => TType::STRING,
-          'elem' => array(
-            'type' => TType::STRING,
-            ),
+          'var' => 'validTxnList',
+          'type' => TType::STRING,
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['dbname'])) {
-        $this->dbname = $vals['dbname'];
+      if (isset($vals['creation_metadata'])) {
+        $this->creation_metadata = $vals['creation_metadata'];
       }
-      if (isset($vals['tbl_names'])) {
-        $this->tbl_names = $vals['tbl_names'];
+      if (isset($vals['validTxnList'])) {
+        $this->validTxnList = $vals['validTxnList'];
       }
     }
   }
@@ -23056,25 +23053,16 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->dbname);
+          if ($ftype == TType::STRUCT) {
+            $this->creation_metadata = new \metastore\CreationMetadata();
+            $xfer += $this->creation_metadata->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
-          if ($ftype == TType::LST) {
-            $this->tbl_names = array();
-            $_size997 = 0;
-            $_etype1000 = 0;
-            $xfer += $input->readListBegin($_etype1000, $_size997);
-            for ($_i1001 = 0; $_i1001 < $_size997; ++$_i1001)
-            {
-              $elem1002 = null;
-              $xfer += $input->readString($elem1002);
-              $this->tbl_names []= $elem1002;
-            }
-            $xfer += $input->readListEnd();
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->validTxnList);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -23092,26 +23080,17 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
   public function write($output) {
     $xfer = 0;
     $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_materialization_invalidation_info_args');
-    if ($this->dbname !== null) {
-      $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1);
-      $xfer += $output->writeString($this->dbname);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->tbl_names !== null) {
-      if (!is_array($this->tbl_names)) {
+    if ($this->creation_metadata !== null) {
+      if (!is_object($this->creation_metadata)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('tbl_names', TType::LST, 2);
-      {
-        $output->writeListBegin(TType::STRING, count($this->tbl_names));
-        {
-          foreach ($this->tbl_names as $iter1003)
-          {
-            $xfer += $output->writeString($iter1003);
-          }
-        }
-        $output->writeListEnd();
-      }
+      $xfer += $output->writeFieldBegin('creation_metadata', TType::STRUCT, 1);
+      $xfer += $this->creation_metadata->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->validTxnList !== null) {
+      $xfer += $output->writeFieldBegin('validTxnList', TType::STRING, 2);
+      $xfer += $output->writeString($this->validTxnList);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -23125,7 +23104,7 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
   static $_TSPEC;
 
   /**
-   * @var array
+   * @var \metastore\Materialization
    */
   public $success = null;
   /**
@@ -23146,16 +23125,8 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
       self::$_TSPEC = array(
         0 => array(
           'var' => 'success',
-          'type' => TType::MAP,
-          'ktype' => TType::STRING,
-          'vtype' => TType::STRUCT,
-          'key' => array(
-            'type' => TType::STRING,
-          ),
-          'val' => array(
-            'type' => TType::STRUCT,
-            'class' => '\metastore\Materialization',
-            ),
+          'type' => TType::STRUCT,
+          'class' => '\metastore\Materialization',
           ),
         1 => array(
           'var' => 'o1',
@@ -23210,22 +23181,9 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
       switch ($fid)
       {
         case 0:
-          if ($ftype == TType::MAP) {
-            $this->success = array();
-            $_size1004 = 0;
-            $_ktype1005 = 0;
-            $_vtype1006 = 0;
-            $xfer += $input->readMapBegin($_ktype1005, $_vtype1006, $_size1004);
-            for ($_i1008 = 0; $_i1008 < $_size1004; ++$_i1008)
-            {
-              $key1009 = '';
-              $val1010 = new \metastore\Materialization();
-              $xfer += $input->readString($key1009);
-              $val1010 = new \metastore\Materialization();
-              $xfer += $val1010->read($input);
-              $this->success[$key1009] = $val1010;
-            }
-            $xfer += $input->readMapEnd();
+          if ($ftype == TType::STRUCT) {
+            $this->success = new \metastore\Materialization();
+            $xfer += $this->success->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -23268,21 +23226,11 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
     $xfer = 0;
     $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_materialization_invalidation_info_result');
     if ($this->success !== null) {
-      if (!is_array($this->success)) {
+      if (!is_object($this->success)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('success', TType::MAP, 0);
-      {
-        $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
-        {
-          foreach ($this->success as $kiter1011 => $viter1012)
-          {
-            $xfer += $output->writeString($kiter1011);
-            $xfer += $viter1012->write($output);
-          }
-        }
-        $output->writeMapEnd();
-      }
+      $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0);
+      $xfer += $this->success->write($output);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->o1 !== null) {
@@ -23790,14 +23738,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1013 = 0;
-            $_etype1016 = 0;
-            $xfer += $input->readListBegin($_etype1016, $_size1013);
-            for ($_i1017 = 0; $_i1017 < $_size1013; ++$_i1017)
+            $_size989 = 0;
+            $_etype992 = 0;
+            $xfer += $input->readListBegin($_etype992, $_size989);
+            for ($_i993 = 0; $_i993 < $_size989; ++$_i993)
             {
-              $elem1018 = null;
-              $xfer += $input->readString($elem1018);
-              $this->success []= $elem1018;
+              $elem994 = null;
+              $xfer += $input->readString($elem994);
+              $this->success []= $elem994;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23849,9 +23797,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1019)
+          foreach ($this->success as $iter995)
           {
-            $xfer += $output->writeString($iter1019);
+            $xfer += $output->writeString($iter995);
           }
         }
         $output->writeListEnd();
@@ -25164,15 +25112,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1020 = 0;
-            $_etype1023 = 0;
-            $xfer += $input->readListBegin($_etype1023, $_size1020);
-            for ($_i1024 = 0; $_i1024 < $_size1020; ++$_i1024)
+            $_size996 = 0;
+            $_etype999 = 0;
+            $xfer += $input->readListBegin($_etype999, $_size996);
+            for ($_i1000 = 0; $_i1000 < $_size996; ++$_i1000)
             {
-              $elem1025 = null;
-              $elem1025 = new \metastore\Partition();
-              $xfer += $elem1025->read($input);
-              $this->new_parts []= $elem1025;
+              $elem1001 = null;
+              $elem1001 = new \metastore\Partition();
+              $xfer += $elem1001->read($input);
+              $this->new_parts []= $elem1001;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25200,9 +25148,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1026)
+          foreach ($this->new_parts as $iter1002)
           {
-            $xfer += $iter1026->write($output);
+            $xfer += $iter1002->write($output);
           }
         }
         $output->writeListEnd();
@@ -25417,15 +25365,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1027 = 0;
-            $_etype1030 = 0;
-            $xfer += $input->readListBegin($_etype1030, $_size1027);
-            for ($_i1031 = 0; $_i1031 < $_size1027; ++$_i1031)
+            $_size1003 = 0;
+            $_etype1006 = 0;
+            $xfer += $input->readListBegin($_etype1006, $_size1003);
+            for ($_i1007 = 0; $_i1007 < $_size1003; ++$_i1007)
             {
-              $elem1032 = null;
-              $elem1032 = new \metastore\PartitionSpec();
-              $xfer += $elem1032->read($input);
-              $this->new_parts []= $elem1032;
+              $elem1008 = null;
+              $elem1008 = new \metastore\PartitionSpec();
+              $xfer += $elem1008->read($input);
+              $this->new_parts []= $elem1008;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25453,9 +25401,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1033)
+          foreach ($this->new_parts as $iter1009)
           {
-            $xfer += $iter1033->write($output);
+            $xfer += $iter1009->write($output);
           }
         }
         $output->writeListEnd();
@@ -25705,14 +25653,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1034 = 0;
-            $_etype1037 = 0;
-            $xfer += $input->readListBegin($_etype1037, $_size1034);
-            for ($_i1038 = 0; $_i1038 < $_size1034; ++$_i1038)
+            $_size1010 = 0;
+            $_etype1013 = 0;
+            $xfer += $input->readListBegin($_etype1013, $_size1010);
+            for ($_i1014 = 0; $_i1014 < $_size1010; ++$_i1014)
             {
-              $elem1039 = null;
-              $xfer += $input->readString($elem1039);
-              $this->part_vals []= $elem1039;
+              $elem1015 = null;
+              $xfer += $input->readString($elem1015);
+              $this->part_vals []= $elem1015;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25750,9 +25698,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1040)
+          foreach ($this->part_vals as $iter1016)
           {
-            $xfer += $output->writeString($iter1040);
+            $xfer += $output->writeString($iter1016);
           }
         }
         $output->writeListEnd();
@@ -26254,14 +26202,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1041 = 0;
-            $_etype1044 = 0;
-            $xfer += $input->readListBegin($_etype1044, $_size1041);
-            for ($_i1045 = 0; $_i1045 < $_size1041; ++$_i1045)
+            $_size1017 = 0;
+            $_etype1020 = 0;
+            $xfer += $input->readListBegin($_etype1020, $_size1017);
+            for ($_i1021 = 0; $_i1021 < $_size1017; ++$_i1021)
             {
-              $elem1046 = null;
-              $xfer += $input->readString($elem1046);
-              $this->part_vals []= $elem1046;
+              $elem1022 = null;
+              $xfer += $input->readString($elem1022);
+              $this->part_vals []= $elem1022;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26307,9 +26255,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1047)
+          foreach ($this->part_vals as $iter1023)
           {
-            $xfer += $output->writeString($iter1047);
+            $xfer += $output->writeString($iter1023);
           }
         }
         $output->writeListEnd();
@@ -27163,14 +27111,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1048 = 0;
-            $_etype1051 = 0;
-            $xfer += $input->readListBegin($_etype1051, $_size1048);
-            for ($_i1052 = 0; $_i1052 < $_size1048; ++$_i1052)
+            $_size1024 = 0;
+            $_etype1027 = 0;
+            $xfer += $input->readListBegin($_etype1027, $_size1024);
+            for ($_i1028 = 0; $_i1028 < $_size1024; ++$_i1028)
             {
-              $elem1053 = null;
-              $xfer += $input->readString($elem1053);
-              $this->part_vals []= $elem1053;
+              $elem1029 = null;
+              $xfer += $input->readString($elem1029);
+              $this->part_vals []= $elem1029;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27215,9 +27163,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1054)
+          foreach ($this->part_vals as $iter1030)
           {
-            $xfer += $output->writeString($iter1054);
+            $xfer += $output->writeString($iter1030);
           }
         }
         $output->writeListEnd();
@@ -27470,14 +27418,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1055 = 0;
-            $_etype1058 = 0;
-            $xfer += $input->readListBegin($_etype1058, $_size1055);
-            for ($_i1059 = 0; $_i1059 < $_size1055; ++$_i1059)
+            $_size1031 = 0;
+            $_etype1034 = 0;
+            $xfer += $input->readListBegin($_etype1034, $_size1031);
+            for ($_i1035 = 0; $_i1035 < $_size1031; ++$_i1035)
             {
-              $elem1060 = null;
-              $xfer += $input->readString($elem1060);
-              $this->part_vals []= $elem1060;
+              $elem1036 = null;
+              $xfer += $input->readString($elem1036);
+              $this->part_vals []= $elem1036;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27530,9 +27478,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1061)
+          foreach ($this->part_vals as $iter1037)
           {
-            $xfer += $output->writeString($iter1061);
+            $xfer += $output->writeString($iter1037);
           }
         }
         $output->writeListEnd();
@@ -28546,14 +28494,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1062 = 0;
-            $_etype1065 = 0;
-            $xfer += $input->readListBegin($_etype1065, $_size1062);
-            for ($_i1066 = 0; $_i1066 < $_size1062; ++$_i1066)
+            $_size1038 = 0;
+            $_etype1041 = 0;
+            $xfer += $input->readListBegin($_etype1041, $_size1038);
+            for ($_i1042 = 0; $_i1042 < $_size1038; ++$_i1042)
             {
-              $elem1067 = null;
-              $xfer += $input->readString($elem1067);
-              $this->part_vals []= $elem1067;
+              $elem1043 = null;
+              $xfer += $input->readString($elem1043);
+              $this->part_vals []= $elem1043;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28591,9 +28539,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1068)
+          foreach ($this->part_vals as $iter1044)
           {
-            $xfer += $output->writeString($iter1068);
+            $xfer += $output->writeString($iter1044);
           }
         }
         $output->writeListEnd();
@@ -28835,17 +28783,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1069 = 0;
-            $_ktype1070 = 0;
-            $_vtype1071 = 0;
-            $xfer += $input->readMapBegin($_ktype1070, $_vtype1071, $_size1069);
-            for ($_i1073 = 0; $_i1073 < $_size1069; ++$_i1073)
+            $_size1045 = 0;
+            $_ktype1046 = 0;
+            $_vtype1047 = 0;
+            $xfer += $input->readMapBegin($_ktype1046, $_vtype1047, $_size1045);
+            for ($_i1049 = 0; $_i1049 < $_size1045; ++$_i1049)
             {
-              $key1074 = '';
-              $val1075 = '';
-              $xfer += $input->readString($key1074);
-              $xfer += $input->readString($val1075);
-              $this->partitionSpecs[$key1074] = $val1075;
+              $key1050 = '';
+              $val1051 = '';
+              $xfer += $input->readString($key1050);
+              $xfer += $input->readString($val1051);
+              $this->partitionSpecs[$key1050] = $val1051;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -28901,10 +28849,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1076 => $viter1077)
+          foreach ($this->partitionSpecs as $kiter1052 => $viter1053)
           {
-            $xfer += $output->writeString($kiter1076);
-            $xfer += $output->writeString($viter1077);
+            $xfer += $output->writeString($kiter1052);
+            $xfer += $output->writeString($viter1053);
           }
         }
         $output->writeMapEnd();
@@ -29216,17 +29164,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1078 = 0;
-            $_ktype1079 = 0;
-            $_vtype1080 = 0;
-            $xfer += $input->readMapBegin($_ktype1079, $_vtype1080, $_size1078);
-            for ($_i1082 = 0; $_i1082 < $_size1078; ++$_i1082)
+            $_size1054 = 0;
+            $_ktype1055 = 0;
+            $_vtype1056 = 0;
+            $xfer += $input->readMapBegin($_ktype1055, $_vtype1056, $_size1054);
+            for ($_i1058 = 0; $_i1058 < $_size1054; ++$_i1058)
             {
-              $key1083 = '';
-              $val1084 = '';
-              $xfer += $input->readString($key1083);
-              $xfer += $input->readString($val1084);
-              $this->partitionSpecs[$key1083] = $val1084;
+              $key1059 = '';
+              $val1060 = '';
+              $xfer += $input->readString($key1059);
+              $xfer += $input->readString($val1060);
+              $this->partitionSpecs[$key1059] = $val1060;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -29282,10 +29230,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1085 => $viter1086)
+          foreach ($this->partitionSpecs as $kiter1061 => $viter1062)
           {
-            $xfer += $output->writeString($kiter1085);
-            $xfer += $output->writeString($viter1086);
+            $xfer += $output->writeString($kiter1061);
+            $xfer += $output->writeString($viter1062);
           }
         }
         $output->writeMapEnd();
@@ -29418,15 +29366,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1087 = 0;
-            $_etype1090 = 0;
-            $xfer += $input->readListBegin($_etype1090, $_size1087);
-            for ($_i1091 = 0; $_i1091 < $_size1087; ++$_i1091)
+            $_size1063 = 0;
+            $_etype1066 = 0;
+            $xfer += $input->readListBegin($_etype1066, $_size1063);
+            for ($_i1067 = 0; $_i1067 < $_size1063; ++$_i1067)
             {
-              $elem1092 = null;
-              $elem1092 = new \metastore\Partition();
-              $xfer += $elem1092->read($input);
-              $this->success []= $elem1092;
+              $elem1068 = null;
+              $elem1068 = new \metastore\Partition();
+              $xfer += $elem1068->read($input);
+              $this->success []= $elem1068;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29486,9 +29434,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1093)
+          foreach ($this->success as $iter1069)
           {
-            $xfer += $iter1093->write($output);
+            $xfer += $iter1069->write($output);
           }
         }
         $output->writeListEnd();
@@ -29634,14 +29582,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1094 = 0;
-            $_etype1097 = 0;
-            $xfer += $input->readListBegin($_etype1097, $_size1094);
-            for ($_i1098 = 0; $_i1098 < $_size1094; ++$_i1098)
+            $_size1070 = 0;
+            $_etype1073 = 0;
+            $xfer += $input->readListBegin($_etype1073, $_size1070);
+            for ($_i1074 = 0; $_i1074 < $_size1070; ++$_i1074)
             {
-              $elem1099 = null;
-              $xfer += $input->readString($elem1099);
-              $this->part_vals []= $elem1099;
+              $elem1075 = null;
+              $xfer += $input->readString($elem1075);
+              $this->part_vals []= $elem1075;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29658,14 +29606,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1100 = 0;
-            $_etype1103 = 0;
-            $xfer += $input->readListBegin($_etype1103, $_size1100);
-            for ($_i1104 = 0; $_i1104 < $_size1100; ++$_i1104)
+            $_size1076 = 0;
+            $_etype1079 = 0;
+            $xfer += $input->readListBegin($_etype1079, $_size1076);
+            for ($_i1080 = 0; $_i1080 < $_size1076; ++$_i1080)
             {
-              $elem1105 = null;
-              $xfer += $input->readString($elem1105);
-              $this->group_names []= $elem1105;
+              $elem1081 = null;
+              $xfer += $input->readString($elem1081);
+              $this->group_names []= $elem1081;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29703,9 +29651,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1106)
+          foreach ($this->part_vals as $iter1082)
           {
-            $xfer += $output->writeString($iter1106);
+            $xfer += $output->writeString($iter1082);
           }
         }
         $output->writeListEnd();
@@ -29725,9 +29673,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1107)
+          foreach ($this->group_names as $iter1083)
           {
-            $xfer += $output->writeString($iter1107);
+            $xfer += $output->writeString($iter1083);
           }
         }
         $output->writeListEnd();
@@ -30318,15 +30266,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1108 = 0;
-            $_etype1111 = 0;
-            $xfer += $input->readListBegin($_etype1111, $_size1108);
-            for ($_i1112 = 0; $_i1112 < $_size1108; ++$_i1112)
+            $_size1084 = 0;
+            $_etype1087 = 0;
+            $xfer += $input->readListBegin($_etype1087, $_size1084);
+            for ($_i1088 = 0; $_i1088 < $_size1084; ++$_i1088)
             {
-              $elem1113 = null;
-              $elem1113 = new \metastore\Partition();
-              $xfer += $elem1113->read($input);
-              $this->success []= $elem1113;
+              $elem1089 = null;
+              $elem1089 = new \metastore\Partition();
+              $xfer += $elem1089->read($input);
+              $this->success []= $elem1089;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30370,9 +30318,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1114)
+          foreach ($this->success as $iter1090)
           {
-            $xfer += $iter1114->write($output);
+            $xfer += $iter1090->write($output);
           }
         }
         $output->writeListEnd();
@@ -30518,14 +30466,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1115 = 0;
-            $_etype1118 = 0;
-            $xfer += $input->readListBegin($_etype1118, $_size1115);
-            for ($_i1119 = 0; $_i1119 < $_size1115; ++$_i1119)
+            $_size1091 = 0;
+            $_etype1094 = 0;
+            $xfer += $input->readListBegin($_etype1094, $_size1091);
+            for ($_i1095 = 0; $_i1095 < $_size1091; ++$_i1095)
             {
-              $elem1120 = null;
-              $xfer += $input->readString($elem1120);
-              $this->group_names []= $elem1120;
+              $elem1096 = null;
+              $xfer += $input->readString($elem1096);
+              $this->group_names []= $elem1096;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30573,9 +30521,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1121)
+          foreach ($this->group_names as $iter1097)
           {
-            $xfer += $output->writeString($iter1121);
+            $xfer += $output->writeString($iter1097);
           }
         }
         $output->writeListEnd();
@@ -30664,15 +30612,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1122 = 0;
-            $_etype1125 = 0;
-            $xfer += $input->readListBegin($_etype1125, $_size1122);
-            for ($_i1126 = 0; $_i1126 < $_size1122; ++$_i1126)
+            $_size1098 = 0;
+            $_etype1101 = 0;
+            $xfer += $input->readListBegin($_etype1101, $_size1098);
+            for ($_i1102 = 0; $_i1102 < $_size1098; ++$_i1102)
             {
-              $elem1127 = null;
-              $elem1127 = new \metastore\Partition();
-              $xfer += $elem1127->read($input);
-              $this->success []= $elem1127;
+              $elem1103 = null;
+              $elem1103 = new \metastore\Partition();
+              $xfer += $elem1103->read($input);
+              $this->success []= $elem1103;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30716,9 +30664,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1128)
+          foreach ($this->success as $iter1104)
           {
-            $xfer += $iter1128->write($output);
+            $xfer += $iter1104->write($output);
           }
         }
         $output->writeListEnd();
@@ -30938,15 +30886,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1129 = 0;
-            $_etype1132 = 0;
-            $xfer += $input->readListBegin($_etype1132, $_size1129);
-            for ($_i1133 = 0; $_i1133 < $_size1129; ++$_i1133)
+            $_size1105 = 0;
+            $_etype1108 = 0;
+            $xfer += $input->readListBegin($_etype1108, $_size1105);
+            for ($_i1109 = 0; $_i1109 < $_size1105; ++$_i1109)
             {
-              $elem1134 = null;
-              $elem1134 = new \metastore\PartitionSpec();
-              $xfer += $elem1134->read($input);
-              $this->success []= $elem1134;
+              $elem1110 = null;
+              $elem1110 = new \metastore\PartitionSpec();
+              $xfer += $elem1110->read($input);
+              $this->success []= $elem1110;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30990,9 +30938,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1135)
+          foreach ($this->success as $iter1111)
           {
-            $xfer += $iter1135->write($output);
+            $xfer += $iter1111->write($output);
           }
         }
         $output->writeListEnd();
@@ -31211,14 +31159,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1136 = 0;
-            $_etype1139 = 0;
-            $xfer += $input->readListBegin($_etype1139, $_size1136);
-            for ($_i1140 = 0; $_i1140 < $_size1136; ++$_i1140)
+            $_size1112 = 0;
+            $_etype1115 = 0;
+            $xfer += $input->readListBegin($_etype1115, $_size1112);
+            for ($_i1116 = 0; $_i1116 < $_size1112; ++$_i1116)
             {
-              $elem1141 = null;
-              $xfer += $input->readString($elem1141);
-              $this->success []= $elem1141;
+              $elem1117 = null;
+              $xfer += $input->readString($elem1117);
+              $this->success []= $elem1117;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31262,9 +31210,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1142)
+          foreach ($this->success as $iter1118)
           {
-            $xfer += $output->writeString($iter1142);
+            $xfer += $output->writeString($iter1118);
           }
         }
         $output->writeListEnd();
@@ -31595,14 +31543,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1143 = 0;
-            $_etype1146 = 0;
-            $xfer += $input->readListBegin($_etype1146, $_size1143);
-            for ($_i1147 = 0; $_i1147 < $_size1143; ++$_i1147)
+            $_size1119 = 0;
+            $_etype1122 = 0;
+            $xfer += $input->readListBegin($_etype1122, $_size1119);
+            for ($_i1123 = 0; $_i1123 < $_size1119; ++$_i1123)
             {
-              $elem1148 = null;
-              $xfer += $input->readString($elem1148);
-              $this->part_vals []= $elem1148;
+              $elem1124 = null;
+              $xfer += $input->readString($elem1124);
+              $this->part_vals []= $elem1124;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31647,9 +31595,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1149)
+          foreach ($this->part_vals as $iter1125)
           {
-            $xfer += $output->writeString($iter1149);
+            $xfer += $output->writeString($iter1125);
           }
         }
         $output->writeListEnd();
@@ -31743,15 +31691,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1150 = 0;
-            $_etype1153 = 0;
-            $xfer += $input->readListBegin($_etype1153, $_size1150);
-            for ($_i1154 = 0; $_i1154 < $_size1150; ++$_i1154)
+            $_size1126 = 0;
+            $_etype1129 = 0;
+            $xfer += $input->readListBegin($_etype1129, $_size1126);
+            for ($_i1130 = 0; $_i1130 < $_size1126; ++$_i1130)
             {
-              $elem1155 = null;
-              $elem1155 = new \metastore\Partition();
-              $xfer += $elem1155->read($input);
-              $this->success []= $elem1155;
+              $elem1131 = null;
+              $elem1131 = new \metastore\Partition();
+              $xfer += $elem1131->read($input);
+              $this->success []= $elem1131;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31795,9 +31743,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1156)
+          foreach ($this->success as $iter1132)
           {
-            $xfer += $iter1156->write($output);
+            $xfer += $iter1132->write($output);
           }
         }
         $output->writeListEnd();
@@ -31944,14 +31892,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1157 = 0;
-            $_etype1160 = 0;
-            $xfer += $input->readListBegin($_etype1160, $_size1157);
-            for ($_i1161 = 0; $_i1161 < $_size1157; ++$_i1161)
+            $_size1133 = 0;
+            $_etype1136 = 0;
+            $xfer += $input->readListBegin($_etype1136, $_size1133);
+            for ($_i1137 = 0; $_i1137 < $_size1133; ++$_i1137)
             {
-              $elem1162 = null;
-              $xfer += $input->readString($elem1162);
-              $this->part_vals []= $elem1162;
+              $elem1138 = null;
+              $xfer += $input->readString($elem1138);
+              $this->part_vals []= $elem1138;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31975,14 +31923,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1163 = 0;
-            $_etype1166 = 0;
-            $xfer += $input->readListBegin($_etype1166, $_size1163);
-            for ($_i1167 = 0; $_i1167 < $_size1163; ++$_i1167)
+            $_size1139 = 0;
+            $_etype1142 = 0;
+            $xfer += $input->readListBegin($_etype1142, $_size1139);
+            for ($_i1143 = 0; $_i1143 < $_size1139; ++$_i1143)
             {
-              $elem1168 = null;
-              $xfer += $input->readString($elem1168);
-              $this->group_names []= $elem1168;
+              $elem1144 = null;
+              $xfer += $input->readString($elem1144);
+              $this->group_names []= $elem1144;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32020,9 +31968,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1169)
+          foreach ($this->part_vals as $iter1145)
           {
-            $xfer += $output->writeString($iter1169);
+            $xfer += $output->writeString($iter1145);
           }
         }
         $output->writeListEnd();
@@ -32047,9 +31995,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1170)
+          foreach ($this->group_names as $iter1146)
           {
-            $xfer += $output->writeString($iter1170);
+            $xfer += $output->writeString($iter1146);
           }
         }
         $output->writeListEnd();
@@ -32138,15 +32086,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1171 = 0;
-            $_etype1174 = 0;
-            $xfer += $input->readListBegin($_etype1174, $_size1171);
-            for ($_i1175 = 0; $_i1175 < $_size1171; ++$_i1175)
+            $_size1147 = 0;
+            $_etype1150 = 0;
+            $xfer += $input->readListBegin($_etype1150, $_size1147);
+            for ($_i1151 = 0; $_i1151 < $_size1147; ++$_i1151)
             {
-              $elem1176 = null;
-              $elem1176 = new \metastore\Partition();
-              $xfer += $elem1176->read($input);
-              $this->success []= $elem1176;
+              $elem1152 = null;
+              $elem1152 = new \metastore\Partition();
+              $xfer += $elem1152->read($input);
+              $this->success []= $elem1152;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32190,9 +32138,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1177)
+          foreach ($this->success as $iter1153)
           {
-            $xfer += $iter1177->write($output);
+            $xfer += $iter1153->write($output);
           }
         }
         $output->writeListEnd();
@@ -32313,14 +32261,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1178 = 0;
-            $_etype1181 = 0;
-            $xfer += $input->readListBegin($_etype1181, $_size1178);
-            for ($_i1182 = 0; $_i1182 < $_size1178; ++$_i1182)
+            $_size1154 = 0;
+            $_etype1157 = 0;
+            $xfer += $input->readListBegin($_etype1157, $_size1154);
+            for ($_i1158 = 0; $_i1158 < $_size1154; ++$_i1158)
             {
-              $elem1183 = null;
-              $xfer += $input->readString($elem1183);
-              $this->part_vals []= $elem1183;
+              $elem1159 = null;
+              $xfer += $input->readString($elem1159);
+              $this->part_vals []= $elem1159;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32365,9 +32313,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1184)
+          foreach ($this->part_vals as $iter1160)
           {
-            $xfer += $output->writeString($iter1184);
+            $xfer += $output->writeString($iter1160);
           }
         }
         $output->writeListEnd();
@@ -32460,14 +32408,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1185 = 0;
-            $_etype1188 = 0;
-            $xfer += $input->readListBegin($_etype1188, $_size1185);
-            for ($_i1189 = 0; $_i1189 < $_size1185; ++$_i1189)
+            $_size1161 = 0;
+            $_etype1164 = 0;
+            $xfer += $input->readListBegin($_etype1164, $_size1161);
+            for ($_i1165 = 0; $_i1165 < $_size1161; ++$_i1165)
             {
-              $elem1190 = null;
-              $xfer += $input->readString($elem1190);
-              $this->success []= $elem1190;
+              $elem1166 = null;
+              $xfer += $input->readString($elem1166);
+              $this->success []= $elem1166;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32511,9 +32459,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1191)
+          foreach ($this->success as $iter1167)
           {
-            $xfer += $output->writeString($iter1191);
+            $xfer += $output->writeString($iter1167);
           }
         }
         $output->writeListEnd();
@@ -32756,15 +32704,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1192 = 0;
-            $_etype1195 = 0;
-            $xfer += $input->readListBegin($_etype1195, $_size1192);
-            for ($_i1196 = 0; $_i1196 < $_size1192; ++$_i1196)
+            $_size1168 = 0;
+            $_etype1171 = 0;
+            $xfer += $input->readListBegin($_etype1171, $_size1168);
+            for ($_i1172 = 0; $_i1172 < $_size1168; ++$_i1172)
             {
-              $elem1197 = null;
-              $elem1197 = new \metastore\Partition();
-              $xfer += $elem1197->read($input);
-              $this->success []= $elem1197;
+              $elem1173 = null;
+              $elem1173 = new \metastore\Partition();
+              $xfer += $elem1173->read($input);
+              $this->success []= $elem1173;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32808,9 +32756,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1198)
+          foreach ($this->success as $iter1174)
           {
-            $xfer += $iter1198->write($output);
+            $xfer += $iter1174->write($output);
           }
         }
         $output->writeListEnd();
@@ -33053,15 +33001,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1199 = 0;
-            $_etype1202 = 0;
-            $xfer += $input->readListBegin($_etype1202, $_size1199);
-            for ($_i1203 = 0; $_i1203 < $_size1199; ++$_i1203)
+            $_size1175 = 0;
+            $_etype1178 = 0;
+            $xfer += $input->readListBegin($_etype1178, $_size1175);
+            for ($_i1179 = 0; $_i1179 < $_size1175; ++$_i1179)
             {
-              $elem1204 = null;
-              $elem1204 = new \metastore\PartitionSpec();
-              $xfer += $elem1204->read($input);
-              $this->success []= $elem1204;
+              $elem1180 = null;
+              $elem1180 = new \metastore\PartitionSpec();
+              $xfer += $elem1180->read($input);
+              $this->success []= $elem1180;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33105,9 +33053,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1205)
+          foreach ($this->success as $iter1181)
           {
-            $xfer += $iter1205->write($output);
+            $xfer += $iter1181->write($output);
           }
         }
         $output->writeListEnd();
@@ -33673,14 +33621,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size1206 = 0;
-            $_etype1209 = 0;
-            $xfer += $input->readListBegin($_etype1209, $_size1206);
-            for ($_i1210 = 0; $_i1210 < $_size1206; ++$_i1210)
+            $_size1182 = 0;
+            $_etype1185 = 0;
+            $xfer += $input->readListBegin($_etype1185, $_size1182);
+            for ($_i1186 = 0; $_i1186 < $_size1182; ++$_i1186)
             {
-              $elem1211 = null;
-              $xfer += $input->readString($elem1211);
-              $this->names []= $elem1211;
+              $elem1187 = null;
+              $xfer += $input->readString($elem1187);
+              $this->names []= $elem1187;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33718,9 +33666,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter1212)
+          foreach ($this->names as $iter1188)
           {
-            $xfer += $output->writeString($iter1212);
+            $xfer += $output->writeString($iter1188);
           }
         }
         $output->writeListEnd();
@@ -33809,15 +33757,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1213 = 0;
-            $_etype1216 = 0;
-            $xfer += $input->readListBegin($_etype1216, $_size1213);
-            for ($_i1217 = 0; $_i1217 < $_size1213; ++$_i1217)
+            $_size1189 = 0;
+            $_etype1192 = 0;
+            $xfer += $input->readListBegin($_etype1192, $_size1189);
+            for ($_i1193 = 0; $_i1193 < $_size1189; ++$_i1193)
             {
-              $elem1218 = null;
-              $elem1218 = new \metastore\Partition();
-              $xfer += $elem1218->read($input);
-              $this->success []= $elem1218;
+              $elem1194 = null;
+              $elem1194 = new \metastore\Partition();
+              $xfer += $elem1194->read($input);
+              $this->success []= $elem1194;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33861,9 +33809,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1219)
+          foreach ($this->success as $iter1195)
           {
-            $xfer += $iter1219->write($output);
+            $xfer += $iter1195->write($output);
           }
         }
         $output->writeListEnd();
@@ -34202,15 +34150,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1220 = 0;
-            $_etype1223 = 0;
-            $xfer += $input->readListBegin($_etype1223, $_size1220);
-            for ($_i1224 = 0; $_i1224 < $_size1220; ++$_i1224)
+            $_size1196 = 0;
+            $_etype1199 = 0;
+            $xfer += $input->readListBegin($_etype1199, $_size1196);
+            for ($_i1200 = 0; $_i1200 < $_size1196; ++$_i1200)
             {
-              $elem1225 = null;
-              $elem1225 = new \metastore\Partition();
-              $xfer += $elem1225->read($input);
-              $this->new_parts []= $elem1225;
+              $elem1201 = null;
+              $elem1201 = new \metastore\Partition();
+              $xfer += $elem1201->read($input);
+              $this->new_parts []= $elem1201;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34248,9 +34196,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1226)
+          foreach ($this->new_parts as $iter1202)
           {
-            $xfer += $iter1226->write($output);
+            $xfer += $iter1202->write($output);
           }
         }
         $output->writeListEnd();
@@ -34465,15 +34413,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1227 = 0;
-            $_etype1230 = 0;
-            $xfer += $input->readListBegin($_etype1230, $_size1227);
-            for ($_i1231 = 0; $_i1231 < $_size1227; ++$_i1231)
+            $_size1203 = 0;
+            $_etype1206 = 0;
+            $xfer += $input->readListBegin($_etype1206, $_size1203);
+            for ($_i1207 = 0; $_i1207 < $_size1203; ++$_i1207)
             {
-              $elem1232 = null;
-              $elem1232 = new \metastore\Partition();
-              $xfer += $elem1232->read($input);
-              $this->new_parts []= $elem1232;
+              $elem1208 = null;
+              $elem1208 = new \metastore\Partition();
+              $xfer += $elem1208->read($input);
+              $this->new_parts []= $elem1208;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34519,9 +34467,9 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1233)
+          foreach ($this->new_parts as $iter1209)
           {
-            $xfer += $iter1233->write($output);
+            $xfer += $iter1209->write($output);
           }
         }
         $output->writeListEnd();
@@ -34999,14 +34947,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1234 = 0;
-            $_etype1237 = 0;
-            $xfer += $input->readListBegin($_etype1237, $_size1234);
-            for ($_i1238 = 0; $_i1238 < $_size1234; ++$_i1238)
+            $_size1210 = 0;
+            $_etype1213 = 0;
+            $xfer += $input->readListBegin($_etype1213, $_size1210);
+            for ($_i1214 = 0; $_i1214 < $_size1210; ++$_i1214)
             {
-              $elem1239 = null;
-              $xfer += $input->readString($elem1239);
-              $this->part_vals []= $elem1239;
+              $elem1215 = null;
+              $xfer += $input->readString($elem1215);
+              $this->part_vals []= $elem1215;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35052,9 +35000,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1240)
+          foreach ($this->part_vals as $iter1216)
           {
-            $xfer += $output->writeString($iter1240);
+            $xfer += $output->writeString($iter1216);
           }
         }
         $output->writeListEnd();
@@ -35239,14 +35187,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1241 = 0;
-            $_etype1244 = 0;
-            $xfer += $input->readListBegin($_etype1244, $_size1241);
-            for ($_i1245 = 0; $_i1245 < $_size1241; ++$_i1245)
+            $_size1217 = 0;
+            $_etype1220 = 0;
+            $xfer += $input->readListBegin($_etype1220, $_size1217);
+            for ($_i1221 = 0; $_i1221 < $_size1217; ++$_i1221)
             {
-              $elem1246 = null;
-              $xfer += $input->readString($elem1246);
-              $this->part_vals []= $elem1246;
+              $elem1222 = null;
+              $xfer += $input->readString($elem1222);
+              $this->part_vals []= $elem1222;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35281,9 +35229,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1247)
+          foreach ($this->part_vals as $iter1223)
           {
-            $xfer += $output->writeString($iter1247);
+            $xfer += $output->writeString($iter1223);
           }
         }
         $output->writeListEnd();
@@ -35737,14 +35685,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype 

<TRUNCATED>

[04/12] hive git commit: HIVE-20006: Make materializations invalidation cache work with multiple active remote metastores (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
index 93b5780..ae12471 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
@@ -20999,6 +20999,10 @@ class CreationMetadata {
    * @var string
    */
   public $validTxnList = null;
+  /**
+   * @var int
+   */
+  public $materializationTime = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -21027,6 +21031,10 @@ class CreationMetadata {
           'var' => 'validTxnList',
           'type' => TType::STRING,
           ),
+        6 => array(
+          'var' => 'materializationTime',
+          'type' => TType::I64,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -21045,6 +21053,9 @@ class CreationMetadata {
       if (isset($vals['validTxnList'])) {
         $this->validTxnList = $vals['validTxnList'];
       }
+      if (isset($vals['materializationTime'])) {
+        $this->materializationTime = $vals['materializationTime'];
+      }
     }
   }
 
@@ -21116,6 +21127,13 @@ class CreationMetadata {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 6:
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->materializationTime);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -21170,6 +21188,11 @@ class CreationMetadata {
       $xfer += $output->writeString($this->validTxnList);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->materializationTime !== null) {
+      $xfer += $output->writeFieldBegin('materializationTime', TType::I64, 6);
+      $xfer += $output->writeI64($this->materializationTime);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -24987,18 +25010,6 @@ class Materialization {
   static $_TSPEC;
 
   /**
-   * @var string[]
-   */
-  public $tablesUsed = null;
-  /**
-   * @var string
-   */
-  public $validTxnList = null;
-  /**
-   * @var int
-   */
-  public $invalidationTime = null;
-  /**
    * @var bool
    */
   public $sourceTablesUpdateDeleteModified = null;
@@ -25007,37 +25018,12 @@ class Materialization {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'tablesUsed',
-          'type' => TType::SET,
-          'etype' => TType::STRING,
-          'elem' => array(
-            'type' => TType::STRING,
-            ),
-          ),
-        2 => array(
-          'var' => 'validTxnList',
-          'type' => TType::STRING,
-          ),
-        3 => array(
-          'var' => 'invalidationTime',
-          'type' => TType::I64,
-          ),
-        4 => array(
           'var' => 'sourceTablesUpdateDeleteModified',
           'type' => TType::BOOL,
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['tablesUsed'])) {
-        $this->tablesUsed = $vals['tablesUsed'];
-      }
-      if (isset($vals['validTxnList'])) {
-        $this->validTxnList = $vals['validTxnList'];
-      }
-      if (isset($vals['invalidationTime'])) {
-        $this->invalidationTime = $vals['invalidationTime'];
-      }
       if (isset($vals['sourceTablesUpdateDeleteModified'])) {
         $this->sourceTablesUpdateDeleteModified = $vals['sourceTablesUpdateDeleteModified'];
       }
@@ -25064,41 +25050,6 @@ class Materialization {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::SET) {
-            $this->tablesUsed = array();
-            $_size763 = 0;
-            $_etype766 = 0;
-            $xfer += $input->readSetBegin($_etype766, $_size763);
-            for ($_i767 = 0; $_i767 < $_size763; ++$_i767)
-            {
-              $elem768 = null;
-              $xfer += $input->readString($elem768);
-              if (is_scalar($elem768)) {
-                $this->tablesUsed[$elem768] = true;
-              } else {
-                $this->tablesUsed []= $elem768;
-              }
-            }
-            $xfer += $input->readSetEnd();
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 2:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->validTxnList);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 3:
-          if ($ftype == TType::I64) {
-            $xfer += $input->readI64($this->invalidationTime);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 4:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->sourceTablesUpdateDeleteModified);
           } else {
@@ -25118,39 +25069,8 @@ class Materialization {
   public function write($output) {
     $xfer = 0;
     $xfer += $output->writeStructBegin('Materialization');
-    if ($this->tablesUsed !== null) {
-      if (!is_array($this->tablesUsed)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('tablesUsed', TType::SET, 1);
-      {
-        $output->writeSetBegin(TType::STRING, count($this->tablesUsed));
-        {
-          foreach ($this->tablesUsed as $iter769 => $iter770)
-          {
-            if (is_scalar($iter770)) {
-            $xfer += $output->writeString($iter769);
-            } else {
-            $xfer += $output->writeString($iter770);
-            }
-          }
-        }
-        $output->writeSetEnd();
-      }
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->validTxnList !== null) {
-      $xfer += $output->writeFieldBegin('validTxnList', TType::STRING, 2);
-      $xfer += $output->writeString($this->validTxnList);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->invalidationTime !== null) {
-      $xfer += $output->writeFieldBegin('invalidationTime', TType::I64, 3);
-      $xfer += $output->writeI64($this->invalidationTime);
-      $xfer += $output->writeFieldEnd();
-    }
     if ($this->sourceTablesUpdateDeleteModified !== null) {
-      $xfer += $output->writeFieldBegin('sourceTablesUpdateDeleteModified', TType::BOOL, 4);
+      $xfer += $output->writeFieldBegin('sourceTablesUpdateDeleteModified', TType::BOOL, 1);
       $xfer += $output->writeBool($this->sourceTablesUpdateDeleteModified);
       $xfer += $output->writeFieldEnd();
     }
@@ -26403,15 +26323,15 @@ class WMFullResourcePlan {
         case 2:
           if ($ftype == TType::LST) {
             $this->pools = array();
-            $_size771 = 0;
-            $_etype774 = 0;
-            $xfer += $input->readListBegin($_etype774, $_size771);
-            for ($_i775 = 0; $_i775 < $_size771; ++$_i775)
+            $_size763 = 0;
+            $_etype766 = 0;
+            $xfer += $input->readListBegin($_etype766, $_size763);
+            for ($_i767 = 0; $_i767 < $_size763; ++$_i767)
             {
-              $elem776 = null;
-              $elem776 = new \metastore\WMPool();
-              $xfer += $elem776->read($input);
-              $this->pools []= $elem776;
+              $elem768 = null;
+              $elem768 = new \metastore\WMPool();
+              $xfer += $elem768->read($input);
+              $this->pools []= $elem768;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26421,15 +26341,15 @@ class WMFullResourcePlan {
         case 3:
           if ($ftype == TType::LST) {
             $this->mappings = array();
-            $_size777 = 0;
-            $_etype780 = 0;
-            $xfer += $input->readListBegin($_etype780, $_size777);
-            for ($_i781 = 0; $_i781 < $_size777; ++$_i781)
+            $_size769 = 0;
+            $_etype772 = 0;
+            $xfer += $input->readListBegin($_etype772, $_size769);
+            for ($_i773 = 0; $_i773 < $_size769; ++$_i773)
             {
-              $elem782 = null;
-              $elem782 = new \metastore\WMMapping();
-              $xfer += $elem782->read($input);
-              $this->mappings []= $elem782;
+              $elem774 = null;
+              $elem774 = new \metastore\WMMapping();
+              $xfer += $elem774->read($input);
+              $this->mappings []= $elem774;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26439,15 +26359,15 @@ class WMFullResourcePlan {
         case 4:
           if ($ftype == TType::LST) {
             $this->triggers = array();
-            $_size783 = 0;
-            $_etype786 = 0;
-            $xfer += $input->readListBegin($_etype786, $_size783);
-            for ($_i787 = 0; $_i787 < $_size783; ++$_i787)
+            $_size775 = 0;
+            $_etype778 = 0;
+            $xfer += $input->readListBegin($_etype778, $_size775);
+            for ($_i779 = 0; $_i779 < $_size775; ++$_i779)
             {
-              $elem788 = null;
-              $elem788 = new \metastore\WMTrigger();
-              $xfer += $elem788->read($input);
-              $this->triggers []= $elem788;
+              $elem780 = null;
+              $elem780 = new \metastore\WMTrigger();
+              $xfer += $elem780->read($input);
+              $this->triggers []= $elem780;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26457,15 +26377,15 @@ class WMFullResourcePlan {
         case 5:
           if ($ftype == TType::LST) {
             $this->poolTriggers = array();
-            $_size789 = 0;
-            $_etype792 = 0;
-            $xfer += $input->readListBegin($_etype792, $_size789);
-            for ($_i793 = 0; $_i793 < $_size789; ++$_i793)
+            $_size781 = 0;
+            $_etype784 = 0;
+            $xfer += $input->readListBegin($_etype784, $_size781);
+            for ($_i785 = 0; $_i785 < $_size781; ++$_i785)
             {
-              $elem794 = null;
-              $elem794 = new \metastore\WMPoolTrigger();
-              $xfer += $elem794->read($input);
-              $this->poolTriggers []= $elem794;
+              $elem786 = null;
+              $elem786 = new \metastore\WMPoolTrigger();
+              $xfer += $elem786->read($input);
+              $this->poolTriggers []= $elem786;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26501,9 +26421,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->pools));
         {
-          foreach ($this->pools as $iter795)
+          foreach ($this->pools as $iter787)
           {
-            $xfer += $iter795->write($output);
+            $xfer += $iter787->write($output);
           }
         }
         $output->writeListEnd();
@@ -26518,9 +26438,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->mappings));
         {
-          foreach ($this->mappings as $iter796)
+          foreach ($this->mappings as $iter788)
           {
-            $xfer += $iter796->write($output);
+            $xfer += $iter788->write($output);
           }
         }
         $output->writeListEnd();
@@ -26535,9 +26455,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->triggers));
         {
-          foreach ($this->triggers as $iter797)
+          foreach ($this->triggers as $iter789)
           {
-            $xfer += $iter797->write($output);
+            $xfer += $iter789->write($output);
           }
         }
         $output->writeListEnd();
@@ -26552,9 +26472,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->poolTriggers));
         {
-          foreach ($this->poolTriggers as $iter798)
+          foreach ($this->poolTriggers as $iter790)
           {
-            $xfer += $iter798->write($output);
+            $xfer += $iter790->write($output);
           }
         }
         $output->writeListEnd();
@@ -27107,15 +27027,15 @@ class WMGetAllResourcePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->resourcePlans = array();
-            $_size799 = 0;
-            $_etype802 = 0;
-            $xfer += $input->readListBegin($_etype802, $_size799);
-            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
+            $_size791 = 0;
+            $_etype794 = 0;
+            $xfer += $input->readListBegin($_etype794, $_size791);
+            for ($_i795 = 0; $_i795 < $_size791; ++$_i795)
             {
-              $elem804 = null;
-              $elem804 = new \metastore\WMResourcePlan();
-              $xfer += $elem804->read($input);
-              $this->resourcePlans []= $elem804;
+              $elem796 = null;
+              $elem796 = new \metastore\WMResourcePlan();
+              $xfer += $elem796->read($input);
+              $this->resourcePlans []= $elem796;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27143,9 +27063,9 @@ class WMGetAllResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->resourcePlans));
         {
-          foreach ($this->resourcePlans as $iter805)
+          foreach ($this->resourcePlans as $iter797)
           {
-            $xfer += $iter805->write($output);
+            $xfer += $iter797->write($output);
           }
         }
         $output->writeListEnd();
@@ -27551,14 +27471,14 @@ class WMValidateResourcePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->errors = array();
-            $_size806 = 0;
-            $_etype809 = 0;
-            $xfer += $input->readListBegin($_etype809, $_size806);
-            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
+            $_size798 = 0;
+            $_etype801 = 0;
+            $xfer += $input->readListBegin($_etype801, $_size798);
+            for ($_i802 = 0; $_i802 < $_size798; ++$_i802)
             {
-              $elem811 = null;
-              $xfer += $input->readString($elem811);
-              $this->errors []= $elem811;
+              $elem803 = null;
+              $xfer += $input->readString($elem803);
+              $this->errors []= $elem803;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27568,14 +27488,14 @@ class WMValidateResourcePlanResponse {
         case 2:
           if ($ftype == TType::LST) {
             $this->warnings = array();
-            $_size812 = 0;
-            $_etype815 = 0;
-            $xfer += $input->readListBegin($_etype815, $_size812);
-            for ($_i816 = 0; $_i816 < $_size812; ++$_i816)
+            $_size804 = 0;
+            $_etype807 = 0;
+            $xfer += $input->readListBegin($_etype807, $_size804);
+            for ($_i808 = 0; $_i808 < $_size804; ++$_i808)
             {
-              $elem817 = null;
-              $xfer += $input->readString($elem817);
-              $this->warnings []= $elem817;
+              $elem809 = null;
+              $xfer += $input->readString($elem809);
+              $this->warnings []= $elem809;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27603,9 +27523,9 @@ class WMValidateResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRING, count($this->errors));
         {
-          foreach ($this->errors as $iter818)
+          foreach ($this->errors as $iter810)
           {
-            $xfer += $output->writeString($iter818);
+            $xfer += $output->writeString($iter810);
           }
         }
         $output->writeListEnd();
@@ -27620,9 +27540,9 @@ class WMValidateResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRING, count($this->warnings));
         {
-          foreach ($this->warnings as $iter819)
+          foreach ($this->warnings as $iter811)
           {
-            $xfer += $output->writeString($iter819);
+            $xfer += $output->writeString($iter811);
           }
         }
         $output->writeListEnd();
@@ -28295,15 +28215,15 @@ class WMGetTriggersForResourePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->triggers = array();
-            $_size820 = 0;
-            $_etype823 = 0;
-            $xfer += $input->readListBegin($_etype823, $_size820);
-            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
+            $_size812 = 0;
+            $_etype815 = 0;
+            $xfer += $input->readListBegin($_etype815, $_size812);
+            for ($_i816 = 0; $_i816 < $_size812; ++$_i816)
             {
-              $elem825 = null;
-              $elem825 = new \metastore\WMTrigger();
-              $xfer += $elem825->read($input);
-              $this->triggers []= $elem825;
+              $elem817 = null;
+              $elem817 = new \metastore\WMTrigger();
+              $xfer += $elem817->read($input);
+              $this->triggers []= $elem817;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28331,9 +28251,9 @@ class WMGetTriggersForResourePlanResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->triggers));
         {
-          foreach ($this->triggers as $iter826)
+          foreach ($this->triggers as $iter818)
           {
-            $xfer += $iter826->write($output);
+            $xfer += $iter818->write($output);
           }
         }
         $output->writeListEnd();
@@ -29917,15 +29837,15 @@ class SchemaVersion {
         case 4:
           if ($ftype == TType::LST) {
             $this->cols = array();
-            $_size827 = 0;
-            $_etype830 = 0;
-            $xfer += $input->readListBegin($_etype830, $_size827);
-            for ($_i831 = 0; $_i831 < $_size827; ++$_i831)
+            $_size819 = 0;
+            $_etype822 = 0;
+            $xfer += $input->readListBegin($_etype822, $_size819);
+            for ($_i823 = 0; $_i823 < $_size819; ++$_i823)
             {
-              $elem832 = null;
-              $elem832 = new \metastore\FieldSchema();
-              $xfer += $elem832->read($input);
-              $this->cols []= $elem832;
+              $elem824 = null;
+              $elem824 = new \metastore\FieldSchema();
+              $xfer += $elem824->read($input);
+              $this->cols []= $elem824;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30014,9 +29934,9 @@ class SchemaVersion {
       {
         $output->writeListBegin(TType::STRUCT, count($this->cols));
         {
-          foreach ($this->cols as $iter833)
+          foreach ($this->cols as $iter825)
           {
-            $xfer += $iter833->write($output);
+            $xfer += $iter825->write($output);
           }
         }
         $output->writeListEnd();
@@ -30338,15 +30258,15 @@ class FindSchemasByColsResp {
         case 1:
           if ($ftype == TType::LST) {
             $this->schemaVersions = array();
-            $_size834 = 0;
-            $_etype837 = 0;
-            $xfer += $input->readListBegin($_etype837, $_size834);
-            for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
+            $_size826 = 0;
+            $_etype829 = 0;
+            $xfer += $input->readListBegin($_etype829, $_size826);
+            for ($_i830 = 0; $_i830 < $_size826; ++$_i830)
             {
-              $elem839 = null;
-              $elem839 = new \metastore\SchemaVersionDescriptor();
-              $xfer += $elem839->read($input);
-              $this->schemaVersions []= $elem839;
+              $elem831 = null;
+              $elem831 = new \metastore\SchemaVersionDescriptor();
+              $xfer += $elem831->read($input);
+              $this->schemaVersions []= $elem831;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30374,9 +30294,9 @@ class FindSchemasByColsResp {
       {
         $output->writeListBegin(TType::STRUCT, count($this->schemaVersions));
         {
-          foreach ($this->schemaVersions as $iter840)
+          foreach ($this->schemaVersions as $iter832)
           {
-            $xfer += $iter840->write($output);
+            $xfer += $iter832->write($output);
           }
         }
         $output->writeListEnd();

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index add9197..e5d943d 100755
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -67,7 +67,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('   get_table_objects_by_name(string dbname,  tbl_names)')
   print('  GetTableResult get_table_req(GetTableRequest req)')
   print('  GetTablesResult get_table_objects_by_name_req(GetTablesRequest req)')
-  print('   get_materialization_invalidation_info(string dbname,  tbl_names)')
+  print('  Materialization get_materialization_invalidation_info(CreationMetadata creation_metadata, string validTxnList)')
   print('  void update_creation_metadata(string catName, string dbname, string tbl_name, CreationMetadata creation_metadata)')
   print('   get_table_names_by_filter(string dbname, string filter, i16 max_tables)')
   print('  void alter_table(string dbname, string tbl_name, Table new_tbl)')
@@ -563,7 +563,7 @@ elif cmd == 'get_materialization_invalidation_info':
   if len(args) != 2:
     print('get_materialization_invalidation_info requires 2 args')
     sys.exit(1)
-  pp.pprint(client.get_materialization_invalidation_info(args[0],eval(args[1]),))
+  pp.pprint(client.get_materialization_invalidation_info(eval(args[0]),args[1],))
 
 elif cmd == 'update_creation_metadata':
   if len(args) != 4:


[12/12] hive git commit: HIVE-20006: Make materializations invalidation cache work with multiple active remote metastores (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
HIVE-20006: Make materializations invalidation cache work with multiple active remote metastores (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master
Commit: 1b5903b035c3b3ac02efbddf36d5438cda97cc91
Parents: ab9e954
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Tue Jun 26 11:37:27 2018 -0700
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Fri Jul 13 23:06:53 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   26 +-
 .../hive/ql/exec/MaterializedViewTask.java      |    2 -
 .../apache/hadoop/hive/ql/metadata/Hive.java    |  124 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   21 +-
 ...terialized_view_create_rewrite_time_window.q |    4 +-
 .../clientpositive/druid/druidmini_mv.q.out     |   85 +-
 .../materialized_view_create_rewrite_5.q.out    |    4 +-
 ...alized_view_create_rewrite_time_window.q.out |   16 +-
 .../llap/materialized_view_rewrite_empty.q.out  |    4 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2351 +++++++-------
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |   52 +-
 .../ThriftHiveMetastore_server.skeleton.cpp     |    2 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 1442 +++++----
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   45 +-
 .../hive/metastore/api/CreationMetadata.java    |  111 +-
 .../metastore/api/FindSchemasByColsResp.java    |   36 +-
 .../hive/metastore/api/Materialization.java     |  409 +--
 .../hive/metastore/api/SchemaVersion.java       |   36 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 2858 +++++++++---------
 .../hive/metastore/api/WMFullResourcePlan.java  |  144 +-
 .../api/WMGetAllResourcePlanResponse.java       |   36 +-
 .../WMGetTriggersForResourePlanResponse.java    |   36 +-
 .../api/WMValidateResourcePlanResponse.java     |   64 +-
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1510 +++++----
 .../src/gen/thrift/gen-php/metastore/Types.php  |  324 +-
 .../hive_metastore/ThriftHiveMetastore-remote   |    4 +-
 .../hive_metastore/ThriftHiveMetastore.py       | 1015 +++----
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  208 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   16 +-
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   20 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |   10 +-
 .../hive/metastore/HiveMetaStoreClient.java     |    7 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |    2 +-
 .../MaterializationsCacheCleanerTask.java       |   63 -
 .../MaterializationsInvalidationCache.java      |  543 ----
 .../MaterializationsRebuildLockCleanerTask.java |   30 +-
 .../hadoop/hive/metastore/ObjectStore.java      |   20 +-
 .../hive/metastore/conf/MetastoreConf.java      |    6 +-
 .../hive/metastore/model/MCreationMetadata.java |   16 +-
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    |   13 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |  309 +-
 .../hadoop/hive/metastore/txn/TxnStore.java     |   38 +-
 .../main/sql/derby/hive-schema-3.1.0.derby.sql  |   14 +-
 .../main/sql/derby/hive-schema-4.0.0.derby.sql  |   14 +-
 .../sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql  |   19 +
 .../main/sql/mssql/hive-schema-3.1.0.mssql.sql  |   17 +-
 .../main/sql/mssql/hive-schema-4.0.0.mssql.sql  |   18 +-
 .../sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql  |   19 +
 .../main/sql/mysql/hive-schema-3.1.0.mysql.sql  |   12 +-
 .../main/sql/mysql/hive-schema-4.0.0.mysql.sql  |   12 +-
 .../sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql  |   20 +
 .../sql/oracle/hive-schema-3.1.0.oracle.sql     |   14 +-
 .../sql/oracle/hive-schema-4.0.0.oracle.sql     |   14 +-
 .../oracle/upgrade-3.0.0-to-3.1.0.oracle.sql    |   19 +
 .../sql/postgres/hive-schema-3.1.0.postgres.sql |   14 +-
 .../sql/postgres/hive-schema-4.0.0.postgres.sql |   26 +-
 .../upgrade-3.0.0-to-3.1.0.postgres.sql         |   19 +
 .../src/main/thrift/hive_metastore.thrift       |    8 +-
 .../HiveMetaStoreClientPreCatalog.java          |    7 +-
 ...stMetaStoreMaterializationsCacheCleaner.java |  328 --
 .../TestTablesCreateDropAlterTruncate.java      |    1 +
 61 files changed, 5720 insertions(+), 6937 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 41fae36..858c630 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1633,13 +1633,13 @@ public class HiveConf extends Configuration {
             "choosing the plan with lower cost among possible plans containing a materialized view\n" +
             "  costbased: Fully cost-based strategy, always use plan with lower cost, independently on whether " +
             "it uses a materialized view or not"),
-    HIVE_MATERIALIZED_VIEW_REWRITING_TIME_WINDOW("hive.materializedview.rewriting.time.window", "0s", new TimeValidator(TimeUnit.SECONDS),
+    HIVE_MATERIALIZED_VIEW_REWRITING_TIME_WINDOW("hive.materializedview.rewriting.time.window", "0min", new TimeValidator(TimeUnit.MINUTES),
         "Time window, specified in seconds, after which outdated materialized views become invalid for automatic query rewriting.\n" +
-        "For instance, if a materialized view is created and afterwards one of its source tables is changed at " +
-        "moment in time t0, the materialized view will not be considered for rewriting anymore after t0 plus " +
-        "the value assigned to this property. Default value 0 means that the materialized view cannot be " +
-        "outdated to be used automatically in query rewriting."),
-    HIVE_MATERIALIZED_VIEW_REWRITING_INCREMENTAL("hive.materializedview.rewriting.incremental", true,
+        "For instance, if more time than the value assigned to the property has passed since the materialized view " +
+        "was created or rebuilt, and one of its source tables has changed since, the materialized view will not be " +
+        "considered for rewriting. Default value 0 means that the materialized view cannot be " +
+        "outdated to be used automatically in query rewriting. Value -1 means to skip this check."),
+    HIVE_MATERIALIZED_VIEW_REWRITING_INCREMENTAL("hive.materializedview.rewriting.incremental", false,
         "Whether to try to execute incremental rewritings based on outdated materializations and\n" +
         "current content of tables. Default value of true effectively amounts to enabling incremental\n" +
         "rebuild for the materializations too."),
@@ -1647,24 +1647,12 @@ public class HiveConf extends Configuration {
         "Whether to try to execute incremental rebuild for the materialized views. Incremental rebuild\n" +
         "tries to modify the original materialization contents to reflect the latest changes to the\n" +
         "materialized view source tables, instead of rebuilding the contents fully. Incremental rebuild\n" +
-        "is based on the materialized view algebraic incremental rewriting. Hence, this requires\n" +
-        "hive.materializedview.rewriting.incremental to be true."),
+        "is based on the materialized view algebraic incremental rewriting."),
     HIVE_MATERIALIZED_VIEW_FILE_FORMAT("hive.materializedview.fileformat", "ORC",
         new StringSet("none", "TextFile", "SequenceFile", "RCfile", "ORC"),
         "Default file format for CREATE MATERIALIZED VIEW statement"),
     HIVE_MATERIALIZED_VIEW_SERDE("hive.materializedview.serde",
         "org.apache.hadoop.hive.ql.io.orc.OrcSerde", "Default SerDe used for materialized views"),
-    HIVE_MATERIALIZATIONS_INVALIDATION_CACHE_IMPL("hive.metastore.materializations.invalidation.impl", "DEFAULT",
-        new StringSet("DEFAULT", "DISABLE"),
-        "The implementation that we should use for the materializations invalidation cache. \n" +
-            "  DEFAULT: Default implementation for invalidation cache\n" +
-            "  DISABLE: Disable invalidation cache (debugging purposes)"),
-    HIVE_MATERIALIZATIONS_INVALIDATION_CACHE_CLEAN_FREQUENCY("hive.metastore.materializations.invalidation.clean.frequency",
-        "3600s", new TimeValidator(TimeUnit.SECONDS), "Frequency at which timer task runs to remove unnecessary transactions information from" +
-        "materializations invalidation cache."),
-    HIVE_MATERIALIZATIONS_INVALIDATION_CACHE_EXPIRY_DURATION("hive.metastore.materializations.invalidation.max.duration",
-        "86400s", new TimeValidator(TimeUnit.SECONDS), "Maximum duration for query producing a materialization. After this time, transactions" +
-        "information that is not relevant for materializations can be removed from invalidation cache."),
 
     // hive.mapjoin.bucket.cache.size has been replaced by hive.smbjoin.cache.row,
     // need to remove by hive .13. Also, do not change default (see SMB operator)

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/ql/src/java/org/apache/hadoop/hive/ql/exec/MaterializedViewTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MaterializedViewTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MaterializedViewTask.java
index 19aef6c..87828b1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MaterializedViewTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MaterializedViewTask.java
@@ -19,9 +19,7 @@
 package org.apache.hadoop.hive.ql.exec;
 
 import com.google.common.collect.ImmutableSet;
-import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
-import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.DriverContext;

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/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 cd62d49..1fe1fb6 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
@@ -84,7 +84,9 @@ import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.HiveStatsUtils;
 import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
+import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.hive.common.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.hive.common.log.InPlaceUpdate;
@@ -193,6 +195,7 @@ import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.common.util.TxnIdUtils;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -1332,7 +1335,7 @@ public class Hive {
    * @return the list of materialized views available for rewriting
    * @throws HiveException
    */
-  public List<RelOptMaterialization> getAllValidMaterializedViews(boolean forceMVContentsUpToDate, String validTxnsList)
+  public List<RelOptMaterialization> getAllValidMaterializedViews(List<String> tablesUsed, boolean forceMVContentsUpToDate)
       throws HiveException {
     // Final result
     List<RelOptMaterialization> result = new ArrayList<>();
@@ -1344,7 +1347,7 @@ public class Hive {
           // Bail out: empty list
           continue;
         }
-        result.addAll(getValidMaterializedViews(dbName, materializedViewNames, forceMVContentsUpToDate, validTxnsList));
+        result.addAll(getValidMaterializedViews(dbName, materializedViewNames, tablesUsed, forceMVContentsUpToDate));
       }
       return result;
     } catch (Exception e) {
@@ -1353,14 +1356,19 @@ public class Hive {
   }
 
   public List<RelOptMaterialization> getValidMaterializedView(String dbName, String materializedViewName,
-      boolean forceMVContentsUpToDate, String validTxnsList) throws HiveException {
-    return getValidMaterializedViews(dbName, ImmutableList.of(materializedViewName), forceMVContentsUpToDate, validTxnsList);
+      List<String> tablesUsed, boolean forceMVContentsUpToDate) throws HiveException {
+    return getValidMaterializedViews(dbName, ImmutableList.of(materializedViewName), tablesUsed, forceMVContentsUpToDate);
   }
 
   private List<RelOptMaterialization> getValidMaterializedViews(String dbName, List<String> materializedViewNames,
-      boolean forceMVContentsUpToDate, String validTxnsList) throws HiveException {
+      List<String> tablesUsed, boolean forceMVContentsUpToDate) throws HiveException {
+    final String validTxnsList = conf.get(ValidTxnList.VALID_TXNS_KEY);
+    final ValidTxnWriteIdList currentTxnWriteIds =
+        SessionState.get().getTxnMgr().getValidWriteIds(tablesUsed, validTxnsList);
     final boolean tryIncrementalRewriting =
         HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_REWRITING_INCREMENTAL);
+    final boolean tryIncrementalRebuild =
+        HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_REBUILD_INCREMENTAL);
     final long defaultDiff =
         HiveConf.getTimeVar(conf, HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_REWRITING_TIME_WINDOW,
             TimeUnit.MILLISECONDS);
@@ -1369,8 +1377,6 @@ public class Hive {
       // Final result
       List<RelOptMaterialization> result = new ArrayList<>();
       List<Table> materializedViewTables = getTableObjects(dbName, materializedViewNames);
-      Map<String, Materialization> databaseInvalidationInfo =
-          getMSC().getMaterializationsInvalidationInfo(dbName, materializedViewNames);
       for (Table materializedViewTable : materializedViewTables) {
         // Check if materialization defined its own invalidation time window
         String timeWindowString = materializedViewTable.getProperty(MATERIALIZED_VIEW_REWRITING_TIME_WINDOW);
@@ -1378,7 +1384,7 @@ public class Hive {
             HiveConf.toTime(timeWindowString,
                 HiveConf.getDefaultTimeUnit(HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_REWRITING_TIME_WINDOW),
                 TimeUnit.MILLISECONDS);
-        Materialization materializationInvInfo = null;
+        CreationMetadata creationMetadata = materializedViewTable.getCreationMetadata();
         boolean outdated = false;
         if (diff < 0L) {
           // We only consider the materialized view to be outdated if forceOutdated = true, i.e.,
@@ -1386,40 +1392,80 @@ public class Hive {
           outdated = forceMVContentsUpToDate;
         } else {
           // Check whether the materialized view is invalidated
-          materializationInvInfo =
-              databaseInvalidationInfo.get(materializedViewTable.getTableName());
-          if (materializationInvInfo == null) {
-            LOG.debug("Materialized view " + materializedViewTable.getFullyQualifiedName() +
-                " ignored for rewriting as there was no information loaded in the invalidation cache");
-            continue;
-          }
-          long invalidationTime = materializationInvInfo.getInvalidationTime();
-          if (invalidationTime == Long.MIN_VALUE) {
-            LOG.debug("Materialized view " + materializedViewTable.getFullyQualifiedName() +
-                " ignored for rewriting as it contains non-transactional tables");
-            continue;
-          }
-          // If the limit is not met, we do not add the materialized view.
-          // If we are doing a rebuild, we do not consider outdated materialized views either.
-          if (diff == 0L || forceMVContentsUpToDate) {
-            if (invalidationTime != 0L) {
-              outdated = true;
+          if (forceMVContentsUpToDate || diff == 0L || creationMetadata.getMaterializationTime() < currentTime - diff) {
+            if (currentTxnWriteIds == null) {
+              LOG.debug("Materialized view " + materializedViewTable.getFullyQualifiedName() +
+                  " ignored for rewriting as we could not obtain current txn ids");
+              continue;
             }
-          } else {
-            if (invalidationTime != 0L && invalidationTime > currentTime - diff) {
-              outdated = true;
+            if (creationMetadata.getValidTxnList() == null ||
+                creationMetadata.getValidTxnList().isEmpty()) {
+              LOG.debug("Materialized view " + materializedViewTable.getFullyQualifiedName() +
+                  " ignored for rewriting as we could not obtain materialization txn ids");
+              continue;
+            }
+            boolean ignore = false;
+            ValidTxnWriteIdList mvTxnWriteIds = new ValidTxnWriteIdList(
+                creationMetadata.getValidTxnList());
+            for (String qName : tablesUsed) {
+              // Note. If the materialized view does not contain a table that is contained in the query,
+              // we do not need to check whether that specific table is outdated or not. If a rewriting
+              // is produced in those cases, it is because that additional table is joined with the
+              // existing tables with an append-columns only join, i.e., PK-FK + not null.
+              if (!creationMetadata.getTablesUsed().contains(qName)) {
+                continue;
+              }
+              ValidWriteIdList tableCurrentWriteIds = currentTxnWriteIds.getTableValidWriteIdList(qName);
+              if (tableCurrentWriteIds == null) {
+                // Uses non-transactional table, cannot be considered
+                LOG.debug("Materialized view " + materializedViewTable.getFullyQualifiedName() +
+                    " ignored for rewriting as it is outdated and cannot be considered for " +
+                    " rewriting because it uses non-transactional table " + qName);
+                ignore = true;
+                break;
+              }
+              ValidWriteIdList tableWriteIds = mvTxnWriteIds.getTableValidWriteIdList(qName);
+              if (tableWriteIds == null) {
+                // This should not happen, but we ignore for safety
+                LOG.warn("Materialized view " + materializedViewTable.getFullyQualifiedName() +
+                    " ignored for rewriting as details about txn ids for table " + qName +
+                    " could not be found in " + mvTxnWriteIds);
+                ignore = true;
+                break;
+              }
+              if (!outdated && !TxnIdUtils.checkEquivalentWriteIds(tableCurrentWriteIds, tableWriteIds)) {
+                LOG.debug("Materialized view " + materializedViewTable.getFullyQualifiedName() +
+                    " contents are outdated");
+                outdated = true;
+              }
+            }
+            if (ignore) {
+              continue;
             }
           }
         }
 
-        if (outdated && (!tryIncrementalRewriting || materializationInvInfo == null
-            || validTxnsList == null || materializationInvInfo.isSourceTablesUpdateDeleteModified())) {
-          // We will not try partial rewriting either because the config specification, this
-          // is a rebuild over some non-transactional table, or there were update/delete
-          // operations in the source tables (not supported yet)
-          LOG.debug("Materialized view " + materializedViewTable.getFullyQualifiedName() +
-              " ignored for rewriting as its contents are outdated");
-          continue;
+        if (outdated) {
+          // The MV is outdated, see whether we should consider it for rewriting or not
+          boolean ignore = false;
+          if (forceMVContentsUpToDate && !tryIncrementalRebuild) {
+            // We will not try partial rewriting for rebuild if incremental rebuild is disabled
+            ignore = true;
+          } else if (!forceMVContentsUpToDate && !tryIncrementalRewriting) {
+            // We will not try partial rewriting for non-rebuild if incremental rewriting is disabled
+            ignore = true;
+          } else {
+            // Obtain additional information if we should try incremental rewriting / rebuild
+            // We will not try partial rewriting if there were update/delete operations on source tables
+            Materialization invalidationInfo = getMSC().getMaterializationInvalidationInfo(
+                creationMetadata, conf.get(ValidTxnList.VALID_TXNS_KEY));
+            ignore = invalidationInfo == null || invalidationInfo.isSourceTablesUpdateDeleteModified();
+          }
+          if (ignore) {
+            LOG.debug("Materialized view " + materializedViewTable.getFullyQualifiedName() +
+                " ignored for rewriting as its contents are outdated");
+            continue;
+          }
         }
 
         // It passed the test, load
@@ -1443,7 +1489,7 @@ public class Hive {
               // so we can produce partial rewritings
               materialization = augmentMaterializationWithTimeInformation(
                   materialization, validTxnsList, new ValidTxnWriteIdList(
-                      materializationInvInfo.getValidTxnList()));
+                      creationMetadata.getValidTxnList()));
             }
             result.add(materialization);
             continue;
@@ -1466,7 +1512,7 @@ public class Hive {
               // so we can produce partial rewritings
               materialization = augmentMaterializationWithTimeInformation(
                   materialization, validTxnsList, new ValidTxnWriteIdList(
-                      materializationInvInfo.getValidTxnList()));
+                      creationMetadata.getValidTxnList()));
             }
             result.add(materialization);
           }

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index e091f38..fecfd0c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -2081,18 +2081,17 @@ public class CalcitePlanner extends SemanticAnalyzer {
       // Add views to planner
       List<RelOptMaterialization> materializations = new ArrayList<>();
       try {
-        final String validTxnsList = conf.get(ValidTxnList.VALID_TXNS_KEY);
         if (mvRebuildMode != MaterializationRebuildMode.NONE) {
           // We only retrieve the materialization corresponding to the rebuild. In turn,
           // we pass 'true' for the forceMVContentsUpToDate parameter, as we cannot allow the
           // materialization contents to be stale for a rebuild if we want to use it.
           materializations = Hive.get().getValidMaterializedView(mvRebuildDbName, mvRebuildName,
-              true, validTxnsList);
+              getTablesUsed(basePlan), true);
         } else {
           // This is not a rebuild, we retrieve all the materializations. In turn, we do not need
           // to force the materialization contents to be up-to-date, as this is not a rebuild, and
           // we apply the user parameters (HIVE_MATERIALIZED_VIEW_REWRITING_TIME_WINDOW) instead.
-          materializations = Hive.get().getAllValidMaterializedViews(false, validTxnsList);
+          materializations = Hive.get().getAllValidMaterializedViews(getTablesUsed(basePlan), false);
         }
         // We need to use the current cluster for the scan operator on views,
         // otherwise the planner will throw an Exception (different planners)
@@ -2169,7 +2168,6 @@ public class CalcitePlanner extends SemanticAnalyzer {
           // A rewriting was produced, we will check whether it was part of an incremental rebuild
           // to try to replace INSERT OVERWRITE by INSERT
           if (mvRebuildMode == MaterializationRebuildMode.INSERT_OVERWRITE_REBUILD &&
-              HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_REWRITING_INCREMENTAL) &&
               HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_REBUILD_INCREMENTAL)) {
             // First we need to check if it is valid to convert to MERGE/INSERT INTO.
             // If we succeed, we modify the plan and afterwards the AST.
@@ -2196,6 +2194,21 @@ public class CalcitePlanner extends SemanticAnalyzer {
       return basePlan;
     }
 
+    private List<String> getTablesUsed(RelNode plan) {
+      List<String> tablesUsed = new ArrayList<>();
+      new RelVisitor() {
+        @Override
+        public void visit(RelNode node, int ordinal, RelNode parent) {
+          if (node instanceof TableScan) {
+            TableScan ts = (TableScan) node;
+            tablesUsed.add(((RelOptHiveTable) ts.getTable()).getHiveTableMD().getFullyQualifiedName());
+          }
+          super.visit(node, ordinal, parent);
+        }
+      }.go(plan);
+      return tablesUsed;
+    }
+
     /**
      * Run the HEP Planner with the given rule set.
      *

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window.q
index c61730e..55c6c04 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window.q
@@ -25,13 +25,13 @@ analyze table cmv_basetable_2_n1 compute statistics for columns;
 
 -- CREATE VIEW WITH REWRITE DISABLED
 EXPLAIN
-CREATE MATERIALIZED VIEW cmv_mat_view_n3 DISABLE REWRITE TBLPROPERTIES('rewriting.time.window'='300s') AS
+CREATE MATERIALIZED VIEW cmv_mat_view_n3 DISABLE REWRITE TBLPROPERTIES('rewriting.time.window'='5min') AS
   SELECT cmv_basetable_n3.a, cmv_basetable_2_n1.c
   FROM cmv_basetable_n3 JOIN cmv_basetable_2_n1 ON (cmv_basetable_n3.a = cmv_basetable_2_n1.a)
   WHERE cmv_basetable_2_n1.c > 10.0
   GROUP BY cmv_basetable_n3.a, cmv_basetable_2_n1.c;
 
-CREATE MATERIALIZED VIEW cmv_mat_view_n3 DISABLE REWRITE TBLPROPERTIES('rewriting.time.window'='300s') AS
+CREATE MATERIALIZED VIEW cmv_mat_view_n3 DISABLE REWRITE TBLPROPERTIES('rewriting.time.window'='5min') AS
   SELECT cmv_basetable_n3.a, cmv_basetable_2_n1.c
   FROM cmv_basetable_n3 JOIN cmv_basetable_2_n1 ON (cmv_basetable_n3.a = cmv_basetable_2_n1.a)
   WHERE cmv_basetable_2_n1.c > 10.0

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/ql/src/test/results/clientpositive/druid/druidmini_mv.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid/druidmini_mv.q.out b/ql/src/test/results/clientpositive/druid/druidmini_mv.q.out
index 383f2dc..54a4ef6 100644
--- a/ql/src/test/results/clientpositive/druid/druidmini_mv.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidmini_mv.q.out
@@ -162,31 +162,35 @@ STAGE PLANS:
       limit: -1
       Processor Tree:
         TableScan
-          alias: cmv_basetable_n2
-          filterExpr: (a = 3) (type: boolean)
-          Filter Operator
-            predicate: (a = 3) (type: boolean)
-            Select Operator
-              expressions: 3 (type: int), c (type: double)
-              outputColumnNames: _col0, _col1
-              ListSink
+          alias: cmv_mat_view2_n0
+          properties:
+            druid.fieldNames vc,c
+            druid.fieldTypes int,double
+            druid.query.json {"queryType":"scan","dataSource":"default.cmv_mat_view2_n0","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"virtualColumns":[{"type":"expression","name":"vc","expression":"3","outputType":"LONG"}],"columns":["vc","c"],"resultFormat":"compactedList"}
+            druid.query.type scan
+          Select Operator
+            expressions: vc (type: int), c (type: double)
+            outputColumnNames: _col0, _col1
+            ListSink
 
 PREHOOK: query: SELECT a, c
 FROM cmv_basetable_n2
 WHERE a = 3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@cmv_basetable_n2
+PREHOOK: Input: default@cmv_mat_view2_n0
 PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: SELECT a, c
 FROM cmv_basetable_n2
 WHERE a = 3
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@cmv_basetable_n2
+POSTHOOK: Input: default@cmv_mat_view2_n0
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-3	15.8
-3	9.8
-3	978.76
-Warning: Shuffle Join MERGEJOIN[13][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+3	15.800000190734863
+3	9.800000190734863
+3	978.760009765625
+Warning: Shuffle Join MERGEJOIN[10][tables = [cmv_mat_view2_n0, $hdt$_0]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT * FROM (
   (SELECT a, c FROM cmv_basetable_n2 WHERE a = 3) table1
@@ -217,36 +221,33 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: cmv_basetable_n2
-                  filterExpr: (a = 3) (type: boolean)
-                  Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: (a = 3) (type: boolean)
-                    Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: c (type: double)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        sort order: 
-                        Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: double)
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: cmv_basetable_n2
-                  filterExpr: ((d = 3) and (a = 3)) (type: boolean)
+                  filterExpr: ((a = 3) and (d = 3)) (type: boolean)
                   Statistics: Num rows: 6 Data size: 96 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: ((a = 3) and (d = 3)) (type: boolean)
                     Statistics: Num rows: 6 Data size: 96 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: c (type: double)
-                      outputColumnNames: _col0
+                      outputColumnNames: _col1
                       Statistics: Num rows: 6 Data size: 96 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         sort order: 
                         Statistics: Num rows: 6 Data size: 96 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: double)
+                        value expressions: _col1 (type: double)
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: cmv_mat_view2_n0
+                  properties:
+                    druid.fieldNames a,c
+                    druid.fieldTypes int,double
+                    druid.query.json {"queryType":"scan","dataSource":"default.cmv_mat_view2_n0","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"columns":["a","c"],"resultFormat":"compactedList"}
+                    druid.query.type scan
+                  Statistics: Num rows: 3 Data size: 36 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 3 Data size: 36 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: a (type: int), c (type: double)
         Reducer 2 
             Reduce Operator Tree:
               Merge Join Operator
@@ -255,15 +256,15 @@ STAGE PLANS:
                 keys:
                   0 
                   1 
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 36 Data size: 1044 Basic stats: COMPLETE Column stats: NONE
+                outputColumnNames: _col0, _col1, _col6
+                Statistics: Num rows: 18 Data size: 522 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: 3 (type: int), _col0 (type: double), 3 (type: int), _col1 (type: double)
+                  expressions: _col0 (type: int), _col1 (type: double), _col0 (type: int), _col6 (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 36 Data size: 1044 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 18 Data size: 522 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 36 Data size: 1044 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 18 Data size: 522 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -275,7 +276,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[13][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[10][tables = [cmv_mat_view2_n0, $hdt$_0]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: SELECT * FROM (
   (SELECT a, c FROM cmv_basetable_n2 WHERE a = 3) table1
   JOIN
@@ -283,6 +284,7 @@ PREHOOK: query: SELECT * FROM (
   ON table1.a = table2.a)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@cmv_basetable_n2
+PREHOOK: Input: default@cmv_mat_view2_n0
 PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: SELECT * FROM (
   (SELECT a, c FROM cmv_basetable_n2 WHERE a = 3) table1
@@ -291,10 +293,11 @@ POSTHOOK: query: SELECT * FROM (
   ON table1.a = table2.a)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@cmv_basetable_n2
+POSTHOOK: Input: default@cmv_mat_view2_n0
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-3	15.8	3	978.76
-3	9.8	3	978.76
-3	978.76	3	978.76
+3	15.800000190734863	3	978.76
+3	9.800000190734863	3	978.76
+3	978.760009765625	3	978.76
 PREHOOK: query: INSERT INTO cmv_basetable_n2 VALUES
  (cast(current_timestamp() AS timestamp), 3, 'charlie', 'charlie_c', 15.8, 1)
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out
index 2c4ee3d..99832ff 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out
@@ -180,12 +180,12 @@ STAGE PLANS:
                     Select Operator
                       expressions: a (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: may be used (ACID table)
         Reducer 2 

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window.q.out
index 44a866d..68e7500 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window.q.out
@@ -73,14 +73,14 @@ POSTHOOK: Input: default@cmv_basetable_2_n1
 POSTHOOK: Output: default@cmv_basetable_2_n1
 #### A masked pattern was here ####
 PREHOOK: query: EXPLAIN
-CREATE MATERIALIZED VIEW cmv_mat_view_n3 DISABLE REWRITE TBLPROPERTIES('rewriting.time.window'='300s') AS
+CREATE MATERIALIZED VIEW cmv_mat_view_n3 DISABLE REWRITE TBLPROPERTIES('rewriting.time.window'='5min') AS
   SELECT cmv_basetable_n3.a, cmv_basetable_2_n1.c
   FROM cmv_basetable_n3 JOIN cmv_basetable_2_n1 ON (cmv_basetable_n3.a = cmv_basetable_2_n1.a)
   WHERE cmv_basetable_2_n1.c > 10.0
   GROUP BY cmv_basetable_n3.a, cmv_basetable_2_n1.c
 PREHOOK: type: CREATE_MATERIALIZED_VIEW
 POSTHOOK: query: EXPLAIN
-CREATE MATERIALIZED VIEW cmv_mat_view_n3 DISABLE REWRITE TBLPROPERTIES('rewriting.time.window'='300s') AS
+CREATE MATERIALIZED VIEW cmv_mat_view_n3 DISABLE REWRITE TBLPROPERTIES('rewriting.time.window'='5min') AS
   SELECT cmv_basetable_n3.a, cmv_basetable_2_n1.c
   FROM cmv_basetable_n3 JOIN cmv_basetable_2_n1 ON (cmv_basetable_n3.a = cmv_basetable_2_n1.a)
   WHERE cmv_basetable_2_n1.c > 10.0
@@ -190,7 +190,7 @@ STAGE PLANS:
         Create View
           columns: a int, c decimal(10,2)
           table properties:
-            rewriting.time.window 300s
+            rewriting.time.window 5min
           expanded text: SELECT `cmv_basetable_n3`.`a`, `cmv_basetable_2_n1`.`c`
   FROM `default`.`cmv_basetable_n3` JOIN `default`.`cmv_basetable_2_n1` ON (`cmv_basetable_n3`.`a` = `cmv_basetable_2_n1`.`a`)
   WHERE `cmv_basetable_2_n1`.`c` > 10.0
@@ -214,7 +214,7 @@ STAGE PLANS:
           hdfs directory: true
 #### A masked pattern was here ####
 
-PREHOOK: query: CREATE MATERIALIZED VIEW cmv_mat_view_n3 DISABLE REWRITE TBLPROPERTIES('rewriting.time.window'='300s') AS
+PREHOOK: query: CREATE MATERIALIZED VIEW cmv_mat_view_n3 DISABLE REWRITE TBLPROPERTIES('rewriting.time.window'='5min') AS
   SELECT cmv_basetable_n3.a, cmv_basetable_2_n1.c
   FROM cmv_basetable_n3 JOIN cmv_basetable_2_n1 ON (cmv_basetable_n3.a = cmv_basetable_2_n1.a)
   WHERE cmv_basetable_2_n1.c > 10.0
@@ -224,7 +224,7 @@ PREHOOK: Input: default@cmv_basetable_2_n1
 PREHOOK: Input: default@cmv_basetable_n3
 PREHOOK: Output: database:default
 PREHOOK: Output: default@cmv_mat_view_n3
-POSTHOOK: query: CREATE MATERIALIZED VIEW cmv_mat_view_n3 DISABLE REWRITE TBLPROPERTIES('rewriting.time.window'='300s') AS
+POSTHOOK: query: CREATE MATERIALIZED VIEW cmv_mat_view_n3 DISABLE REWRITE TBLPROPERTIES('rewriting.time.window'='5min') AS
   SELECT cmv_basetable_n3.a, cmv_basetable_2_n1.c
   FROM cmv_basetable_n3 JOIN cmv_basetable_2_n1 ON (cmv_basetable_n3.a = cmv_basetable_2_n1.a)
   WHERE cmv_basetable_2_n1.c > 10.0
@@ -256,7 +256,7 @@ Table Parameters:
 	numFiles            	2                   
 	numRows             	2                   
 	rawDataSize         	232                 
-	rewriting.time.window	300s                
+	rewriting.time.window	5min                
 	totalSize           	608                 
 #### A masked pattern was here ####
 	 	 
@@ -480,7 +480,7 @@ Table Parameters:
 	numFiles            	2                   
 	numRows             	2                   
 	rawDataSize         	232                 
-	rewriting.time.window	300s                
+	rewriting.time.window	5min                
 	totalSize           	608                 
 #### A masked pattern was here ####
 	 	 
@@ -786,7 +786,7 @@ Table Parameters:
 	numFiles            	2                   
 	numRows             	3                   
 	rawDataSize         	348                 
-	rewriting.time.window	300s                
+	rewriting.time.window	5min                
 	totalSize           	628                 
 #### A masked pattern was here ####
 	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_empty.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_empty.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_empty.q.out
index 2e6f00c..ac8cc35 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_empty.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_empty.q.out
@@ -55,7 +55,7 @@ STAGE PLANS:
       limit: -1
       Processor Tree:
         TableScan
-          alias: emps_mv_rewrite_empty
+          alias: default.emps_mv_rewrite_empty_mv1
           filterExpr: (empid < 120) (type: boolean)
           Filter Operator
             predicate: (empid < 120) (type: boolean)
@@ -67,10 +67,12 @@ STAGE PLANS:
 PREHOOK: query: select * from emps_mv_rewrite_empty where empid < 120
 PREHOOK: type: QUERY
 PREHOOK: Input: default@emps_mv_rewrite_empty
+PREHOOK: Input: default@emps_mv_rewrite_empty_mv1
 #### A masked pattern was here ####
 POSTHOOK: query: select * from emps_mv_rewrite_empty where empid < 120
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@emps_mv_rewrite_empty
+POSTHOOK: Input: default@emps_mv_rewrite_empty_mv1
 #### A masked pattern was here ####
 PREHOOK: query: drop materialized view emps_mv_rewrite_empty_mv1
 PREHOOK: type: DROP_MATERIALIZED_VIEW


[08/12] hive git commit: HIVE-20006: Make materializations invalidation cache work with multiple active remote metastores (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h
index f5913fc..d0c299b 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -8742,8 +8742,9 @@ inline std::ostream& operator<<(std::ostream& out, const BasicTxnInfo& obj)
 }
 
 typedef struct _CreationMetadata__isset {
-  _CreationMetadata__isset() : validTxnList(false) {}
+  _CreationMetadata__isset() : validTxnList(false), materializationTime(false) {}
   bool validTxnList :1;
+  bool materializationTime :1;
 } _CreationMetadata__isset;
 
 class CreationMetadata {
@@ -8751,7 +8752,7 @@ class CreationMetadata {
 
   CreationMetadata(const CreationMetadata&);
   CreationMetadata& operator=(const CreationMetadata&);
-  CreationMetadata() : catName(), dbName(), tblName(), validTxnList() {
+  CreationMetadata() : catName(), dbName(), tblName(), validTxnList(), materializationTime(0) {
   }
 
   virtual ~CreationMetadata() throw();
@@ -8760,6 +8761,7 @@ class CreationMetadata {
   std::string tblName;
   std::set<std::string>  tablesUsed;
   std::string validTxnList;
+  int64_t materializationTime;
 
   _CreationMetadata__isset __isset;
 
@@ -8773,6 +8775,8 @@ class CreationMetadata {
 
   void __set_validTxnList(const std::string& val);
 
+  void __set_materializationTime(const int64_t val);
+
   bool operator == (const CreationMetadata & rhs) const
   {
     if (!(catName == rhs.catName))
@@ -8787,6 +8791,10 @@ class CreationMetadata {
       return false;
     else if (__isset.validTxnList && !(validTxnList == rhs.validTxnList))
       return false;
+    if (__isset.materializationTime != rhs.__isset.materializationTime)
+      return false;
+    else if (__isset.materializationTime && !(materializationTime == rhs.materializationTime))
+      return false;
     return true;
   }
   bool operator != (const CreationMetadata &rhs) const {
@@ -10452,52 +10460,23 @@ inline std::ostream& operator<<(std::ostream& out, const TableMeta& obj)
   return out;
 }
 
-typedef struct _Materialization__isset {
-  _Materialization__isset() : validTxnList(false), invalidationTime(false), sourceTablesUpdateDeleteModified(false) {}
-  bool validTxnList :1;
-  bool invalidationTime :1;
-  bool sourceTablesUpdateDeleteModified :1;
-} _Materialization__isset;
 
 class Materialization {
  public:
 
   Materialization(const Materialization&);
   Materialization& operator=(const Materialization&);
-  Materialization() : validTxnList(), invalidationTime(0), sourceTablesUpdateDeleteModified(0) {
+  Materialization() : sourceTablesUpdateDeleteModified(0) {
   }
 
   virtual ~Materialization() throw();
-  std::set<std::string>  tablesUsed;
-  std::string validTxnList;
-  int64_t invalidationTime;
   bool sourceTablesUpdateDeleteModified;
 
-  _Materialization__isset __isset;
-
-  void __set_tablesUsed(const std::set<std::string> & val);
-
-  void __set_validTxnList(const std::string& val);
-
-  void __set_invalidationTime(const int64_t val);
-
   void __set_sourceTablesUpdateDeleteModified(const bool val);
 
   bool operator == (const Materialization & rhs) const
   {
-    if (!(tablesUsed == rhs.tablesUsed))
-      return false;
-    if (__isset.validTxnList != rhs.__isset.validTxnList)
-      return false;
-    else if (__isset.validTxnList && !(validTxnList == rhs.validTxnList))
-      return false;
-    if (__isset.invalidationTime != rhs.__isset.invalidationTime)
-      return false;
-    else if (__isset.invalidationTime && !(invalidationTime == rhs.invalidationTime))
-      return false;
-    if (__isset.sourceTablesUpdateDeleteModified != rhs.__isset.sourceTablesUpdateDeleteModified)
-      return false;
-    else if (__isset.sourceTablesUpdateDeleteModified && !(sourceTablesUpdateDeleteModified == rhs.sourceTablesUpdateDeleteModified))
+    if (!(sourceTablesUpdateDeleteModified == rhs.sourceTablesUpdateDeleteModified))
       return false;
     return true;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
index 611bf6f..281dada 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
@@ -43,6 +43,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tblName", org.apache.thrift.protocol.TType.STRING, (short)3);
   private static final org.apache.thrift.protocol.TField TABLES_USED_FIELD_DESC = new org.apache.thrift.protocol.TField("tablesUsed", org.apache.thrift.protocol.TType.SET, (short)4);
   private static final org.apache.thrift.protocol.TField VALID_TXN_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validTxnList", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField MATERIALIZATION_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("materializationTime", org.apache.thrift.protocol.TType.I64, (short)6);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -55,6 +56,7 @@ import org.slf4j.LoggerFactory;
   private String tblName; // required
   private Set<String> tablesUsed; // required
   private String validTxnList; // optional
+  private long materializationTime; // 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 {
@@ -62,7 +64,8 @@ import org.slf4j.LoggerFactory;
     DB_NAME((short)2, "dbName"),
     TBL_NAME((short)3, "tblName"),
     TABLES_USED((short)4, "tablesUsed"),
-    VALID_TXN_LIST((short)5, "validTxnList");
+    VALID_TXN_LIST((short)5, "validTxnList"),
+    MATERIALIZATION_TIME((short)6, "materializationTime");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -87,6 +90,8 @@ import org.slf4j.LoggerFactory;
           return TABLES_USED;
         case 5: // VALID_TXN_LIST
           return VALID_TXN_LIST;
+        case 6: // MATERIALIZATION_TIME
+          return MATERIALIZATION_TIME;
         default:
           return null;
       }
@@ -127,7 +132,9 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
-  private static final _Fields optionals[] = {_Fields.VALID_TXN_LIST};
+  private static final int __MATERIALIZATIONTIME_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.VALID_TXN_LIST,_Fields.MATERIALIZATION_TIME};
   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);
@@ -142,6 +149,8 @@ import org.slf4j.LoggerFactory;
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
     tmpMap.put(_Fields.VALID_TXN_LIST, new org.apache.thrift.meta_data.FieldMetaData("validTxnList", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.MATERIALIZATION_TIME, new org.apache.thrift.meta_data.FieldMetaData("materializationTime", 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(CreationMetadata.class, metaDataMap);
   }
@@ -166,6 +175,7 @@ import org.slf4j.LoggerFactory;
    * Performs a deep copy on <i>other</i>.
    */
   public CreationMetadata(CreationMetadata other) {
+    __isset_bitfield = other.__isset_bitfield;
     if (other.isSetCatName()) {
       this.catName = other.catName;
     }
@@ -182,6 +192,7 @@ import org.slf4j.LoggerFactory;
     if (other.isSetValidTxnList()) {
       this.validTxnList = other.validTxnList;
     }
+    this.materializationTime = other.materializationTime;
   }
 
   public CreationMetadata deepCopy() {
@@ -195,6 +206,8 @@ import org.slf4j.LoggerFactory;
     this.tblName = null;
     this.tablesUsed = null;
     this.validTxnList = null;
+    setMaterializationTimeIsSet(false);
+    this.materializationTime = 0;
   }
 
   public String getCatName() {
@@ -327,6 +340,28 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public long getMaterializationTime() {
+    return this.materializationTime;
+  }
+
+  public void setMaterializationTime(long materializationTime) {
+    this.materializationTime = materializationTime;
+    setMaterializationTimeIsSet(true);
+  }
+
+  public void unsetMaterializationTime() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MATERIALIZATIONTIME_ISSET_ID);
+  }
+
+  /** Returns true if field materializationTime is set (has been assigned a value) and false otherwise */
+  public boolean isSetMaterializationTime() {
+    return EncodingUtils.testBit(__isset_bitfield, __MATERIALIZATIONTIME_ISSET_ID);
+  }
+
+  public void setMaterializationTimeIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MATERIALIZATIONTIME_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case CAT_NAME:
@@ -369,6 +404,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case MATERIALIZATION_TIME:
+      if (value == null) {
+        unsetMaterializationTime();
+      } else {
+        setMaterializationTime((Long)value);
+      }
+      break;
+
     }
   }
 
@@ -389,6 +432,9 @@ import org.slf4j.LoggerFactory;
     case VALID_TXN_LIST:
       return getValidTxnList();
 
+    case MATERIALIZATION_TIME:
+      return getMaterializationTime();
+
     }
     throw new IllegalStateException();
   }
@@ -410,6 +456,8 @@ import org.slf4j.LoggerFactory;
       return isSetTablesUsed();
     case VALID_TXN_LIST:
       return isSetValidTxnList();
+    case MATERIALIZATION_TIME:
+      return isSetMaterializationTime();
     }
     throw new IllegalStateException();
   }
@@ -472,6 +520,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_materializationTime = true && this.isSetMaterializationTime();
+    boolean that_present_materializationTime = true && that.isSetMaterializationTime();
+    if (this_present_materializationTime || that_present_materializationTime) {
+      if (!(this_present_materializationTime && that_present_materializationTime))
+        return false;
+      if (this.materializationTime != that.materializationTime)
+        return false;
+    }
+
     return true;
   }
 
@@ -504,6 +561,11 @@ import org.slf4j.LoggerFactory;
     if (present_validTxnList)
       list.add(validTxnList);
 
+    boolean present_materializationTime = true && (isSetMaterializationTime());
+    list.add(present_materializationTime);
+    if (present_materializationTime)
+      list.add(materializationTime);
+
     return list.hashCode();
   }
 
@@ -565,6 +627,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetMaterializationTime()).compareTo(other.isSetMaterializationTime());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetMaterializationTime()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.materializationTime, other.materializationTime);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -626,6 +698,12 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetMaterializationTime()) {
+      if (!first) sb.append(", ");
+      sb.append("materializationTime:");
+      sb.append(this.materializationTime);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -661,6 +739,8 @@ import org.slf4j.LoggerFactory;
 
   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);
@@ -735,6 +815,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 6: // MATERIALIZATION_TIME
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.materializationTime = iprot.readI64();
+              struct.setMaterializationTimeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -782,6 +870,11 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
+      if (struct.isSetMaterializationTime()) {
+        oprot.writeFieldBegin(MATERIALIZATION_TIME_FIELD_DESC);
+        oprot.writeI64(struct.materializationTime);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -813,10 +906,16 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetValidTxnList()) {
         optionals.set(0);
       }
-      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetMaterializationTime()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
       if (struct.isSetValidTxnList()) {
         oprot.writeString(struct.validTxnList);
       }
+      if (struct.isSetMaterializationTime()) {
+        oprot.writeI64(struct.materializationTime);
+      }
     }
 
     @Override
@@ -839,11 +938,15 @@ import org.slf4j.LoggerFactory;
         }
       }
       struct.setTablesUsedIsSet(true);
-      BitSet incoming = iprot.readBitSet(1);
+      BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         struct.validTxnList = iprot.readString();
         struct.setValidTxnListIsSet(true);
       }
+      if (incoming.get(1)) {
+        struct.materializationTime = iprot.readI64();
+        struct.setMaterializationTimeIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
index 8f5b4e5..79d9fc6 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // SCHEMA_VERSIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list944 = iprot.readListBegin();
-                struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list944.size);
-                SchemaVersionDescriptor _elem945;
-                for (int _i946 = 0; _i946 < _list944.size; ++_i946)
+                org.apache.thrift.protocol.TList _list936 = iprot.readListBegin();
+                struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list936.size);
+                SchemaVersionDescriptor _elem937;
+                for (int _i938 = 0; _i938 < _list936.size; ++_i938)
                 {
-                  _elem945 = new SchemaVersionDescriptor();
-                  _elem945.read(iprot);
-                  struct.schemaVersions.add(_elem945);
+                  _elem937 = new SchemaVersionDescriptor();
+                  _elem937.read(iprot);
+                  struct.schemaVersions.add(_elem937);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(SCHEMA_VERSIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.schemaVersions.size()));
-          for (SchemaVersionDescriptor _iter947 : struct.schemaVersions)
+          for (SchemaVersionDescriptor _iter939 : struct.schemaVersions)
           {
-            _iter947.write(oprot);
+            _iter939.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetSchemaVersions()) {
         {
           oprot.writeI32(struct.schemaVersions.size());
-          for (SchemaVersionDescriptor _iter948 : struct.schemaVersions)
+          for (SchemaVersionDescriptor _iter940 : struct.schemaVersions)
           {
-            _iter948.write(oprot);
+            _iter940.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list949 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list949.size);
-          SchemaVersionDescriptor _elem950;
-          for (int _i951 = 0; _i951 < _list949.size; ++_i951)
+          org.apache.thrift.protocol.TList _list941 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list941.size);
+          SchemaVersionDescriptor _elem942;
+          for (int _i943 = 0; _i943 < _list941.size; ++_i943)
           {
-            _elem950 = new SchemaVersionDescriptor();
-            _elem950.read(iprot);
-            struct.schemaVersions.add(_elem950);
+            _elem942 = new SchemaVersionDescriptor();
+            _elem942.read(iprot);
+            struct.schemaVersions.add(_elem942);
           }
         }
         struct.setSchemaVersionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
index 3510995..0972c5e 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
@@ -38,10 +38,7 @@ import org.slf4j.LoggerFactory;
 @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class Materialization implements org.apache.thrift.TBase<Materialization, Materialization._Fields>, java.io.Serializable, Cloneable, Comparable<Materialization> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Materialization");
 
-  private static final org.apache.thrift.protocol.TField TABLES_USED_FIELD_DESC = new org.apache.thrift.protocol.TField("tablesUsed", org.apache.thrift.protocol.TType.SET, (short)1);
-  private static final org.apache.thrift.protocol.TField VALID_TXN_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validTxnList", org.apache.thrift.protocol.TType.STRING, (short)2);
-  private static final org.apache.thrift.protocol.TField INVALIDATION_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("invalidationTime", org.apache.thrift.protocol.TType.I64, (short)3);
-  private static final org.apache.thrift.protocol.TField SOURCE_TABLES_UPDATE_DELETE_MODIFIED_FIELD_DESC = new org.apache.thrift.protocol.TField("sourceTablesUpdateDeleteModified", org.apache.thrift.protocol.TType.BOOL, (short)4);
+  private static final org.apache.thrift.protocol.TField SOURCE_TABLES_UPDATE_DELETE_MODIFIED_FIELD_DESC = new org.apache.thrift.protocol.TField("sourceTablesUpdateDeleteModified", org.apache.thrift.protocol.TType.BOOL, (short)1);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -49,17 +46,11 @@ import org.slf4j.LoggerFactory;
     schemes.put(TupleScheme.class, new MaterializationTupleSchemeFactory());
   }
 
-  private Set<String> tablesUsed; // required
-  private String validTxnList; // optional
-  private long invalidationTime; // optional
-  private boolean sourceTablesUpdateDeleteModified; // optional
+  private boolean sourceTablesUpdateDeleteModified; // 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 {
-    TABLES_USED((short)1, "tablesUsed"),
-    VALID_TXN_LIST((short)2, "validTxnList"),
-    INVALIDATION_TIME((short)3, "invalidationTime"),
-    SOURCE_TABLES_UPDATE_DELETE_MODIFIED((short)4, "sourceTablesUpdateDeleteModified");
+    SOURCE_TABLES_UPDATE_DELETE_MODIFIED((short)1, "sourceTablesUpdateDeleteModified");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -74,13 +65,7 @@ import org.slf4j.LoggerFactory;
      */
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 1: // TABLES_USED
-          return TABLES_USED;
-        case 2: // VALID_TXN_LIST
-          return VALID_TXN_LIST;
-        case 3: // INVALIDATION_TIME
-          return INVALIDATION_TIME;
-        case 4: // SOURCE_TABLES_UPDATE_DELETE_MODIFIED
+        case 1: // SOURCE_TABLES_UPDATE_DELETE_MODIFIED
           return SOURCE_TABLES_UPDATE_DELETE_MODIFIED;
         default:
           return null;
@@ -122,21 +107,12 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
-  private static final int __INVALIDATIONTIME_ISSET_ID = 0;
-  private static final int __SOURCETABLESUPDATEDELETEMODIFIED_ISSET_ID = 1;
+  private static final int __SOURCETABLESUPDATEDELETEMODIFIED_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.VALID_TXN_LIST,_Fields.INVALIDATION_TIME,_Fields.SOURCE_TABLES_UPDATE_DELETE_MODIFIED};
   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.TABLES_USED, new org.apache.thrift.meta_data.FieldMetaData("tablesUsed", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET, 
-            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
-    tmpMap.put(_Fields.VALID_TXN_LIST, new org.apache.thrift.meta_data.FieldMetaData("validTxnList", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.INVALIDATION_TIME, new org.apache.thrift.meta_data.FieldMetaData("invalidationTime", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-    tmpMap.put(_Fields.SOURCE_TABLES_UPDATE_DELETE_MODIFIED, new org.apache.thrift.meta_data.FieldMetaData("sourceTablesUpdateDeleteModified", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+    tmpMap.put(_Fields.SOURCE_TABLES_UPDATE_DELETE_MODIFIED, new org.apache.thrift.meta_data.FieldMetaData("sourceTablesUpdateDeleteModified", 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(Materialization.class, metaDataMap);
@@ -146,10 +122,11 @@ import org.slf4j.LoggerFactory;
   }
 
   public Materialization(
-    Set<String> tablesUsed)
+    boolean sourceTablesUpdateDeleteModified)
   {
     this();
-    this.tablesUsed = tablesUsed;
+    this.sourceTablesUpdateDeleteModified = sourceTablesUpdateDeleteModified;
+    setSourceTablesUpdateDeleteModifiedIsSet(true);
   }
 
   /**
@@ -157,14 +134,6 @@ import org.slf4j.LoggerFactory;
    */
   public Materialization(Materialization other) {
     __isset_bitfield = other.__isset_bitfield;
-    if (other.isSetTablesUsed()) {
-      Set<String> __this__tablesUsed = new HashSet<String>(other.tablesUsed);
-      this.tablesUsed = __this__tablesUsed;
-    }
-    if (other.isSetValidTxnList()) {
-      this.validTxnList = other.validTxnList;
-    }
-    this.invalidationTime = other.invalidationTime;
     this.sourceTablesUpdateDeleteModified = other.sourceTablesUpdateDeleteModified;
   }
 
@@ -174,97 +143,10 @@ import org.slf4j.LoggerFactory;
 
   @Override
   public void clear() {
-    this.tablesUsed = null;
-    this.validTxnList = null;
-    setInvalidationTimeIsSet(false);
-    this.invalidationTime = 0;
     setSourceTablesUpdateDeleteModifiedIsSet(false);
     this.sourceTablesUpdateDeleteModified = false;
   }
 
-  public int getTablesUsedSize() {
-    return (this.tablesUsed == null) ? 0 : this.tablesUsed.size();
-  }
-
-  public java.util.Iterator<String> getTablesUsedIterator() {
-    return (this.tablesUsed == null) ? null : this.tablesUsed.iterator();
-  }
-
-  public void addToTablesUsed(String elem) {
-    if (this.tablesUsed == null) {
-      this.tablesUsed = new HashSet<String>();
-    }
-    this.tablesUsed.add(elem);
-  }
-
-  public Set<String> getTablesUsed() {
-    return this.tablesUsed;
-  }
-
-  public void setTablesUsed(Set<String> tablesUsed) {
-    this.tablesUsed = tablesUsed;
-  }
-
-  public void unsetTablesUsed() {
-    this.tablesUsed = null;
-  }
-
-  /** Returns true if field tablesUsed is set (has been assigned a value) and false otherwise */
-  public boolean isSetTablesUsed() {
-    return this.tablesUsed != null;
-  }
-
-  public void setTablesUsedIsSet(boolean value) {
-    if (!value) {
-      this.tablesUsed = null;
-    }
-  }
-
-  public String getValidTxnList() {
-    return this.validTxnList;
-  }
-
-  public void setValidTxnList(String validTxnList) {
-    this.validTxnList = validTxnList;
-  }
-
-  public void unsetValidTxnList() {
-    this.validTxnList = null;
-  }
-
-  /** Returns true if field validTxnList is set (has been assigned a value) and false otherwise */
-  public boolean isSetValidTxnList() {
-    return this.validTxnList != null;
-  }
-
-  public void setValidTxnListIsSet(boolean value) {
-    if (!value) {
-      this.validTxnList = null;
-    }
-  }
-
-  public long getInvalidationTime() {
-    return this.invalidationTime;
-  }
-
-  public void setInvalidationTime(long invalidationTime) {
-    this.invalidationTime = invalidationTime;
-    setInvalidationTimeIsSet(true);
-  }
-
-  public void unsetInvalidationTime() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __INVALIDATIONTIME_ISSET_ID);
-  }
-
-  /** Returns true if field invalidationTime is set (has been assigned a value) and false otherwise */
-  public boolean isSetInvalidationTime() {
-    return EncodingUtils.testBit(__isset_bitfield, __INVALIDATIONTIME_ISSET_ID);
-  }
-
-  public void setInvalidationTimeIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __INVALIDATIONTIME_ISSET_ID, value);
-  }
-
   public boolean isSourceTablesUpdateDeleteModified() {
     return this.sourceTablesUpdateDeleteModified;
   }
@@ -289,30 +171,6 @@ import org.slf4j.LoggerFactory;
 
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
-    case TABLES_USED:
-      if (value == null) {
-        unsetTablesUsed();
-      } else {
-        setTablesUsed((Set<String>)value);
-      }
-      break;
-
-    case VALID_TXN_LIST:
-      if (value == null) {
-        unsetValidTxnList();
-      } else {
-        setValidTxnList((String)value);
-      }
-      break;
-
-    case INVALIDATION_TIME:
-      if (value == null) {
-        unsetInvalidationTime();
-      } else {
-        setInvalidationTime((Long)value);
-      }
-      break;
-
     case SOURCE_TABLES_UPDATE_DELETE_MODIFIED:
       if (value == null) {
         unsetSourceTablesUpdateDeleteModified();
@@ -326,15 +184,6 @@ import org.slf4j.LoggerFactory;
 
   public Object getFieldValue(_Fields field) {
     switch (field) {
-    case TABLES_USED:
-      return getTablesUsed();
-
-    case VALID_TXN_LIST:
-      return getValidTxnList();
-
-    case INVALIDATION_TIME:
-      return getInvalidationTime();
-
     case SOURCE_TABLES_UPDATE_DELETE_MODIFIED:
       return isSourceTablesUpdateDeleteModified();
 
@@ -349,12 +198,6 @@ import org.slf4j.LoggerFactory;
     }
 
     switch (field) {
-    case TABLES_USED:
-      return isSetTablesUsed();
-    case VALID_TXN_LIST:
-      return isSetValidTxnList();
-    case INVALIDATION_TIME:
-      return isSetInvalidationTime();
     case SOURCE_TABLES_UPDATE_DELETE_MODIFIED:
       return isSetSourceTablesUpdateDeleteModified();
     }
@@ -374,35 +217,8 @@ import org.slf4j.LoggerFactory;
     if (that == null)
       return false;
 
-    boolean this_present_tablesUsed = true && this.isSetTablesUsed();
-    boolean that_present_tablesUsed = true && that.isSetTablesUsed();
-    if (this_present_tablesUsed || that_present_tablesUsed) {
-      if (!(this_present_tablesUsed && that_present_tablesUsed))
-        return false;
-      if (!this.tablesUsed.equals(that.tablesUsed))
-        return false;
-    }
-
-    boolean this_present_validTxnList = true && this.isSetValidTxnList();
-    boolean that_present_validTxnList = true && that.isSetValidTxnList();
-    if (this_present_validTxnList || that_present_validTxnList) {
-      if (!(this_present_validTxnList && that_present_validTxnList))
-        return false;
-      if (!this.validTxnList.equals(that.validTxnList))
-        return false;
-    }
-
-    boolean this_present_invalidationTime = true && this.isSetInvalidationTime();
-    boolean that_present_invalidationTime = true && that.isSetInvalidationTime();
-    if (this_present_invalidationTime || that_present_invalidationTime) {
-      if (!(this_present_invalidationTime && that_present_invalidationTime))
-        return false;
-      if (this.invalidationTime != that.invalidationTime)
-        return false;
-    }
-
-    boolean this_present_sourceTablesUpdateDeleteModified = true && this.isSetSourceTablesUpdateDeleteModified();
-    boolean that_present_sourceTablesUpdateDeleteModified = true && that.isSetSourceTablesUpdateDeleteModified();
+    boolean this_present_sourceTablesUpdateDeleteModified = true;
+    boolean that_present_sourceTablesUpdateDeleteModified = true;
     if (this_present_sourceTablesUpdateDeleteModified || that_present_sourceTablesUpdateDeleteModified) {
       if (!(this_present_sourceTablesUpdateDeleteModified && that_present_sourceTablesUpdateDeleteModified))
         return false;
@@ -417,22 +233,7 @@ import org.slf4j.LoggerFactory;
   public int hashCode() {
     List<Object> list = new ArrayList<Object>();
 
-    boolean present_tablesUsed = true && (isSetTablesUsed());
-    list.add(present_tablesUsed);
-    if (present_tablesUsed)
-      list.add(tablesUsed);
-
-    boolean present_validTxnList = true && (isSetValidTxnList());
-    list.add(present_validTxnList);
-    if (present_validTxnList)
-      list.add(validTxnList);
-
-    boolean present_invalidationTime = true && (isSetInvalidationTime());
-    list.add(present_invalidationTime);
-    if (present_invalidationTime)
-      list.add(invalidationTime);
-
-    boolean present_sourceTablesUpdateDeleteModified = true && (isSetSourceTablesUpdateDeleteModified());
+    boolean present_sourceTablesUpdateDeleteModified = true;
     list.add(present_sourceTablesUpdateDeleteModified);
     if (present_sourceTablesUpdateDeleteModified)
       list.add(sourceTablesUpdateDeleteModified);
@@ -448,36 +249,6 @@ import org.slf4j.LoggerFactory;
 
     int lastComparison = 0;
 
-    lastComparison = Boolean.valueOf(isSetTablesUsed()).compareTo(other.isSetTablesUsed());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetTablesUsed()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tablesUsed, other.tablesUsed);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetValidTxnList()).compareTo(other.isSetValidTxnList());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetValidTxnList()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.validTxnList, other.validTxnList);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetInvalidationTime()).compareTo(other.isSetInvalidationTime());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetInvalidationTime()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.invalidationTime, other.invalidationTime);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
     lastComparison = Boolean.valueOf(isSetSourceTablesUpdateDeleteModified()).compareTo(other.isSetSourceTablesUpdateDeleteModified());
     if (lastComparison != 0) {
       return lastComparison;
@@ -508,43 +279,17 @@ import org.slf4j.LoggerFactory;
     StringBuilder sb = new StringBuilder("Materialization(");
     boolean first = true;
 
-    sb.append("tablesUsed:");
-    if (this.tablesUsed == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.tablesUsed);
-    }
+    sb.append("sourceTablesUpdateDeleteModified:");
+    sb.append(this.sourceTablesUpdateDeleteModified);
     first = false;
-    if (isSetValidTxnList()) {
-      if (!first) sb.append(", ");
-      sb.append("validTxnList:");
-      if (this.validTxnList == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.validTxnList);
-      }
-      first = false;
-    }
-    if (isSetInvalidationTime()) {
-      if (!first) sb.append(", ");
-      sb.append("invalidationTime:");
-      sb.append(this.invalidationTime);
-      first = false;
-    }
-    if (isSetSourceTablesUpdateDeleteModified()) {
-      if (!first) sb.append(", ");
-      sb.append("sourceTablesUpdateDeleteModified:");
-      sb.append(this.sourceTablesUpdateDeleteModified);
-      first = false;
-    }
     sb.append(")");
     return sb.toString();
   }
 
   public void validate() throws org.apache.thrift.TException {
     // check for required fields
-    if (!isSetTablesUsed()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tablesUsed' is unset! Struct:" + toString());
+    if (!isSetSourceTablesUpdateDeleteModified()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'sourceTablesUpdateDeleteModified' is unset! Struct:" + toString());
     }
 
     // check for sub-struct validity
@@ -586,41 +331,7 @@ import org.slf4j.LoggerFactory;
           break;
         }
         switch (schemeField.id) {
-          case 1: // TABLES_USED
-            if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
-              {
-                org.apache.thrift.protocol.TSet _set864 = iprot.readSetBegin();
-                struct.tablesUsed = new HashSet<String>(2*_set864.size);
-                String _elem865;
-                for (int _i866 = 0; _i866 < _set864.size; ++_i866)
-                {
-                  _elem865 = iprot.readString();
-                  struct.tablesUsed.add(_elem865);
-                }
-                iprot.readSetEnd();
-              }
-              struct.setTablesUsedIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 2: // VALID_TXN_LIST
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.validTxnList = iprot.readString();
-              struct.setValidTxnListIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 3: // INVALIDATION_TIME
-            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-              struct.invalidationTime = iprot.readI64();
-              struct.setInvalidationTimeIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 4: // SOURCE_TABLES_UPDATE_DELETE_MODIFIED
+          case 1: // SOURCE_TABLES_UPDATE_DELETE_MODIFIED
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.sourceTablesUpdateDeleteModified = iprot.readBool();
               struct.setSourceTablesUpdateDeleteModifiedIsSet(true);
@@ -641,35 +352,9 @@ import org.slf4j.LoggerFactory;
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
-      if (struct.tablesUsed != null) {
-        oprot.writeFieldBegin(TABLES_USED_FIELD_DESC);
-        {
-          oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.tablesUsed.size()));
-          for (String _iter867 : struct.tablesUsed)
-          {
-            oprot.writeString(_iter867);
-          }
-          oprot.writeSetEnd();
-        }
-        oprot.writeFieldEnd();
-      }
-      if (struct.validTxnList != null) {
-        if (struct.isSetValidTxnList()) {
-          oprot.writeFieldBegin(VALID_TXN_LIST_FIELD_DESC);
-          oprot.writeString(struct.validTxnList);
-          oprot.writeFieldEnd();
-        }
-      }
-      if (struct.isSetInvalidationTime()) {
-        oprot.writeFieldBegin(INVALIDATION_TIME_FIELD_DESC);
-        oprot.writeI64(struct.invalidationTime);
-        oprot.writeFieldEnd();
-      }
-      if (struct.isSetSourceTablesUpdateDeleteModified()) {
-        oprot.writeFieldBegin(SOURCE_TABLES_UPDATE_DELETE_MODIFIED_FIELD_DESC);
-        oprot.writeBool(struct.sourceTablesUpdateDeleteModified);
-        oprot.writeFieldEnd();
-      }
+      oprot.writeFieldBegin(SOURCE_TABLES_UPDATE_DELETE_MODIFIED_FIELD_DESC);
+      oprot.writeBool(struct.sourceTablesUpdateDeleteModified);
+      oprot.writeFieldEnd();
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -687,62 +372,14 @@ import org.slf4j.LoggerFactory;
     @Override
     public void write(org.apache.thrift.protocol.TProtocol prot, Materialization struct) throws org.apache.thrift.TException {
       TTupleProtocol oprot = (TTupleProtocol) prot;
-      {
-        oprot.writeI32(struct.tablesUsed.size());
-        for (String _iter868 : struct.tablesUsed)
-        {
-          oprot.writeString(_iter868);
-        }
-      }
-      BitSet optionals = new BitSet();
-      if (struct.isSetValidTxnList()) {
-        optionals.set(0);
-      }
-      if (struct.isSetInvalidationTime()) {
-        optionals.set(1);
-      }
-      if (struct.isSetSourceTablesUpdateDeleteModified()) {
-        optionals.set(2);
-      }
-      oprot.writeBitSet(optionals, 3);
-      if (struct.isSetValidTxnList()) {
-        oprot.writeString(struct.validTxnList);
-      }
-      if (struct.isSetInvalidationTime()) {
-        oprot.writeI64(struct.invalidationTime);
-      }
-      if (struct.isSetSourceTablesUpdateDeleteModified()) {
-        oprot.writeBool(struct.sourceTablesUpdateDeleteModified);
-      }
+      oprot.writeBool(struct.sourceTablesUpdateDeleteModified);
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, Materialization struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      {
-        org.apache.thrift.protocol.TSet _set869 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.tablesUsed = new HashSet<String>(2*_set869.size);
-        String _elem870;
-        for (int _i871 = 0; _i871 < _set869.size; ++_i871)
-        {
-          _elem870 = iprot.readString();
-          struct.tablesUsed.add(_elem870);
-        }
-      }
-      struct.setTablesUsedIsSet(true);
-      BitSet incoming = iprot.readBitSet(3);
-      if (incoming.get(0)) {
-        struct.validTxnList = iprot.readString();
-        struct.setValidTxnListIsSet(true);
-      }
-      if (incoming.get(1)) {
-        struct.invalidationTime = iprot.readI64();
-        struct.setInvalidationTimeIsSet(true);
-      }
-      if (incoming.get(2)) {
-        struct.sourceTablesUpdateDeleteModified = iprot.readBool();
-        struct.setSourceTablesUpdateDeleteModifiedIsSet(true);
-      }
+      struct.sourceTablesUpdateDeleteModified = iprot.readBool();
+      struct.setSourceTablesUpdateDeleteModifiedIsSet(true);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
index 88d7e3f..935af04 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
@@ -1119,14 +1119,14 @@ import org.slf4j.LoggerFactory;
           case 4: // COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list936 = iprot.readListBegin();
-                struct.cols = new ArrayList<FieldSchema>(_list936.size);
-                FieldSchema _elem937;
-                for (int _i938 = 0; _i938 < _list936.size; ++_i938)
+                org.apache.thrift.protocol.TList _list928 = iprot.readListBegin();
+                struct.cols = new ArrayList<FieldSchema>(_list928.size);
+                FieldSchema _elem929;
+                for (int _i930 = 0; _i930 < _list928.size; ++_i930)
                 {
-                  _elem937 = new FieldSchema();
-                  _elem937.read(iprot);
-                  struct.cols.add(_elem937);
+                  _elem929 = new FieldSchema();
+                  _elem929.read(iprot);
+                  struct.cols.add(_elem929);
                 }
                 iprot.readListEnd();
               }
@@ -1212,9 +1212,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.cols.size()));
-          for (FieldSchema _iter939 : struct.cols)
+          for (FieldSchema _iter931 : struct.cols)
           {
-            _iter939.write(oprot);
+            _iter931.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1323,9 +1323,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetCols()) {
         {
           oprot.writeI32(struct.cols.size());
-          for (FieldSchema _iter940 : struct.cols)
+          for (FieldSchema _iter932 : struct.cols)
           {
-            _iter940.write(oprot);
+            _iter932.write(oprot);
           }
         }
       }
@@ -1368,14 +1368,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TList _list941 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.cols = new ArrayList<FieldSchema>(_list941.size);
-          FieldSchema _elem942;
-          for (int _i943 = 0; _i943 < _list941.size; ++_i943)
+          org.apache.thrift.protocol.TList _list933 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.cols = new ArrayList<FieldSchema>(_list933.size);
+          FieldSchema _elem934;
+          for (int _i935 = 0; _i935 < _list933.size; ++_i935)
           {
-            _elem942 = new FieldSchema();
-            _elem942.read(iprot);
-            struct.cols.add(_elem942);
+            _elem934 = new FieldSchema();
+            _elem934.read(iprot);
+            struct.cols.add(_elem934);
           }
         }
         struct.setColsIsSet(true);


[10/12] hive git commit: HIVE-20006: Make materializations invalidation cache work with multiple active remote metastores (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index 352f5c7..a547de1 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -65,7 +65,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void get_table_objects_by_name(std::vector<Table> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names) = 0;
   virtual void get_table_req(GetTableResult& _return, const GetTableRequest& req) = 0;
   virtual void get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req) = 0;
-  virtual void get_materialization_invalidation_info(std::map<std::string, Materialization> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names) = 0;
+  virtual void get_materialization_invalidation_info(Materialization& _return, const CreationMetadata& creation_metadata, const std::string& validTxnList) = 0;
   virtual void update_creation_metadata(const std::string& catName, const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata) = 0;
   virtual void get_table_names_by_filter(std::vector<std::string> & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables) = 0;
   virtual void alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) = 0;
@@ -390,7 +390,7 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void get_table_objects_by_name_req(GetTablesResult& /* _return */, const GetTablesRequest& /* req */) {
     return;
   }
-  void get_materialization_invalidation_info(std::map<std::string, Materialization> & /* _return */, const std::string& /* dbname */, const std::vector<std::string> & /* tbl_names */) {
+  void get_materialization_invalidation_info(Materialization& /* _return */, const CreationMetadata& /* creation_metadata */, const std::string& /* validTxnList */) {
     return;
   }
   void update_creation_metadata(const std::string& /* catName */, const std::string& /* dbname */, const std::string& /* tbl_name */, const CreationMetadata& /* creation_metadata */) {
@@ -6131,9 +6131,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_req_presult {
 };
 
 typedef struct _ThriftHiveMetastore_get_materialization_invalidation_info_args__isset {
-  _ThriftHiveMetastore_get_materialization_invalidation_info_args__isset() : dbname(false), tbl_names(false) {}
-  bool dbname :1;
-  bool tbl_names :1;
+  _ThriftHiveMetastore_get_materialization_invalidation_info_args__isset() : creation_metadata(false), validTxnList(false) {}
+  bool creation_metadata :1;
+  bool validTxnList :1;
 } _ThriftHiveMetastore_get_materialization_invalidation_info_args__isset;
 
 class ThriftHiveMetastore_get_materialization_invalidation_info_args {
@@ -6141,24 +6141,24 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
 
   ThriftHiveMetastore_get_materialization_invalidation_info_args(const ThriftHiveMetastore_get_materialization_invalidation_info_args&);
   ThriftHiveMetastore_get_materialization_invalidation_info_args& operator=(const ThriftHiveMetastore_get_materialization_invalidation_info_args&);
-  ThriftHiveMetastore_get_materialization_invalidation_info_args() : dbname() {
+  ThriftHiveMetastore_get_materialization_invalidation_info_args() : validTxnList() {
   }
 
   virtual ~ThriftHiveMetastore_get_materialization_invalidation_info_args() throw();
-  std::string dbname;
-  std::vector<std::string>  tbl_names;
+  CreationMetadata creation_metadata;
+  std::string validTxnList;
 
   _ThriftHiveMetastore_get_materialization_invalidation_info_args__isset __isset;
 
-  void __set_dbname(const std::string& val);
+  void __set_creation_metadata(const CreationMetadata& val);
 
-  void __set_tbl_names(const std::vector<std::string> & val);
+  void __set_validTxnList(const std::string& val);
 
   bool operator == (const ThriftHiveMetastore_get_materialization_invalidation_info_args & rhs) const
   {
-    if (!(dbname == rhs.dbname))
+    if (!(creation_metadata == rhs.creation_metadata))
       return false;
-    if (!(tbl_names == rhs.tbl_names))
+    if (!(validTxnList == rhs.validTxnList))
       return false;
     return true;
   }
@@ -6179,8 +6179,8 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_pargs {
 
 
   virtual ~ThriftHiveMetastore_get_materialization_invalidation_info_pargs() throw();
-  const std::string* dbname;
-  const std::vector<std::string> * tbl_names;
+  const CreationMetadata* creation_metadata;
+  const std::string* validTxnList;
 
   uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
 
@@ -6203,14 +6203,14 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
   }
 
   virtual ~ThriftHiveMetastore_get_materialization_invalidation_info_result() throw();
-  std::map<std::string, Materialization>  success;
+  Materialization success;
   MetaException o1;
   InvalidOperationException o2;
   UnknownDBException o3;
 
   _ThriftHiveMetastore_get_materialization_invalidation_info_result__isset __isset;
 
-  void __set_success(const std::map<std::string, Materialization> & val);
+  void __set_success(const Materialization& val);
 
   void __set_o1(const MetaException& val);
 
@@ -6254,7 +6254,7 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_presult {
 
 
   virtual ~ThriftHiveMetastore_get_materialization_invalidation_info_presult() throw();
-  std::map<std::string, Materialization> * success;
+  Materialization* success;
   MetaException o1;
   InvalidOperationException o2;
   UnknownDBException o3;
@@ -26460,9 +26460,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req);
   void send_get_table_objects_by_name_req(const GetTablesRequest& req);
   void recv_get_table_objects_by_name_req(GetTablesResult& _return);
-  void get_materialization_invalidation_info(std::map<std::string, Materialization> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names);
-  void send_get_materialization_invalidation_info(const std::string& dbname, const std::vector<std::string> & tbl_names);
-  void recv_get_materialization_invalidation_info(std::map<std::string, Materialization> & _return);
+  void get_materialization_invalidation_info(Materialization& _return, const CreationMetadata& creation_metadata, const std::string& validTxnList);
+  void send_get_materialization_invalidation_info(const CreationMetadata& creation_metadata, const std::string& validTxnList);
+  void recv_get_materialization_invalidation_info(Materialization& _return);
   void update_creation_metadata(const std::string& catName, const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
   void send_update_creation_metadata(const std::string& catName, const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
   void recv_update_creation_metadata();
@@ -27826,13 +27826,13 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     return;
   }
 
-  void get_materialization_invalidation_info(std::map<std::string, Materialization> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names) {
+  void get_materialization_invalidation_info(Materialization& _return, const CreationMetadata& creation_metadata, const std::string& validTxnList) {
     size_t sz = ifaces_.size();
     size_t i = 0;
     for (; i < (sz - 1); ++i) {
-      ifaces_[i]->get_materialization_invalidation_info(_return, dbname, tbl_names);
+      ifaces_[i]->get_materialization_invalidation_info(_return, creation_metadata, validTxnList);
     }
-    ifaces_[i]->get_materialization_invalidation_info(_return, dbname, tbl_names);
+    ifaces_[i]->get_materialization_invalidation_info(_return, creation_metadata, validTxnList);
     return;
   }
 
@@ -29562,9 +29562,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req);
   int32_t send_get_table_objects_by_name_req(const GetTablesRequest& req);
   void recv_get_table_objects_by_name_req(GetTablesResult& _return, const int32_t seqid);
-  void get_materialization_invalidation_info(std::map<std::string, Materialization> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names);
-  int32_t send_get_materialization_invalidation_info(const std::string& dbname, const std::vector<std::string> & tbl_names);
-  void recv_get_materialization_invalidation_info(std::map<std::string, Materialization> & _return, const int32_t seqid);
+  void get_materialization_invalidation_info(Materialization& _return, const CreationMetadata& creation_metadata, const std::string& validTxnList);
+  int32_t send_get_materialization_invalidation_info(const CreationMetadata& creation_metadata, const std::string& validTxnList);
+  void recv_get_materialization_invalidation_info(Materialization& _return, const int32_t seqid);
   void update_creation_metadata(const std::string& catName, const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
   int32_t send_update_creation_metadata(const std::string& catName, const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
   void recv_update_creation_metadata(const int32_t seqid);

http://git-wip-us.apache.org/repos/asf/hive/blob/1b5903b0/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index 789c150..5819b17 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -237,7 +237,7 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("get_table_objects_by_name_req\n");
   }
 
-  void get_materialization_invalidation_info(std::map<std::string, Materialization> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names) {
+  void get_materialization_invalidation_info(Materialization& _return, const CreationMetadata& creation_metadata, const std::string& validTxnList) {
     // Your implementation goes here
     printf("get_materialization_invalidation_info\n");
   }