You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2018/01/20 00:11:47 UTC

[1/8] hive git commit: HIVE-18231 : validate resource plan - part 2 - validate action and trigger expressions (Harish Jaiprakash, reviewed by Sergey Shelukhin)

Repository: hive
Updated Branches:
  refs/heads/master 6d890faf2 -> 90d236aff


http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index a6bae3b..ecc4644 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -7417,10 +7417,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     public WMValidateResourcePlanResponse validate_resource_plan(WMValidateResourcePlanRequest request)
         throws NoSuchObjectException, MetaException, TException {
       try {
-        List<String> errors = getMS().validateResourcePlan(request.getResourcePlanName());
-        WMValidateResourcePlanResponse resp = new WMValidateResourcePlanResponse();
-        resp.setErrors(errors);
-        return resp;
+        return getMS().validateResourcePlan(request.getResourcePlanName());
       } catch (MetaException e) {
         LOG.error("Exception while trying to validate resource plan", e);
         throw e;

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index c44099f..3a468b1 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -2695,11 +2695,11 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   @Override
-  public List<String> validateResourcePlan(String resourcePlanName)
+  public WMValidateResourcePlanResponse validateResourcePlan(String resourcePlanName)
       throws NoSuchObjectException, InvalidObjectException, MetaException, TException {
     WMValidateResourcePlanRequest request = new WMValidateResourcePlanRequest();
     request.setResourcePlanName(resourcePlanName);
-    return client.validate_resource_plan(request).getErrors();
+    return client.validate_resource_plan(request);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 238c5ed..8ec8b3b 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -110,6 +110,7 @@ import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.utils.ObjectPair;
 import org.apache.thrift.TException;
@@ -1827,7 +1828,7 @@ public interface IMetaStoreClient {
 
   WMFullResourcePlan getActiveResourcePlan() throws MetaException, TException;
 
-  List<String> validateResourcePlan(String resourcePlanName)
+  WMValidateResourcePlanResponse validateResourcePlan(String resourcePlanName)
       throws NoSuchObjectException, InvalidObjectException, MetaException, TException;
 
   void createWMTrigger(WMTrigger trigger)

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 6f5d8a6..b3d99a1 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -140,6 +140,7 @@ import org.apache.hadoop.hive.metastore.api.WMPoolTrigger;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlanStatus;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.datasource.DataSourceProvider;
@@ -10128,10 +10129,10 @@ public class ObjectStore implements RawStore, Configurable {
     if (doValidate) {
       // Note: this may use additional inputs from the caller, e.g. maximum query
       // parallelism in the cluster based on physical constraints.
-      List<String> planErrors = getResourcePlanErrors(mResourcePlan);
-      if (!planErrors.isEmpty()) {
+      WMValidateResourcePlanResponse response = getResourcePlanErrors(mResourcePlan);
+      if (!response.getErrors().isEmpty()) {
         throw new InvalidOperationException(
-            "ResourcePlan: " + name + " is invalid: " + planErrors);
+            "ResourcePlan: " + name + " is invalid: " + response.getErrors());
       }
     }
     if (doActivate) {
@@ -10164,8 +10165,6 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   private static class PoolData {
-    int queryParallelism = 0;
-    int totalChildrenQueryParallelism = 0;
     double totalChildrenAllocFraction = 0;
     boolean found = false;
     boolean hasChildren = false;
@@ -10180,12 +10179,16 @@ public class ObjectStore implements RawStore, Configurable {
     return poolData;
   }
 
-  private List<String> getResourcePlanErrors(MWMResourcePlan mResourcePlan) {
-    List<String> errors = new ArrayList<>();
-    if (mResourcePlan.getQueryParallelism() != null && mResourcePlan.getQueryParallelism() < 1) {
-      errors.add("Query parallelism should for resource plan be positive. Got: " +
-        mResourcePlan.getQueryParallelism());
+  private WMValidateResourcePlanResponse getResourcePlanErrors(MWMResourcePlan mResourcePlan) {
+    WMValidateResourcePlanResponse response = new WMValidateResourcePlanResponse();
+    response.setErrors(new ArrayList());
+    response.setWarnings(new ArrayList());
+    Integer rpParallelism = mResourcePlan.getQueryParallelism();
+    if (rpParallelism != null && rpParallelism < 1) {
+      response.addToErrors("Query parallelism should for resource plan be positive. Got: " +
+          rpParallelism);
     }
+    int totalQueryParallelism = 0;
     Map<String, PoolData> poolInfo = new HashMap<>();
     for (MWMPool pool : mResourcePlan.getPools()) {
       PoolData currentPoolData = getPoolData(poolInfo, pool.getPath());
@@ -10195,45 +10198,54 @@ public class ObjectStore implements RawStore, Configurable {
       parentPoolData.hasChildren = true;
       parentPoolData.totalChildrenAllocFraction += pool.getAllocFraction();
       if (pool.getQueryParallelism() != null && pool.getQueryParallelism() < 1) {
-        errors.add("Invalid query parallelism for pool: " + pool.getPath());
+        response.addToErrors("Invalid query parallelism for pool: " + pool.getPath());
       } else {
-        currentPoolData.queryParallelism = pool.getQueryParallelism();
-        parentPoolData.totalChildrenQueryParallelism += pool.getQueryParallelism();
+        totalQueryParallelism += pool.getQueryParallelism();
       }
       if (!MetaStoreUtils.isValidSchedulingPolicy(pool.getSchedulingPolicy())) {
-        errors.add("Invalid scheduling policy "
-            + pool.getSchedulingPolicy() + " for pool: " + pool.getPath());
+        response.addToErrors("Invalid scheduling policy " + pool.getSchedulingPolicy() +
+            " for pool: " + pool.getPath());
+      }
+    }
+    if (rpParallelism != null) {
+      if (rpParallelism < totalQueryParallelism) {
+        response.addToErrors("Sum of all pools' query parallelism: " + totalQueryParallelism  +
+            " exceeds resource plan query parallelism: " + rpParallelism);
+      } else if (rpParallelism != totalQueryParallelism) {
+        response.addToWarnings("Sum of all pools' query parallelism: " + totalQueryParallelism  +
+            " is less than resource plan query parallelism: " + rpParallelism);
       }
     }
     for (Entry<String, PoolData> entry : poolInfo.entrySet()) {
-      PoolData poolData = entry.getValue();
+      final PoolData poolData = entry.getValue();
+      final boolean isRoot = entry.getKey().isEmpty();
       // Special case for root parent
-      if (entry.getKey().equals("")) {
+      if (isRoot) {
         poolData.found = true;
-        poolData.queryParallelism = mResourcePlan.getQueryParallelism() == null ?
-            poolData.totalChildrenQueryParallelism : mResourcePlan.getQueryParallelism();
+        if (!poolData.hasChildren) {
+          response.addToErrors("Root has no children");
+        } else if (Math.abs(1.0 - poolData.totalChildrenAllocFraction) > 0.001) {
+          response.addToErrors("Sum of root children pools' alloc fraction should be 1.0 got: " +
+              poolData.totalChildrenAllocFraction + " for pool: " + entry.getKey());
+        }
       }
       if (!poolData.found) {
-        errors.add("Pool does not exists but has children: " + entry.getKey());
+        response.addToErrors("Pool does not exists but has children: " + entry.getKey());
       }
       if (poolData.hasChildren) {
-        if (Math.abs(1.0 - poolData.totalChildrenAllocFraction) > 0.001) {
-          errors.add("Sum of children pools' alloc fraction should be equal 1.0 got: " +
-              poolData.totalChildrenAllocFraction + " for pool: " + entry.getKey());
-        }
-        if (poolData.queryParallelism != poolData.totalChildrenQueryParallelism) {
-          errors.add("Sum of children pools' query parallelism: " +
-              poolData.totalChildrenQueryParallelism + " is not equal to pool parallelism: " +
-              poolData.queryParallelism + " for pool: " + entry.getKey());
+        if (!isRoot && 1.0 <= poolData.totalChildrenAllocFraction) {
+          response.addToErrors("Sum of children pools' alloc fraction should be less than 1 got: "
+              + poolData.totalChildrenAllocFraction + " for pool: " + entry.getKey());
         }
       }
     }
-    // TODO: validate trigger and action expressions. mResourcePlan.getTriggers()
-    return errors;
+    // trigger and action expressions are not validated here, since counters are not
+    // available and grammar check is there in the language itself.
+    return response;
   }
 
   @Override
-  public List<String> validateResourcePlan(String name)
+  public WMValidateResourcePlanResponse validateResourcePlan(String name)
       throws NoSuchObjectException, InvalidObjectException, MetaException {
     name = normalizeIdentifier(name);
     Query query = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
index f4eff4c..e4e7d42 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
-import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -62,6 +61,7 @@ import org.apache.hadoop.hive.metastore.api.WMNullablePool;
 import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
@@ -771,7 +771,7 @@ public interface RawStore extends Configurable {
 
   WMFullResourcePlan getActiveResourcePlan() throws MetaException;
 
-  List<String> validateResourcePlan(String name)
+  WMValidateResourcePlanResponse validateResourcePlan(String name)
       throws NoSuchObjectException, InvalidObjectException, MetaException;
 
   void dropResourcePlan(String name) throws NoSuchObjectException, MetaException;

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
index 5598540..80aa3bc 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
@@ -80,6 +80,7 @@ import org.apache.hadoop.hive.metastore.api.WMNullablePool;
 import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.cache.SharedCache.StatsType;
 import org.apache.hadoop.hive.metastore.columnstats.aggr.ColumnStatsAggregator;
 import org.apache.hadoop.hive.metastore.columnstats.aggr.ColumnStatsAggregatorFactory;
@@ -2634,7 +2635,7 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<String> validateResourcePlan(String name)
+  public WMValidateResourcePlanResponse validateResourcePlan(String name)
       throws NoSuchObjectException, InvalidObjectException, MetaException {
     return rawStore.validateResourcePlan(name);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/thrift/hive_metastore.thrift b/standalone-metastore/src/main/thrift/hive_metastore.thrift
index b77afac..93f3e53 100644
--- a/standalone-metastore/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/src/main/thrift/hive_metastore.thrift
@@ -1179,6 +1179,7 @@ struct WMValidateResourcePlanRequest {
 
 struct WMValidateResourcePlanResponse {
   1: optional list<string> errors;
+  2: optional list<string> warnings;
 }
 
 struct WMDropResourcePlanRequest {

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index e0a67bc..9100c73 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.hive.metastore.api.WMNullablePool;
 import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
@@ -974,7 +975,7 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public List<String> validateResourcePlan(String name)
+  public WMValidateResourcePlanResponse validateResourcePlan(String name)
       throws NoSuchObjectException, InvalidObjectException, MetaException {
     return objectStore.validateResourcePlan(name);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index aad2914..86e72d8 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hive.metastore.api.WMNullablePool;
 import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
@@ -984,9 +985,9 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public List<String> validateResourcePlan(String name)
+  public WMValidateResourcePlanResponse validateResourcePlan(String name)
       throws NoSuchObjectException, InvalidObjectException, MetaException {
-    return Collections.emptyList();
+    return null;
   }
 
   @Override


[3/8] hive git commit: HIVE-18231 : validate resource plan - part 2 - validate action and trigger expressions (Harish Jaiprakash, reviewed by Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index e383e9a..bf7d466 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -12984,14 +12984,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size725 = 0;
-            $_etype728 = 0;
-            $xfer += $input->readListBegin($_etype728, $_size725);
-            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
+            $_size732 = 0;
+            $_etype735 = 0;
+            $xfer += $input->readListBegin($_etype735, $_size732);
+            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
             {
-              $elem730 = null;
-              $xfer += $input->readString($elem730);
-              $this->success []= $elem730;
+              $elem737 = null;
+              $xfer += $input->readString($elem737);
+              $this->success []= $elem737;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13027,9 +13027,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter731)
+          foreach ($this->success as $iter738)
           {
-            $xfer += $output->writeString($iter731);
+            $xfer += $output->writeString($iter738);
           }
         }
         $output->writeListEnd();
@@ -13160,14 +13160,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size732 = 0;
-            $_etype735 = 0;
-            $xfer += $input->readListBegin($_etype735, $_size732);
-            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
+            $_size739 = 0;
+            $_etype742 = 0;
+            $xfer += $input->readListBegin($_etype742, $_size739);
+            for ($_i743 = 0; $_i743 < $_size739; ++$_i743)
             {
-              $elem737 = null;
-              $xfer += $input->readString($elem737);
-              $this->success []= $elem737;
+              $elem744 = null;
+              $xfer += $input->readString($elem744);
+              $this->success []= $elem744;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13203,9 +13203,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter738)
+          foreach ($this->success as $iter745)
           {
-            $xfer += $output->writeString($iter738);
+            $xfer += $output->writeString($iter745);
           }
         }
         $output->writeListEnd();
@@ -14206,18 +14206,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size739 = 0;
-            $_ktype740 = 0;
-            $_vtype741 = 0;
-            $xfer += $input->readMapBegin($_ktype740, $_vtype741, $_size739);
-            for ($_i743 = 0; $_i743 < $_size739; ++$_i743)
+            $_size746 = 0;
+            $_ktype747 = 0;
+            $_vtype748 = 0;
+            $xfer += $input->readMapBegin($_ktype747, $_vtype748, $_size746);
+            for ($_i750 = 0; $_i750 < $_size746; ++$_i750)
             {
-              $key744 = '';
-              $val745 = new \metastore\Type();
-              $xfer += $input->readString($key744);
-              $val745 = new \metastore\Type();
-              $xfer += $val745->read($input);
-              $this->success[$key744] = $val745;
+              $key751 = '';
+              $val752 = new \metastore\Type();
+              $xfer += $input->readString($key751);
+              $val752 = new \metastore\Type();
+              $xfer += $val752->read($input);
+              $this->success[$key751] = $val752;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -14253,10 +14253,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter746 => $viter747)
+          foreach ($this->success as $kiter753 => $viter754)
           {
-            $xfer += $output->writeString($kiter746);
-            $xfer += $viter747->write($output);
+            $xfer += $output->writeString($kiter753);
+            $xfer += $viter754->write($output);
           }
         }
         $output->writeMapEnd();
@@ -14460,15 +14460,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size748 = 0;
-            $_etype751 = 0;
-            $xfer += $input->readListBegin($_etype751, $_size748);
-            for ($_i752 = 0; $_i752 < $_size748; ++$_i752)
+            $_size755 = 0;
+            $_etype758 = 0;
+            $xfer += $input->readListBegin($_etype758, $_size755);
+            for ($_i759 = 0; $_i759 < $_size755; ++$_i759)
             {
-              $elem753 = null;
-              $elem753 = new \metastore\FieldSchema();
-              $xfer += $elem753->read($input);
-              $this->success []= $elem753;
+              $elem760 = null;
+              $elem760 = new \metastore\FieldSchema();
+              $xfer += $elem760->read($input);
+              $this->success []= $elem760;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14520,9 +14520,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter754)
+          foreach ($this->success as $iter761)
           {
-            $xfer += $iter754->write($output);
+            $xfer += $iter761->write($output);
           }
         }
         $output->writeListEnd();
@@ -14764,15 +14764,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size755 = 0;
-            $_etype758 = 0;
-            $xfer += $input->readListBegin($_etype758, $_size755);
-            for ($_i759 = 0; $_i759 < $_size755; ++$_i759)
+            $_size762 = 0;
+            $_etype765 = 0;
+            $xfer += $input->readListBegin($_etype765, $_size762);
+            for ($_i766 = 0; $_i766 < $_size762; ++$_i766)
             {
-              $elem760 = null;
-              $elem760 = new \metastore\FieldSchema();
-              $xfer += $elem760->read($input);
-              $this->success []= $elem760;
+              $elem767 = null;
+              $elem767 = new \metastore\FieldSchema();
+              $xfer += $elem767->read($input);
+              $this->success []= $elem767;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14824,9 +14824,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter761)
+          foreach ($this->success as $iter768)
           {
-            $xfer += $iter761->write($output);
+            $xfer += $iter768->write($output);
           }
         }
         $output->writeListEnd();
@@ -15040,15 +15040,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size762 = 0;
-            $_etype765 = 0;
-            $xfer += $input->readListBegin($_etype765, $_size762);
-            for ($_i766 = 0; $_i766 < $_size762; ++$_i766)
+            $_size769 = 0;
+            $_etype772 = 0;
+            $xfer += $input->readListBegin($_etype772, $_size769);
+            for ($_i773 = 0; $_i773 < $_size769; ++$_i773)
             {
-              $elem767 = null;
-              $elem767 = new \metastore\FieldSchema();
-              $xfer += $elem767->read($input);
-              $this->success []= $elem767;
+              $elem774 = null;
+              $elem774 = new \metastore\FieldSchema();
+              $xfer += $elem774->read($input);
+              $this->success []= $elem774;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15100,9 +15100,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter768)
+          foreach ($this->success as $iter775)
           {
-            $xfer += $iter768->write($output);
+            $xfer += $iter775->write($output);
           }
         }
         $output->writeListEnd();
@@ -15344,15 +15344,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size769 = 0;
-            $_etype772 = 0;
-            $xfer += $input->readListBegin($_etype772, $_size769);
-            for ($_i773 = 0; $_i773 < $_size769; ++$_i773)
+            $_size776 = 0;
+            $_etype779 = 0;
+            $xfer += $input->readListBegin($_etype779, $_size776);
+            for ($_i780 = 0; $_i780 < $_size776; ++$_i780)
             {
-              $elem774 = null;
-              $elem774 = new \metastore\FieldSchema();
-              $xfer += $elem774->read($input);
-              $this->success []= $elem774;
+              $elem781 = null;
+              $elem781 = new \metastore\FieldSchema();
+              $xfer += $elem781->read($input);
+              $this->success []= $elem781;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15404,9 +15404,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter775)
+          foreach ($this->success as $iter782)
           {
-            $xfer += $iter775->write($output);
+            $xfer += $iter782->write($output);
           }
         }
         $output->writeListEnd();
@@ -16046,15 +16046,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size776 = 0;
-            $_etype779 = 0;
-            $xfer += $input->readListBegin($_etype779, $_size776);
-            for ($_i780 = 0; $_i780 < $_size776; ++$_i780)
+            $_size783 = 0;
+            $_etype786 = 0;
+            $xfer += $input->readListBegin($_etype786, $_size783);
+            for ($_i787 = 0; $_i787 < $_size783; ++$_i787)
             {
-              $elem781 = null;
-              $elem781 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem781->read($input);
-              $this->primaryKeys []= $elem781;
+              $elem788 = null;
+              $elem788 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem788->read($input);
+              $this->primaryKeys []= $elem788;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16064,15 +16064,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size782 = 0;
-            $_etype785 = 0;
-            $xfer += $input->readListBegin($_etype785, $_size782);
-            for ($_i786 = 0; $_i786 < $_size782; ++$_i786)
+            $_size789 = 0;
+            $_etype792 = 0;
+            $xfer += $input->readListBegin($_etype792, $_size789);
+            for ($_i793 = 0; $_i793 < $_size789; ++$_i793)
             {
-              $elem787 = null;
-              $elem787 = new \metastore\SQLForeignKey();
-              $xfer += $elem787->read($input);
-              $this->foreignKeys []= $elem787;
+              $elem794 = null;
+              $elem794 = new \metastore\SQLForeignKey();
+              $xfer += $elem794->read($input);
+              $this->foreignKeys []= $elem794;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16082,15 +16082,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 4:
           if ($ftype == TType::LST) {
             $this->uniqueConstraints = array();
-            $_size788 = 0;
-            $_etype791 = 0;
-            $xfer += $input->readListBegin($_etype791, $_size788);
-            for ($_i792 = 0; $_i792 < $_size788; ++$_i792)
+            $_size795 = 0;
+            $_etype798 = 0;
+            $xfer += $input->readListBegin($_etype798, $_size795);
+            for ($_i799 = 0; $_i799 < $_size795; ++$_i799)
             {
-              $elem793 = null;
-              $elem793 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem793->read($input);
-              $this->uniqueConstraints []= $elem793;
+              $elem800 = null;
+              $elem800 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem800->read($input);
+              $this->uniqueConstraints []= $elem800;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16100,15 +16100,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->notNullConstraints = array();
-            $_size794 = 0;
-            $_etype797 = 0;
-            $xfer += $input->readListBegin($_etype797, $_size794);
-            for ($_i798 = 0; $_i798 < $_size794; ++$_i798)
+            $_size801 = 0;
+            $_etype804 = 0;
+            $xfer += $input->readListBegin($_etype804, $_size801);
+            for ($_i805 = 0; $_i805 < $_size801; ++$_i805)
             {
-              $elem799 = null;
-              $elem799 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem799->read($input);
-              $this->notNullConstraints []= $elem799;
+              $elem806 = null;
+              $elem806 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem806->read($input);
+              $this->notNullConstraints []= $elem806;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16144,9 +16144,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter800)
+          foreach ($this->primaryKeys as $iter807)
           {
-            $xfer += $iter800->write($output);
+            $xfer += $iter807->write($output);
           }
         }
         $output->writeListEnd();
@@ -16161,9 +16161,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter801)
+          foreach ($this->foreignKeys as $iter808)
           {
-            $xfer += $iter801->write($output);
+            $xfer += $iter808->write($output);
           }
         }
         $output->writeListEnd();
@@ -16178,9 +16178,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
         {
-          foreach ($this->uniqueConstraints as $iter802)
+          foreach ($this->uniqueConstraints as $iter809)
           {
-            $xfer += $iter802->write($output);
+            $xfer += $iter809->write($output);
           }
         }
         $output->writeListEnd();
@@ -16195,9 +16195,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
         {
-          foreach ($this->notNullConstraints as $iter803)
+          foreach ($this->notNullConstraints as $iter810)
           {
-            $xfer += $iter803->write($output);
+            $xfer += $iter810->write($output);
           }
         }
         $output->writeListEnd();
@@ -17833,14 +17833,14 @@ class ThriftHiveMetastore_truncate_table_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size804 = 0;
-            $_etype807 = 0;
-            $xfer += $input->readListBegin($_etype807, $_size804);
-            for ($_i808 = 0; $_i808 < $_size804; ++$_i808)
+            $_size811 = 0;
+            $_etype814 = 0;
+            $xfer += $input->readListBegin($_etype814, $_size811);
+            for ($_i815 = 0; $_i815 < $_size811; ++$_i815)
             {
-              $elem809 = null;
-              $xfer += $input->readString($elem809);
-              $this->partNames []= $elem809;
+              $elem816 = null;
+              $xfer += $input->readString($elem816);
+              $this->partNames []= $elem816;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17878,9 +17878,9 @@ class ThriftHiveMetastore_truncate_table_args {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter810)
+          foreach ($this->partNames as $iter817)
           {
-            $xfer += $output->writeString($iter810);
+            $xfer += $output->writeString($iter817);
           }
         }
         $output->writeListEnd();
@@ -18131,14 +18131,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size811 = 0;
-            $_etype814 = 0;
-            $xfer += $input->readListBegin($_etype814, $_size811);
-            for ($_i815 = 0; $_i815 < $_size811; ++$_i815)
+            $_size818 = 0;
+            $_etype821 = 0;
+            $xfer += $input->readListBegin($_etype821, $_size818);
+            for ($_i822 = 0; $_i822 < $_size818; ++$_i822)
             {
-              $elem816 = null;
-              $xfer += $input->readString($elem816);
-              $this->success []= $elem816;
+              $elem823 = null;
+              $xfer += $input->readString($elem823);
+              $this->success []= $elem823;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18174,9 +18174,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter817)
+          foreach ($this->success as $iter824)
           {
-            $xfer += $output->writeString($iter817);
+            $xfer += $output->writeString($iter824);
           }
         }
         $output->writeListEnd();
@@ -18378,14 +18378,14 @@ class ThriftHiveMetastore_get_tables_by_type_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size818 = 0;
-            $_etype821 = 0;
-            $xfer += $input->readListBegin($_etype821, $_size818);
-            for ($_i822 = 0; $_i822 < $_size818; ++$_i822)
+            $_size825 = 0;
+            $_etype828 = 0;
+            $xfer += $input->readListBegin($_etype828, $_size825);
+            for ($_i829 = 0; $_i829 < $_size825; ++$_i829)
             {
-              $elem823 = null;
-              $xfer += $input->readString($elem823);
-              $this->success []= $elem823;
+              $elem830 = null;
+              $xfer += $input->readString($elem830);
+              $this->success []= $elem830;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18421,9 +18421,9 @@ class ThriftHiveMetastore_get_tables_by_type_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter824)
+          foreach ($this->success as $iter831)
           {
-            $xfer += $output->writeString($iter824);
+            $xfer += $output->writeString($iter831);
           }
         }
         $output->writeListEnd();
@@ -18579,14 +18579,14 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size825 = 0;
-            $_etype828 = 0;
-            $xfer += $input->readListBegin($_etype828, $_size825);
-            for ($_i829 = 0; $_i829 < $_size825; ++$_i829)
+            $_size832 = 0;
+            $_etype835 = 0;
+            $xfer += $input->readListBegin($_etype835, $_size832);
+            for ($_i836 = 0; $_i836 < $_size832; ++$_i836)
             {
-              $elem830 = null;
-              $xfer += $input->readString($elem830);
-              $this->success []= $elem830;
+              $elem837 = null;
+              $xfer += $input->readString($elem837);
+              $this->success []= $elem837;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18622,9 +18622,9 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter831)
+          foreach ($this->success as $iter838)
           {
-            $xfer += $output->writeString($iter831);
+            $xfer += $output->writeString($iter838);
           }
         }
         $output->writeListEnd();
@@ -18729,14 +18729,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size832 = 0;
-            $_etype835 = 0;
-            $xfer += $input->readListBegin($_etype835, $_size832);
-            for ($_i836 = 0; $_i836 < $_size832; ++$_i836)
+            $_size839 = 0;
+            $_etype842 = 0;
+            $xfer += $input->readListBegin($_etype842, $_size839);
+            for ($_i843 = 0; $_i843 < $_size839; ++$_i843)
             {
-              $elem837 = null;
-              $xfer += $input->readString($elem837);
-              $this->tbl_types []= $elem837;
+              $elem844 = null;
+              $xfer += $input->readString($elem844);
+              $this->tbl_types []= $elem844;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18774,9 +18774,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter838)
+          foreach ($this->tbl_types as $iter845)
           {
-            $xfer += $output->writeString($iter838);
+            $xfer += $output->writeString($iter845);
           }
         }
         $output->writeListEnd();
@@ -18853,15 +18853,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size839 = 0;
-            $_etype842 = 0;
-            $xfer += $input->readListBegin($_etype842, $_size839);
-            for ($_i843 = 0; $_i843 < $_size839; ++$_i843)
+            $_size846 = 0;
+            $_etype849 = 0;
+            $xfer += $input->readListBegin($_etype849, $_size846);
+            for ($_i850 = 0; $_i850 < $_size846; ++$_i850)
             {
-              $elem844 = null;
-              $elem844 = new \metastore\TableMeta();
-              $xfer += $elem844->read($input);
-              $this->success []= $elem844;
+              $elem851 = null;
+              $elem851 = new \metastore\TableMeta();
+              $xfer += $elem851->read($input);
+              $this->success []= $elem851;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18897,9 +18897,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter845)
+          foreach ($this->success as $iter852)
           {
-            $xfer += $iter845->write($output);
+            $xfer += $iter852->write($output);
           }
         }
         $output->writeListEnd();
@@ -19055,14 +19055,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size846 = 0;
-            $_etype849 = 0;
-            $xfer += $input->readListBegin($_etype849, $_size846);
-            for ($_i850 = 0; $_i850 < $_size846; ++$_i850)
+            $_size853 = 0;
+            $_etype856 = 0;
+            $xfer += $input->readListBegin($_etype856, $_size853);
+            for ($_i857 = 0; $_i857 < $_size853; ++$_i857)
             {
-              $elem851 = null;
-              $xfer += $input->readString($elem851);
-              $this->success []= $elem851;
+              $elem858 = null;
+              $xfer += $input->readString($elem858);
+              $this->success []= $elem858;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19098,9 +19098,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter852)
+          foreach ($this->success as $iter859)
           {
-            $xfer += $output->writeString($iter852);
+            $xfer += $output->writeString($iter859);
           }
         }
         $output->writeListEnd();
@@ -19415,14 +19415,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size853 = 0;
-            $_etype856 = 0;
-            $xfer += $input->readListBegin($_etype856, $_size853);
-            for ($_i857 = 0; $_i857 < $_size853; ++$_i857)
+            $_size860 = 0;
+            $_etype863 = 0;
+            $xfer += $input->readListBegin($_etype863, $_size860);
+            for ($_i864 = 0; $_i864 < $_size860; ++$_i864)
             {
-              $elem858 = null;
-              $xfer += $input->readString($elem858);
-              $this->tbl_names []= $elem858;
+              $elem865 = null;
+              $xfer += $input->readString($elem865);
+              $this->tbl_names []= $elem865;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19455,9 +19455,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter859)
+          foreach ($this->tbl_names as $iter866)
           {
-            $xfer += $output->writeString($iter859);
+            $xfer += $output->writeString($iter866);
           }
         }
         $output->writeListEnd();
@@ -19522,15 +19522,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size860 = 0;
-            $_etype863 = 0;
-            $xfer += $input->readListBegin($_etype863, $_size860);
-            for ($_i864 = 0; $_i864 < $_size860; ++$_i864)
+            $_size867 = 0;
+            $_etype870 = 0;
+            $xfer += $input->readListBegin($_etype870, $_size867);
+            for ($_i871 = 0; $_i871 < $_size867; ++$_i871)
             {
-              $elem865 = null;
-              $elem865 = new \metastore\Table();
-              $xfer += $elem865->read($input);
-              $this->success []= $elem865;
+              $elem872 = null;
+              $elem872 = new \metastore\Table();
+              $xfer += $elem872->read($input);
+              $this->success []= $elem872;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19558,9 +19558,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter866)
+          foreach ($this->success as $iter873)
           {
-            $xfer += $iter866->write($output);
+            $xfer += $iter873->write($output);
           }
         }
         $output->writeListEnd();
@@ -20087,14 +20087,14 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size867 = 0;
-            $_etype870 = 0;
-            $xfer += $input->readListBegin($_etype870, $_size867);
-            for ($_i871 = 0; $_i871 < $_size867; ++$_i871)
+            $_size874 = 0;
+            $_etype877 = 0;
+            $xfer += $input->readListBegin($_etype877, $_size874);
+            for ($_i878 = 0; $_i878 < $_size874; ++$_i878)
             {
-              $elem872 = null;
-              $xfer += $input->readString($elem872);
-              $this->tbl_names []= $elem872;
+              $elem879 = null;
+              $xfer += $input->readString($elem879);
+              $this->tbl_names []= $elem879;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20127,9 +20127,9 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter873)
+          foreach ($this->tbl_names as $iter880)
           {
-            $xfer += $output->writeString($iter873);
+            $xfer += $output->writeString($iter880);
           }
         }
         $output->writeListEnd();
@@ -20234,18 +20234,18 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size874 = 0;
-            $_ktype875 = 0;
-            $_vtype876 = 0;
-            $xfer += $input->readMapBegin($_ktype875, $_vtype876, $_size874);
-            for ($_i878 = 0; $_i878 < $_size874; ++$_i878)
+            $_size881 = 0;
+            $_ktype882 = 0;
+            $_vtype883 = 0;
+            $xfer += $input->readMapBegin($_ktype882, $_vtype883, $_size881);
+            for ($_i885 = 0; $_i885 < $_size881; ++$_i885)
             {
-              $key879 = '';
-              $val880 = new \metastore\Materialization();
-              $xfer += $input->readString($key879);
-              $val880 = new \metastore\Materialization();
-              $xfer += $val880->read($input);
-              $this->success[$key879] = $val880;
+              $key886 = '';
+              $val887 = new \metastore\Materialization();
+              $xfer += $input->readString($key886);
+              $val887 = new \metastore\Materialization();
+              $xfer += $val887->read($input);
+              $this->success[$key886] = $val887;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -20297,10 +20297,10 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter881 => $viter882)
+          foreach ($this->success as $kiter888 => $viter889)
           {
-            $xfer += $output->writeString($kiter881);
-            $xfer += $viter882->write($output);
+            $xfer += $output->writeString($kiter888);
+            $xfer += $viter889->write($output);
           }
         }
         $output->writeMapEnd();
@@ -20536,14 +20536,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size883 = 0;
-            $_etype886 = 0;
-            $xfer += $input->readListBegin($_etype886, $_size883);
-            for ($_i887 = 0; $_i887 < $_size883; ++$_i887)
+            $_size890 = 0;
+            $_etype893 = 0;
+            $xfer += $input->readListBegin($_etype893, $_size890);
+            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
             {
-              $elem888 = null;
-              $xfer += $input->readString($elem888);
-              $this->success []= $elem888;
+              $elem895 = null;
+              $xfer += $input->readString($elem895);
+              $this->success []= $elem895;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20595,9 +20595,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter889)
+          foreach ($this->success as $iter896)
           {
-            $xfer += $output->writeString($iter889);
+            $xfer += $output->writeString($iter896);
           }
         }
         $output->writeListEnd();
@@ -21910,15 +21910,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size890 = 0;
-            $_etype893 = 0;
-            $xfer += $input->readListBegin($_etype893, $_size890);
-            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
+            $_size897 = 0;
+            $_etype900 = 0;
+            $xfer += $input->readListBegin($_etype900, $_size897);
+            for ($_i901 = 0; $_i901 < $_size897; ++$_i901)
             {
-              $elem895 = null;
-              $elem895 = new \metastore\Partition();
-              $xfer += $elem895->read($input);
-              $this->new_parts []= $elem895;
+              $elem902 = null;
+              $elem902 = new \metastore\Partition();
+              $xfer += $elem902->read($input);
+              $this->new_parts []= $elem902;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21946,9 +21946,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter896)
+          foreach ($this->new_parts as $iter903)
           {
-            $xfer += $iter896->write($output);
+            $xfer += $iter903->write($output);
           }
         }
         $output->writeListEnd();
@@ -22163,15 +22163,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size897 = 0;
-            $_etype900 = 0;
-            $xfer += $input->readListBegin($_etype900, $_size897);
-            for ($_i901 = 0; $_i901 < $_size897; ++$_i901)
+            $_size904 = 0;
+            $_etype907 = 0;
+            $xfer += $input->readListBegin($_etype907, $_size904);
+            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
             {
-              $elem902 = null;
-              $elem902 = new \metastore\PartitionSpec();
-              $xfer += $elem902->read($input);
-              $this->new_parts []= $elem902;
+              $elem909 = null;
+              $elem909 = new \metastore\PartitionSpec();
+              $xfer += $elem909->read($input);
+              $this->new_parts []= $elem909;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22199,9 +22199,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter903)
+          foreach ($this->new_parts as $iter910)
           {
-            $xfer += $iter903->write($output);
+            $xfer += $iter910->write($output);
           }
         }
         $output->writeListEnd();
@@ -22451,14 +22451,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size904 = 0;
-            $_etype907 = 0;
-            $xfer += $input->readListBegin($_etype907, $_size904);
-            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
+            $_size911 = 0;
+            $_etype914 = 0;
+            $xfer += $input->readListBegin($_etype914, $_size911);
+            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
             {
-              $elem909 = null;
-              $xfer += $input->readString($elem909);
-              $this->part_vals []= $elem909;
+              $elem916 = null;
+              $xfer += $input->readString($elem916);
+              $this->part_vals []= $elem916;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22496,9 +22496,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter910)
+          foreach ($this->part_vals as $iter917)
           {
-            $xfer += $output->writeString($iter910);
+            $xfer += $output->writeString($iter917);
           }
         }
         $output->writeListEnd();
@@ -23000,14 +23000,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size911 = 0;
-            $_etype914 = 0;
-            $xfer += $input->readListBegin($_etype914, $_size911);
-            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
+            $_size918 = 0;
+            $_etype921 = 0;
+            $xfer += $input->readListBegin($_etype921, $_size918);
+            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
             {
-              $elem916 = null;
-              $xfer += $input->readString($elem916);
-              $this->part_vals []= $elem916;
+              $elem923 = null;
+              $xfer += $input->readString($elem923);
+              $this->part_vals []= $elem923;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23053,9 +23053,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter917)
+          foreach ($this->part_vals as $iter924)
           {
-            $xfer += $output->writeString($iter917);
+            $xfer += $output->writeString($iter924);
           }
         }
         $output->writeListEnd();
@@ -23909,14 +23909,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size918 = 0;
-            $_etype921 = 0;
-            $xfer += $input->readListBegin($_etype921, $_size918);
-            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
+            $_size925 = 0;
+            $_etype928 = 0;
+            $xfer += $input->readListBegin($_etype928, $_size925);
+            for ($_i929 = 0; $_i929 < $_size925; ++$_i929)
             {
-              $elem923 = null;
-              $xfer += $input->readString($elem923);
-              $this->part_vals []= $elem923;
+              $elem930 = null;
+              $xfer += $input->readString($elem930);
+              $this->part_vals []= $elem930;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23961,9 +23961,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter924)
+          foreach ($this->part_vals as $iter931)
           {
-            $xfer += $output->writeString($iter924);
+            $xfer += $output->writeString($iter931);
           }
         }
         $output->writeListEnd();
@@ -24216,14 +24216,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size925 = 0;
-            $_etype928 = 0;
-            $xfer += $input->readListBegin($_etype928, $_size925);
-            for ($_i929 = 0; $_i929 < $_size925; ++$_i929)
+            $_size932 = 0;
+            $_etype935 = 0;
+            $xfer += $input->readListBegin($_etype935, $_size932);
+            for ($_i936 = 0; $_i936 < $_size932; ++$_i936)
             {
-              $elem930 = null;
-              $xfer += $input->readString($elem930);
-              $this->part_vals []= $elem930;
+              $elem937 = null;
+              $xfer += $input->readString($elem937);
+              $this->part_vals []= $elem937;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24276,9 +24276,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter931)
+          foreach ($this->part_vals as $iter938)
           {
-            $xfer += $output->writeString($iter931);
+            $xfer += $output->writeString($iter938);
           }
         }
         $output->writeListEnd();
@@ -25292,14 +25292,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size932 = 0;
-            $_etype935 = 0;
-            $xfer += $input->readListBegin($_etype935, $_size932);
-            for ($_i936 = 0; $_i936 < $_size932; ++$_i936)
+            $_size939 = 0;
+            $_etype942 = 0;
+            $xfer += $input->readListBegin($_etype942, $_size939);
+            for ($_i943 = 0; $_i943 < $_size939; ++$_i943)
             {
-              $elem937 = null;
-              $xfer += $input->readString($elem937);
-              $this->part_vals []= $elem937;
+              $elem944 = null;
+              $xfer += $input->readString($elem944);
+              $this->part_vals []= $elem944;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25337,9 +25337,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter938)
+          foreach ($this->part_vals as $iter945)
           {
-            $xfer += $output->writeString($iter938);
+            $xfer += $output->writeString($iter945);
           }
         }
         $output->writeListEnd();
@@ -25581,17 +25581,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size939 = 0;
-            $_ktype940 = 0;
-            $_vtype941 = 0;
-            $xfer += $input->readMapBegin($_ktype940, $_vtype941, $_size939);
-            for ($_i943 = 0; $_i943 < $_size939; ++$_i943)
+            $_size946 = 0;
+            $_ktype947 = 0;
+            $_vtype948 = 0;
+            $xfer += $input->readMapBegin($_ktype947, $_vtype948, $_size946);
+            for ($_i950 = 0; $_i950 < $_size946; ++$_i950)
             {
-              $key944 = '';
-              $val945 = '';
-              $xfer += $input->readString($key944);
-              $xfer += $input->readString($val945);
-              $this->partitionSpecs[$key944] = $val945;
+              $key951 = '';
+              $val952 = '';
+              $xfer += $input->readString($key951);
+              $xfer += $input->readString($val952);
+              $this->partitionSpecs[$key951] = $val952;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25647,10 +25647,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter946 => $viter947)
+          foreach ($this->partitionSpecs as $kiter953 => $viter954)
           {
-            $xfer += $output->writeString($kiter946);
-            $xfer += $output->writeString($viter947);
+            $xfer += $output->writeString($kiter953);
+            $xfer += $output->writeString($viter954);
           }
         }
         $output->writeMapEnd();
@@ -25962,17 +25962,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size948 = 0;
-            $_ktype949 = 0;
-            $_vtype950 = 0;
-            $xfer += $input->readMapBegin($_ktype949, $_vtype950, $_size948);
-            for ($_i952 = 0; $_i952 < $_size948; ++$_i952)
+            $_size955 = 0;
+            $_ktype956 = 0;
+            $_vtype957 = 0;
+            $xfer += $input->readMapBegin($_ktype956, $_vtype957, $_size955);
+            for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
             {
-              $key953 = '';
-              $val954 = '';
-              $xfer += $input->readString($key953);
-              $xfer += $input->readString($val954);
-              $this->partitionSpecs[$key953] = $val954;
+              $key960 = '';
+              $val961 = '';
+              $xfer += $input->readString($key960);
+              $xfer += $input->readString($val961);
+              $this->partitionSpecs[$key960] = $val961;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -26028,10 +26028,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter955 => $viter956)
+          foreach ($this->partitionSpecs as $kiter962 => $viter963)
           {
-            $xfer += $output->writeString($kiter955);
-            $xfer += $output->writeString($viter956);
+            $xfer += $output->writeString($kiter962);
+            $xfer += $output->writeString($viter963);
           }
         }
         $output->writeMapEnd();
@@ -26164,15 +26164,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size957 = 0;
-            $_etype960 = 0;
-            $xfer += $input->readListBegin($_etype960, $_size957);
-            for ($_i961 = 0; $_i961 < $_size957; ++$_i961)
+            $_size964 = 0;
+            $_etype967 = 0;
+            $xfer += $input->readListBegin($_etype967, $_size964);
+            for ($_i968 = 0; $_i968 < $_size964; ++$_i968)
             {
-              $elem962 = null;
-              $elem962 = new \metastore\Partition();
-              $xfer += $elem962->read($input);
-              $this->success []= $elem962;
+              $elem969 = null;
+              $elem969 = new \metastore\Partition();
+              $xfer += $elem969->read($input);
+              $this->success []= $elem969;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26232,9 +26232,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter963)
+          foreach ($this->success as $iter970)
           {
-            $xfer += $iter963->write($output);
+            $xfer += $iter970->write($output);
           }
         }
         $output->writeListEnd();
@@ -26380,14 +26380,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size964 = 0;
-            $_etype967 = 0;
-            $xfer += $input->readListBegin($_etype967, $_size964);
-            for ($_i968 = 0; $_i968 < $_size964; ++$_i968)
+            $_size971 = 0;
+            $_etype974 = 0;
+            $xfer += $input->readListBegin($_etype974, $_size971);
+            for ($_i975 = 0; $_i975 < $_size971; ++$_i975)
             {
-              $elem969 = null;
-              $xfer += $input->readString($elem969);
-              $this->part_vals []= $elem969;
+              $elem976 = null;
+              $xfer += $input->readString($elem976);
+              $this->part_vals []= $elem976;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26404,14 +26404,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size970 = 0;
-            $_etype973 = 0;
-            $xfer += $input->readListBegin($_etype973, $_size970);
-            for ($_i974 = 0; $_i974 < $_size970; ++$_i974)
+            $_size977 = 0;
+            $_etype980 = 0;
+            $xfer += $input->readListBegin($_etype980, $_size977);
+            for ($_i981 = 0; $_i981 < $_size977; ++$_i981)
             {
-              $elem975 = null;
-              $xfer += $input->readString($elem975);
-              $this->group_names []= $elem975;
+              $elem982 = null;
+              $xfer += $input->readString($elem982);
+              $this->group_names []= $elem982;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26449,9 +26449,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter976)
+          foreach ($this->part_vals as $iter983)
           {
-            $xfer += $output->writeString($iter976);
+            $xfer += $output->writeString($iter983);
           }
         }
         $output->writeListEnd();
@@ -26471,9 +26471,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter977)
+          foreach ($this->group_names as $iter984)
           {
-            $xfer += $output->writeString($iter977);
+            $xfer += $output->writeString($iter984);
           }
         }
         $output->writeListEnd();
@@ -27064,15 +27064,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size978 = 0;
-            $_etype981 = 0;
-            $xfer += $input->readListBegin($_etype981, $_size978);
-            for ($_i982 = 0; $_i982 < $_size978; ++$_i982)
+            $_size985 = 0;
+            $_etype988 = 0;
+            $xfer += $input->readListBegin($_etype988, $_size985);
+            for ($_i989 = 0; $_i989 < $_size985; ++$_i989)
             {
-              $elem983 = null;
-              $elem983 = new \metastore\Partition();
-              $xfer += $elem983->read($input);
-              $this->success []= $elem983;
+              $elem990 = null;
+              $elem990 = new \metastore\Partition();
+              $xfer += $elem990->read($input);
+              $this->success []= $elem990;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27116,9 +27116,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter984)
+          foreach ($this->success as $iter991)
           {
-            $xfer += $iter984->write($output);
+            $xfer += $iter991->write($output);
           }
         }
         $output->writeListEnd();
@@ -27264,14 +27264,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size985 = 0;
-            $_etype988 = 0;
-            $xfer += $input->readListBegin($_etype988, $_size985);
-            for ($_i989 = 0; $_i989 < $_size985; ++$_i989)
+            $_size992 = 0;
+            $_etype995 = 0;
+            $xfer += $input->readListBegin($_etype995, $_size992);
+            for ($_i996 = 0; $_i996 < $_size992; ++$_i996)
             {
-              $elem990 = null;
-              $xfer += $input->readString($elem990);
-              $this->group_names []= $elem990;
+              $elem997 = null;
+              $xfer += $input->readString($elem997);
+              $this->group_names []= $elem997;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27319,9 +27319,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter991)
+          foreach ($this->group_names as $iter998)
           {
-            $xfer += $output->writeString($iter991);
+            $xfer += $output->writeString($iter998);
           }
         }
         $output->writeListEnd();
@@ -27410,15 +27410,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size992 = 0;
-            $_etype995 = 0;
-            $xfer += $input->readListBegin($_etype995, $_size992);
-            for ($_i996 = 0; $_i996 < $_size992; ++$_i996)
+            $_size999 = 0;
+            $_etype1002 = 0;
+            $xfer += $input->readListBegin($_etype1002, $_size999);
+            for ($_i1003 = 0; $_i1003 < $_size999; ++$_i1003)
             {
-              $elem997 = null;
-              $elem997 = new \metastore\Partition();
-              $xfer += $elem997->read($input);
-              $this->success []= $elem997;
+              $elem1004 = null;
+              $elem1004 = new \metastore\Partition();
+              $xfer += $elem1004->read($input);
+              $this->success []= $elem1004;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27462,9 +27462,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter998)
+          foreach ($this->success as $iter1005)
           {
-            $xfer += $iter998->write($output);
+            $xfer += $iter1005->write($output);
           }
         }
         $output->writeListEnd();
@@ -27684,15 +27684,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size999 = 0;
-            $_etype1002 = 0;
-            $xfer += $input->readListBegin($_etype1002, $_size999);
-            for ($_i1003 = 0; $_i1003 < $_size999; ++$_i1003)
+            $_size1006 = 0;
+            $_etype1009 = 0;
+            $xfer += $input->readListBegin($_etype1009, $_size1006);
+            for ($_i1010 = 0; $_i1010 < $_size1006; ++$_i1010)
             {
-              $elem1004 = null;
-              $elem1004 = new \metastore\PartitionSpec();
-              $xfer += $elem1004->read($input);
-              $this->success []= $elem1004;
+              $elem1011 = null;
+              $elem1011 = new \metastore\PartitionSpec();
+              $xfer += $elem1011->read($input);
+              $this->success []= $elem1011;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27736,9 +27736,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1005)
+          foreach ($this->success as $iter1012)
           {
-            $xfer += $iter1005->write($output);
+            $xfer += $iter1012->write($output);
           }
         }
         $output->writeListEnd();
@@ -27957,14 +27957,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1006 = 0;
-            $_etype1009 = 0;
-            $xfer += $input->readListBegin($_etype1009, $_size1006);
-            for ($_i1010 = 0; $_i1010 < $_size1006; ++$_i1010)
+            $_size1013 = 0;
+            $_etype1016 = 0;
+            $xfer += $input->readListBegin($_etype1016, $_size1013);
+            for ($_i1017 = 0; $_i1017 < $_size1013; ++$_i1017)
             {
-              $elem1011 = null;
-              $xfer += $input->readString($elem1011);
-              $this->success []= $elem1011;
+              $elem1018 = null;
+              $xfer += $input->readString($elem1018);
+              $this->success []= $elem1018;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28008,9 +28008,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1012)
+          foreach ($this->success as $iter1019)
           {
-            $xfer += $output->writeString($iter1012);
+            $xfer += $output->writeString($iter1019);
           }
         }
         $output->writeListEnd();
@@ -28341,14 +28341,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1013 = 0;
-            $_etype1016 = 0;
-            $xfer += $input->readListBegin($_etype1016, $_size1013);
-            for ($_i1017 = 0; $_i1017 < $_size1013; ++$_i1017)
+            $_size1020 = 0;
+            $_etype1023 = 0;
+            $xfer += $input->readListBegin($_etype1023, $_size1020);
+            for ($_i1024 = 0; $_i1024 < $_size1020; ++$_i1024)
             {
-              $elem1018 = null;
-              $xfer += $input->readString($elem1018);
-              $this->part_vals []= $elem1018;
+              $elem1025 = null;
+              $xfer += $input->readString($elem1025);
+              $this->part_vals []= $elem1025;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28393,9 +28393,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1019)
+          foreach ($this->part_vals as $iter1026)
           {
-            $xfer += $output->writeString($iter1019);
+            $xfer += $output->writeString($iter1026);
           }
         }
         $output->writeListEnd();
@@ -28489,15 +28489,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1020 = 0;
-            $_etype1023 = 0;
-            $xfer += $input->readListBegin($_etype1023, $_size1020);
-            for ($_i1024 = 0; $_i1024 < $_size1020; ++$_i1024)
+            $_size1027 = 0;
+            $_etype1030 = 0;
+            $xfer += $input->readListBegin($_etype1030, $_size1027);
+            for ($_i1031 = 0; $_i1031 < $_size1027; ++$_i1031)
             {
-              $elem1025 = null;
-              $elem1025 = new \metastore\Partition();
-              $xfer += $elem1025->read($input);
-              $this->success []= $elem1025;
+              $elem1032 = null;
+              $elem1032 = new \metastore\Partition();
+              $xfer += $elem1032->read($input);
+              $this->success []= $elem1032;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28541,9 +28541,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1026)
+          foreach ($this->success as $iter1033)
           {
-            $xfer += $iter1026->write($output);
+            $xfer += $iter1033->write($output);
           }
         }
         $output->writeListEnd();
@@ -28690,14 +28690,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1027 = 0;
-            $_etype1030 = 0;
-            $xfer += $input->readListBegin($_etype1030, $_size1027);
-            for ($_i1031 = 0; $_i1031 < $_size1027; ++$_i1031)
+            $_size1034 = 0;
+            $_etype1037 = 0;
+            $xfer += $input->readListBegin($_etype1037, $_size1034);
+            for ($_i1038 = 0; $_i1038 < $_size1034; ++$_i1038)
             {
-              $elem1032 = null;
-              $xfer += $input->readString($elem1032);
-              $this->part_vals []= $elem1032;
+              $elem1039 = null;
+              $xfer += $input->readString($elem1039);
+              $this->part_vals []= $elem1039;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28721,14 +28721,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1033 = 0;
-            $_etype1036 = 0;
-            $xfer += $input->readListBegin($_etype1036, $_size1033);
-            for ($_i1037 = 0; $_i1037 < $_size1033; ++$_i1037)
+            $_size1040 = 0;
+            $_etype1043 = 0;
+            $xfer += $input->readListBegin($_etype1043, $_size1040);
+            for ($_i1044 = 0; $_i1044 < $_size1040; ++$_i1044)
             {
-              $elem1038 = null;
-              $xfer += $input->readString($elem1038);
-              $this->group_names []= $elem1038;
+              $elem1045 = null;
+              $xfer += $input->readString($elem1045);
+              $this->group_names []= $elem1045;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28766,9 +28766,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1039)
+          foreach ($this->part_vals as $iter1046)
           {
-            $xfer += $output->writeString($iter1039);
+            $xfer += $output->writeString($iter1046);
           }
         }
         $output->writeListEnd();
@@ -28793,9 +28793,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1040)
+          foreach ($this->group_names as $iter1047)
           {
-            $xfer += $output->writeString($iter1040);
+            $xfer += $output->writeString($iter1047);
           }
         }
         $output->writeListEnd();
@@ -28884,15 +28884,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1041 = 0;
-            $_etype1044 = 0;
-            $xfer += $input->readListBegin($_etype1044, $_size1041);
-            for ($_i1045 = 0; $_i1045 < $_size1041; ++$_i1045)
+            $_size1048 = 0;
+            $_etype1051 = 0;
+            $xfer += $input->readListBegin($_etype1051, $_size1048);
+            for ($_i1052 = 0; $_i1052 < $_size1048; ++$_i1052)
             {
-              $elem1046 = null;
-              $elem1046 = new \metastore\Partition();
-              $xfer += $elem1046->read($input);
-              $this->success []= $elem1046;
+              $elem1053 = null;
+              $elem1053 = new \metastore\Partition();
+              $xfer += $elem1053->read($input);
+              $this->success []= $elem1053;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28936,9 +28936,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1047)
+          foreach ($this->success as $iter1054)
           {
-            $xfer += $iter1047->write($output);
+            $xfer += $iter1054->write($output);
           }
         }
         $output->writeListEnd();
@@ -29059,14 +29059,14 @@ class ThriftHiveMetastore_get_partition_names_ps_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)
+            $_size1055 = 0;
+            $_etype1058 = 0;
+            $xfer += $input->readListBegin($_etype1058, $_size1055);
+            for ($_i1059 = 0; $_i1059 < $_size1055; ++$_i1059)
             {
-              $elem1053 = null;
-              $xfer += $input->readString($elem1053);
-              $this->part_vals []= $elem1053;
+              $elem1060 = null;
+              $xfer += $input->readString($elem1060);
+              $this->part_vals []= $elem1060;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29111,9 +29111,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1054)
+          foreach ($this->part_vals as $iter1061)
           {
-            $xfer += $output->writeString($iter1054);
+            $xfer += $output->writeString($iter1061);
           }
         }
         $output->writeListEnd();
@@ -29206,14 +29206,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1055 = 0;
-            $_etype1058 = 0;
-            $xfer += $input->readListBegin($_etype1058, $_size1055);
-            for ($_i1059 = 0; $_i1059 < $_size1055; ++$_i1059)
+            $_size1062 = 0;
+            $_etype1065 = 0;
+            $xfer += $input->readListBegin($_etype1065, $_size1062);
+            for ($_i1066 = 0; $_i1066 < $_size1062; ++$_i1066)
             {
-              $elem1060 = null;
-              $xfer += $input->readString($elem1060);
-              $this->success []= $elem1060;
+              $elem1067 = null;
+              $xfer += $input->readString($elem1067);
+              $this->success []= $elem1067;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29257,9 +29257,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1061)
+          foreach ($this->success as $iter1068)
           {
-            $xfer += $output->writeString($iter1061);
+            $xfer += $output->writeString($iter1068);
           }
         }
         $output->writeListEnd();
@@ -29502,15 +29502,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1062 = 0;
-            $_etype1065 = 0;
-            $xfer += $input->readListBegin($_etype1065, $_size1062);
-            for ($_i1066 = 0; $_i1066 < $_size1062; ++$_i1066)
+            $_size1069 = 0;
+            $_etype1072 = 0;
+            $xfer += $input->readListBegin($_etype1072, $_size1069);
+            for ($_i1073 = 0; $_i1073 < $_size1069; ++$_i1073)
             {
-              $elem1067 = null;
-              $elem1067 = new \metastore\Partition();
-              $xfer += $elem1067->read($input);
-              $this->success []= $elem1067;
+              $elem1074 = null;
+              $elem1074 = new \metastore\Partition();
+              $xfer += $elem1074->read($input);
+              $this->success []= $elem1074;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29554,9 +29554,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1068)
+          foreach ($this->success as $iter1075)
           {
-            $xfer += $iter1068->write($output);
+            $xfer += $iter1075->write($output);
           }
         }
         $output->writeListEnd();
@@ -29799,15 +29799,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1069 = 0;
-            $_etype1072 = 0;
-            $xfer += $input->readListBegin($_etype1072, $_size1069);
-            for ($_i1073 = 0; $_i1073 < $_size1069; ++$_i1073)
+            $_size1076 = 0;
+            $_etype1079 = 0;
+            $xfer += $input->readListBegin($_etype1079, $_size1076);
+            for ($_i1080 = 0; $_i1080 < $_size1076; ++$_i1080)
             {
-              $elem1074 = null;
-              $elem1074 = new \metastore\PartitionSpec();
-              $xfer += $elem1074->read($input);
-              $this->success []= $elem1074;
+              $elem1081 = null;
+              $elem1081 = new \metastore\PartitionSpec();
+              $xfer += $elem1081->read($input);
+              $this->success []= $elem1081;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29851,9 +29851,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1075)
+          foreach ($this->success as $iter1082)
           {
-            $xfer += $iter1075->write($output);
+            $xfer += $iter1082->write($output);
           }
         }
         $output->writeListEnd();
@@ -30419,14 +30419,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size1076 = 0;
-            $_etype1079 = 0;
-            $xfer += $input->readListBegin($_etype1079, $_size1076);
-            for ($_i1080 = 0; $_i1080 < $_size1076; ++$_i1080)
+            $_size1083 = 0;
+            $_etype1086 = 0;
+            $xfer += $input->readListBegin($_etype1086, $_size1083);
+            for ($_i1087 = 0; $_i1087 < $_size1083; ++$_i1087)
             {
-              $elem1081 = null;
-              $xfer += $input->readString($elem1081);
-              $this->names []= $elem1081;
+              $elem1088 = null;
+              $xfer += $input->readString($elem1088);
+              $this->names []= $elem1088;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30464,9 +30464,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter1082)
+          foreach ($this->names as $iter1089)
           {
-            $xfer += $output->writeString($iter1082);
+            $xfer += $output->writeString($iter1089);
           }
         }
         $output->writeListEnd();
@@ -30555,15 +30555,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1083 = 0;
-            $_etype1086 = 0;
-            $xfer += $input->readListBegin($_etype1086, $_size1083);
-            for ($_i1087 = 0; $_i1087 < $_size1083; ++$_i1087)
+            $_size1090 = 0;
+            $_etype1093 = 0;
+            $xfer += $input->readListBegin($_etype1093, $_size1090);
+            for ($_i1094 = 0; $_i1094 < $_size1090; ++$_i1094)
             {
-              $elem1088 = null;
-              $elem1088 = new \metastore\Partition();
-              $xfer += $elem1088->read($input);
-              $this->success []= $elem1088;
+              $elem1095 = null;
+              $elem1095 = new \metastore\Partition();
+              $xfer += $elem1095->read($input);
+              $this->success []= $elem1095;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30607,9 +30607,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1089)
+          foreach ($this->success as $iter1096)
           {
-            $xfer += $iter1089->write($output);
+            $xfer += $iter1096->write($output);
           }
         }
         $output->writeListEnd();
@@ -30948,15 +30948,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1090 = 0;
-            $_etype1093 = 0;
-            $xfer += $input->readListBegin($_etype1093, $_size1090);
-            for ($_i1094 = 0; $_i1094 < $_size1090; ++$_i1094)
+            $_size1097 = 0;
+            $_etype1100 = 0;
+            $xfer += $input->readListBegin($_etype1100, $_size1097);
+            for ($_i1101 = 0; $_i1101 < $_size1097; ++$_i1101)
             {
-              $elem1095 = null;
-              $elem1095 = new \metastore\Partition();
-              $xfer += $elem1095->read($input);
-              $this->new_parts []= $elem1095;
+              $elem1102 = null;
+              $elem1102 = new \metastore\Partition();
+              $xfer += $elem1102->read($input);
+              $this->new_parts []= $elem1102;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30994,9 +30994,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1096)
+          foreach ($this->new_parts as $iter1103)
           {
-            $xfer += $iter1096->write($output);
+            $xfer += $iter1103->write($output);
           }
         }
         $output->writeListEnd();
@@ -31211,15 +31211,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1097 = 0;
-            $_etype1100 = 0;
-            $xfer += $input->readListBegin($_etype1100, $_size1097);
-            for ($_i1101 = 0; $_i1101 < $_size1097; ++$_i1101)
+            $_size1104 = 0;
+            $_etype1107 = 0;
+            $xfer += $input->readListBegin($_etype1107, $_size1104);
+            for ($_i1108 = 0; $_i1108 < $_size1104; ++$_i1108)
             {
-              $elem1102 = null;
-              $elem1102 = new \metastore\Partition();
-              $xfer += $elem1102->read($input);
-              $this->new_parts []= $elem1102;
+              $elem1109 = null;
+              $elem1109 = new \metastore\Partition();
+              $xfer += $elem1109->read($input);
+              $this->new_parts []= $elem1109;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31265,9 +31265,9 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1103)
+          foreach ($this->new_parts as $iter1110)
           {
-            $xfer += $iter1103->write($output);
+            $xfer += $iter1110->write($output);
           }
         }
         $output->writeListEnd();
@@ -31745,14 +31745,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1104 = 0;
-            $_etype1107 = 0;
-            $xfer += $input->readListBegin($_etype1107, $_size1104);
-            for ($_i1108 = 0; $_i1108 < $_size1104; ++$_i1108)
+            $_size1111 = 0;
+            $_etype1114 = 0;
+            $xfer += $input->readListBegin($_etype1114, $_size1111);
+            for ($_i1115 = 0; $_i1115 < $_size1111; ++$_i1115)
             {
-              $elem1109 = null;
-              $xfer += $input->readString($elem1109);
-              $this->part_vals []= $elem1109;
+              $elem1116 = null;
+              $xfer += $input->readString($elem1116);
+              $this->part_vals []= $elem1116;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31798,9 +31798,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1110)
+          foreach ($this->part_vals as $iter1117)
           {
-            $xfer += $output->writeString($iter1110);
+            $xfer += $output->writeString($iter1117);
           }
         }
         $output->writeListEnd();
@@ -31985,14 +31985,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1111 = 0;
-            $_etype1114 = 0;
-            $xfer += $input->readListBegin($_etype1114, $_size1111);
-            for ($_i1115 = 0; $_i1115 < $_size1111; ++$_i1115)
+            $_size1118 = 0;
+            $_etype1121 = 0;
+            $xfer += $input->readListBegin($_etype1121, $_size1118);
+            for ($_i1122 = 0; $_i1122 < $_size1118; ++$_i1122)
             {
-              $elem1116 = null;
-              $xfer += $input->readString($elem1116);
-              $this->part_vals []= $elem1116;
+              $elem1123 = null;
+              $xfer += $input->readString($elem1123);
+              $this->part_vals []= $elem1123;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32027,9 +32027,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1117)
+          foreach ($this->part_vals as $iter1124)
           {
-            $xfer += $output->writeString($iter1117);
+            $xfer += $output->writeString($iter1124);
           }
         }
         $output->writeListEnd();
@@ -32483,14 +32483,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1118 = 0;
-            $_etype1121 = 0;
-            $xfer += $input->readListBegin($_etype1121, $_size1118);
-            for ($_i1122 = 0; $_i1122 < $_size1118; ++$_i1122)
+            $_size1125 = 0;
+            $_etype1128 = 0;
+            $xfer += $input->readListBegin($_etype1128, $_size1125);
+            for ($_i1129 = 0; $_i1129 < $_size1125; ++$_i1129)
             {
-              $elem1123 = null;
-              $xfer += $input->readString($elem1123);
-              $this->success []= $elem1123;
+              $elem1130 = null;
+              $xfer += $input->readString($elem1130);
+              $this->success []= $elem1130;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32526,9 +32526,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1124)
+          foreach ($this->success as $iter1131)
           {
-            $xfer += $output->writeString($iter1124);
+            $xfer += $output->writeString($iter1131);
           }
         }
         $output->writeListEnd();
@@ -32688,17 +32688,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size1125 = 0;
-            $_ktype1126 = 0;
-            $_vtype1127 = 0;
-            $xfer += $input->readMapBegin($_ktype1126, $_vtype1127, $_size1125);
-            for ($_i1129 = 0; $_i1129 < $_size1125; ++$_i1129)
+            $_size1132 = 0;
+            $_ktype1133 = 0;
+            $_vtype1134 = 0;
+            $xfer += $input->readMapBegin($_ktype1133, $_vtype1134, $_size1132);
+            for ($_i1136 = 0; $_i1136 < $_size1132; ++$_i1136)
             {
-              $key1130 = '';
-              $val1131 = '';
-              $xfer += $input->readString($key1130);
-              $xfer += $input->readString($val1131);
-              $this->success[$key1130] = $val1131;
+              $key1137 = '';
+              $val1138 = '';
+              $xfer += $input->readString($key1137);
+              $xfer += $input->readString($val1138);
+              $this->success[$key1137] = $val1138;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -32734,10 +32734,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter1132 => $viter1133)
+          foreach ($this->success as $kiter1139 => $viter1140)
           {
-            $xfer += $output->writeString($kiter1132);
-            $xfer += $output->writeString($viter1133);
+            $xfer += $output->writeString($kiter1139);
+            $xfer += $output->writeString($viter1140);
           }
         }
         $output->writeMapEnd();
@@ -32857,17 +32857,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size1134 = 0;
-            $_ktype1135 = 0;
-            $_vtype1136 = 0;
-            $xfer += $input->readMapBegin($_ktype1135, $_vtype1136, $_size1134);
-            for ($_i1138 = 0; $_i1138 < $_size1134; ++$_i1138)
+            $_size1141 = 0;
+            $_ktype1142 = 0;
+            $_vtype1143 = 0;
+            $xfer += $input->readMapBegin($_ktype1142, $_vtype1143, $_size1141);
+            for ($_i1145 = 0; $_i1145 < $_size1141; ++$_i1145)
             {
-              $key1139 = '';
-              $val1140 = '';
-              $xfer += $input->readString($key1139);
-              $xfer += $input->readString($val1140);
-              $this->part_vals[$key1139] = $val1140;
+              $key1146 = '';
+              $val1147 = '';
+              $xfer += $input->readString($key1146);
+              $xfer += $input->readString($val1147);
+              $this->part_vals[$key1146] = $val1147;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -32912,10 +32912,10 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter1141 => $viter1142)
+          foreach ($this->part_vals as $kiter1148 => $viter1149)
           {
-            $xfer += $output->writeString($kiter1141);
-            $xfer += $output->writeString($viter1142);
+            $xfer += $output->writeString($kiter1148);
+            $xfer += $output->writeString($viter1149);
           }
         }
         $output->writeMapEnd();
@@ -33237,17 +33237,17 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size1143 = 0;
-            $_ktype1144 = 0;
-            $_vtype1145 = 0;
-            $xfer += $input->readMapBegin($_ktype1144, $_vtype1145, $_size1143);
-            for ($_i1147 = 0; $_i1147 < $_size1143; ++$_i1147)
+            $_size1150 = 0;
+            $_ktype1151 = 0;
+            $_vtype1152 = 0;
+            $xfer += $input->readMapBegin($_ktype1151, $_vtype1152, $_size1150);
+            for ($_i1154 = 0; $_i1154 < $_size1150; ++$_i1154)
             {
-              $key1148 = '';
-              $val1149 = '';
-              $xfer += $input->readString($key1148);
-              $xfer += $input->readString($val1149);
-              $this->part_vals[$key1148] = $val1149;
+              $key1155 = '';
+              $val1156 = '';
+              $xfer += $input->readString($key1155);
+              $xfer += $input->readString($val1156);
+              $this->part_vals[$key1155] = $val1156;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -33292,10 +33292,10 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter1150 => $viter1151)
+          foreach ($this->part_vals as $kiter1157 => $viter1158)
           {
-            $xfer += $output->writeString($kiter1150);
-            $xfer += $output->writeString($viter1151);
+            $xfer += $output->writeString($kiter1157);
+            $xfer += $output->writeString($viter1158);
           }
         }
         $output->writeMapEnd();
@@ -34769,15 +34769,15 @@ class ThriftHiveMetastore_get_indexes_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1152 = 0;
-            $_etype1155 = 0;
-            $xfer += $input->readListBegin($_etype1155, $_size1152);
-            for ($_i1156 = 0; $_i1156 < $_size1152; ++$_i1156)
+            $_size1159 = 0;
+            $_etype1162 = 0;
+            $xfer += $input->readListBegin($_etype1162, $_size1159);
+            for ($_i1163 = 0; $_i1163 < $_size1159; ++$_i1163)
             {
-              $elem1157 = null;
-              $elem1157 = new \metastore\Index();
-              $xfer += $elem1157->read($input);
-              $this->success []= $elem1157;
+              $elem1164 = null;
+              $elem1164 = new \metastore\Index();
+              $xfer += $elem1164->read($input);
+              $this->success []= $elem1164;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34821,9 +34821,9 @@ class ThriftHiveMetastore_get_indexes_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1158)
+          foreach ($this->success as $iter1165)
           {
-            $xfer += $iter1158->write($output);
+            $xfer += $iter1165->write($output);
           }
         }
         $output->writeListEnd();
@@ -35030,14 +35030,14 @@ class ThriftHiveMetastore_get_index_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1159 = 0;
-            $_etype1162 = 0;
-            $xfer += $input->readListBegin($_etype1162, $_size1159);
-            for ($_i1163 = 0; $_i1163 < $_size1159; ++$_i1163)
+            $_size1166 = 0;
+            $_etype1169 = 0;
+            $xfer += $input->readListBegin($_etype1169, $_size1166);
+            for ($_i1170 = 0; $_i1170 < $_size1166; ++$_i1170)
             {
-              $elem1164 = null;
-              $xfer += $input->readString($elem1164);
-              $this->success []= $elem1164;
+              $elem1171 = null;
+              $xfer += $input->readString($elem1171);
+              $this->success []= $elem1171;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35073,9 +35073,9 @@ class ThriftHiveMetastore_get_index_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1165)
+          foreach ($this->success as $iter1172)
           {
-            $xfer += $output->writeString($iter1165);
+            $xfer += $output->writeString($iter1172);
           }
         }
         $output->writeListEnd();
@@ -39389,14 +39389,14 @@ class ThriftHiveMetastore_get_functions_r

<TRUNCATED>

[6/8] hive git commit: HIVE-18231 : validate resource plan - part 2 - validate action and trigger expressions (Harish Jaiprakash, reviewed by Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 11e9447..27f8c0f 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -23822,6 +23822,11 @@ void WMValidateResourcePlanResponse::__set_errors(const std::vector<std::string>
 __isset.errors = true;
 }
 
+void WMValidateResourcePlanResponse::__set_warnings(const std::vector<std::string> & val) {
+  this->warnings = val;
+__isset.warnings = true;
+}
+
 uint32_t WMValidateResourcePlanResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -23863,6 +23868,26 @@ uint32_t WMValidateResourcePlanResponse::read(::apache::thrift::protocol::TProto
           xfer += iprot->skip(ftype);
         }
         break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->warnings.clear();
+            uint32_t _size965;
+            ::apache::thrift::protocol::TType _etype968;
+            xfer += iprot->readListBegin(_etype968, _size965);
+            this->warnings.resize(_size965);
+            uint32_t _i969;
+            for (_i969 = 0; _i969 < _size965; ++_i969)
+            {
+              xfer += iprot->readString(this->warnings[_i969]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.warnings = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -23884,10 +23909,23 @@ 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 _iter965;
-      for (_iter965 = this->errors.begin(); _iter965 != this->errors.end(); ++_iter965)
+      std::vector<std::string> ::const_iterator _iter970;
+      for (_iter970 = this->errors.begin(); _iter970 != this->errors.end(); ++_iter970)
+      {
+        xfer += oprot->writeString((*_iter970));
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.warnings) {
+    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 _iter971;
+      for (_iter971 = this->warnings.begin(); _iter971 != this->warnings.end(); ++_iter971)
       {
-        xfer += oprot->writeString((*_iter965));
+        xfer += oprot->writeString((*_iter971));
       }
       xfer += oprot->writeListEnd();
     }
@@ -23901,22 +23939,26 @@ uint32_t WMValidateResourcePlanResponse::write(::apache::thrift::protocol::TProt
 void swap(WMValidateResourcePlanResponse &a, WMValidateResourcePlanResponse &b) {
   using ::std::swap;
   swap(a.errors, b.errors);
+  swap(a.warnings, b.warnings);
   swap(a.__isset, b.__isset);
 }
 
-WMValidateResourcePlanResponse::WMValidateResourcePlanResponse(const WMValidateResourcePlanResponse& other966) {
-  errors = other966.errors;
-  __isset = other966.__isset;
+WMValidateResourcePlanResponse::WMValidateResourcePlanResponse(const WMValidateResourcePlanResponse& other972) {
+  errors = other972.errors;
+  warnings = other972.warnings;
+  __isset = other972.__isset;
 }
-WMValidateResourcePlanResponse& WMValidateResourcePlanResponse::operator=(const WMValidateResourcePlanResponse& other967) {
-  errors = other967.errors;
-  __isset = other967.__isset;
+WMValidateResourcePlanResponse& WMValidateResourcePlanResponse::operator=(const WMValidateResourcePlanResponse& other973) {
+  errors = other973.errors;
+  warnings = other973.warnings;
+  __isset = other973.__isset;
   return *this;
 }
 void WMValidateResourcePlanResponse::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
   out << "WMValidateResourcePlanResponse(";
   out << "errors="; (__isset.errors ? (out << to_string(errors)) : (out << "<null>"));
+  out << ", " << "warnings="; (__isset.warnings ? (out << to_string(warnings)) : (out << "<null>"));
   out << ")";
 }
 
@@ -23992,13 +24034,13 @@ void swap(WMDropResourcePlanRequest &a, WMDropResourcePlanRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMDropResourcePlanRequest::WMDropResourcePlanRequest(const WMDropResourcePlanRequest& other968) {
-  resourcePlanName = other968.resourcePlanName;
-  __isset = other968.__isset;
+WMDropResourcePlanRequest::WMDropResourcePlanRequest(const WMDropResourcePlanRequest& other974) {
+  resourcePlanName = other974.resourcePlanName;
+  __isset = other974.__isset;
 }
-WMDropResourcePlanRequest& WMDropResourcePlanRequest::operator=(const WMDropResourcePlanRequest& other969) {
-  resourcePlanName = other969.resourcePlanName;
-  __isset = other969.__isset;
+WMDropResourcePlanRequest& WMDropResourcePlanRequest::operator=(const WMDropResourcePlanRequest& other975) {
+  resourcePlanName = other975.resourcePlanName;
+  __isset = other975.__isset;
   return *this;
 }
 void WMDropResourcePlanRequest::printTo(std::ostream& out) const {
@@ -24057,11 +24099,11 @@ void swap(WMDropResourcePlanResponse &a, WMDropResourcePlanResponse &b) {
   (void) b;
 }
 
-WMDropResourcePlanResponse::WMDropResourcePlanResponse(const WMDropResourcePlanResponse& other970) {
-  (void) other970;
+WMDropResourcePlanResponse::WMDropResourcePlanResponse(const WMDropResourcePlanResponse& other976) {
+  (void) other976;
 }
-WMDropResourcePlanResponse& WMDropResourcePlanResponse::operator=(const WMDropResourcePlanResponse& other971) {
-  (void) other971;
+WMDropResourcePlanResponse& WMDropResourcePlanResponse::operator=(const WMDropResourcePlanResponse& other977) {
+  (void) other977;
   return *this;
 }
 void WMDropResourcePlanResponse::printTo(std::ostream& out) const {
@@ -24142,13 +24184,13 @@ void swap(WMCreateTriggerRequest &a, WMCreateTriggerRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMCreateTriggerRequest::WMCreateTriggerRequest(const WMCreateTriggerRequest& other972) {
-  trigger = other972.trigger;
-  __isset = other972.__isset;
+WMCreateTriggerRequest::WMCreateTriggerRequest(const WMCreateTriggerRequest& other978) {
+  trigger = other978.trigger;
+  __isset = other978.__isset;
 }
-WMCreateTriggerRequest& WMCreateTriggerRequest::operator=(const WMCreateTriggerRequest& other973) {
-  trigger = other973.trigger;
-  __isset = other973.__isset;
+WMCreateTriggerRequest& WMCreateTriggerRequest::operator=(const WMCreateTriggerRequest& other979) {
+  trigger = other979.trigger;
+  __isset = other979.__isset;
   return *this;
 }
 void WMCreateTriggerRequest::printTo(std::ostream& out) const {
@@ -24207,11 +24249,11 @@ void swap(WMCreateTriggerResponse &a, WMCreateTriggerResponse &b) {
   (void) b;
 }
 
-WMCreateTriggerResponse::WMCreateTriggerResponse(const WMCreateTriggerResponse& other974) {
-  (void) other974;
+WMCreateTriggerResponse::WMCreateTriggerResponse(const WMCreateTriggerResponse& other980) {
+  (void) other980;
 }
-WMCreateTriggerResponse& WMCreateTriggerResponse::operator=(const WMCreateTriggerResponse& other975) {
-  (void) other975;
+WMCreateTriggerResponse& WMCreateTriggerResponse::operator=(const WMCreateTriggerResponse& other981) {
+  (void) other981;
   return *this;
 }
 void WMCreateTriggerResponse::printTo(std::ostream& out) const {
@@ -24292,13 +24334,13 @@ void swap(WMAlterTriggerRequest &a, WMAlterTriggerRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMAlterTriggerRequest::WMAlterTriggerRequest(const WMAlterTriggerRequest& other976) {
-  trigger = other976.trigger;
-  __isset = other976.__isset;
+WMAlterTriggerRequest::WMAlterTriggerRequest(const WMAlterTriggerRequest& other982) {
+  trigger = other982.trigger;
+  __isset = other982.__isset;
 }
-WMAlterTriggerRequest& WMAlterTriggerRequest::operator=(const WMAlterTriggerRequest& other977) {
-  trigger = other977.trigger;
-  __isset = other977.__isset;
+WMAlterTriggerRequest& WMAlterTriggerRequest::operator=(const WMAlterTriggerRequest& other983) {
+  trigger = other983.trigger;
+  __isset = other983.__isset;
   return *this;
 }
 void WMAlterTriggerRequest::printTo(std::ostream& out) const {
@@ -24357,11 +24399,11 @@ void swap(WMAlterTriggerResponse &a, WMAlterTriggerResponse &b) {
   (void) b;
 }
 
-WMAlterTriggerResponse::WMAlterTriggerResponse(const WMAlterTriggerResponse& other978) {
-  (void) other978;
+WMAlterTriggerResponse::WMAlterTriggerResponse(const WMAlterTriggerResponse& other984) {
+  (void) other984;
 }
-WMAlterTriggerResponse& WMAlterTriggerResponse::operator=(const WMAlterTriggerResponse& other979) {
-  (void) other979;
+WMAlterTriggerResponse& WMAlterTriggerResponse::operator=(const WMAlterTriggerResponse& other985) {
+  (void) other985;
   return *this;
 }
 void WMAlterTriggerResponse::printTo(std::ostream& out) const {
@@ -24461,15 +24503,15 @@ void swap(WMDropTriggerRequest &a, WMDropTriggerRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMDropTriggerRequest::WMDropTriggerRequest(const WMDropTriggerRequest& other980) {
-  resourcePlanName = other980.resourcePlanName;
-  triggerName = other980.triggerName;
-  __isset = other980.__isset;
+WMDropTriggerRequest::WMDropTriggerRequest(const WMDropTriggerRequest& other986) {
+  resourcePlanName = other986.resourcePlanName;
+  triggerName = other986.triggerName;
+  __isset = other986.__isset;
 }
-WMDropTriggerRequest& WMDropTriggerRequest::operator=(const WMDropTriggerRequest& other981) {
-  resourcePlanName = other981.resourcePlanName;
-  triggerName = other981.triggerName;
-  __isset = other981.__isset;
+WMDropTriggerRequest& WMDropTriggerRequest::operator=(const WMDropTriggerRequest& other987) {
+  resourcePlanName = other987.resourcePlanName;
+  triggerName = other987.triggerName;
+  __isset = other987.__isset;
   return *this;
 }
 void WMDropTriggerRequest::printTo(std::ostream& out) const {
@@ -24529,11 +24571,11 @@ void swap(WMDropTriggerResponse &a, WMDropTriggerResponse &b) {
   (void) b;
 }
 
-WMDropTriggerResponse::WMDropTriggerResponse(const WMDropTriggerResponse& other982) {
-  (void) other982;
+WMDropTriggerResponse::WMDropTriggerResponse(const WMDropTriggerResponse& other988) {
+  (void) other988;
 }
-WMDropTriggerResponse& WMDropTriggerResponse::operator=(const WMDropTriggerResponse& other983) {
-  (void) other983;
+WMDropTriggerResponse& WMDropTriggerResponse::operator=(const WMDropTriggerResponse& other989) {
+  (void) other989;
   return *this;
 }
 void WMDropTriggerResponse::printTo(std::ostream& out) const {
@@ -24614,13 +24656,13 @@ void swap(WMGetTriggersForResourePlanRequest &a, WMGetTriggersForResourePlanRequ
   swap(a.__isset, b.__isset);
 }
 
-WMGetTriggersForResourePlanRequest::WMGetTriggersForResourePlanRequest(const WMGetTriggersForResourePlanRequest& other984) {
-  resourcePlanName = other984.resourcePlanName;
-  __isset = other984.__isset;
+WMGetTriggersForResourePlanRequest::WMGetTriggersForResourePlanRequest(const WMGetTriggersForResourePlanRequest& other990) {
+  resourcePlanName = other990.resourcePlanName;
+  __isset = other990.__isset;
 }
-WMGetTriggersForResourePlanRequest& WMGetTriggersForResourePlanRequest::operator=(const WMGetTriggersForResourePlanRequest& other985) {
-  resourcePlanName = other985.resourcePlanName;
-  __isset = other985.__isset;
+WMGetTriggersForResourePlanRequest& WMGetTriggersForResourePlanRequest::operator=(const WMGetTriggersForResourePlanRequest& other991) {
+  resourcePlanName = other991.resourcePlanName;
+  __isset = other991.__isset;
   return *this;
 }
 void WMGetTriggersForResourePlanRequest::printTo(std::ostream& out) const {
@@ -24665,14 +24707,14 @@ uint32_t WMGetTriggersForResourePlanResponse::read(::apache::thrift::protocol::T
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->triggers.clear();
-            uint32_t _size986;
-            ::apache::thrift::protocol::TType _etype989;
-            xfer += iprot->readListBegin(_etype989, _size986);
-            this->triggers.resize(_size986);
-            uint32_t _i990;
-            for (_i990 = 0; _i990 < _size986; ++_i990)
+            uint32_t _size992;
+            ::apache::thrift::protocol::TType _etype995;
+            xfer += iprot->readListBegin(_etype995, _size992);
+            this->triggers.resize(_size992);
+            uint32_t _i996;
+            for (_i996 = 0; _i996 < _size992; ++_i996)
             {
-              xfer += this->triggers[_i990].read(iprot);
+              xfer += this->triggers[_i996].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -24702,10 +24744,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 _iter991;
-      for (_iter991 = this->triggers.begin(); _iter991 != this->triggers.end(); ++_iter991)
+      std::vector<WMTrigger> ::const_iterator _iter997;
+      for (_iter997 = this->triggers.begin(); _iter997 != this->triggers.end(); ++_iter997)
       {
-        xfer += (*_iter991).write(oprot);
+        xfer += (*_iter997).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -24722,13 +24764,13 @@ void swap(WMGetTriggersForResourePlanResponse &a, WMGetTriggersForResourePlanRes
   swap(a.__isset, b.__isset);
 }
 
-WMGetTriggersForResourePlanResponse::WMGetTriggersForResourePlanResponse(const WMGetTriggersForResourePlanResponse& other992) {
-  triggers = other992.triggers;
-  __isset = other992.__isset;
+WMGetTriggersForResourePlanResponse::WMGetTriggersForResourePlanResponse(const WMGetTriggersForResourePlanResponse& other998) {
+  triggers = other998.triggers;
+  __isset = other998.__isset;
 }
-WMGetTriggersForResourePlanResponse& WMGetTriggersForResourePlanResponse::operator=(const WMGetTriggersForResourePlanResponse& other993) {
-  triggers = other993.triggers;
-  __isset = other993.__isset;
+WMGetTriggersForResourePlanResponse& WMGetTriggersForResourePlanResponse::operator=(const WMGetTriggersForResourePlanResponse& other999) {
+  triggers = other999.triggers;
+  __isset = other999.__isset;
   return *this;
 }
 void WMGetTriggersForResourePlanResponse::printTo(std::ostream& out) const {
@@ -24810,13 +24852,13 @@ void swap(WMCreatePoolRequest &a, WMCreatePoolRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMCreatePoolRequest::WMCreatePoolRequest(const WMCreatePoolRequest& other994) {
-  pool = other994.pool;
-  __isset = other994.__isset;
+WMCreatePoolRequest::WMCreatePoolRequest(const WMCreatePoolRequest& other1000) {
+  pool = other1000.pool;
+  __isset = other1000.__isset;
 }
-WMCreatePoolRequest& WMCreatePoolRequest::operator=(const WMCreatePoolRequest& other995) {
-  pool = other995.pool;
-  __isset = other995.__isset;
+WMCreatePoolRequest& WMCreatePoolRequest::operator=(const WMCreatePoolRequest& other1001) {
+  pool = other1001.pool;
+  __isset = other1001.__isset;
   return *this;
 }
 void WMCreatePoolRequest::printTo(std::ostream& out) const {
@@ -24875,11 +24917,11 @@ void swap(WMCreatePoolResponse &a, WMCreatePoolResponse &b) {
   (void) b;
 }
 
-WMCreatePoolResponse::WMCreatePoolResponse(const WMCreatePoolResponse& other996) {
-  (void) other996;
+WMCreatePoolResponse::WMCreatePoolResponse(const WMCreatePoolResponse& other1002) {
+  (void) other1002;
 }
-WMCreatePoolResponse& WMCreatePoolResponse::operator=(const WMCreatePoolResponse& other997) {
-  (void) other997;
+WMCreatePoolResponse& WMCreatePoolResponse::operator=(const WMCreatePoolResponse& other1003) {
+  (void) other1003;
   return *this;
 }
 void WMCreatePoolResponse::printTo(std::ostream& out) const {
@@ -24979,15 +25021,15 @@ void swap(WMAlterPoolRequest &a, WMAlterPoolRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMAlterPoolRequest::WMAlterPoolRequest(const WMAlterPoolRequest& other998) {
-  pool = other998.pool;
-  poolPath = other998.poolPath;
-  __isset = other998.__isset;
+WMAlterPoolRequest::WMAlterPoolRequest(const WMAlterPoolRequest& other1004) {
+  pool = other1004.pool;
+  poolPath = other1004.poolPath;
+  __isset = other1004.__isset;
 }
-WMAlterPoolRequest& WMAlterPoolRequest::operator=(const WMAlterPoolRequest& other999) {
-  pool = other999.pool;
-  poolPath = other999.poolPath;
-  __isset = other999.__isset;
+WMAlterPoolRequest& WMAlterPoolRequest::operator=(const WMAlterPoolRequest& other1005) {
+  pool = other1005.pool;
+  poolPath = other1005.poolPath;
+  __isset = other1005.__isset;
   return *this;
 }
 void WMAlterPoolRequest::printTo(std::ostream& out) const {
@@ -25047,11 +25089,11 @@ void swap(WMAlterPoolResponse &a, WMAlterPoolResponse &b) {
   (void) b;
 }
 
-WMAlterPoolResponse::WMAlterPoolResponse(const WMAlterPoolResponse& other1000) {
-  (void) other1000;
+WMAlterPoolResponse::WMAlterPoolResponse(const WMAlterPoolResponse& other1006) {
+  (void) other1006;
 }
-WMAlterPoolResponse& WMAlterPoolResponse::operator=(const WMAlterPoolResponse& other1001) {
-  (void) other1001;
+WMAlterPoolResponse& WMAlterPoolResponse::operator=(const WMAlterPoolResponse& other1007) {
+  (void) other1007;
   return *this;
 }
 void WMAlterPoolResponse::printTo(std::ostream& out) const {
@@ -25151,15 +25193,15 @@ void swap(WMDropPoolRequest &a, WMDropPoolRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMDropPoolRequest::WMDropPoolRequest(const WMDropPoolRequest& other1002) {
-  resourcePlanName = other1002.resourcePlanName;
-  poolPath = other1002.poolPath;
-  __isset = other1002.__isset;
+WMDropPoolRequest::WMDropPoolRequest(const WMDropPoolRequest& other1008) {
+  resourcePlanName = other1008.resourcePlanName;
+  poolPath = other1008.poolPath;
+  __isset = other1008.__isset;
 }
-WMDropPoolRequest& WMDropPoolRequest::operator=(const WMDropPoolRequest& other1003) {
-  resourcePlanName = other1003.resourcePlanName;
-  poolPath = other1003.poolPath;
-  __isset = other1003.__isset;
+WMDropPoolRequest& WMDropPoolRequest::operator=(const WMDropPoolRequest& other1009) {
+  resourcePlanName = other1009.resourcePlanName;
+  poolPath = other1009.poolPath;
+  __isset = other1009.__isset;
   return *this;
 }
 void WMDropPoolRequest::printTo(std::ostream& out) const {
@@ -25219,11 +25261,11 @@ void swap(WMDropPoolResponse &a, WMDropPoolResponse &b) {
   (void) b;
 }
 
-WMDropPoolResponse::WMDropPoolResponse(const WMDropPoolResponse& other1004) {
-  (void) other1004;
+WMDropPoolResponse::WMDropPoolResponse(const WMDropPoolResponse& other1010) {
+  (void) other1010;
 }
-WMDropPoolResponse& WMDropPoolResponse::operator=(const WMDropPoolResponse& other1005) {
-  (void) other1005;
+WMDropPoolResponse& WMDropPoolResponse::operator=(const WMDropPoolResponse& other1011) {
+  (void) other1011;
   return *this;
 }
 void WMDropPoolResponse::printTo(std::ostream& out) const {
@@ -25323,15 +25365,15 @@ void swap(WMCreateOrUpdateMappingRequest &a, WMCreateOrUpdateMappingRequest &b)
   swap(a.__isset, b.__isset);
 }
 
-WMCreateOrUpdateMappingRequest::WMCreateOrUpdateMappingRequest(const WMCreateOrUpdateMappingRequest& other1006) {
-  mapping = other1006.mapping;
-  update = other1006.update;
-  __isset = other1006.__isset;
+WMCreateOrUpdateMappingRequest::WMCreateOrUpdateMappingRequest(const WMCreateOrUpdateMappingRequest& other1012) {
+  mapping = other1012.mapping;
+  update = other1012.update;
+  __isset = other1012.__isset;
 }
-WMCreateOrUpdateMappingRequest& WMCreateOrUpdateMappingRequest::operator=(const WMCreateOrUpdateMappingRequest& other1007) {
-  mapping = other1007.mapping;
-  update = other1007.update;
-  __isset = other1007.__isset;
+WMCreateOrUpdateMappingRequest& WMCreateOrUpdateMappingRequest::operator=(const WMCreateOrUpdateMappingRequest& other1013) {
+  mapping = other1013.mapping;
+  update = other1013.update;
+  __isset = other1013.__isset;
   return *this;
 }
 void WMCreateOrUpdateMappingRequest::printTo(std::ostream& out) const {
@@ -25391,11 +25433,11 @@ void swap(WMCreateOrUpdateMappingResponse &a, WMCreateOrUpdateMappingResponse &b
   (void) b;
 }
 
-WMCreateOrUpdateMappingResponse::WMCreateOrUpdateMappingResponse(const WMCreateOrUpdateMappingResponse& other1008) {
-  (void) other1008;
+WMCreateOrUpdateMappingResponse::WMCreateOrUpdateMappingResponse(const WMCreateOrUpdateMappingResponse& other1014) {
+  (void) other1014;
 }
-WMCreateOrUpdateMappingResponse& WMCreateOrUpdateMappingResponse::operator=(const WMCreateOrUpdateMappingResponse& other1009) {
-  (void) other1009;
+WMCreateOrUpdateMappingResponse& WMCreateOrUpdateMappingResponse::operator=(const WMCreateOrUpdateMappingResponse& other1015) {
+  (void) other1015;
   return *this;
 }
 void WMCreateOrUpdateMappingResponse::printTo(std::ostream& out) const {
@@ -25476,13 +25518,13 @@ void swap(WMDropMappingRequest &a, WMDropMappingRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMDropMappingRequest::WMDropMappingRequest(const WMDropMappingRequest& other1010) {
-  mapping = other1010.mapping;
-  __isset = other1010.__isset;
+WMDropMappingRequest::WMDropMappingRequest(const WMDropMappingRequest& other1016) {
+  mapping = other1016.mapping;
+  __isset = other1016.__isset;
 }
-WMDropMappingRequest& WMDropMappingRequest::operator=(const WMDropMappingRequest& other1011) {
-  mapping = other1011.mapping;
-  __isset = other1011.__isset;
+WMDropMappingRequest& WMDropMappingRequest::operator=(const WMDropMappingRequest& other1017) {
+  mapping = other1017.mapping;
+  __isset = other1017.__isset;
   return *this;
 }
 void WMDropMappingRequest::printTo(std::ostream& out) const {
@@ -25541,11 +25583,11 @@ void swap(WMDropMappingResponse &a, WMDropMappingResponse &b) {
   (void) b;
 }
 
-WMDropMappingResponse::WMDropMappingResponse(const WMDropMappingResponse& other1012) {
-  (void) other1012;
+WMDropMappingResponse::WMDropMappingResponse(const WMDropMappingResponse& other1018) {
+  (void) other1018;
 }
-WMDropMappingResponse& WMDropMappingResponse::operator=(const WMDropMappingResponse& other1013) {
-  (void) other1013;
+WMDropMappingResponse& WMDropMappingResponse::operator=(const WMDropMappingResponse& other1019) {
+  (void) other1019;
   return *this;
 }
 void WMDropMappingResponse::printTo(std::ostream& out) const {
@@ -25683,19 +25725,19 @@ void swap(WMCreateOrDropTriggerToPoolMappingRequest &a, WMCreateOrDropTriggerToP
   swap(a.__isset, b.__isset);
 }
 
-WMCreateOrDropTriggerToPoolMappingRequest::WMCreateOrDropTriggerToPoolMappingRequest(const WMCreateOrDropTriggerToPoolMappingRequest& other1014) {
-  resourcePlanName = other1014.resourcePlanName;
-  triggerName = other1014.triggerName;
-  poolPath = other1014.poolPath;
-  drop = other1014.drop;
-  __isset = other1014.__isset;
+WMCreateOrDropTriggerToPoolMappingRequest::WMCreateOrDropTriggerToPoolMappingRequest(const WMCreateOrDropTriggerToPoolMappingRequest& other1020) {
+  resourcePlanName = other1020.resourcePlanName;
+  triggerName = other1020.triggerName;
+  poolPath = other1020.poolPath;
+  drop = other1020.drop;
+  __isset = other1020.__isset;
 }
-WMCreateOrDropTriggerToPoolMappingRequest& WMCreateOrDropTriggerToPoolMappingRequest::operator=(const WMCreateOrDropTriggerToPoolMappingRequest& other1015) {
-  resourcePlanName = other1015.resourcePlanName;
-  triggerName = other1015.triggerName;
-  poolPath = other1015.poolPath;
-  drop = other1015.drop;
-  __isset = other1015.__isset;
+WMCreateOrDropTriggerToPoolMappingRequest& WMCreateOrDropTriggerToPoolMappingRequest::operator=(const WMCreateOrDropTriggerToPoolMappingRequest& other1021) {
+  resourcePlanName = other1021.resourcePlanName;
+  triggerName = other1021.triggerName;
+  poolPath = other1021.poolPath;
+  drop = other1021.drop;
+  __isset = other1021.__isset;
   return *this;
 }
 void WMCreateOrDropTriggerToPoolMappingRequest::printTo(std::ostream& out) const {
@@ -25757,11 +25799,11 @@ void swap(WMCreateOrDropTriggerToPoolMappingResponse &a, WMCreateOrDropTriggerTo
   (void) b;
 }
 
-WMCreateOrDropTriggerToPoolMappingResponse::WMCreateOrDropTriggerToPoolMappingResponse(const WMCreateOrDropTriggerToPoolMappingResponse& other1016) {
-  (void) other1016;
+WMCreateOrDropTriggerToPoolMappingResponse::WMCreateOrDropTriggerToPoolMappingResponse(const WMCreateOrDropTriggerToPoolMappingResponse& other1022) {
+  (void) other1022;
 }
-WMCreateOrDropTriggerToPoolMappingResponse& WMCreateOrDropTriggerToPoolMappingResponse::operator=(const WMCreateOrDropTriggerToPoolMappingResponse& other1017) {
-  (void) other1017;
+WMCreateOrDropTriggerToPoolMappingResponse& WMCreateOrDropTriggerToPoolMappingResponse::operator=(const WMCreateOrDropTriggerToPoolMappingResponse& other1023) {
+  (void) other1023;
   return *this;
 }
 void WMCreateOrDropTriggerToPoolMappingResponse::printTo(std::ostream& out) const {
@@ -25840,13 +25882,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other1018) : TException() {
-  message = other1018.message;
-  __isset = other1018.__isset;
+MetaException::MetaException(const MetaException& other1024) : TException() {
+  message = other1024.message;
+  __isset = other1024.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other1019) {
-  message = other1019.message;
-  __isset = other1019.__isset;
+MetaException& MetaException::operator=(const MetaException& other1025) {
+  message = other1025.message;
+  __isset = other1025.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -25937,13 +25979,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other1020) : TException() {
-  message = other1020.message;
-  __isset = other1020.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other1026) : TException() {
+  message = other1026.message;
+  __isset = other1026.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other1021) {
-  message = other1021.message;
-  __isset = other1021.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other1027) {
+  message = other1027.message;
+  __isset = other1027.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -26034,13 +26076,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other1022) : TException() {
-  message = other1022.message;
-  __isset = other1022.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other1028) : TException() {
+  message = other1028.message;
+  __isset = other1028.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other1023) {
-  message = other1023.message;
-  __isset = other1023.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other1029) {
+  message = other1029.message;
+  __isset = other1029.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -26131,13 +26173,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other1024) : TException() {
-  message = other1024.message;
-  __isset = other1024.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other1030) : TException() {
+  message = other1030.message;
+  __isset = other1030.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other1025) {
-  message = other1025.message;
-  __isset = other1025.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other1031) {
+  message = other1031.message;
+  __isset = other1031.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -26228,13 +26270,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other1026) : TException() {
-  message = other1026.message;
-  __isset = other1026.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other1032) : TException() {
+  message = other1032.message;
+  __isset = other1032.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other1027) {
-  message = other1027.message;
-  __isset = other1027.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other1033) {
+  message = other1033.message;
+  __isset = other1033.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -26325,13 +26367,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other1028) : TException() {
-  message = other1028.message;
-  __isset = other1028.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other1034) : TException() {
+  message = other1034.message;
+  __isset = other1034.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other1029) {
-  message = other1029.message;
-  __isset = other1029.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other1035) {
+  message = other1035.message;
+  __isset = other1035.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -26422,13 +26464,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other1030) : TException() {
-  message = other1030.message;
-  __isset = other1030.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other1036) : TException() {
+  message = other1036.message;
+  __isset = other1036.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other1031) {
-  message = other1031.message;
-  __isset = other1031.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other1037) {
+  message = other1037.message;
+  __isset = other1037.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -26519,13 +26561,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other1032) : TException() {
-  message = other1032.message;
-  __isset = other1032.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other1038) : TException() {
+  message = other1038.message;
+  __isset = other1038.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other1033) {
-  message = other1033.message;
-  __isset = other1033.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other1039) {
+  message = other1039.message;
+  __isset = other1039.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -26616,13 +26658,13 @@ void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other1034) : TException() {
-  message = other1034.message;
-  __isset = other1034.__isset;
+IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other1040) : TException() {
+  message = other1040.message;
+  __isset = other1040.__isset;
 }
-IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other1035) {
-  message = other1035.message;
-  __isset = other1035.__isset;
+IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other1041) {
+  message = other1041.message;
+  __isset = other1041.__isset;
   return *this;
 }
 void IndexAlreadyExistsException::printTo(std::ostream& out) const {
@@ -26713,13 +26755,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other1036) : TException() {
-  message = other1036.message;
-  __isset = other1036.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other1042) : TException() {
+  message = other1042.message;
+  __isset = other1042.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other1037) {
-  message = other1037.message;
-  __isset = other1037.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other1043) {
+  message = other1043.message;
+  __isset = other1043.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -26810,13 +26852,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other1038) : TException() {
-  message = other1038.message;
-  __isset = other1038.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other1044) : TException() {
+  message = other1044.message;
+  __isset = other1044.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other1039) {
-  message = other1039.message;
-  __isset = other1039.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other1045) {
+  message = other1045.message;
+  __isset = other1045.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -26907,13 +26949,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other1040) : TException() {
-  message = other1040.message;
-  __isset = other1040.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other1046) : TException() {
+  message = other1046.message;
+  __isset = other1046.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other1041) {
-  message = other1041.message;
-  __isset = other1041.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other1047) {
+  message = other1047.message;
+  __isset = other1047.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -27004,13 +27046,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other1042) : TException() {
-  message = other1042.message;
-  __isset = other1042.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other1048) : TException() {
+  message = other1048.message;
+  __isset = other1048.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other1043) {
-  message = other1043.message;
-  __isset = other1043.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other1049) {
+  message = other1049.message;
+  __isset = other1049.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -27101,13 +27143,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other1044) : TException() {
-  message = other1044.message;
-  __isset = other1044.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other1050) : TException() {
+  message = other1050.message;
+  __isset = other1050.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other1045) {
-  message = other1045.message;
-  __isset = other1045.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other1051) {
+  message = other1051.message;
+  __isset = other1051.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -27198,13 +27240,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other1046) : TException() {
-  message = other1046.message;
-  __isset = other1046.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other1052) : TException() {
+  message = other1052.message;
+  __isset = other1052.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other1047) {
-  message = other1047.message;
-  __isset = other1047.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other1053) {
+  message = other1053.message;
+  __isset = other1053.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -27295,13 +27337,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other1048) : TException() {
-  message = other1048.message;
-  __isset = other1048.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other1054) : TException() {
+  message = other1054.message;
+  __isset = other1054.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other1049) {
-  message = other1049.message;
-  __isset = other1049.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other1055) {
+  message = other1055.message;
+  __isset = other1055.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index 4cc6417..df646a7 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -9851,8 +9851,9 @@ inline std::ostream& operator<<(std::ostream& out, const WMValidateResourcePlanR
 }
 
 typedef struct _WMValidateResourcePlanResponse__isset {
-  _WMValidateResourcePlanResponse__isset() : errors(false) {}
+  _WMValidateResourcePlanResponse__isset() : errors(false), warnings(false) {}
   bool errors :1;
+  bool warnings :1;
 } _WMValidateResourcePlanResponse__isset;
 
 class WMValidateResourcePlanResponse {
@@ -9865,17 +9866,24 @@ class WMValidateResourcePlanResponse {
 
   virtual ~WMValidateResourcePlanResponse() throw();
   std::vector<std::string>  errors;
+  std::vector<std::string>  warnings;
 
   _WMValidateResourcePlanResponse__isset __isset;
 
   void __set_errors(const std::vector<std::string> & val);
 
+  void __set_warnings(const std::vector<std::string> & val);
+
   bool operator == (const WMValidateResourcePlanResponse & rhs) const
   {
     if (__isset.errors != rhs.__isset.errors)
       return false;
     else if (__isset.errors && !(errors == rhs.errors))
       return false;
+    if (__isset.warnings != rhs.__isset.warnings)
+      return false;
+    else if (__isset.warnings && !(warnings == rhs.warnings))
+      return false;
     return true;
   }
   bool operator != (const WMValidateResourcePlanResponse &rhs) const {


[8/8] hive git commit: HIVE-18231 : validate resource plan - part 2 - validate action and trigger expressions (Harish Jaiprakash, reviewed by Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-18231 : validate resource plan - part 2 - validate action and trigger expressions (Harish Jaiprakash, reviewed by Sergey Shelukhin)


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

Branch: refs/heads/master
Commit: 90d236affcb19b52ca66029e6646c5d751dc5f02
Parents: 6d890fa
Author: sergey <se...@apache.org>
Authored: Fri Jan 19 16:11:38 2018 -0800
Committer: sergey <se...@apache.org>
Committed: Fri Jan 19 16:11:38 2018 -0800

----------------------------------------------------------------------
 .../listener/DummyRawStoreFailEvent.java        |    3 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |    5 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |    3 +-
 .../formatting/JsonMetaDataFormatter.java       |   19 +-
 .../formatting/MetaDataFormatUtils.java         |    3 +
 .../metadata/formatting/MetaDataFormatter.java  |    4 +-
 .../formatting/TextMetaDataFormatter.java       |   13 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |   15 +
 .../hadoop/hive/ql/parse/ParseDriver.java       |   46 +-
 .../hadoop/hive/ql/parse/ResourcePlanParser.g   |   46 +-
 .../org/apache/hadoop/hive/ql/wm/Action.java    |   46 +-
 .../hadoop/hive/ql/wm/ExpressionFactory.java    |   33 +-
 .../formatting/TestJsonRPFormatter.java         |   10 +-
 .../hive/ql/wm/TestExpressionFactory.java       |  115 +
 .../apache/hadoop/hive/ql/wm/TestTrigger.java   |   46 +-
 .../test/queries/clientpositive/resourceplan.q  |   45 +-
 .../clientpositive/llap/resourceplan.q.out      |  260 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2326 ++++++++--------
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  524 ++--
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   10 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 2536 +++++++++---------
 .../WMGetTriggersForResourePlanResponse.java    |   36 +-
 .../api/WMValidateResourcePlanResponse.java     |  183 +-
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1428 +++++-----
 .../src/gen/thrift/gen-php/metastore/Types.php  |   73 +-
 .../hive_metastore/ThriftHiveMetastore.py       |  962 +++----
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |   41 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |    4 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |    5 +-
 .../hive/metastore/HiveMetaStoreClient.java     |    4 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |    3 +-
 .../hadoop/hive/metastore/ObjectStore.java      |   74 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |    4 +-
 .../hive/metastore/cache/CachedStore.java       |    3 +-
 .../src/main/thrift/hive_metastore.thrift       |    1 +
 .../DummyRawStoreControlledCommit.java          |    3 +-
 .../DummyRawStoreForJdoConnection.java          |    5 +-
 37 files changed, 4763 insertions(+), 4174 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
index 9ec41e3..78b2637 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
@@ -60,6 +60,7 @@ import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
@@ -1011,7 +1012,7 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public List<String> validateResourcePlan(String name)
+  public WMValidateResourcePlanResponse validateResourcePlan(String name)
       throws NoSuchObjectException, InvalidObjectException, MetaException {
     return objectStore.validateResourcePlan(name);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 96fc330..51ef390 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -109,6 +109,7 @@ import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlanStatus;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
@@ -717,9 +718,9 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
   private int alterResourcePlan(Hive db, AlterResourcePlanDesc desc) throws HiveException {
     if (desc.shouldValidate()) {
-      List<String> errors = db.validateResourcePlan(desc.getResourcePlanName());
+      WMValidateResourcePlanResponse result = db.validateResourcePlan(desc.getResourcePlanName());
       try (DataOutputStream out = getOutputStream(desc.getResFile())) {
-        formatter.showErrors(out, errors);
+        formatter.showErrors(out, result);
       } catch (IOException e) {
         throw new HiveException(e);
       };

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/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 f1610eb..23983d8 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
@@ -137,6 +137,7 @@ import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -4877,7 +4878,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
   }
 
-  public List<String> validateResourcePlan(String rpName) throws HiveException {
+  public WMValidateResourcePlanResponse validateResourcePlan(String rpName) throws HiveException {
     try {
       return getMSC().validateResourcePlan(rpName);
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
index 3142901..77e5678 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -50,7 +51,6 @@ import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.PrimaryKeyInfo;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.metadata.UniqueConstraint;
-import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.JsonGenerator;
 import org.codehaus.jackson.map.ObjectMapper;
 
@@ -578,15 +578,26 @@ public class JsonMetaDataFormatter implements MetaDataFormatter {
   }
 
   @Override
-  public void showErrors(DataOutputStream out, List<String> errors) throws HiveException {
+  public void showErrors(DataOutputStream out, WMValidateResourcePlanResponse response)
+      throws HiveException {
     JsonGenerator generator = null;
     try {
       generator = new ObjectMapper().getJsonFactory().createJsonGenerator(out);
-      generator.writeStartArray();
-      for (String error : errors) {
+      generator.writeStartObject();
+
+      generator.writeArrayFieldStart("errors");
+      for (String error : response.getErrors()) {
+        generator.writeString(error);
+      }
+      generator.writeEndArray();
+
+      generator.writeArrayFieldStart("warnings");
+      for (String error : response.getWarnings()) {
         generator.writeString(error);
       }
       generator.writeEndArray();
+
+      generator.writeEndObject();
     } catch (IOException e) {
       throw new HiveException(e);
     } finally {

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
index f4cbbd4..3b87824 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
@@ -782,6 +782,7 @@ public final class MetaDataFormatUtils {
       rpFormatter.endTriggers();
       rpFormatter.startMappings();
       for (Map.Entry<String, List<String>> mappingsOfType : mappings.entrySet()) {
+        mappingsOfType.getValue().sort(String::compareTo);
         rpFormatter.formatMappingType(mappingsOfType.getKey(), mappingsOfType.getValue());
       }
       if (isDefault) {
@@ -807,6 +808,8 @@ public final class MetaDataFormatUtils {
       for (PoolTreeNode child : children) {
         child.sortChildren();
       }
+      triggers.sort((WMTrigger t1, WMTrigger t2)
+          -> t1.getTriggerName().compareTo(t2.getTriggerName()));
     }
 
     static PoolTreeNode makePoolTree(WMFullResourcePlan fullRp) {

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java
index 50d2f57..88d5554 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -128,6 +129,7 @@ public interface MetaDataFormatter {
   void showFullResourcePlan(DataOutputStream out, WMFullResourcePlan resourcePlan)
       throws HiveException;
 
-  void showErrors(DataOutputStream out, List<String> errors) throws HiveException;
+  void showErrors(DataOutputStream out, WMValidateResourcePlanResponse errors)
+      throws HiveException;
 }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
index 74c5998..b743df0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo;
 import org.apache.hadoop.hive.ql.metadata.Hive;
@@ -724,10 +725,16 @@ class TextMetaDataFormatter implements MetaDataFormatter {
     out.write(str(val));
   }
 
-  public void showErrors(DataOutputStream out, List<String> errors) throws HiveException {
+  public void showErrors(DataOutputStream out, WMValidateResourcePlanResponse response)
+      throws HiveException {
     try {
-      for (String error : errors) {
-        out.write(error.getBytes("UTF-8"));
+      for (String error : response.getErrors()) {
+        write(out, error);
+        out.write(terminator);
+      }
+      for (String warning : response.getWarnings()) {
+        write(out, "warn: ");
+        write(out, warning);
         out.write(terminator);
       }
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index ef2aa34..78cbf25 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -469,6 +469,21 @@ ByteLengthLiteral
     (Digit)+ ('b' | 'B' | 'k' | 'K' | 'm' | 'M' | 'g' | 'G')
     ;
 
+TimeFullLiteral
+    :
+    (Digit)+ ('NS' | 'NSEC' | 'NSECS' | 'NANOSECOND' | 'NANOSECONDS' |
+          'US' | 'USEC' | 'USECS' | 'MICROSECOND' | 'MICROSECONDS' |
+          'MS' | 'MSEC' | 'MSECS' | 'MILLISECOND' | 'MILLISECONDS' |
+          'SEC' | 'SECS' | 'SECOND' | 'SECONDS' |
+          'MIN' | 'MINS' | 'MINUTE' | 'MINUTES' |
+          'HOUR' | 'HOURS' | 'DAY' | 'DAYS')
+    ;
+
+ByteLengthFullLiteral
+    :
+    (Digit)+ ('KB' | 'MB' | 'GB' | 'TB' | 'PB')
+    ;
+
 Number
     :
     (Digit)+ ( DOT (Digit)* (Exponent)? | Exponent)?

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java
index e6f0b22..bda3c21 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java
@@ -32,8 +32,6 @@ import org.antlr.runtime.tree.TreeAdaptor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.sun.tools.extcheck.Main;
-
 import org.apache.hadoop.hive.ql.Context;
 
 /**
@@ -294,7 +292,7 @@ public class ParseDriver {
       throw new ParseException(parser.errors);
     }
 
-    return (ASTNode) r.getTree();
+    return r.getTree();
   }
   public ASTNode parseExpression(String command) throws ParseException {
     LOG.info("Parsing expression: " + command);
@@ -321,4 +319,46 @@ public class ParseDriver {
 
     return (ASTNode) r.getTree();
   }
+
+  public ASTNode parseTriggerExpression(String command) throws ParseException {
+    HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command));
+    TokenRewriteStream tokens = new TokenRewriteStream(lexer);
+    HiveParser parser = new HiveParser(tokens);
+    parser.setTreeAdaptor(adaptor);
+    HiveParser_ResourcePlanParser.triggerExpressionStandalone_return r = null;
+    try {
+      r = parser.gResourcePlanParser.triggerExpressionStandalone();
+    } catch (RecognitionException e) {
+      e.printStackTrace();
+      throw new ParseException(parser.errors);
+    }
+    if (lexer.getErrors().size() != 0) {
+      throw new ParseException(lexer.getErrors());
+    } else if (parser.errors.size() != 0) {
+      throw new ParseException(parser.errors);
+    }
+
+    return r.getTree();
+  }
+
+  public ASTNode parseTriggerActionExpression(String command) throws ParseException {
+    HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command));
+    TokenRewriteStream tokens = new TokenRewriteStream(lexer);
+    HiveParser parser = new HiveParser(tokens);
+    parser.setTreeAdaptor(adaptor);
+    HiveParser_ResourcePlanParser.triggerActionExpressionStandalone_return r = null;
+    try {
+      r = parser.gResourcePlanParser.triggerActionExpressionStandalone();
+    } catch (RecognitionException e) {
+      e.printStackTrace();
+      throw new ParseException(parser.errors);
+    }
+    if (lexer.getErrors().size() != 0) {
+      throw new ParseException(lexer.getErrors());
+    } else if (parser.errors.size() != 0) {
+      throw new ParseException(parser.errors);
+    }
+
+    return r.getTree();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/java/org/apache/hadoop/hive/ql/parse/ResourcePlanParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ResourcePlanParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/ResourcePlanParser.g
index e3ea1f2..21f2d45 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ResourcePlanParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ResourcePlanParser.g
@@ -16,14 +16,33 @@
 */
 parser grammar ResourcePlanParser;
 
-options
-{
+options {
   output=AST;
   ASTLabelType=ASTNode;
   backtrack=false;
   k=3;
 }
 
+@members {
+  @Override
+  public Object recoverFromMismatchedSet(IntStream input,
+      RecognitionException re, BitSet follow) throws RecognitionException {
+    return gParent.recoverFromMismatchedSet(input, re, follow);
+  }
+
+  @Override
+  public void displayRecognitionError(String[] tokenNames,
+      RecognitionException e) {
+    gParent.displayRecognitionError(tokenNames, e);
+  }
+}
+
+@rulecatch {
+  catch (RecognitionException e) {
+    throw e;
+  }
+}
+
 resourcePlanDdlStatements
     : createResourcePlanStatement
     | alterResourcePlanStatement
@@ -133,9 +152,15 @@ poolPath
 triggerExpression
 @init { gParent.pushMsg("triggerExpression", state); }
 @after { gParent.popMsg(state); }
-    : triggerOrExpression -> ^(TOK_TRIGGER_EXPRESSION triggerOrExpression)
+    : triggerAtomExpression -> ^(TOK_TRIGGER_EXPRESSION triggerAtomExpression)
     ;
 
+triggerExpressionStandalone : triggerExpression EOF ;
+
+/*
+  The rules triggerOrExpression and triggerAndExpression are not being used right now.
+  Only > operator is supported, this should be changed if logic in ExpressionFactory changes.
+*/
 triggerOrExpression
 @init { gParent.pushMsg("triggerOrExpression", state); }
 @after { gParent.popMsg(state); }
@@ -151,22 +176,21 @@ triggerAndExpression
 triggerAtomExpression
 @init { gParent.pushMsg("triggerAtomExpression", state); }
 @after { gParent.popMsg(state); }
-    : (identifier comparisionOperator triggerLiteral)
-    | (LPAREN triggerOrExpression RPAREN)
+    : identifier comparisionOperator triggerLiteral
     ;
 
 triggerLiteral
 @init { gParent.pushMsg("triggerLiteral", state); }
 @after { gParent.popMsg(state); }
-    : (Number (KW_HOUR|KW_MINUTE|KW_SECOND)?)
-    | ByteLengthLiteral
-    | StringLiteral
+    : Number
+    | TimeFullLiteral
+    | ByteLengthFullLiteral
     ;
 
 comparisionOperator
 @init { gParent.pushMsg("comparisionOperator", state); }
 @after { gParent.popMsg(state); }
-    : EQUAL | LESSTHAN | LESSTHANOREQUALTO | GREATERTHAN | GREATERTHANOREQUALTO
+    : GREATERTHAN
     ;
 
 triggerActionExpression
@@ -176,6 +200,8 @@ triggerActionExpression
     | (KW_MOVE^ KW_TO! poolPath)
     ;
 
+triggerActionExpressionStandalone : triggerActionExpression EOF ;
+
 createTriggerStatement
 @init { gParent.pushMsg("create trigger statement", state); }
 @after { gParent.popMsg(state); }
@@ -263,7 +289,7 @@ createMappingStatement
 alterMappingStatement
 @init { gParent.pushMsg("alter mapping statement", state); }
 @after { gParent.popMsg(state); }
-    : (KW_ALTER mappingType=(KW_USER | KW_GROUP | KW_APPLICATION) KW_MAPPING
+    : (KW_ALTER mappingType=(KW_USER | KW_GROUP | KW_APPLICATION)
          KW_MAPPING name=StringLiteral
          KW_IN rpName=identifier ((KW_TO path=poolPath) | unmanaged)
          (KW_WITH KW_ORDER order=Number)?)

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/java/org/apache/hadoop/hive/ql/wm/Action.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/wm/Action.java b/ql/src/java/org/apache/hadoop/hive/ql/wm/Action.java
index 921ad54..7258ad5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/wm/Action.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/wm/Action.java
@@ -17,6 +17,12 @@ package org.apache.hadoop.hive.ql.wm;
 
 import java.util.Objects;
 
+import org.antlr.runtime.tree.Tree;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.apache.hadoop.hive.ql.parse.ParseException;
+
 /**
  * Action that gets invoked for trigger violations.
  */
@@ -46,16 +52,40 @@ public class Action {
   private final String poolName;
 
   public static Action fromMetastoreExpression(String metastoreActionExpression) {
-    if (metastoreActionExpression.equalsIgnoreCase(Type.KILL_QUERY.getDisplayName())) {
+    ParseDriver driver = new ParseDriver();
+    ASTNode node = null;
+    try {
+      node = driver.parseTriggerActionExpression(metastoreActionExpression);
+    } catch (ParseException e) {
+      throw new IllegalArgumentException(
+          "Invalid action expression: " + metastoreActionExpression, e);
+    }
+    if (node == null || node.getChildCount() != 2 ||
+        node.getChild(1).getType() != HiveParser.EOF) {
+      throw new IllegalArgumentException(
+          "Invalid action expression: " + metastoreActionExpression);
+    }
+    node = (ASTNode) node.getChild(0);
+    switch (node.getType()) {
+    case HiveParser.KW_KILL:
+      if (node.getChildCount() != 0) {
+        throw new IllegalArgumentException("Invalid KILL action");
+      }
       return new Action(Type.KILL_QUERY);
-    } else {
-      final String poolName = metastoreActionExpression.substring(Type.MOVE_TO_POOL.getDisplayName().length()).trim();
-      if (poolName.isEmpty()) {
-        throw new IllegalArgumentException("Invalid move action expression (" + metastoreActionExpression + "). Pool " +
-          "name is empty");
-      } else {
-        return new Action(Type.MOVE_TO_POOL, poolName);
+    case HiveParser.KW_MOVE: {
+      if (node.getChildCount() != 1) {
+        throw new IllegalArgumentException("Invalid move to action, expected poolPath");
+      }
+      Tree poolNode = node.getChild(0);
+      StringBuilder poolPath = new StringBuilder(poolNode.getText());
+      for (int i = 0; i < poolNode.getChildCount(); ++i) {
+        poolPath.append(poolNode.getChild(0).getText());
       }
+      return new Action(Type.MOVE_TO_POOL, poolPath.toString());
+    }
+    default:
+      throw new IllegalArgumentException("Unhandled action expression, type: " + node.getType() +
+          ": " + metastoreActionExpression);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/java/org/apache/hadoop/hive/ql/wm/ExpressionFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/wm/ExpressionFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/wm/ExpressionFactory.java
index 953faa8..2299a1b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/wm/ExpressionFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/wm/ExpressionFactory.java
@@ -19,6 +19,10 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.Validator;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.apache.hadoop.hive.ql.parse.ParseException;
 
 /**
  * Factory to create expressions
@@ -30,16 +34,33 @@ public class ExpressionFactory {
       return null;
     }
 
-    // TODO: Only ">" predicate is supported right now, this has to be extended to support expression tree when
-    // multiple conditions are required. HIVE-17622
+    ParseDriver driver = new ParseDriver();
+    ASTNode node = null;
+    try {
+      node = driver.parseTriggerExpression(expression);
+    } catch (ParseException e) {
+      throw new IllegalArgumentException("Invalid expression: " + expression, e);
+    }
+    if (node.getChildCount() == 2 && node.getChild(1).getType() == HiveParser.EOF) {
+      node = (ASTNode) node.getChild(0);
+    }
+    if (node.getType() != HiveParser.TOK_TRIGGER_EXPRESSION) {
+      throw new IllegalArgumentException(
+          "Expected trigger expression, got: " + node.toStringTree());
+    }
+
+    if (node.getChildCount() != 3) {
+      throw new IllegalArgumentException("Only single > condition supported: " + expression);
+    }
 
-    String[] tokens = expression.split(Expression.Predicate.GREATER_THAN.getSymbol());
-    if (tokens.length != 2) {
+    // Only ">" predicate is supported right now, this has to be extended to support
+    // expression tree when multiple conditions are required. HIVE-17622
+    if (node.getChild(1).getType() != HiveParser.GREATERTHAN) {
       throw new IllegalArgumentException("Invalid predicate in expression");
     }
 
-    final String counterName = tokens[0].trim();
-    final String counterValueStr = tokens[1].trim();
+    final String counterName = node.getChild(0).getText();
+    final String counterValueStr = node.getChild(2).getText().toLowerCase();
     if (counterName.isEmpty()) {
       throw new IllegalArgumentException("Counter name cannot be empty!");
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/test/org/apache/hadoop/hive/ql/metadata/formatting/TestJsonRPFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/formatting/TestJsonRPFormatter.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/formatting/TestJsonRPFormatter.java
index ed854e7..9e8d290 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/formatting/TestJsonRPFormatter.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/formatting/TestJsonRPFormatter.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertTrue;
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.util.ArrayList;
+import java.util.HashSet;
 
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
@@ -152,8 +153,13 @@ public class TestJsonRPFormatter {
     JsonNode type0 = pool2.get("mappings").get(0);
     assertEquals("user", type0.get("type").asText());
     assertTrue(type0.get("values").isArray());
-    assertEquals("foo", type0.get("values").get(0).asText());
-    assertEquals("bar", type0.get("values").get(1).asText());
+    assertEquals(2, type0.get("values").size());
+    HashSet<String> vals = new HashSet<>();
+    for (int i = 0; i < type0.get("values").size(); ++i) {
+      vals.add(type0.get("values").get(i).asText());
+    }
+    assertTrue(vals.contains("foo"));
+    assertTrue(vals.contains("bar"));
 
     JsonNode pool1 = jsonTree.get("pools").get(1);
     assertEquals("pool1", pool1.get("name").asText());

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/test/org/apache/hadoop/hive/ql/wm/TestExpressionFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/wm/TestExpressionFactory.java b/ql/src/test/org/apache/hadoop/hive/ql/wm/TestExpressionFactory.java
new file mode 100644
index 0000000..074794c
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/wm/TestExpressionFactory.java
@@ -0,0 +1,115 @@
+/*
+ * 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.ql.wm;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import org.apache.hadoop.hive.ql.wm.Expression.Predicate;
+import org.junit.Test;
+
+public class TestExpressionFactory {
+  @Test
+  public void testSize() {
+    Expression expr = null;
+
+    expr = ExpressionFactory.fromString("BYTES_READ > 5");
+    assertNotNull(expr);
+    assertEquals(Predicate.GREATER_THAN, expr.getPredicate());
+    assertEquals("BYTES_READ", expr.getCounterLimit().getName());
+    assertEquals(5, expr.getCounterLimit().getLimit());
+
+    expr = ExpressionFactory.fromString("BYTES_READ > 5kb");
+    assertNotNull(expr);
+    assertEquals(Predicate.GREATER_THAN, expr.getPredicate());
+    assertEquals("BYTES_READ", expr.getCounterLimit().getName());
+    assertEquals(5 * (1 << 10), expr.getCounterLimit().getLimit());
+
+    expr = ExpressionFactory.fromString("BYTES_READ > 2mb");
+    assertNotNull(expr);
+    assertEquals(Predicate.GREATER_THAN, expr.getPredicate());
+    assertEquals("BYTES_READ", expr.getCounterLimit().getName());
+    assertEquals(2 * (1 << 20), expr.getCounterLimit().getLimit());
+
+    expr = ExpressionFactory.fromString("BYTES_READ > 3gb");
+    assertNotNull(expr);
+    assertEquals(Predicate.GREATER_THAN, expr.getPredicate());
+    assertEquals("BYTES_READ", expr.getCounterLimit().getName());
+    assertEquals(3L * (1 << 30), expr.getCounterLimit().getLimit());
+
+    expr = ExpressionFactory.fromString("SHUFFLE_BYTES > 7tb");
+    assertNotNull(expr);
+    assertEquals(Predicate.GREATER_THAN, expr.getPredicate());
+    assertEquals("SHUFFLE_BYTES", expr.getCounterLimit().getName());
+    assertEquals(7L * (1L << 40), expr.getCounterLimit().getLimit());
+
+    expr = ExpressionFactory.fromString("SHUFFLE_BYTES > 6pb");
+    assertNotNull(expr);
+    assertEquals(Predicate.GREATER_THAN, expr.getPredicate());
+    assertEquals("SHUFFLE_BYTES", expr.getCounterLimit().getName());
+    assertEquals(6L * (1L << 50), expr.getCounterLimit().getLimit());
+
+    expr = ExpressionFactory.fromString("BYTES_WRITTEN > 27");
+    assertNotNull(expr);
+    assertEquals(Predicate.GREATER_THAN, expr.getPredicate());
+    assertEquals("BYTES_WRITTEN", expr.getCounterLimit().getName());
+    assertEquals(27, expr.getCounterLimit().getLimit());
+  }
+
+  @Test
+  public void testTime() {
+    Expression expr = null;
+
+    expr = ExpressionFactory.fromString("ELAPSED_TIME > 1");
+    assertNotNull(expr);
+    assertEquals(Predicate.GREATER_THAN, expr.getPredicate());
+    assertEquals("ELAPSED_TIME", expr.getCounterLimit().getName());
+    assertEquals(1, expr.getCounterLimit().getLimit());
+
+    expr = ExpressionFactory.fromString("ELAPSED_TIME > 1ms");
+    assertNotNull(expr);
+    assertEquals(Predicate.GREATER_THAN, expr.getPredicate());
+    assertEquals("ELAPSED_TIME", expr.getCounterLimit().getName());
+    assertEquals(1, expr.getCounterLimit().getLimit());
+
+    expr = ExpressionFactory.fromString("ELAPSED_TIME > 1sec");
+    assertNotNull(expr);
+    assertEquals(Predicate.GREATER_THAN, expr.getPredicate());
+    assertEquals("ELAPSED_TIME", expr.getCounterLimit().getName());
+    assertEquals(1000, expr.getCounterLimit().getLimit());
+
+    expr = ExpressionFactory.fromString("ELAPSED_TIME > 1min");
+    assertNotNull(expr);
+    assertEquals(Predicate.GREATER_THAN, expr.getPredicate());
+    assertEquals("ELAPSED_TIME", expr.getCounterLimit().getName());
+    assertEquals(60 * 1000, expr.getCounterLimit().getLimit());
+
+    expr = ExpressionFactory.fromString("ELAPSED_TIME > 1hour");
+    assertNotNull(expr);
+    assertEquals(Predicate.GREATER_THAN, expr.getPredicate());
+    assertEquals("ELAPSED_TIME", expr.getCounterLimit().getName());
+    assertEquals(3600 * 1000, expr.getCounterLimit().getLimit());
+
+    expr = ExpressionFactory.fromString("ELAPSED_TIME > 1day");
+    assertNotNull(expr);
+    assertEquals(Predicate.GREATER_THAN, expr.getPredicate());
+    assertEquals("ELAPSED_TIME", expr.getCounterLimit().getName());
+    assertEquals(24 * 3600 * 1000, expr.getCounterLimit().getLimit());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/test/org/apache/hadoop/hive/ql/wm/TestTrigger.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/wm/TestTrigger.java b/ql/src/test/org/apache/hadoop/hive/ql/wm/TestTrigger.java
index b686783..a3e8336 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/wm/TestTrigger.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/wm/TestTrigger.java
@@ -185,25 +185,25 @@ public class TestTrigger {
     assertEquals(expected, expression);
     assertEquals(expected.hashCode(), expression.hashCode());
 
-    expression = ExpressionFactory.fromString(" SHUFFLE_BYTES > 1 gB");
+    expression = ExpressionFactory.fromString(" SHUFFLE_BYTES > 1gB");
     expected = ExpressionFactory.createExpression(new FileSystemCounterLimit("",
       FileSystemCounterLimit.FSCounter.SHUFFLE_BYTES, 1024 * 1024 * 1024));
     assertEquals(expected, expression);
     assertEquals(expected.hashCode(), expression.hashCode());
 
-    expression = ExpressionFactory.fromString(" SHUFFLE_BYTES > 1 TB");
+    expression = ExpressionFactory.fromString(" SHUFFLE_BYTES > 1TB");
     expected = ExpressionFactory.createExpression(new FileSystemCounterLimit("",
       FileSystemCounterLimit.FSCounter.SHUFFLE_BYTES, 1024L * 1024 * 1024 * 1024));
     assertEquals(expected, expression);
     assertEquals(expected.hashCode(), expression.hashCode());
 
-    expression = ExpressionFactory.fromString(" SHUFFLE_BYTES > 100 B");
+    expression = ExpressionFactory.fromString(" SHUFFLE_BYTES > 100");
     expected = ExpressionFactory.createExpression(new FileSystemCounterLimit("",
       FileSystemCounterLimit.FSCounter.SHUFFLE_BYTES, 100));
     assertEquals(expected, expression);
     assertEquals(expected.hashCode(), expression.hashCode());
 
-    expression = ExpressionFactory.fromString(" SHUFFLE_BYTES > 100bytes");
+    expression = ExpressionFactory.fromString(" SHUFFLE_BYTES > 100");
     expected = ExpressionFactory.createExpression(new FileSystemCounterLimit("",
       FileSystemCounterLimit.FSCounter.SHUFFLE_BYTES, 100));
     assertEquals(expected, expression);
@@ -213,38 +213,38 @@ public class TestTrigger {
   @Test
   public void testIllegalSizeCounterValue1() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Invalid size unit");
+    thrown.expectMessage("Invalid expression:  SHUFFLE_BYTES > 300GiB");
     ExpressionFactory.fromString(" SHUFFLE_BYTES > 300GiB");
   }
 
   @Test
   public void testIllegalSizeCounterValue2() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Invalid size unit");
+    thrown.expectMessage("Invalid expression:  SHUFFLE_BYTES > 300 foo");
     ExpressionFactory.fromString(" SHUFFLE_BYTES > 300 foo");
   }
 
   @Test
   public void testTimeValidationInTrigger() {
-    Expression expression = ExpressionFactory.fromString(" elapsed_TIME > 300 s");
+    Expression expression = ExpressionFactory.fromString(" elapsed_TIME > 300sec");
     Expression expected = ExpressionFactory.createExpression(new TimeCounterLimit(TimeCounterLimit.TimeCounter
       .ELAPSED_TIME, 300000));
     assertEquals(expected, expression);
     assertEquals(expected.hashCode(), expression.hashCode());
 
-    expression = ExpressionFactory.fromString(" elapsed_TIME > 300 seconds");
+    expression = ExpressionFactory.fromString(" elapsed_TIME > 300seconds");
     expected = ExpressionFactory.createExpression(new TimeCounterLimit(TimeCounterLimit.TimeCounter
       .ELAPSED_TIME, 300000));
     assertEquals(expected, expression);
     assertEquals(expected.hashCode(), expression.hashCode());
 
-    expression = ExpressionFactory.fromString(" elapsed_TIME > 300 sec");
+    expression = ExpressionFactory.fromString(" elapsed_TIME > 300sec");
     expected = ExpressionFactory.createExpression(new TimeCounterLimit(TimeCounterLimit.TimeCounter
       .ELAPSED_TIME, 300000));
     assertEquals(expected, expression);
     assertEquals(expected.hashCode(), expression.hashCode());
 
-    expression = ExpressionFactory.fromString(" elapsed_TIME > 300s");
+    expression = ExpressionFactory.fromString(" elapsed_TIME > 300second");
     expected = ExpressionFactory.createExpression(new TimeCounterLimit(TimeCounterLimit.TimeCounter
       .ELAPSED_TIME, 300000));
     assertEquals(expected, expression);
@@ -262,7 +262,7 @@ public class TestTrigger {
     assertEquals(expected, expression);
     assertEquals(expected.hashCode(), expression.hashCode());
 
-    expression = ExpressionFactory.fromString(" elapsed_TIME > 300000000 microseconds");
+    expression = ExpressionFactory.fromString(" elapsed_TIME > 300000000microseconds");
     expected = ExpressionFactory.createExpression(new TimeCounterLimit(TimeCounterLimit.TimeCounter
       .ELAPSED_TIME, 300000));
     assertEquals(expected, expression);
@@ -278,14 +278,14 @@ public class TestTrigger {
   @Test
   public void testIllegalTimeCounterValue1() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Invalid time unit");
-    ExpressionFactory.fromString(" elapsed_TIME > 300 light years");
+    thrown.expectMessage("Invalid expression:  elapsed_TIME > 300lightyears");
+    ExpressionFactory.fromString(" elapsed_TIME > 300lightyears");
   }
 
   @Test
   public void testIllegalTimeCounterValue2() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Invalid time unit");
+    thrown.expectMessage("Invalid expression:  elapsed_TIME > 300secTOR");
     ExpressionFactory.fromString(" elapsed_TIME > 300secTOR");
   }
 
@@ -296,7 +296,7 @@ public class TestTrigger {
     assertEquals("MOVE TO etl", Action.fromMetastoreExpression("MOVE TO etl").toString());
 
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Invalid move action expression (MOVE TO    ). Pool name is empty");
+    thrown.expectMessage("Invalid action expression: MOVE TO  ");
     assertEquals(Action.Type.MOVE_TO_POOL, Action.fromMetastoreExpression("MOVE TO    ").getType());
   }
 
@@ -327,56 +327,56 @@ public class TestTrigger {
   @Test
   public void testIllegalExpressionsUnsupportedPredicate() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Invalid predicate in expression");
+    thrown.expectMessage("Invalid expression: BYTES_READ < 1024");
     ExpressionFactory.fromString("BYTES_READ < 1024");
   }
 
   @Test
   public void testIllegalExpressionsMissingLimit() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Invalid predicate in expression");
+    thrown.expectMessage("Invalid expression: BYTES_READ >");
     ExpressionFactory.fromString("BYTES_READ >");
   }
 
   @Test
   public void testIllegalExpressionsMissingCounter() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Counter name cannot be empty!");
+    thrown.expectMessage("Invalid expression: > 1024");
     ExpressionFactory.fromString("> 1024");
   }
 
   @Test
   public void testIllegalExpressionsMultipleLimit() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Invalid predicate in expression");
+    thrown.expectMessage("Invalid expression: BYTES_READ > 1024 > 1025");
     ExpressionFactory.fromString("BYTES_READ > 1024 > 1025");
   }
 
   @Test
   public void testIllegalExpressionsMultipleCounters() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Invalid predicate in expression");
+    thrown.expectMessage("Invalid expression: BYTES_READ > BYTES_READ > 1025");
     ExpressionFactory.fromString("BYTES_READ > BYTES_READ > 1025");
   }
 
   @Test
   public void testIllegalExpressionsInvalidLimitPost() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Invalid size unit");
+    thrown.expectMessage("Invalid expression: BYTES_READ > 1024aaaa");
     ExpressionFactory.fromString("BYTES_READ > 1024aaaa");
   }
 
   @Test
   public void testIllegalExpressionsInvalidLimitPre() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Invalid counter value");
+    thrown.expectMessage("Invalid expression: BYTES_READ > foo1024");
     ExpressionFactory.fromString("BYTES_READ > foo1024");
   }
 
   @Test
   public void testIllegalExpressionsInvalidNegativeLimit() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Illegal value for counter limit. Expected a positive long value.");
+    thrown.expectMessage("Invalid expression: BYTES_READ > -1024");
     ExpressionFactory.fromString("BYTES_READ > -1024");
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/test/queries/clientpositive/resourceplan.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/resourceplan.q b/ql/src/test/queries/clientpositive/resourceplan.q
index d2aec73..7314585 100644
--- a/ql/src/test/queries/clientpositive/resourceplan.q
+++ b/ql/src/test/queries/clientpositive/resourceplan.q
@@ -27,7 +27,8 @@ SHOW RESOURCE PLAN plan_1;
 SELECT * FROM SYS.WM_RESOURCEPLANS;
 
 -- Create and show plan_2.
-CREATE RESOURCE PLAN plan_2 WITH QUERY_PARALLELISM=4;
+CREATE RESOURCE PLAN plan_2 WITH QUERY_PARALLELISM=5;
+ALTER RESOURCE PLAN plan_2 SET QUERY_PARALLELISM=10;
 SHOW RESOURCE PLANS;
 SHOW RESOURCE PLAN plan_2;
 SELECT * FROM SYS.WM_RESOURCEPLANS;
@@ -147,30 +148,38 @@ SELECT * FROM SYS.WM_RESOURCEPLANS;
 
 CREATE RESOURCE PLAN plan_1;
 
-CREATE TRIGGER plan_1.trigger_1 WHEN BYTES_READ > 10k AND BYTES_READ <= 1M OR ELAPSED_TIME > 30 SECOND AND ELAPSED_TIME < 1 MINUTE DO KILL;
+CREATE TRIGGER plan_1.trigger_1 WHEN BYTES_READ > 10kb DO KILL;
 SELECT * FROM SYS.WM_TRIGGERS;
 
 -- Duplicate should fail.
-CREATE TRIGGER plan_1.trigger_1 WHEN BYTES_READ = 10G DO KILL;
+CREATE TRIGGER plan_1.trigger_1 WHEN ELAPSED_TIME > 300 DO KILL;
 
-CREATE TRIGGER plan_1.trigger_2 WHEN BYTES_READ > 100 DO MOVE TO slow_pool;
+-- Invalid triggers should fail.
+CREATE TRIGGER plan_1.trigger_2 WHEN ELAPSED_TIME > 30sec AND BYTES_READ > 10 DO MOVE TO slow_pool;
+CREATE TRIGGER plan_1.trigger_2 WHEN ELAPSED_TIME > 30second OR BYTES_READ > 10 DO MOVE TO slow_pool;
+CREATE TRIGGER plan_1.trigger_2 WHEN ELAPSED_TIME >= 30seconds DO MOVE TO slow_pool;
+CREATE TRIGGER plan_1.trigger_2 WHEN ELAPSED_TIME < 30hour DO MOVE TO slow_pool;
+CREATE TRIGGER plan_1.trigger_2 WHEN ELAPSED_TIME <= 30min DO MOVE TO slow_pool;
+CREATE TRIGGER plan_1.trigger_2 WHEN ELAPSED_TIME = 0day DO MOVE TO slow_pool;
+
+CREATE TRIGGER plan_1.trigger_2 WHEN ELAPSED_TIME > 30hour DO MOVE TO slow_pool;
 SELECT * FROM SYS.WM_TRIGGERS;
 
-ALTER TRIGGER plan_1.trigger_1 WHEN BYTES_READ = 1000 DO KILL;
+ALTER TRIGGER plan_1.trigger_1 WHEN BYTES_READ > 1min DO KILL;
 SELECT * FROM SYS.WM_TRIGGERS;
 
 DROP TRIGGER plan_1.trigger_1;
 SELECT * FROM SYS.WM_TRIGGERS;
 
 -- No edit on active resource plan.
-CREATE TRIGGER plan_2.trigger_1 WHEN BYTES_READ = 0m DO MOVE TO null_pool;
+CREATE TRIGGER plan_2.trigger_1 WHEN BYTES_READ > 100mb DO MOVE TO null_pool;
 
 -- Add trigger with reserved keywords.
-CREATE TRIGGER `table`.`table` WHEN BYTES_WRITTEN > 100K DO MOVE TO `table`;
-CREATE TRIGGER `table`.`trigger` WHEN BYTES_WRITTEN > 100K DO MOVE TO `default`;
-CREATE TRIGGER `table`.`database` WHEN BYTES_WRITTEN > 1M DO MOVE TO `default`;
+CREATE TRIGGER `table`.`table` WHEN BYTES_WRITTEN > 100KB DO MOVE TO `table`;
+CREATE TRIGGER `table`.`trigger` WHEN BYTES_WRITTEN > 100MB DO MOVE TO `default`;
+CREATE TRIGGER `table`.`database` WHEN BYTES_WRITTEN > 1GB DO MOVE TO `default`;
 CREATE TRIGGER `table`.`trigger1` WHEN ELAPSED_TIME > 10 DO KILL;
-CREATE TRIGGER `table`.`trigger2` WHEN BYTES_READ > 100 DO KILL;
+CREATE TRIGGER `table`.`trigger2` WHEN ELAPSED_TIME > 1hour DO KILL;
 SELECT * FROM SYS.WM_TRIGGERS;
 DROP TRIGGER `table`.`database`;
 SELECT * FROM SYS.WM_TRIGGERS;
@@ -179,17 +188,17 @@ SELECT * FROM SYS.WM_TRIGGERS;
 ALTER RESOURCE PLAN plan_1 ENABLE;
 SELECT * FROM SYS.WM_RESOURCEPLANS;
 DROP TRIGGER plan_1.trigger_2;
-ALTER TRIGGER plan_1.trigger_2 WHEN BYTES_READ = 1000g DO KILL;
+ALTER TRIGGER plan_1.trigger_2 WHEN BYTES_READ > 1000gb DO KILL;
 
 -- Cannot drop/change trigger from active plan.
 ALTER RESOURCE PLAN plan_1 ACTIVATE;
 SELECT * FROM SYS.WM_RESOURCEPLANS;
 DROP TRIGGER plan_1.trigger_2;
-ALTER TRIGGER plan_1.trigger_2 WHEN BYTES_READ = 1000K DO KILL;
+ALTER TRIGGER plan_1.trigger_2 WHEN BYTES_READ > 1000KB DO KILL;
 
 -- Once disabled we should be able to change it.
 ALTER RESOURCE PLAN plan_2 DISABLE;
-CREATE TRIGGER plan_2.trigger_1 WHEN BYTES_READ = 0 DO MOVE TO null_pool;
+CREATE TRIGGER plan_2.trigger_1 WHEN BYTES_READ > 0 DO MOVE TO null_pool;
 SELECT * FROM SYS.WM_TRIGGERS;
 
 
@@ -212,13 +221,13 @@ CREATE POOL plan_2.default.c1 WITH
     ALLOC_FRACTION=0.3, QUERY_PARALLELISM=3, SCHEDULING_POLICY='fair';
 
 CREATE POOL plan_2.default.c2 WITH
-    QUERY_PARALLELISM=2, SCHEDULING_POLICY='fair', ALLOC_FRACTION=0.2;
+    QUERY_PARALLELISM=2, SCHEDULING_POLICY='fair', ALLOC_FRACTION=0.7;
 
--- Cannot activate c1 + c2 = 0.5
+-- Cannot activate c1 + c2 = 1.0
 ALTER RESOURCE PLAN plan_2 VALIDATE;
 ALTER RESOURCE PLAN plan_2 ENABLE ACTIVATE;
 
-ALTER POOL plan_2.default.c2 SET ALLOC_FRACTION = 0.7, QUERY_PARALLELISM = 1;
+ALTER POOL plan_2.default.c2 SET ALLOC_FRACTION = 0.5, QUERY_PARALLELISM = 1;
 ALTER POOL plan_2.default.c2 SET SCHEDULING_POLICY='fair';
 SELECT * FROM SYS.WM_POOLS;
 ALTER POOL plan_2.default.c2 UNSET SCHEDULING_POLICY;
@@ -371,8 +380,8 @@ SELECT * FROM SYS.WM_MAPPINGS;
 CREATE RESOURCE PLAN plan_4a LIKE plan_4;
 CREATE POOL plan_4a.pool1 WITH SCHEDULING_POLICY='fair', QUERY_PARALLELISM=2, ALLOC_FRACTION=0.0;
 CREATE USER MAPPING "user1" IN plan_4a TO pool1;
-CREATE TRIGGER plan_4a.trigger_1 WHEN BYTES_READ = 10G DO KILL;
-CREATE TRIGGER plan_4a.trigger_2 WHEN BYTES_READ = 11G DO KILL;
+CREATE TRIGGER plan_4a.trigger_1 WHEN BYTES_READ > 10GB DO KILL;
+CREATE TRIGGER plan_4a.trigger_2 WHEN BYTES_READ > 11GB DO KILL;
 ALTER POOL plan_4a.pool1 ADD TRIGGER trigger_2;
 
 CREATE RESOURCE PLAN plan_4b LIKE plan_4a;

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/ql/src/test/results/clientpositive/llap/resourceplan.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/resourceplan.q.out b/ql/src/test/results/clientpositive/llap/resourceplan.q.out
index 68508e9..b23720d 100644
--- a/ql/src/test/results/clientpositive/llap/resourceplan.q.out
+++ b/ql/src/test/results/clientpositive/llap/resourceplan.q.out
@@ -3210,22 +3210,26 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_1	DISABLED	NULL	default
-PREHOOK: query: CREATE RESOURCE PLAN plan_2 WITH QUERY_PARALLELISM=4
+PREHOOK: query: CREATE RESOURCE PLAN plan_2 WITH QUERY_PARALLELISM=5
 PREHOOK: type: CREATE RESOURCEPLAN
-POSTHOOK: query: CREATE RESOURCE PLAN plan_2 WITH QUERY_PARALLELISM=4
+POSTHOOK: query: CREATE RESOURCE PLAN plan_2 WITH QUERY_PARALLELISM=5
 POSTHOOK: type: CREATE RESOURCEPLAN
+PREHOOK: query: ALTER RESOURCE PLAN plan_2 SET QUERY_PARALLELISM=10
+PREHOOK: type: ALTER RESOURCEPLAN
+POSTHOOK: query: ALTER RESOURCE PLAN plan_2 SET QUERY_PARALLELISM=10
+POSTHOOK: type: ALTER RESOURCEPLAN
 PREHOOK: query: SHOW RESOURCE PLANS
 PREHOOK: type: SHOW RESOURCEPLAN
 POSTHOOK: query: SHOW RESOURCE PLANS
 POSTHOOK: type: SHOW RESOURCEPLAN
 plan_1	DISABLED	 
-plan_2	DISABLED	4
+plan_2	DISABLED	10
 PREHOOK: query: SHOW RESOURCE PLAN plan_2
 PREHOOK: type: SHOW RESOURCEPLAN
 POSTHOOK: query: SHOW RESOURCE PLAN plan_2
 POSTHOOK: type: SHOW RESOURCEPLAN
-plan_2[status=DISABLED,parallelism=4,defaultPool=default]
- +  default[allocFraction=1.0,schedulingPolicy=null,parallelism=4]
+plan_2[status=DISABLED,parallelism=10,defaultPool=default]
+ +  default[allocFraction=1.0,schedulingPolicy=null,parallelism=5]
      |  mapped for default
 PREHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 PREHOOK: type: QUERY
@@ -3236,7 +3240,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_1	DISABLED	NULL	default
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 FAILED: SemanticException Invalid create arguments (tok_create_rp plan_3 (tok_query_parallelism 5) (tok_default_pool all))
 PREHOOK: query: ALTER RESOURCE PLAN plan_1 RENAME TO plan_2
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3250,7 +3254,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_1	DISABLED	NULL	default
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_1 RENAME TO plan_3
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_1 RENAME TO plan_3
@@ -3263,7 +3267,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 plan_3	DISABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 SET QUERY_PARALLELISM = 4
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3277,7 +3281,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 plan_3	DISABLED	4	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 UNSET QUERY_PARALLELISM
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3291,7 +3295,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 plan_3	DISABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 SET QUERY_PARALLELISM = 30, DEFAULT POOL = default1
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3304,7 +3308,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 plan_3	DISABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 ENABLE
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3328,7 +3332,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 plan_3	DISABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 ACTIVATE
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3341,7 +3345,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 plan_3	DISABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 DISABLE
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3355,7 +3359,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 plan_3	DISABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 ENABLE
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3369,7 +3373,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 plan_3	ENABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 ACTIVATE
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3383,7 +3387,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 plan_3	ACTIVE	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 ACTIVATE
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3397,7 +3401,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 plan_3	ACTIVE	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 ENABLE
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3410,7 +3414,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 plan_3	ACTIVE	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 DISABLE
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3423,7 +3427,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 plan_3	ACTIVE	NULL	default
 PREHOOK: query: DISABLE WORKLOAD MANAGEMENT
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3437,7 +3441,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 plan_3	ENABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 DISABLE
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3455,7 +3459,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	DISABLED	4	default
+plan_2	DISABLED	10	default
 plan_3	ACTIVE	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_2 ENABLE
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3469,7 +3473,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	ENABLED	4	default
+plan_2	ENABLED	10	default
 plan_3	ACTIVE	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_2 ACTIVATE
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3483,7 +3487,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	ACTIVE	4	default
+plan_2	ACTIVE	10	default
 plan_3	ENABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 ENABLE
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3497,7 +3501,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	ACTIVE	4	default
+plan_2	ACTIVE	10	default
 plan_3	ENABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 DISABLE
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3511,7 +3515,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	ACTIVE	4	default
+plan_2	ACTIVE	10	default
 plan_3	DISABLED	NULL	default
 PREHOOK: query: DROP RESOURCE PLAN plan_2
 PREHOOK: type: DROP RESOURCEPLAN
@@ -3528,7 +3532,7 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	ACTIVE	4	default
+plan_2	ACTIVE	10	default
 PREHOOK: query: CREATE RESOURCE PLAN `table`
 PREHOOK: type: CREATE RESOURCEPLAN
 POSTHOOK: query: CREATE RESOURCE PLAN `table`
@@ -3545,15 +3549,15 @@ POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
-plan_2	ACTIVE	4	default
+plan_2	ACTIVE	10	default
 table	DISABLED	1	default
 PREHOOK: query: CREATE RESOURCE PLAN plan_1
 PREHOOK: type: CREATE RESOURCEPLAN
 POSTHOOK: query: CREATE RESOURCE PLAN plan_1
 POSTHOOK: type: CREATE RESOURCEPLAN
-PREHOOK: query: CREATE TRIGGER plan_1.trigger_1 WHEN BYTES_READ > 10k AND BYTES_READ <= 1M OR ELAPSED_TIME > 30 SECOND AND ELAPSED_TIME < 1 MINUTE DO KILL
+PREHOOK: query: CREATE TRIGGER plan_1.trigger_1 WHEN BYTES_READ > 10kb DO KILL
 PREHOOK: type: CREATE TRIGGER
-POSTHOOK: query: CREATE TRIGGER plan_1.trigger_1 WHEN BYTES_READ > 10k AND BYTES_READ <= 1M OR ELAPSED_TIME > 30 SECOND AND ELAPSED_TIME < 1 MINUTE DO KILL
+POSTHOOK: query: CREATE TRIGGER plan_1.trigger_1 WHEN BYTES_READ > 10kb DO KILL
 POSTHOOK: type: CREATE TRIGGER
 PREHOOK: query: SELECT * FROM SYS.WM_TRIGGERS
 PREHOOK: type: QUERY
@@ -3563,13 +3567,19 @@ POSTHOOK: query: SELECT * FROM SYS.WM_TRIGGERS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_triggers
 #### A masked pattern was here ####
-plan_1	trigger_1	BYTES_READ > 10k AND BYTES_READ <= 1M OR ELAPSED_TIME > 30 SECOND AND ELAPSED_TIME < 1 MINUTE	KILL
-PREHOOK: query: CREATE TRIGGER plan_1.trigger_1 WHEN BYTES_READ = 10G DO KILL
+plan_1	trigger_1	BYTES_READ > 10kb	KILL
+PREHOOK: query: CREATE TRIGGER plan_1.trigger_1 WHEN ELAPSED_TIME > 300 DO KILL
 PREHOOK: type: CREATE TRIGGER
 FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. AlreadyExistsException(message:Trigger already exists, use alter: )
-PREHOOK: query: CREATE TRIGGER plan_1.trigger_2 WHEN BYTES_READ > 100 DO MOVE TO slow_pool
+FAILED: ParseException line 4:58 mismatched input 'AND' expecting DO near '30sec' in create trigger statement
+FAILED: ParseException line 2:61 mismatched input 'OR' expecting DO near '30second' in create trigger statement
+FAILED: ParseException line 2:50 mismatched input '>=' expecting > near 'ELAPSED_TIME' in comparisionOperator
+FAILED: ParseException line 2:50 mismatched input '<' expecting > near 'ELAPSED_TIME' in comparisionOperator
+FAILED: ParseException line 2:50 mismatched input '<=' expecting > near 'ELAPSED_TIME' in comparisionOperator
+FAILED: ParseException line 2:50 mismatched input '=' expecting > near 'ELAPSED_TIME' in comparisionOperator
+PREHOOK: query: CREATE TRIGGER plan_1.trigger_2 WHEN ELAPSED_TIME > 30hour DO MOVE TO slow_pool
 PREHOOK: type: CREATE TRIGGER
-POSTHOOK: query: CREATE TRIGGER plan_1.trigger_2 WHEN BYTES_READ > 100 DO MOVE TO slow_pool
+POSTHOOK: query: CREATE TRIGGER plan_1.trigger_2 WHEN ELAPSED_TIME > 30hour DO MOVE TO slow_pool
 POSTHOOK: type: CREATE TRIGGER
 PREHOOK: query: SELECT * FROM SYS.WM_TRIGGERS
 PREHOOK: type: QUERY
@@ -3579,11 +3589,11 @@ POSTHOOK: query: SELECT * FROM SYS.WM_TRIGGERS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_triggers
 #### A masked pattern was here ####
-plan_1	trigger_1	BYTES_READ > 10k AND BYTES_READ <= 1M OR ELAPSED_TIME > 30 SECOND AND ELAPSED_TIME < 1 MINUTE	KILL
-plan_1	trigger_2	BYTES_READ > 100	MOVE TO slow_pool
-PREHOOK: query: ALTER TRIGGER plan_1.trigger_1 WHEN BYTES_READ = 1000 DO KILL
+plan_1	trigger_1	BYTES_READ > 10kb	KILL
+plan_1	trigger_2	ELAPSED_TIME > 30hour	MOVE TO slow_pool
+PREHOOK: query: ALTER TRIGGER plan_1.trigger_1 WHEN BYTES_READ > 1min DO KILL
 PREHOOK: type: ALTER TRIGGER
-POSTHOOK: query: ALTER TRIGGER plan_1.trigger_1 WHEN BYTES_READ = 1000 DO KILL
+POSTHOOK: query: ALTER TRIGGER plan_1.trigger_1 WHEN BYTES_READ > 1min DO KILL
 POSTHOOK: type: ALTER TRIGGER
 PREHOOK: query: SELECT * FROM SYS.WM_TRIGGERS
 PREHOOK: type: QUERY
@@ -3593,8 +3603,8 @@ POSTHOOK: query: SELECT * FROM SYS.WM_TRIGGERS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_triggers
 #### A masked pattern was here ####
-plan_1	trigger_1	BYTES_READ = 1000	KILL
-plan_1	trigger_2	BYTES_READ > 100	MOVE TO slow_pool
+plan_1	trigger_1	BYTES_READ > 1min	KILL
+plan_1	trigger_2	ELAPSED_TIME > 30hour	MOVE TO slow_pool
 PREHOOK: query: DROP TRIGGER plan_1.trigger_1
 PREHOOK: type: DROP TRIGGER
 POSTHOOK: query: DROP TRIGGER plan_1.trigger_1
@@ -3607,29 +3617,29 @@ POSTHOOK: query: SELECT * FROM SYS.WM_TRIGGERS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_triggers
 #### A masked pattern was here ####
-plan_1	trigger_2	BYTES_READ > 100	MOVE TO slow_pool
-PREHOOK: query: CREATE TRIGGER plan_2.trigger_1 WHEN BYTES_READ = 0m DO MOVE TO null_pool
+plan_1	trigger_2	ELAPSED_TIME > 30hour	MOVE TO slow_pool
+PREHOOK: query: CREATE TRIGGER plan_2.trigger_1 WHEN BYTES_READ > 100mb DO MOVE TO null_pool
 PREHOOK: type: CREATE TRIGGER
 FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidOperationException(message:Resource plan must be disabled to edit it.)
-PREHOOK: query: CREATE TRIGGER `table`.`table` WHEN BYTES_WRITTEN > 100K DO MOVE TO `table`
+PREHOOK: query: CREATE TRIGGER `table`.`table` WHEN BYTES_WRITTEN > 100KB DO MOVE TO `table`
 PREHOOK: type: CREATE TRIGGER
-POSTHOOK: query: CREATE TRIGGER `table`.`table` WHEN BYTES_WRITTEN > 100K DO MOVE TO `table`
+POSTHOOK: query: CREATE TRIGGER `table`.`table` WHEN BYTES_WRITTEN > 100KB DO MOVE TO `table`
 POSTHOOK: type: CREATE TRIGGER
-PREHOOK: query: CREATE TRIGGER `table`.`trigger` WHEN BYTES_WRITTEN > 100K DO MOVE TO `default`
+PREHOOK: query: CREATE TRIGGER `table`.`trigger` WHEN BYTES_WRITTEN > 100MB DO MOVE TO `default`
 PREHOOK: type: CREATE TRIGGER
-POSTHOOK: query: CREATE TRIGGER `table`.`trigger` WHEN BYTES_WRITTEN > 100K DO MOVE TO `default`
+POSTHOOK: query: CREATE TRIGGER `table`.`trigger` WHEN BYTES_WRITTEN > 100MB DO MOVE TO `default`
 POSTHOOK: type: CREATE TRIGGER
-PREHOOK: query: CREATE TRIGGER `table`.`database` WHEN BYTES_WRITTEN > 1M DO MOVE TO `default`
+PREHOOK: query: CREATE TRIGGER `table`.`database` WHEN BYTES_WRITTEN > 1GB DO MOVE TO `default`
 PREHOOK: type: CREATE TRIGGER
-POSTHOOK: query: CREATE TRIGGER `table`.`database` WHEN BYTES_WRITTEN > 1M DO MOVE TO `default`
+POSTHOOK: query: CREATE TRIGGER `table`.`database` WHEN BYTES_WRITTEN > 1GB DO MOVE TO `default`
 POSTHOOK: type: CREATE TRIGGER
 PREHOOK: query: CREATE TRIGGER `table`.`trigger1` WHEN ELAPSED_TIME > 10 DO KILL
 PREHOOK: type: CREATE TRIGGER
 POSTHOOK: query: CREATE TRIGGER `table`.`trigger1` WHEN ELAPSED_TIME > 10 DO KILL
 POSTHOOK: type: CREATE TRIGGER
-PREHOOK: query: CREATE TRIGGER `table`.`trigger2` WHEN BYTES_READ > 100 DO KILL
+PREHOOK: query: CREATE TRIGGER `table`.`trigger2` WHEN ELAPSED_TIME > 1hour DO KILL
 PREHOOK: type: CREATE TRIGGER
-POSTHOOK: query: CREATE TRIGGER `table`.`trigger2` WHEN BYTES_READ > 100 DO KILL
+POSTHOOK: query: CREATE TRIGGER `table`.`trigger2` WHEN ELAPSED_TIME > 1hour DO KILL
 POSTHOOK: type: CREATE TRIGGER
 PREHOOK: query: SELECT * FROM SYS.WM_TRIGGERS
 PREHOOK: type: QUERY
@@ -3639,12 +3649,12 @@ POSTHOOK: query: SELECT * FROM SYS.WM_TRIGGERS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_triggers
 #### A masked pattern was here ####
-plan_1	trigger_2	BYTES_READ > 100	MOVE TO slow_pool
-table	database	BYTES_WRITTEN > 1M	MOVE TO default
-table	table	BYTES_WRITTEN > 100K	MOVE TO table
-table	trigger	BYTES_WRITTEN > 100K	MOVE TO default
+plan_1	trigger_2	ELAPSED_TIME > 30hour	MOVE TO slow_pool
+table	database	BYTES_WRITTEN > 1GB	MOVE TO default
+table	table	BYTES_WRITTEN > 100KB	MOVE TO table
+table	trigger	BYTES_WRITTEN > 100MB	MOVE TO default
 table	trigger1	ELAPSED_TIME > 10	KILL
-table	trigger2	BYTES_READ > 100	KILL
+table	trigger2	ELAPSED_TIME > 1hour	KILL
 PREHOOK: query: DROP TRIGGER `table`.`database`
 PREHOOK: type: DROP TRIGGER
 POSTHOOK: query: DROP TRIGGER `table`.`database`
@@ -3657,11 +3667,11 @@ POSTHOOK: query: SELECT * FROM SYS.WM_TRIGGERS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_triggers
 #### A masked pattern was here ####
-plan_1	trigger_2	BYTES_READ > 100	MOVE TO slow_pool
-table	table	BYTES_WRITTEN > 100K	MOVE TO table
-table	trigger	BYTES_WRITTEN > 100K	MOVE TO default
+plan_1	trigger_2	ELAPSED_TIME > 30hour	MOVE TO slow_pool
+table	table	BYTES_WRITTEN > 100KB	MOVE TO table
+table	trigger	BYTES_WRITTEN > 100MB	MOVE TO default
 table	trigger1	ELAPSED_TIME > 10	KILL
-table	trigger2	BYTES_READ > 100	KILL
+table	trigger2	ELAPSED_TIME > 1hour	KILL
 PREHOOK: query: ALTER RESOURCE PLAN plan_1 ENABLE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_1 ENABLE
@@ -3675,12 +3685,12 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_1	ENABLED	NULL	default
-plan_2	ACTIVE	4	default
+plan_2	ACTIVE	10	default
 table	DISABLED	1	default
 PREHOOK: query: DROP TRIGGER plan_1.trigger_2
 PREHOOK: type: DROP TRIGGER
 FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidOperationException(message:Resource plan must be disabled to edit it.)
-PREHOOK: query: ALTER TRIGGER plan_1.trigger_2 WHEN BYTES_READ = 1000g DO KILL
+PREHOOK: query: ALTER TRIGGER plan_1.trigger_2 WHEN BYTES_READ > 1000gb DO KILL
 PREHOOK: type: ALTER TRIGGER
 FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidOperationException(message:Resource plan must be disabled to edit it.)
 PREHOOK: query: ALTER RESOURCE PLAN plan_1 ACTIVATE
@@ -3696,21 +3706,21 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_1	ACTIVE	NULL	default
-plan_2	ENABLED	4	default
+plan_2	ENABLED	10	default
 table	DISABLED	1	default
 PREHOOK: query: DROP TRIGGER plan_1.trigger_2
 PREHOOK: type: DROP TRIGGER
 FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidOperationException(message:Resource plan must be disabled to edit it.)
-PREHOOK: query: ALTER TRIGGER plan_1.trigger_2 WHEN BYTES_READ = 1000K DO KILL
+PREHOOK: query: ALTER TRIGGER plan_1.trigger_2 WHEN BYTES_READ > 1000KB DO KILL
 PREHOOK: type: ALTER TRIGGER
 FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidOperationException(message:Resource plan must be disabled to edit it.)
 PREHOOK: query: ALTER RESOURCE PLAN plan_2 DISABLE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_2 DISABLE
 POSTHOOK: type: ALTER RESOURCEPLAN
-PREHOOK: query: CREATE TRIGGER plan_2.trigger_1 WHEN BYTES_READ = 0 DO MOVE TO null_pool
+PREHOOK: query: CREATE TRIGGER plan_2.trigger_1 WHEN BYTES_READ > 0 DO MOVE TO null_pool
 PREHOOK: type: CREATE TRIGGER
-POSTHOOK: query: CREATE TRIGGER plan_2.trigger_1 WHEN BYTES_READ = 0 DO MOVE TO null_pool
+POSTHOOK: query: CREATE TRIGGER plan_2.trigger_1 WHEN BYTES_READ > 0 DO MOVE TO null_pool
 POSTHOOK: type: CREATE TRIGGER
 PREHOOK: query: SELECT * FROM SYS.WM_TRIGGERS
 PREHOOK: type: QUERY
@@ -3720,12 +3730,12 @@ POSTHOOK: query: SELECT * FROM SYS.WM_TRIGGERS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_triggers
 #### A masked pattern was here ####
-plan_1	trigger_2	BYTES_READ > 100	MOVE TO slow_pool
-plan_2	trigger_1	BYTES_READ = 0	MOVE TO null_pool
-table	table	BYTES_WRITTEN > 100K	MOVE TO table
-table	trigger	BYTES_WRITTEN > 100K	MOVE TO default
+plan_1	trigger_2	ELAPSED_TIME > 30hour	MOVE TO slow_pool
+plan_2	trigger_1	BYTES_READ > 0	MOVE TO null_pool
+table	table	BYTES_WRITTEN > 100KB	MOVE TO table
+table	trigger	BYTES_WRITTEN > 100MB	MOVE TO default
 table	trigger1	ELAPSED_TIME > 10	KILL
-table	trigger2	BYTES_READ > 100	KILL
+table	trigger2	ELAPSED_TIME > 1hour	KILL
 PREHOOK: query: CREATE POOL plan_1.default WITH
    ALLOC_FRACTION=1.0, QUERY_PARALLELISM=5, SCHEDULING_POLICY='default'
 PREHOOK: type: CREATE POOL
@@ -3743,7 +3753,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_pools
 #### A masked pattern was here ####
 plan_1	default	1.0	4	NULL
-plan_2	default	1.0	4	NULL
+plan_2	default	1.0	5	NULL
 table	default	1.0	4	NULL
 FAILED: SemanticException Invalid scheduling policy invalid
 PREHOOK: query: CREATE POOL plan_2.default.c1 WITH
@@ -3753,23 +3763,22 @@ POSTHOOK: query: CREATE POOL plan_2.default.c1 WITH
     ALLOC_FRACTION=0.3, QUERY_PARALLELISM=3, SCHEDULING_POLICY='fair'
 POSTHOOK: type: CREATE POOL
 PREHOOK: query: CREATE POOL plan_2.default.c2 WITH
-    QUERY_PARALLELISM=2, SCHEDULING_POLICY='fair', ALLOC_FRACTION=0.2
+    QUERY_PARALLELISM=2, SCHEDULING_POLICY='fair', ALLOC_FRACTION=0.7
 PREHOOK: type: CREATE POOL
 POSTHOOK: query: CREATE POOL plan_2.default.c2 WITH
-    QUERY_PARALLELISM=2, SCHEDULING_POLICY='fair', ALLOC_FRACTION=0.2
+    QUERY_PARALLELISM=2, SCHEDULING_POLICY='fair', ALLOC_FRACTION=0.7
 POSTHOOK: type: CREATE POOL
 PREHOOK: query: ALTER RESOURCE PLAN plan_2 VALIDATE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_2 VALIDATE
 POSTHOOK: type: ALTER RESOURCEPLAN
-Sum of children pools' alloc fraction should be equal 1.0 got: 0.5 for pool: default
-Sum of children pools' query parallelism: 5 is not equal to pool parallelism: 4 for pool: default
+Sum of children pools' alloc fraction should be less than 1 got: 1.0 for pool: default
 PREHOOK: query: ALTER RESOURCE PLAN plan_2 ENABLE ACTIVATE
 PREHOOK: type: ALTER RESOURCEPLAN
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidOperationException(message:ResourcePlan: plan_2 is invalid: [Sum of children pools' alloc fraction should be equal 1.0 got: 0.5 for pool: default, Sum of children pools' query parallelism: 5 is not equal to pool parallelism: 4 for pool: default])
-PREHOOK: query: ALTER POOL plan_2.default.c2 SET ALLOC_FRACTION = 0.7, QUERY_PARALLELISM = 1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidOperationException(message:ResourcePlan: plan_2 is invalid: [Sum of children pools' alloc fraction should be less than 1 got: 1.0 for pool: default])
+PREHOOK: query: ALTER POOL plan_2.default.c2 SET ALLOC_FRACTION = 0.5, QUERY_PARALLELISM = 1
 PREHOOK: type: ALTER POOL
-POSTHOOK: query: ALTER POOL plan_2.default.c2 SET ALLOC_FRACTION = 0.7, QUERY_PARALLELISM = 1
+POSTHOOK: query: ALTER POOL plan_2.default.c2 SET ALLOC_FRACTION = 0.5, QUERY_PARALLELISM = 1
 POSTHOOK: type: ALTER POOL
 PREHOOK: query: ALTER POOL plan_2.default.c2 SET SCHEDULING_POLICY='fair'
 PREHOOK: type: ALTER POOL
@@ -3784,9 +3793,9 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_pools
 #### A masked pattern was here ####
 plan_1	default	1.0	4	NULL
-plan_2	default	1.0	4	NULL
+plan_2	default	1.0	5	NULL
 plan_2	default.c1	0.3	3	fair
-plan_2	default.c2	0.7	1	fair
+plan_2	default.c2	0.5	1	fair
 table	default	1.0	4	NULL
 PREHOOK: query: ALTER POOL plan_2.default.c2 UNSET SCHEDULING_POLICY
 PREHOOK: type: ALTER POOL
@@ -3801,14 +3810,15 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_pools
 #### A masked pattern was here ####
 plan_1	default	1.0	4	NULL
-plan_2	default	1.0	4	NULL
+plan_2	default	1.0	5	NULL
 plan_2	default.c1	0.3	3	fair
-plan_2	default.c2	0.7	1	NULL
+plan_2	default.c2	0.5	1	NULL
 table	default	1.0	4	NULL
 PREHOOK: query: ALTER RESOURCE PLAN plan_2 VALIDATE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_2 VALIDATE
 POSTHOOK: type: ALTER RESOURCEPLAN
+warn: Sum of all pools' query parallelism: 9 is less than resource plan query parallelism: 10
 PREHOOK: query: ALTER RESOURCE PLAN plan_2 ENABLE ACTIVATE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_2 ENABLE ACTIVATE
@@ -3834,9 +3844,9 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_pools
 #### A masked pattern was here ####
 plan_1	default	1.0	4	NULL
-plan_2	def	1.0	4	NULL
+plan_2	def	1.0	5	NULL
 plan_2	def.c1	0.3	3	fair
-plan_2	def.c2	0.7	1	NULL
+plan_2	def.c2	0.5	1	NULL
 table	default	1.0	4	NULL
 PREHOOK: query: DROP POOL plan_2.default
 PREHOOK: type: DROP POOL
@@ -3850,9 +3860,9 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_pools
 #### A masked pattern was here ####
 plan_1	default	1.0	4	NULL
-plan_2	def	1.0	4	NULL
+plan_2	def	1.0	5	NULL
 plan_2	def.c1	0.3	3	fair
-plan_2	def.c2	0.7	1	NULL
+plan_2	def.c2	0.5	1	NULL
 table	default	1.0	4	NULL
 PREHOOK: query: CREATE POOL plan_2.child1.child2 WITH
     QUERY_PARALLELISM=2, SCHEDULING_POLICY='fifo', ALLOC_FRACTION=0.8
@@ -3895,9 +3905,9 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_pools
 #### A masked pattern was here ####
 plan_1	default	1.0	4	NULL
-plan_2	def	1.0	4	NULL
+plan_2	def	1.0	5	NULL
 plan_2	def.c1	0.3	3	fair
-plan_2	def.c2	0.7	1	NULL
+plan_2	def.c2	0.5	1	NULL
 table	default	1.0	4	NULL
 table	table	0.0	1	fifo
 table	table.pool1	0.9	3	fair
@@ -3916,9 +3926,9 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_pools
 #### A masked pattern was here ####
 plan_1	default	1.0	4	NULL
-plan_2	def	1.0	4	NULL
+plan_2	def	1.0	5	NULL
 plan_2	def.c1	0.3	3	fair
-plan_2	def.c2	0.7	1	NULL
+plan_2	def.c2	0.5	1	NULL
 table	default	1.0	4	NULL
 table	table	0.0	1	fifo
 table	table.pool	0.9	3	fair
@@ -3936,9 +3946,9 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_pools
 #### A masked pattern was here ####
 plan_1	default	1.0	4	NULL
-plan_2	def	1.0	4	NULL
+plan_2	def	1.0	5	NULL
 plan_2	def.c1	0.3	3	fair
-plan_2	def.c2	0.7	1	NULL
+plan_2	def.c2	0.5	1	NULL
 table	default	1.0	4	NULL
 table	table	0.0	1	fifo
 table	table.pool	0.9	3	fair
@@ -3956,9 +3966,9 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_pools
 #### A masked pattern was here ####
 plan_1	default	1.0	4	NULL
-plan_2	def	1.0	4	NULL
+plan_2	def	1.0	5	NULL
 plan_2	def.c1	0.3	3	fair
-plan_2	def.c2	0.7	1	NULL
+plan_2	def.c2	0.5	1	NULL
 table	default	1.0	4	NULL
 table	table	0.0	1	fifo
 table	table.pool	0.9	3	fair
@@ -3973,7 +3983,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_1	ACTIVE	NULL	default
-plan_2	DISABLED	4	def
+plan_2	DISABLED	10	def
 table	DISABLED	1	default
 PREHOOK: query: ALTER RESOURCE PLAN `table` SET DEFAULT POOL = `table`.pool
 PREHOOK: type: ALTER RESOURCEPLAN
@@ -3992,9 +4002,9 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_pools
 #### A masked pattern was here ####
 plan_1	default	1.0	4	NULL
-plan_2	def	1.0	4	NULL
+plan_2	def	1.0	5	NULL
 plan_2	def.c1	0.3	3	fair
-plan_2	def.c2	0.7	1	NULL
+plan_2	def.c2	0.5	1	NULL
 table	table	0.0	1	fifo
 table	table.pool	0.9	3	fair
 table	table.pool.child1	0.3	1	fair
@@ -4012,7 +4022,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_1	ACTIVE	NULL	default
-plan_2	DISABLED	4	def
+plan_2	DISABLED	10	def
 table	DISABLED	1	NULL
 PREHOOK: query: ALTER POOL plan_2.def.c1 ADD TRIGGER trigger_1
 PREHOOK: type: ALTER POOL
@@ -4068,18 +4078,18 @@ POSTHOOK: query: SHOW RESOURCE PLAN `table`
 POSTHOOK: type: SHOW RESOURCEPLAN
 table[status=DISABLED,parallelism=1,defaultPool=null]
  +  table[allocFraction=0.0,schedulingPolicy=fifo,parallelism=1]
-     |  trigger table: if (BYTES_WRITTEN > 100K) { MOVE TO table }
+     |  trigger table: if (BYTES_WRITTEN > 100KB) { MOVE TO table }
      +  pool[allocFraction=0.9,schedulingPolicy=fair,parallelism=3]
          +  child2[allocFraction=0.7,schedulingPolicy=fair,parallelism=3]
-             |  trigger trigger2: if (BYTES_READ > 100) { KILL }
              |  trigger trigger1: if (ELAPSED_TIME > 10) { KILL }
+             |  trigger trigger2: if (ELAPSED_TIME > 1hour) { KILL }
          +  child1[allocFraction=0.3,schedulingPolicy=fair,parallelism=1]
+             |  trigger table: if (BYTES_WRITTEN > 100KB) { MOVE TO table }
              |  trigger trigger1: if (ELAPSED_TIME > 10) { KILL }
-             |  trigger table: if (BYTES_WRITTEN > 100K) { MOVE TO table }
  +  <unmanaged queries>
      |  trigger trigger1: if (ELAPSED_TIME > 10) { KILL }
  +  <unused triggers>
-     |  trigger trigger: if (BYTES_WRITTEN > 100K) { MOVE TO default }
+     |  trigger trigger: if (BYTES_WRITTEN > 100MB) { MOVE TO default }
 PREHOOK: query: ALTER TRIGGER `table`.`trigger1` DROP FROM POOL `table`.pool.child2
 PREHOOK: type: ALTER POOL
 POSTHOOK: query: ALTER TRIGGER `table`.`trigger1` DROP FROM POOL `table`.pool.child2
@@ -4179,12 +4189,12 @@ PREHOOK: query: SHOW RESOURCE PLAN plan_2
 PREHOOK: type: SHOW RESOURCEPLAN
 POSTHOOK: query: SHOW RESOURCE PLAN plan_2
 POSTHOOK: type: SHOW RESOURCEPLAN
-plan_2[status=DISABLED,parallelism=4,defaultPool=def]
- +  def[allocFraction=1.0,schedulingPolicy=null,parallelism=4]
+plan_2[status=DISABLED,parallelism=10,defaultPool=def]
+ +  def[allocFraction=1.0,schedulingPolicy=null,parallelism=5]
      |  mapped for users: user2
      |  mapped for default
-     +  c2[allocFraction=0.7,schedulingPolicy=null,parallelism=1]
-         |  trigger trigger_1: if (BYTES_READ = 0) { MOVE TO null_pool }
+     +  c2[allocFraction=0.5,schedulingPolicy=null,parallelism=1]
+         |  trigger trigger_1: if (BYTES_READ > 0) { MOVE TO null_pool }
          |  mapped for groups: group2
      +  c1[allocFraction=0.3,schedulingPolicy=fair,parallelism=3]
          |  mapped for groups: group1
@@ -4247,11 +4257,11 @@ PREHOOK: query: SHOW RESOURCE PLAN plan_2
 PREHOOK: type: SHOW RESOURCEPLAN
 POSTHOOK: query: SHOW RESOURCE PLAN plan_2
 POSTHOOK: type: SHOW RESOURCEPLAN
-plan_2[status=DISABLED,parallelism=4,defaultPool=def]
- +  def[allocFraction=1.0,schedulingPolicy=null,parallelism=4]
+plan_2[status=DISABLED,parallelism=10,defaultPool=def]
+ +  def[allocFraction=1.0,schedulingPolicy=null,parallelism=5]
      |  mapped for default
-     +  c2[allocFraction=0.7,schedulingPolicy=null,parallelism=1]
-         |  trigger trigger_1: if (BYTES_READ = 0) { MOVE TO null_pool }
+     +  c2[allocFraction=0.5,schedulingPolicy=null,parallelism=1]
+         |  trigger trigger_1: if (BYTES_READ > 0) { MOVE TO null_pool }
      +  c1[allocFraction=0.3,schedulingPolicy=fair,parallelism=3]
          |  mapped for groups: group1
  +  <unmanaged queries>
@@ -4297,11 +4307,11 @@ POSTHOOK: query: SELECT * FROM SYS.WM_TRIGGERS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_triggers
 #### A masked pattern was here ####
-plan_1	trigger_2	BYTES_READ > 100	MOVE TO slow_pool
-table	table	BYTES_WRITTEN > 100K	MOVE TO table
-table	trigger	BYTES_WRITTEN > 100K	MOVE TO default
+plan_1	trigger_2	ELAPSED_TIME > 30hour	MOVE TO slow_pool
+table	table	BYTES_WRITTEN > 100KB	MOVE TO table
+table	trigger	BYTES_WRITTEN > 100MB	MOVE TO default
 table	trigger1	ELAPSED_TIME > 10	KILL
-table	trigger2	BYTES_READ > 100	KILL
+table	trigger2	ELAPSED_TIME > 1hour	KILL
 PREHOOK: query: SELECT * FROM SYS.WM_POOLS_TO_TRIGGERS
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@wm_pools_to_triggers
@@ -4331,13 +4341,13 @@ PREHOOK: query: CREATE USER MAPPING "user1" IN plan_4a TO pool1
 PREHOOK: type: CREATE MAPPING
 POSTHOOK: query: CREATE USER MAPPING "user1" IN plan_4a TO pool1
 POSTHOOK: type: CREATE MAPPING
-PREHOOK: query: CREATE TRIGGER plan_4a.trigger_1 WHEN BYTES_READ = 10G DO KILL
+PREHOOK: query: CREATE TRIGGER plan_4a.trigger_1 WHEN BYTES_READ > 10GB DO KILL
 PREHOOK: type: CREATE TRIGGER
-POSTHOOK: query: CREATE TRIGGER plan_4a.trigger_1 WHEN BYTES_READ = 10G DO KILL
+POSTHOOK: query: CREATE TRIGGER plan_4a.trigger_1 WHEN BYTES_READ > 10GB DO KILL
 POSTHOOK: type: CREATE TRIGGER
-PREHOOK: query: CREATE TRIGGER plan_4a.trigger_2 WHEN BYTES_READ = 11G DO KILL
+PREHOOK: query: CREATE TRIGGER plan_4a.trigger_2 WHEN BYTES_READ > 11GB DO KILL
 PREHOOK: type: CREATE TRIGGER
-POSTHOOK: query: CREATE TRIGGER plan_4a.trigger_2 WHEN BYTES_READ = 11G DO KILL
+POSTHOOK: query: CREATE TRIGGER plan_4a.trigger_2 WHEN BYTES_READ > 11GB DO KILL
 POSTHOOK: type: CREATE TRIGGER
 PREHOOK: query: ALTER POOL plan_4a.pool1 ADD TRIGGER trigger_2
 PREHOOK: type: ALTER POOL
@@ -4391,15 +4401,15 @@ POSTHOOK: query: SELECT * FROM SYS.WM_TRIGGERS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_triggers
 #### A masked pattern was here ####
-plan_1	trigger_2	BYTES_READ > 100	MOVE TO slow_pool
-plan_4a	trigger_1	BYTES_READ = 10G	KILL
-plan_4a	trigger_2	BYTES_READ = 11G	KILL
-plan_4b	trigger_1	BYTES_READ = 10G	KILL
-plan_4b	trigger_2	BYTES_READ = 11G	KILL
-table	table	BYTES_WRITTEN > 100K	MOVE TO table
-table	trigger	BYTES_WRITTEN > 100K	MOVE TO default
+plan_1	trigger_2	ELAPSED_TIME > 30hour	MOVE TO slow_pool
+plan_4a	trigger_1	BYTES_READ > 10GB	KILL
+plan_4a	trigger_2	BYTES_READ > 11GB	KILL
+plan_4b	trigger_1	BYTES_READ > 10GB	KILL
+plan_4b	trigger_2	BYTES_READ > 11GB	KILL
+table	table	BYTES_WRITTEN > 100KB	MOVE TO table
+table	trigger	BYTES_WRITTEN > 100MB	MOVE TO default
 table	trigger1	ELAPSED_TIME > 10	KILL
-table	trigger2	BYTES_READ > 100	KILL
+table	trigger2	ELAPSED_TIME > 1hour	KILL
 PREHOOK: query: SELECT * FROM SYS.WM_POOLS_TO_TRIGGERS
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@wm_pools_to_triggers


[4/8] hive git commit: HIVE-18231 : validate resource plan - part 2 - validate action and trigger expressions (Harish Jaiprakash, reviewed by Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
index 9b09603..4d4894a 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
+++ b/standalone-metastore/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 _list812 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list812.size);
-                WMTrigger _elem813;
-                for (int _i814 = 0; _i814 < _list812.size; ++_i814)
+                org.apache.thrift.protocol.TList _list820 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list820.size);
+                WMTrigger _elem821;
+                for (int _i822 = 0; _i822 < _list820.size; ++_i822)
                 {
-                  _elem813 = new WMTrigger();
-                  _elem813.read(iprot);
-                  struct.triggers.add(_elem813);
+                  _elem821 = new WMTrigger();
+                  _elem821.read(iprot);
+                  struct.triggers.add(_elem821);
                 }
                 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 _iter815 : struct.triggers)
+            for (WMTrigger _iter823 : struct.triggers)
             {
-              _iter815.write(oprot);
+              _iter823.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter816 : struct.triggers)
+          for (WMTrigger _iter824 : struct.triggers)
           {
-            _iter816.write(oprot);
+            _iter824.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list817 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list817.size);
-          WMTrigger _elem818;
-          for (int _i819 = 0; _i819 < _list817.size; ++_i819)
+          org.apache.thrift.protocol.TList _list825 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list825.size);
+          WMTrigger _elem826;
+          for (int _i827 = 0; _i827 < _list825.size; ++_i827)
           {
-            _elem818 = new WMTrigger();
-            _elem818.read(iprot);
-            struct.triggers.add(_elem818);
+            _elem826 = new WMTrigger();
+            _elem826.read(iprot);
+            struct.triggers.add(_elem826);
           }
         }
         struct.setTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
index a8781b8..ea8f3aa 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
@@ -39,6 +39,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WMValidateResourcePlanResponse");
 
   private static final org.apache.thrift.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift.protocol.TField("errors", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField WARNINGS_FIELD_DESC = new org.apache.thrift.protocol.TField("warnings", org.apache.thrift.protocol.TType.LIST, (short)2);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -47,10 +48,12 @@ import org.slf4j.LoggerFactory;
   }
 
   private List<String> errors; // optional
+  private List<String> warnings; // 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 {
-    ERRORS((short)1, "errors");
+    ERRORS((short)1, "errors"),
+    WARNINGS((short)2, "warnings");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -67,6 +70,8 @@ import org.slf4j.LoggerFactory;
       switch(fieldId) {
         case 1: // ERRORS
           return ERRORS;
+        case 2: // WARNINGS
+          return WARNINGS;
         default:
           return null;
       }
@@ -107,13 +112,16 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
-  private static final _Fields optionals[] = {_Fields.ERRORS};
+  private static final _Fields optionals[] = {_Fields.ERRORS,_Fields.WARNINGS};
   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.ERRORS, new org.apache.thrift.meta_data.FieldMetaData("errors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.WARNINGS, new org.apache.thrift.meta_data.FieldMetaData("warnings", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        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(WMValidateResourcePlanResponse.class, metaDataMap);
   }
@@ -129,6 +137,10 @@ import org.slf4j.LoggerFactory;
       List<String> __this__errors = new ArrayList<String>(other.errors);
       this.errors = __this__errors;
     }
+    if (other.isSetWarnings()) {
+      List<String> __this__warnings = new ArrayList<String>(other.warnings);
+      this.warnings = __this__warnings;
+    }
   }
 
   public WMValidateResourcePlanResponse deepCopy() {
@@ -138,6 +150,7 @@ import org.slf4j.LoggerFactory;
   @Override
   public void clear() {
     this.errors = null;
+    this.warnings = null;
   }
 
   public int getErrorsSize() {
@@ -178,6 +191,44 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public int getWarningsSize() {
+    return (this.warnings == null) ? 0 : this.warnings.size();
+  }
+
+  public java.util.Iterator<String> getWarningsIterator() {
+    return (this.warnings == null) ? null : this.warnings.iterator();
+  }
+
+  public void addToWarnings(String elem) {
+    if (this.warnings == null) {
+      this.warnings = new ArrayList<String>();
+    }
+    this.warnings.add(elem);
+  }
+
+  public List<String> getWarnings() {
+    return this.warnings;
+  }
+
+  public void setWarnings(List<String> warnings) {
+    this.warnings = warnings;
+  }
+
+  public void unsetWarnings() {
+    this.warnings = null;
+  }
+
+  /** Returns true if field warnings is set (has been assigned a value) and false otherwise */
+  public boolean isSetWarnings() {
+    return this.warnings != null;
+  }
+
+  public void setWarningsIsSet(boolean value) {
+    if (!value) {
+      this.warnings = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case ERRORS:
@@ -188,6 +239,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case WARNINGS:
+      if (value == null) {
+        unsetWarnings();
+      } else {
+        setWarnings((List<String>)value);
+      }
+      break;
+
     }
   }
 
@@ -196,6 +255,9 @@ import org.slf4j.LoggerFactory;
     case ERRORS:
       return getErrors();
 
+    case WARNINGS:
+      return getWarnings();
+
     }
     throw new IllegalStateException();
   }
@@ -209,6 +271,8 @@ import org.slf4j.LoggerFactory;
     switch (field) {
     case ERRORS:
       return isSetErrors();
+    case WARNINGS:
+      return isSetWarnings();
     }
     throw new IllegalStateException();
   }
@@ -235,6 +299,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_warnings = true && this.isSetWarnings();
+    boolean that_present_warnings = true && that.isSetWarnings();
+    if (this_present_warnings || that_present_warnings) {
+      if (!(this_present_warnings && that_present_warnings))
+        return false;
+      if (!this.warnings.equals(that.warnings))
+        return false;
+    }
+
     return true;
   }
 
@@ -247,6 +320,11 @@ import org.slf4j.LoggerFactory;
     if (present_errors)
       list.add(errors);
 
+    boolean present_warnings = true && (isSetWarnings());
+    list.add(present_warnings);
+    if (present_warnings)
+      list.add(warnings);
+
     return list.hashCode();
   }
 
@@ -268,6 +346,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetWarnings()).compareTo(other.isSetWarnings());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetWarnings()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.warnings, other.warnings);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -297,6 +385,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetWarnings()) {
+      if (!first) sb.append(", ");
+      sb.append("warnings:");
+      if (this.warnings == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.warnings);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -358,6 +456,24 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 2: // WARNINGS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list807 = iprot.readListBegin();
+                struct.warnings = new ArrayList<String>(_list807.size);
+                String _elem808;
+                for (int _i809 = 0; _i809 < _list807.size; ++_i809)
+                {
+                  _elem808 = iprot.readString();
+                  struct.warnings.add(_elem808);
+                }
+                iprot.readListEnd();
+              }
+              struct.setWarningsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -376,9 +492,23 @@ 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 _iter807 : struct.errors)
+            for (String _iter810 : struct.errors)
             {
-              oprot.writeString(_iter807);
+              oprot.writeString(_iter810);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.warnings != null) {
+        if (struct.isSetWarnings()) {
+          oprot.writeFieldBegin(WARNINGS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.warnings.size()));
+            for (String _iter811 : struct.warnings)
+            {
+              oprot.writeString(_iter811);
             }
             oprot.writeListEnd();
           }
@@ -406,13 +536,25 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetErrors()) {
         optionals.set(0);
       }
-      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetWarnings()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
       if (struct.isSetErrors()) {
         {
           oprot.writeI32(struct.errors.size());
-          for (String _iter808 : struct.errors)
+          for (String _iter812 : struct.errors)
+          {
+            oprot.writeString(_iter812);
+          }
+        }
+      }
+      if (struct.isSetWarnings()) {
+        {
+          oprot.writeI32(struct.warnings.size());
+          for (String _iter813 : struct.warnings)
           {
-            oprot.writeString(_iter808);
+            oprot.writeString(_iter813);
           }
         }
       }
@@ -421,20 +563,33 @@ import org.slf4j.LoggerFactory;
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, WMValidateResourcePlanResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(1);
+      BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list809 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.errors = new ArrayList<String>(_list809.size);
-          String _elem810;
-          for (int _i811 = 0; _i811 < _list809.size; ++_i811)
+          org.apache.thrift.protocol.TList _list814 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.errors = new ArrayList<String>(_list814.size);
+          String _elem815;
+          for (int _i816 = 0; _i816 < _list814.size; ++_i816)
           {
-            _elem810 = iprot.readString();
-            struct.errors.add(_elem810);
+            _elem815 = iprot.readString();
+            struct.errors.add(_elem815);
           }
         }
         struct.setErrorsIsSet(true);
       }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TList _list817 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.warnings = new ArrayList<String>(_list817.size);
+          String _elem818;
+          for (int _i819 = 0; _i819 < _list817.size; ++_i819)
+          {
+            _elem818 = iprot.readString();
+            struct.warnings.add(_elem818);
+          }
+        }
+        struct.setWarningsIsSet(true);
+      }
     }
   }
 


[7/8] hive git commit: HIVE-18231 : validate resource plan - part 2 - validate action and trigger expressions (Harish Jaiprakash, reviewed by Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 49e99d3..8b78230 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -1240,14 +1240,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1050;
-            ::apache::thrift::protocol::TType _etype1053;
-            xfer += iprot->readListBegin(_etype1053, _size1050);
-            this->success.resize(_size1050);
-            uint32_t _i1054;
-            for (_i1054 = 0; _i1054 < _size1050; ++_i1054)
+            uint32_t _size1056;
+            ::apache::thrift::protocol::TType _etype1059;
+            xfer += iprot->readListBegin(_etype1059, _size1056);
+            this->success.resize(_size1056);
+            uint32_t _i1060;
+            for (_i1060 = 0; _i1060 < _size1056; ++_i1060)
             {
-              xfer += iprot->readString(this->success[_i1054]);
+              xfer += iprot->readString(this->success[_i1060]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1286,10 +1286,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1055;
-      for (_iter1055 = this->success.begin(); _iter1055 != this->success.end(); ++_iter1055)
+      std::vector<std::string> ::const_iterator _iter1061;
+      for (_iter1061 = this->success.begin(); _iter1061 != this->success.end(); ++_iter1061)
       {
-        xfer += oprot->writeString((*_iter1055));
+        xfer += oprot->writeString((*_iter1061));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1334,14 +1334,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1056;
-            ::apache::thrift::protocol::TType _etype1059;
-            xfer += iprot->readListBegin(_etype1059, _size1056);
-            (*(this->success)).resize(_size1056);
-            uint32_t _i1060;
-            for (_i1060 = 0; _i1060 < _size1056; ++_i1060)
+            uint32_t _size1062;
+            ::apache::thrift::protocol::TType _etype1065;
+            xfer += iprot->readListBegin(_etype1065, _size1062);
+            (*(this->success)).resize(_size1062);
+            uint32_t _i1066;
+            for (_i1066 = 0; _i1066 < _size1062; ++_i1066)
             {
-              xfer += iprot->readString((*(this->success))[_i1060]);
+              xfer += iprot->readString((*(this->success))[_i1066]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1458,14 +1458,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1061;
-            ::apache::thrift::protocol::TType _etype1064;
-            xfer += iprot->readListBegin(_etype1064, _size1061);
-            this->success.resize(_size1061);
-            uint32_t _i1065;
-            for (_i1065 = 0; _i1065 < _size1061; ++_i1065)
+            uint32_t _size1067;
+            ::apache::thrift::protocol::TType _etype1070;
+            xfer += iprot->readListBegin(_etype1070, _size1067);
+            this->success.resize(_size1067);
+            uint32_t _i1071;
+            for (_i1071 = 0; _i1071 < _size1067; ++_i1071)
             {
-              xfer += iprot->readString(this->success[_i1065]);
+              xfer += iprot->readString(this->success[_i1071]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1504,10 +1504,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1066;
-      for (_iter1066 = this->success.begin(); _iter1066 != this->success.end(); ++_iter1066)
+      std::vector<std::string> ::const_iterator _iter1072;
+      for (_iter1072 = this->success.begin(); _iter1072 != this->success.end(); ++_iter1072)
       {
-        xfer += oprot->writeString((*_iter1066));
+        xfer += oprot->writeString((*_iter1072));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1552,14 +1552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1067;
-            ::apache::thrift::protocol::TType _etype1070;
-            xfer += iprot->readListBegin(_etype1070, _size1067);
-            (*(this->success)).resize(_size1067);
-            uint32_t _i1071;
-            for (_i1071 = 0; _i1071 < _size1067; ++_i1071)
+            uint32_t _size1073;
+            ::apache::thrift::protocol::TType _etype1076;
+            xfer += iprot->readListBegin(_etype1076, _size1073);
+            (*(this->success)).resize(_size1073);
+            uint32_t _i1077;
+            for (_i1077 = 0; _i1077 < _size1073; ++_i1077)
             {
-              xfer += iprot->readString((*(this->success))[_i1071]);
+              xfer += iprot->readString((*(this->success))[_i1077]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2621,17 +2621,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1072;
-            ::apache::thrift::protocol::TType _ktype1073;
-            ::apache::thrift::protocol::TType _vtype1074;
-            xfer += iprot->readMapBegin(_ktype1073, _vtype1074, _size1072);
-            uint32_t _i1076;
-            for (_i1076 = 0; _i1076 < _size1072; ++_i1076)
+            uint32_t _size1078;
+            ::apache::thrift::protocol::TType _ktype1079;
+            ::apache::thrift::protocol::TType _vtype1080;
+            xfer += iprot->readMapBegin(_ktype1079, _vtype1080, _size1078);
+            uint32_t _i1082;
+            for (_i1082 = 0; _i1082 < _size1078; ++_i1082)
             {
-              std::string _key1077;
-              xfer += iprot->readString(_key1077);
-              Type& _val1078 = this->success[_key1077];
-              xfer += _val1078.read(iprot);
+              std::string _key1083;
+              xfer += iprot->readString(_key1083);
+              Type& _val1084 = this->success[_key1083];
+              xfer += _val1084.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2670,11 +2670,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter1079;
-      for (_iter1079 = this->success.begin(); _iter1079 != this->success.end(); ++_iter1079)
+      std::map<std::string, Type> ::const_iterator _iter1085;
+      for (_iter1085 = this->success.begin(); _iter1085 != this->success.end(); ++_iter1085)
       {
-        xfer += oprot->writeString(_iter1079->first);
-        xfer += _iter1079->second.write(oprot);
+        xfer += oprot->writeString(_iter1085->first);
+        xfer += _iter1085->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -2719,17 +2719,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1080;
-            ::apache::thrift::protocol::TType _ktype1081;
-            ::apache::thrift::protocol::TType _vtype1082;
-            xfer += iprot->readMapBegin(_ktype1081, _vtype1082, _size1080);
-            uint32_t _i1084;
-            for (_i1084 = 0; _i1084 < _size1080; ++_i1084)
+            uint32_t _size1086;
+            ::apache::thrift::protocol::TType _ktype1087;
+            ::apache::thrift::protocol::TType _vtype1088;
+            xfer += iprot->readMapBegin(_ktype1087, _vtype1088, _size1086);
+            uint32_t _i1090;
+            for (_i1090 = 0; _i1090 < _size1086; ++_i1090)
             {
-              std::string _key1085;
-              xfer += iprot->readString(_key1085);
-              Type& _val1086 = (*(this->success))[_key1085];
-              xfer += _val1086.read(iprot);
+              std::string _key1091;
+              xfer += iprot->readString(_key1091);
+              Type& _val1092 = (*(this->success))[_key1091];
+              xfer += _val1092.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2883,14 +2883,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1087;
-            ::apache::thrift::protocol::TType _etype1090;
-            xfer += iprot->readListBegin(_etype1090, _size1087);
-            this->success.resize(_size1087);
-            uint32_t _i1091;
-            for (_i1091 = 0; _i1091 < _size1087; ++_i1091)
+            uint32_t _size1093;
+            ::apache::thrift::protocol::TType _etype1096;
+            xfer += iprot->readListBegin(_etype1096, _size1093);
+            this->success.resize(_size1093);
+            uint32_t _i1097;
+            for (_i1097 = 0; _i1097 < _size1093; ++_i1097)
             {
-              xfer += this->success[_i1091].read(iprot);
+              xfer += this->success[_i1097].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2945,10 +2945,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1092;
-      for (_iter1092 = this->success.begin(); _iter1092 != this->success.end(); ++_iter1092)
+      std::vector<FieldSchema> ::const_iterator _iter1098;
+      for (_iter1098 = this->success.begin(); _iter1098 != this->success.end(); ++_iter1098)
       {
-        xfer += (*_iter1092).write(oprot);
+        xfer += (*_iter1098).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3001,14 +3001,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1093;
-            ::apache::thrift::protocol::TType _etype1096;
-            xfer += iprot->readListBegin(_etype1096, _size1093);
-            (*(this->success)).resize(_size1093);
-            uint32_t _i1097;
-            for (_i1097 = 0; _i1097 < _size1093; ++_i1097)
+            uint32_t _size1099;
+            ::apache::thrift::protocol::TType _etype1102;
+            xfer += iprot->readListBegin(_etype1102, _size1099);
+            (*(this->success)).resize(_size1099);
+            uint32_t _i1103;
+            for (_i1103 = 0; _i1103 < _size1099; ++_i1103)
             {
-              xfer += (*(this->success))[_i1097].read(iprot);
+              xfer += (*(this->success))[_i1103].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3194,14 +3194,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1098;
-            ::apache::thrift::protocol::TType _etype1101;
-            xfer += iprot->readListBegin(_etype1101, _size1098);
-            this->success.resize(_size1098);
-            uint32_t _i1102;
-            for (_i1102 = 0; _i1102 < _size1098; ++_i1102)
+            uint32_t _size1104;
+            ::apache::thrift::protocol::TType _etype1107;
+            xfer += iprot->readListBegin(_etype1107, _size1104);
+            this->success.resize(_size1104);
+            uint32_t _i1108;
+            for (_i1108 = 0; _i1108 < _size1104; ++_i1108)
             {
-              xfer += this->success[_i1102].read(iprot);
+              xfer += this->success[_i1108].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3256,10 +3256,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1103;
-      for (_iter1103 = this->success.begin(); _iter1103 != this->success.end(); ++_iter1103)
+      std::vector<FieldSchema> ::const_iterator _iter1109;
+      for (_iter1109 = this->success.begin(); _iter1109 != this->success.end(); ++_iter1109)
       {
-        xfer += (*_iter1103).write(oprot);
+        xfer += (*_iter1109).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3312,14 +3312,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1104;
-            ::apache::thrift::protocol::TType _etype1107;
-            xfer += iprot->readListBegin(_etype1107, _size1104);
-            (*(this->success)).resize(_size1104);
-            uint32_t _i1108;
-            for (_i1108 = 0; _i1108 < _size1104; ++_i1108)
+            uint32_t _size1110;
+            ::apache::thrift::protocol::TType _etype1113;
+            xfer += iprot->readListBegin(_etype1113, _size1110);
+            (*(this->success)).resize(_size1110);
+            uint32_t _i1114;
+            for (_i1114 = 0; _i1114 < _size1110; ++_i1114)
             {
-              xfer += (*(this->success))[_i1108].read(iprot);
+              xfer += (*(this->success))[_i1114].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3489,14 +3489,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1109;
-            ::apache::thrift::protocol::TType _etype1112;
-            xfer += iprot->readListBegin(_etype1112, _size1109);
-            this->success.resize(_size1109);
-            uint32_t _i1113;
-            for (_i1113 = 0; _i1113 < _size1109; ++_i1113)
+            uint32_t _size1115;
+            ::apache::thrift::protocol::TType _etype1118;
+            xfer += iprot->readListBegin(_etype1118, _size1115);
+            this->success.resize(_size1115);
+            uint32_t _i1119;
+            for (_i1119 = 0; _i1119 < _size1115; ++_i1119)
             {
-              xfer += this->success[_i1113].read(iprot);
+              xfer += this->success[_i1119].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3551,10 +3551,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1114;
-      for (_iter1114 = this->success.begin(); _iter1114 != this->success.end(); ++_iter1114)
+      std::vector<FieldSchema> ::const_iterator _iter1120;
+      for (_iter1120 = this->success.begin(); _iter1120 != this->success.end(); ++_iter1120)
       {
-        xfer += (*_iter1114).write(oprot);
+        xfer += (*_iter1120).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3607,14 +3607,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1115;
-            ::apache::thrift::protocol::TType _etype1118;
-            xfer += iprot->readListBegin(_etype1118, _size1115);
-            (*(this->success)).resize(_size1115);
-            uint32_t _i1119;
-            for (_i1119 = 0; _i1119 < _size1115; ++_i1119)
+            uint32_t _size1121;
+            ::apache::thrift::protocol::TType _etype1124;
+            xfer += iprot->readListBegin(_etype1124, _size1121);
+            (*(this->success)).resize(_size1121);
+            uint32_t _i1125;
+            for (_i1125 = 0; _i1125 < _size1121; ++_i1125)
             {
-              xfer += (*(this->success))[_i1119].read(iprot);
+              xfer += (*(this->success))[_i1125].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3800,14 +3800,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1120;
-            ::apache::thrift::protocol::TType _etype1123;
-            xfer += iprot->readListBegin(_etype1123, _size1120);
-            this->success.resize(_size1120);
-            uint32_t _i1124;
-            for (_i1124 = 0; _i1124 < _size1120; ++_i1124)
+            uint32_t _size1126;
+            ::apache::thrift::protocol::TType _etype1129;
+            xfer += iprot->readListBegin(_etype1129, _size1126);
+            this->success.resize(_size1126);
+            uint32_t _i1130;
+            for (_i1130 = 0; _i1130 < _size1126; ++_i1130)
             {
-              xfer += this->success[_i1124].read(iprot);
+              xfer += this->success[_i1130].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3862,10 +3862,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1125;
-      for (_iter1125 = this->success.begin(); _iter1125 != this->success.end(); ++_iter1125)
+      std::vector<FieldSchema> ::const_iterator _iter1131;
+      for (_iter1131 = this->success.begin(); _iter1131 != this->success.end(); ++_iter1131)
       {
-        xfer += (*_iter1125).write(oprot);
+        xfer += (*_iter1131).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3918,14 +3918,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1126;
-            ::apache::thrift::protocol::TType _etype1129;
-            xfer += iprot->readListBegin(_etype1129, _size1126);
-            (*(this->success)).resize(_size1126);
-            uint32_t _i1130;
-            for (_i1130 = 0; _i1130 < _size1126; ++_i1130)
+            uint32_t _size1132;
+            ::apache::thrift::protocol::TType _etype1135;
+            xfer += iprot->readListBegin(_etype1135, _size1132);
+            (*(this->success)).resize(_size1132);
+            uint32_t _i1136;
+            for (_i1136 = 0; _i1136 < _size1132; ++_i1136)
             {
-              xfer += (*(this->success))[_i1130].read(iprot);
+              xfer += (*(this->success))[_i1136].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4518,14 +4518,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size1131;
-            ::apache::thrift::protocol::TType _etype1134;
-            xfer += iprot->readListBegin(_etype1134, _size1131);
-            this->primaryKeys.resize(_size1131);
-            uint32_t _i1135;
-            for (_i1135 = 0; _i1135 < _size1131; ++_i1135)
+            uint32_t _size1137;
+            ::apache::thrift::protocol::TType _etype1140;
+            xfer += iprot->readListBegin(_etype1140, _size1137);
+            this->primaryKeys.resize(_size1137);
+            uint32_t _i1141;
+            for (_i1141 = 0; _i1141 < _size1137; ++_i1141)
             {
-              xfer += this->primaryKeys[_i1135].read(iprot);
+              xfer += this->primaryKeys[_i1141].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4538,14 +4538,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size1136;
-            ::apache::thrift::protocol::TType _etype1139;
-            xfer += iprot->readListBegin(_etype1139, _size1136);
-            this->foreignKeys.resize(_size1136);
-            uint32_t _i1140;
-            for (_i1140 = 0; _i1140 < _size1136; ++_i1140)
+            uint32_t _size1142;
+            ::apache::thrift::protocol::TType _etype1145;
+            xfer += iprot->readListBegin(_etype1145, _size1142);
+            this->foreignKeys.resize(_size1142);
+            uint32_t _i1146;
+            for (_i1146 = 0; _i1146 < _size1142; ++_i1146)
             {
-              xfer += this->foreignKeys[_i1140].read(iprot);
+              xfer += this->foreignKeys[_i1146].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4558,14 +4558,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size1141;
-            ::apache::thrift::protocol::TType _etype1144;
-            xfer += iprot->readListBegin(_etype1144, _size1141);
-            this->uniqueConstraints.resize(_size1141);
-            uint32_t _i1145;
-            for (_i1145 = 0; _i1145 < _size1141; ++_i1145)
+            uint32_t _size1147;
+            ::apache::thrift::protocol::TType _etype1150;
+            xfer += iprot->readListBegin(_etype1150, _size1147);
+            this->uniqueConstraints.resize(_size1147);
+            uint32_t _i1151;
+            for (_i1151 = 0; _i1151 < _size1147; ++_i1151)
             {
-              xfer += this->uniqueConstraints[_i1145].read(iprot);
+              xfer += this->uniqueConstraints[_i1151].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4578,14 +4578,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraints.clear();
-            uint32_t _size1146;
-            ::apache::thrift::protocol::TType _etype1149;
-            xfer += iprot->readListBegin(_etype1149, _size1146);
-            this->notNullConstraints.resize(_size1146);
-            uint32_t _i1150;
-            for (_i1150 = 0; _i1150 < _size1146; ++_i1150)
+            uint32_t _size1152;
+            ::apache::thrift::protocol::TType _etype1155;
+            xfer += iprot->readListBegin(_etype1155, _size1152);
+            this->notNullConstraints.resize(_size1152);
+            uint32_t _i1156;
+            for (_i1156 = 0; _i1156 < _size1152; ++_i1156)
             {
-              xfer += this->notNullConstraints[_i1150].read(iprot);
+              xfer += this->notNullConstraints[_i1156].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4618,10 +4618,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 _iter1151;
-    for (_iter1151 = this->primaryKeys.begin(); _iter1151 != this->primaryKeys.end(); ++_iter1151)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1157;
+    for (_iter1157 = this->primaryKeys.begin(); _iter1157 != this->primaryKeys.end(); ++_iter1157)
     {
-      xfer += (*_iter1151).write(oprot);
+      xfer += (*_iter1157).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4630,10 +4630,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 _iter1152;
-    for (_iter1152 = this->foreignKeys.begin(); _iter1152 != this->foreignKeys.end(); ++_iter1152)
+    std::vector<SQLForeignKey> ::const_iterator _iter1158;
+    for (_iter1158 = this->foreignKeys.begin(); _iter1158 != this->foreignKeys.end(); ++_iter1158)
     {
-      xfer += (*_iter1152).write(oprot);
+      xfer += (*_iter1158).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4642,10 +4642,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 _iter1153;
-    for (_iter1153 = this->uniqueConstraints.begin(); _iter1153 != this->uniqueConstraints.end(); ++_iter1153)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1159;
+    for (_iter1159 = this->uniqueConstraints.begin(); _iter1159 != this->uniqueConstraints.end(); ++_iter1159)
     {
-      xfer += (*_iter1153).write(oprot);
+      xfer += (*_iter1159).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4654,10 +4654,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 _iter1154;
-    for (_iter1154 = this->notNullConstraints.begin(); _iter1154 != this->notNullConstraints.end(); ++_iter1154)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1160;
+    for (_iter1160 = this->notNullConstraints.begin(); _iter1160 != this->notNullConstraints.end(); ++_iter1160)
     {
-      xfer += (*_iter1154).write(oprot);
+      xfer += (*_iter1160).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4685,10 +4685,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 _iter1155;
-    for (_iter1155 = (*(this->primaryKeys)).begin(); _iter1155 != (*(this->primaryKeys)).end(); ++_iter1155)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1161;
+    for (_iter1161 = (*(this->primaryKeys)).begin(); _iter1161 != (*(this->primaryKeys)).end(); ++_iter1161)
     {
-      xfer += (*_iter1155).write(oprot);
+      xfer += (*_iter1161).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4697,10 +4697,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 _iter1156;
-    for (_iter1156 = (*(this->foreignKeys)).begin(); _iter1156 != (*(this->foreignKeys)).end(); ++_iter1156)
+    std::vector<SQLForeignKey> ::const_iterator _iter1162;
+    for (_iter1162 = (*(this->foreignKeys)).begin(); _iter1162 != (*(this->foreignKeys)).end(); ++_iter1162)
     {
-      xfer += (*_iter1156).write(oprot);
+      xfer += (*_iter1162).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4709,10 +4709,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 _iter1157;
-    for (_iter1157 = (*(this->uniqueConstraints)).begin(); _iter1157 != (*(this->uniqueConstraints)).end(); ++_iter1157)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1163;
+    for (_iter1163 = (*(this->uniqueConstraints)).begin(); _iter1163 != (*(this->uniqueConstraints)).end(); ++_iter1163)
     {
-      xfer += (*_iter1157).write(oprot);
+      xfer += (*_iter1163).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4721,10 +4721,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 _iter1158;
-    for (_iter1158 = (*(this->notNullConstraints)).begin(); _iter1158 != (*(this->notNullConstraints)).end(); ++_iter1158)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1164;
+    for (_iter1164 = (*(this->notNullConstraints)).begin(); _iter1164 != (*(this->notNullConstraints)).end(); ++_iter1164)
     {
-      xfer += (*_iter1158).write(oprot);
+      xfer += (*_iter1164).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -6478,14 +6478,14 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size1159;
-            ::apache::thrift::protocol::TType _etype1162;
-            xfer += iprot->readListBegin(_etype1162, _size1159);
-            this->partNames.resize(_size1159);
-            uint32_t _i1163;
-            for (_i1163 = 0; _i1163 < _size1159; ++_i1163)
+            uint32_t _size1165;
+            ::apache::thrift::protocol::TType _etype1168;
+            xfer += iprot->readListBegin(_etype1168, _size1165);
+            this->partNames.resize(_size1165);
+            uint32_t _i1169;
+            for (_i1169 = 0; _i1169 < _size1165; ++_i1169)
             {
-              xfer += iprot->readString(this->partNames[_i1163]);
+              xfer += iprot->readString(this->partNames[_i1169]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6522,10 +6522,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 _iter1164;
-    for (_iter1164 = this->partNames.begin(); _iter1164 != this->partNames.end(); ++_iter1164)
+    std::vector<std::string> ::const_iterator _iter1170;
+    for (_iter1170 = this->partNames.begin(); _iter1170 != this->partNames.end(); ++_iter1170)
     {
-      xfer += oprot->writeString((*_iter1164));
+      xfer += oprot->writeString((*_iter1170));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6557,10 +6557,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 _iter1165;
-    for (_iter1165 = (*(this->partNames)).begin(); _iter1165 != (*(this->partNames)).end(); ++_iter1165)
+    std::vector<std::string> ::const_iterator _iter1171;
+    for (_iter1171 = (*(this->partNames)).begin(); _iter1171 != (*(this->partNames)).end(); ++_iter1171)
     {
-      xfer += oprot->writeString((*_iter1165));
+      xfer += oprot->writeString((*_iter1171));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6804,14 +6804,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1166;
-            ::apache::thrift::protocol::TType _etype1169;
-            xfer += iprot->readListBegin(_etype1169, _size1166);
-            this->success.resize(_size1166);
-            uint32_t _i1170;
-            for (_i1170 = 0; _i1170 < _size1166; ++_i1170)
+            uint32_t _size1172;
+            ::apache::thrift::protocol::TType _etype1175;
+            xfer += iprot->readListBegin(_etype1175, _size1172);
+            this->success.resize(_size1172);
+            uint32_t _i1176;
+            for (_i1176 = 0; _i1176 < _size1172; ++_i1176)
             {
-              xfer += iprot->readString(this->success[_i1170]);
+              xfer += iprot->readString(this->success[_i1176]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6850,10 +6850,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 _iter1171;
-      for (_iter1171 = this->success.begin(); _iter1171 != this->success.end(); ++_iter1171)
+      std::vector<std::string> ::const_iterator _iter1177;
+      for (_iter1177 = this->success.begin(); _iter1177 != this->success.end(); ++_iter1177)
       {
-        xfer += oprot->writeString((*_iter1171));
+        xfer += oprot->writeString((*_iter1177));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6898,14 +6898,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1172;
-            ::apache::thrift::protocol::TType _etype1175;
-            xfer += iprot->readListBegin(_etype1175, _size1172);
-            (*(this->success)).resize(_size1172);
-            uint32_t _i1176;
-            for (_i1176 = 0; _i1176 < _size1172; ++_i1176)
+            uint32_t _size1178;
+            ::apache::thrift::protocol::TType _etype1181;
+            xfer += iprot->readListBegin(_etype1181, _size1178);
+            (*(this->success)).resize(_size1178);
+            uint32_t _i1182;
+            for (_i1182 = 0; _i1182 < _size1178; ++_i1182)
             {
-              xfer += iprot->readString((*(this->success))[_i1176]);
+              xfer += iprot->readString((*(this->success))[_i1182]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7075,14 +7075,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 _size1177;
-            ::apache::thrift::protocol::TType _etype1180;
-            xfer += iprot->readListBegin(_etype1180, _size1177);
-            this->success.resize(_size1177);
-            uint32_t _i1181;
-            for (_i1181 = 0; _i1181 < _size1177; ++_i1181)
+            uint32_t _size1183;
+            ::apache::thrift::protocol::TType _etype1186;
+            xfer += iprot->readListBegin(_etype1186, _size1183);
+            this->success.resize(_size1183);
+            uint32_t _i1187;
+            for (_i1187 = 0; _i1187 < _size1183; ++_i1187)
             {
-              xfer += iprot->readString(this->success[_i1181]);
+              xfer += iprot->readString(this->success[_i1187]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7121,10 +7121,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 _iter1182;
-      for (_iter1182 = this->success.begin(); _iter1182 != this->success.end(); ++_iter1182)
+      std::vector<std::string> ::const_iterator _iter1188;
+      for (_iter1188 = this->success.begin(); _iter1188 != this->success.end(); ++_iter1188)
       {
-        xfer += oprot->writeString((*_iter1182));
+        xfer += oprot->writeString((*_iter1188));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7169,14 +7169,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1183;
-            ::apache::thrift::protocol::TType _etype1186;
-            xfer += iprot->readListBegin(_etype1186, _size1183);
-            (*(this->success)).resize(_size1183);
-            uint32_t _i1187;
-            for (_i1187 = 0; _i1187 < _size1183; ++_i1187)
+            uint32_t _size1189;
+            ::apache::thrift::protocol::TType _etype1192;
+            xfer += iprot->readListBegin(_etype1192, _size1189);
+            (*(this->success)).resize(_size1189);
+            uint32_t _i1193;
+            for (_i1193 = 0; _i1193 < _size1189; ++_i1193)
             {
-              xfer += iprot->readString((*(this->success))[_i1187]);
+              xfer += iprot->readString((*(this->success))[_i1193]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7314,14 +7314,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1188;
-            ::apache::thrift::protocol::TType _etype1191;
-            xfer += iprot->readListBegin(_etype1191, _size1188);
-            this->success.resize(_size1188);
-            uint32_t _i1192;
-            for (_i1192 = 0; _i1192 < _size1188; ++_i1192)
+            uint32_t _size1194;
+            ::apache::thrift::protocol::TType _etype1197;
+            xfer += iprot->readListBegin(_etype1197, _size1194);
+            this->success.resize(_size1194);
+            uint32_t _i1198;
+            for (_i1198 = 0; _i1198 < _size1194; ++_i1198)
             {
-              xfer += iprot->readString(this->success[_i1192]);
+              xfer += iprot->readString(this->success[_i1198]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7360,10 +7360,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 _iter1193;
-      for (_iter1193 = this->success.begin(); _iter1193 != this->success.end(); ++_iter1193)
+      std::vector<std::string> ::const_iterator _iter1199;
+      for (_iter1199 = this->success.begin(); _iter1199 != this->success.end(); ++_iter1199)
       {
-        xfer += oprot->writeString((*_iter1193));
+        xfer += oprot->writeString((*_iter1199));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7408,14 +7408,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_presult::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1194;
-            ::apache::thrift::protocol::TType _etype1197;
-            xfer += iprot->readListBegin(_etype1197, _size1194);
-            (*(this->success)).resize(_size1194);
-            uint32_t _i1198;
-            for (_i1198 = 0; _i1198 < _size1194; ++_i1198)
+            uint32_t _size1200;
+            ::apache::thrift::protocol::TType _etype1203;
+            xfer += iprot->readListBegin(_etype1203, _size1200);
+            (*(this->success)).resize(_size1200);
+            uint32_t _i1204;
+            for (_i1204 = 0; _i1204 < _size1200; ++_i1204)
             {
-              xfer += iprot->readString((*(this->success))[_i1198]);
+              xfer += iprot->readString((*(this->success))[_i1204]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7490,14 +7490,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 _size1199;
-            ::apache::thrift::protocol::TType _etype1202;
-            xfer += iprot->readListBegin(_etype1202, _size1199);
-            this->tbl_types.resize(_size1199);
-            uint32_t _i1203;
-            for (_i1203 = 0; _i1203 < _size1199; ++_i1203)
+            uint32_t _size1205;
+            ::apache::thrift::protocol::TType _etype1208;
+            xfer += iprot->readListBegin(_etype1208, _size1205);
+            this->tbl_types.resize(_size1205);
+            uint32_t _i1209;
+            for (_i1209 = 0; _i1209 < _size1205; ++_i1209)
             {
-              xfer += iprot->readString(this->tbl_types[_i1203]);
+              xfer += iprot->readString(this->tbl_types[_i1209]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7534,10 +7534,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 _iter1204;
-    for (_iter1204 = this->tbl_types.begin(); _iter1204 != this->tbl_types.end(); ++_iter1204)
+    std::vector<std::string> ::const_iterator _iter1210;
+    for (_iter1210 = this->tbl_types.begin(); _iter1210 != this->tbl_types.end(); ++_iter1210)
     {
-      xfer += oprot->writeString((*_iter1204));
+      xfer += oprot->writeString((*_iter1210));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7569,10 +7569,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 _iter1205;
-    for (_iter1205 = (*(this->tbl_types)).begin(); _iter1205 != (*(this->tbl_types)).end(); ++_iter1205)
+    std::vector<std::string> ::const_iterator _iter1211;
+    for (_iter1211 = (*(this->tbl_types)).begin(); _iter1211 != (*(this->tbl_types)).end(); ++_iter1211)
     {
-      xfer += oprot->writeString((*_iter1205));
+      xfer += oprot->writeString((*_iter1211));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7613,14 +7613,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1206;
-            ::apache::thrift::protocol::TType _etype1209;
-            xfer += iprot->readListBegin(_etype1209, _size1206);
-            this->success.resize(_size1206);
-            uint32_t _i1210;
-            for (_i1210 = 0; _i1210 < _size1206; ++_i1210)
+            uint32_t _size1212;
+            ::apache::thrift::protocol::TType _etype1215;
+            xfer += iprot->readListBegin(_etype1215, _size1212);
+            this->success.resize(_size1212);
+            uint32_t _i1216;
+            for (_i1216 = 0; _i1216 < _size1212; ++_i1216)
             {
-              xfer += this->success[_i1210].read(iprot);
+              xfer += this->success[_i1216].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7659,10 +7659,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 _iter1211;
-      for (_iter1211 = this->success.begin(); _iter1211 != this->success.end(); ++_iter1211)
+      std::vector<TableMeta> ::const_iterator _iter1217;
+      for (_iter1217 = this->success.begin(); _iter1217 != this->success.end(); ++_iter1217)
       {
-        xfer += (*_iter1211).write(oprot);
+        xfer += (*_iter1217).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -7707,14 +7707,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1212;
-            ::apache::thrift::protocol::TType _etype1215;
-            xfer += iprot->readListBegin(_etype1215, _size1212);
-            (*(this->success)).resize(_size1212);
-            uint32_t _i1216;
-            for (_i1216 = 0; _i1216 < _size1212; ++_i1216)
+            uint32_t _size1218;
+            ::apache::thrift::protocol::TType _etype1221;
+            xfer += iprot->readListBegin(_etype1221, _size1218);
+            (*(this->success)).resize(_size1218);
+            uint32_t _i1222;
+            for (_i1222 = 0; _i1222 < _size1218; ++_i1222)
             {
-              xfer += (*(this->success))[_i1216].read(iprot);
+              xfer += (*(this->success))[_i1222].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7852,14 +7852,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1217;
-            ::apache::thrift::protocol::TType _etype1220;
-            xfer += iprot->readListBegin(_etype1220, _size1217);
-            this->success.resize(_size1217);
-            uint32_t _i1221;
-            for (_i1221 = 0; _i1221 < _size1217; ++_i1221)
+            uint32_t _size1223;
+            ::apache::thrift::protocol::TType _etype1226;
+            xfer += iprot->readListBegin(_etype1226, _size1223);
+            this->success.resize(_size1223);
+            uint32_t _i1227;
+            for (_i1227 = 0; _i1227 < _size1223; ++_i1227)
             {
-              xfer += iprot->readString(this->success[_i1221]);
+              xfer += iprot->readString(this->success[_i1227]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7898,10 +7898,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 _iter1222;
-      for (_iter1222 = this->success.begin(); _iter1222 != this->success.end(); ++_iter1222)
+      std::vector<std::string> ::const_iterator _iter1228;
+      for (_iter1228 = this->success.begin(); _iter1228 != this->success.end(); ++_iter1228)
       {
-        xfer += oprot->writeString((*_iter1222));
+        xfer += oprot->writeString((*_iter1228));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7946,14 +7946,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1223;
-            ::apache::thrift::protocol::TType _etype1226;
-            xfer += iprot->readListBegin(_etype1226, _size1223);
-            (*(this->success)).resize(_size1223);
-            uint32_t _i1227;
-            for (_i1227 = 0; _i1227 < _size1223; ++_i1227)
+            uint32_t _size1229;
+            ::apache::thrift::protocol::TType _etype1232;
+            xfer += iprot->readListBegin(_etype1232, _size1229);
+            (*(this->success)).resize(_size1229);
+            uint32_t _i1233;
+            for (_i1233 = 0; _i1233 < _size1229; ++_i1233)
             {
-              xfer += iprot->readString((*(this->success))[_i1227]);
+              xfer += iprot->readString((*(this->success))[_i1233]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8263,14 +8263,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 _size1228;
-            ::apache::thrift::protocol::TType _etype1231;
-            xfer += iprot->readListBegin(_etype1231, _size1228);
-            this->tbl_names.resize(_size1228);
-            uint32_t _i1232;
-            for (_i1232 = 0; _i1232 < _size1228; ++_i1232)
+            uint32_t _size1234;
+            ::apache::thrift::protocol::TType _etype1237;
+            xfer += iprot->readListBegin(_etype1237, _size1234);
+            this->tbl_names.resize(_size1234);
+            uint32_t _i1238;
+            for (_i1238 = 0; _i1238 < _size1234; ++_i1238)
             {
-              xfer += iprot->readString(this->tbl_names[_i1232]);
+              xfer += iprot->readString(this->tbl_names[_i1238]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8303,10 +8303,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 _iter1233;
-    for (_iter1233 = this->tbl_names.begin(); _iter1233 != this->tbl_names.end(); ++_iter1233)
+    std::vector<std::string> ::const_iterator _iter1239;
+    for (_iter1239 = this->tbl_names.begin(); _iter1239 != this->tbl_names.end(); ++_iter1239)
     {
-      xfer += oprot->writeString((*_iter1233));
+      xfer += oprot->writeString((*_iter1239));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8334,10 +8334,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 _iter1234;
-    for (_iter1234 = (*(this->tbl_names)).begin(); _iter1234 != (*(this->tbl_names)).end(); ++_iter1234)
+    std::vector<std::string> ::const_iterator _iter1240;
+    for (_iter1240 = (*(this->tbl_names)).begin(); _iter1240 != (*(this->tbl_names)).end(); ++_iter1240)
     {
-      xfer += oprot->writeString((*_iter1234));
+      xfer += oprot->writeString((*_iter1240));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8378,14 +8378,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 _size1235;
-            ::apache::thrift::protocol::TType _etype1238;
-            xfer += iprot->readListBegin(_etype1238, _size1235);
-            this->success.resize(_size1235);
-            uint32_t _i1239;
-            for (_i1239 = 0; _i1239 < _size1235; ++_i1239)
+            uint32_t _size1241;
+            ::apache::thrift::protocol::TType _etype1244;
+            xfer += iprot->readListBegin(_etype1244, _size1241);
+            this->success.resize(_size1241);
+            uint32_t _i1245;
+            for (_i1245 = 0; _i1245 < _size1241; ++_i1245)
             {
-              xfer += this->success[_i1239].read(iprot);
+              xfer += this->success[_i1245].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8416,10 +8416,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 _iter1240;
-      for (_iter1240 = this->success.begin(); _iter1240 != this->success.end(); ++_iter1240)
+      std::vector<Table> ::const_iterator _iter1246;
+      for (_iter1246 = this->success.begin(); _iter1246 != this->success.end(); ++_iter1246)
       {
-        xfer += (*_iter1240).write(oprot);
+        xfer += (*_iter1246).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -8460,14 +8460,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 _size1241;
-            ::apache::thrift::protocol::TType _etype1244;
-            xfer += iprot->readListBegin(_etype1244, _size1241);
-            (*(this->success)).resize(_size1241);
-            uint32_t _i1245;
-            for (_i1245 = 0; _i1245 < _size1241; ++_i1245)
+            uint32_t _size1247;
+            ::apache::thrift::protocol::TType _etype1250;
+            xfer += iprot->readListBegin(_etype1250, _size1247);
+            (*(this->success)).resize(_size1247);
+            uint32_t _i1251;
+            for (_i1251 = 0; _i1251 < _size1247; ++_i1251)
             {
-              xfer += (*(this->success))[_i1245].read(iprot);
+              xfer += (*(this->success))[_i1251].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9000,14 +9000,14 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size1246;
-            ::apache::thrift::protocol::TType _etype1249;
-            xfer += iprot->readListBegin(_etype1249, _size1246);
-            this->tbl_names.resize(_size1246);
-            uint32_t _i1250;
-            for (_i1250 = 0; _i1250 < _size1246; ++_i1250)
+            uint32_t _size1252;
+            ::apache::thrift::protocol::TType _etype1255;
+            xfer += iprot->readListBegin(_etype1255, _size1252);
+            this->tbl_names.resize(_size1252);
+            uint32_t _i1256;
+            for (_i1256 = 0; _i1256 < _size1252; ++_i1256)
             {
-              xfer += iprot->readString(this->tbl_names[_i1250]);
+              xfer += iprot->readString(this->tbl_names[_i1256]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9040,10 +9040,10 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::write(:
   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 _iter1251;
-    for (_iter1251 = this->tbl_names.begin(); _iter1251 != this->tbl_names.end(); ++_iter1251)
+    std::vector<std::string> ::const_iterator _iter1257;
+    for (_iter1257 = this->tbl_names.begin(); _iter1257 != this->tbl_names.end(); ++_iter1257)
     {
-      xfer += oprot->writeString((*_iter1251));
+      xfer += oprot->writeString((*_iter1257));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9071,10 +9071,10 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_pargs::write(
   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 _iter1252;
-    for (_iter1252 = (*(this->tbl_names)).begin(); _iter1252 != (*(this->tbl_names)).end(); ++_iter1252)
+    std::vector<std::string> ::const_iterator _iter1258;
+    for (_iter1258 = (*(this->tbl_names)).begin(); _iter1258 != (*(this->tbl_names)).end(); ++_iter1258)
     {
-      xfer += oprot->writeString((*_iter1252));
+      xfer += oprot->writeString((*_iter1258));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9115,17 +9115,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::read(
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1253;
-            ::apache::thrift::protocol::TType _ktype1254;
-            ::apache::thrift::protocol::TType _vtype1255;
-            xfer += iprot->readMapBegin(_ktype1254, _vtype1255, _size1253);
-            uint32_t _i1257;
-            for (_i1257 = 0; _i1257 < _size1253; ++_i1257)
+            uint32_t _size1259;
+            ::apache::thrift::protocol::TType _ktype1260;
+            ::apache::thrift::protocol::TType _vtype1261;
+            xfer += iprot->readMapBegin(_ktype1260, _vtype1261, _size1259);
+            uint32_t _i1263;
+            for (_i1263 = 0; _i1263 < _size1259; ++_i1263)
             {
-              std::string _key1258;
-              xfer += iprot->readString(_key1258);
-              Materialization& _val1259 = this->success[_key1258];
-              xfer += _val1259.read(iprot);
+              std::string _key1264;
+              xfer += iprot->readString(_key1264);
+              Materialization& _val1265 = this->success[_key1264];
+              xfer += _val1265.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -9180,11 +9180,11 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::write
     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 _iter1260;
-      for (_iter1260 = this->success.begin(); _iter1260 != this->success.end(); ++_iter1260)
+      std::map<std::string, Materialization> ::const_iterator _iter1266;
+      for (_iter1266 = this->success.begin(); _iter1266 != this->success.end(); ++_iter1266)
       {
-        xfer += oprot->writeString(_iter1260->first);
-        xfer += _iter1260->second.write(oprot);
+        xfer += oprot->writeString(_iter1266->first);
+        xfer += _iter1266->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -9237,17 +9237,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_presult::read
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1261;
-            ::apache::thrift::protocol::TType _ktype1262;
-            ::apache::thrift::protocol::TType _vtype1263;
-            xfer += iprot->readMapBegin(_ktype1262, _vtype1263, _size1261);
-            uint32_t _i1265;
-            for (_i1265 = 0; _i1265 < _size1261; ++_i1265)
+            uint32_t _size1267;
+            ::apache::thrift::protocol::TType _ktype1268;
+            ::apache::thrift::protocol::TType _vtype1269;
+            xfer += iprot->readMapBegin(_ktype1268, _vtype1269, _size1267);
+            uint32_t _i1271;
+            for (_i1271 = 0; _i1271 < _size1267; ++_i1271)
             {
-              std::string _key1266;
-              xfer += iprot->readString(_key1266);
-              Materialization& _val1267 = (*(this->success))[_key1266];
-              xfer += _val1267.read(iprot);
+              std::string _key1272;
+              xfer += iprot->readString(_key1272);
+              Materialization& _val1273 = (*(this->success))[_key1272];
+              xfer += _val1273.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -9433,14 +9433,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 _size1268;
-            ::apache::thrift::protocol::TType _etype1271;
-            xfer += iprot->readListBegin(_etype1271, _size1268);
-            this->success.resize(_size1268);
-            uint32_t _i1272;
-            for (_i1272 = 0; _i1272 < _size1268; ++_i1272)
+            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)
             {
-              xfer += iprot->readString(this->success[_i1272]);
+              xfer += iprot->readString(this->success[_i1278]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9495,10 +9495,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 _iter1273;
-      for (_iter1273 = this->success.begin(); _iter1273 != this->success.end(); ++_iter1273)
+      std::vector<std::string> ::const_iterator _iter1279;
+      for (_iter1279 = this->success.begin(); _iter1279 != this->success.end(); ++_iter1279)
       {
-        xfer += oprot->writeString((*_iter1273));
+        xfer += oprot->writeString((*_iter1279));
       }
       xfer += oprot->writeListEnd();
     }
@@ -9551,14 +9551,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 _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 _size1280;
+            ::apache::thrift::protocol::TType _etype1283;
+            xfer += iprot->readListBegin(_etype1283, _size1280);
+            (*(this->success)).resize(_size1280);
+            uint32_t _i1284;
+            for (_i1284 = 0; _i1284 < _size1280; ++_i1284)
             {
-              xfer += iprot->readString((*(this->success))[_i1278]);
+              xfer += iprot->readString((*(this->success))[_i1284]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10892,14 +10892,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size1279;
-            ::apache::thrift::protocol::TType _etype1282;
-            xfer += iprot->readListBegin(_etype1282, _size1279);
-            this->new_parts.resize(_size1279);
-            uint32_t _i1283;
-            for (_i1283 = 0; _i1283 < _size1279; ++_i1283)
+            uint32_t _size1285;
+            ::apache::thrift::protocol::TType _etype1288;
+            xfer += iprot->readListBegin(_etype1288, _size1285);
+            this->new_parts.resize(_size1285);
+            uint32_t _i1289;
+            for (_i1289 = 0; _i1289 < _size1285; ++_i1289)
             {
-              xfer += this->new_parts[_i1283].read(iprot);
+              xfer += this->new_parts[_i1289].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10928,10 +10928,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 _iter1284;
-    for (_iter1284 = this->new_parts.begin(); _iter1284 != this->new_parts.end(); ++_iter1284)
+    std::vector<Partition> ::const_iterator _iter1290;
+    for (_iter1290 = this->new_parts.begin(); _iter1290 != this->new_parts.end(); ++_iter1290)
     {
-      xfer += (*_iter1284).write(oprot);
+      xfer += (*_iter1290).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10955,10 +10955,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 _iter1285;
-    for (_iter1285 = (*(this->new_parts)).begin(); _iter1285 != (*(this->new_parts)).end(); ++_iter1285)
+    std::vector<Partition> ::const_iterator _iter1291;
+    for (_iter1291 = (*(this->new_parts)).begin(); _iter1291 != (*(this->new_parts)).end(); ++_iter1291)
     {
-      xfer += (*_iter1285).write(oprot);
+      xfer += (*_iter1291).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11167,14 +11167,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 _size1286;
-            ::apache::thrift::protocol::TType _etype1289;
-            xfer += iprot->readListBegin(_etype1289, _size1286);
-            this->new_parts.resize(_size1286);
-            uint32_t _i1290;
-            for (_i1290 = 0; _i1290 < _size1286; ++_i1290)
+            uint32_t _size1292;
+            ::apache::thrift::protocol::TType _etype1295;
+            xfer += iprot->readListBegin(_etype1295, _size1292);
+            this->new_parts.resize(_size1292);
+            uint32_t _i1296;
+            for (_i1296 = 0; _i1296 < _size1292; ++_i1296)
             {
-              xfer += this->new_parts[_i1290].read(iprot);
+              xfer += this->new_parts[_i1296].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11203,10 +11203,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 _iter1291;
-    for (_iter1291 = this->new_parts.begin(); _iter1291 != this->new_parts.end(); ++_iter1291)
+    std::vector<PartitionSpec> ::const_iterator _iter1297;
+    for (_iter1297 = this->new_parts.begin(); _iter1297 != this->new_parts.end(); ++_iter1297)
     {
-      xfer += (*_iter1291).write(oprot);
+      xfer += (*_iter1297).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11230,10 +11230,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 _iter1292;
-    for (_iter1292 = (*(this->new_parts)).begin(); _iter1292 != (*(this->new_parts)).end(); ++_iter1292)
+    std::vector<PartitionSpec> ::const_iterator _iter1298;
+    for (_iter1298 = (*(this->new_parts)).begin(); _iter1298 != (*(this->new_parts)).end(); ++_iter1298)
     {
-      xfer += (*_iter1292).write(oprot);
+      xfer += (*_iter1298).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11458,14 +11458,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1293;
-            ::apache::thrift::protocol::TType _etype1296;
-            xfer += iprot->readListBegin(_etype1296, _size1293);
-            this->part_vals.resize(_size1293);
-            uint32_t _i1297;
-            for (_i1297 = 0; _i1297 < _size1293; ++_i1297)
+            uint32_t _size1299;
+            ::apache::thrift::protocol::TType _etype1302;
+            xfer += iprot->readListBegin(_etype1302, _size1299);
+            this->part_vals.resize(_size1299);
+            uint32_t _i1303;
+            for (_i1303 = 0; _i1303 < _size1299; ++_i1303)
             {
-              xfer += iprot->readString(this->part_vals[_i1297]);
+              xfer += iprot->readString(this->part_vals[_i1303]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11502,10 +11502,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 _iter1298;
-    for (_iter1298 = this->part_vals.begin(); _iter1298 != this->part_vals.end(); ++_iter1298)
+    std::vector<std::string> ::const_iterator _iter1304;
+    for (_iter1304 = this->part_vals.begin(); _iter1304 != this->part_vals.end(); ++_iter1304)
     {
-      xfer += oprot->writeString((*_iter1298));
+      xfer += oprot->writeString((*_iter1304));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11537,10 +11537,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 _iter1299;
-    for (_iter1299 = (*(this->part_vals)).begin(); _iter1299 != (*(this->part_vals)).end(); ++_iter1299)
+    std::vector<std::string> ::const_iterator _iter1305;
+    for (_iter1305 = (*(this->part_vals)).begin(); _iter1305 != (*(this->part_vals)).end(); ++_iter1305)
     {
-      xfer += oprot->writeString((*_iter1299));
+      xfer += oprot->writeString((*_iter1305));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12012,14 +12012,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1300;
-            ::apache::thrift::protocol::TType _etype1303;
-            xfer += iprot->readListBegin(_etype1303, _size1300);
-            this->part_vals.resize(_size1300);
-            uint32_t _i1304;
-            for (_i1304 = 0; _i1304 < _size1300; ++_i1304)
+            uint32_t _size1306;
+            ::apache::thrift::protocol::TType _etype1309;
+            xfer += iprot->readListBegin(_etype1309, _size1306);
+            this->part_vals.resize(_size1306);
+            uint32_t _i1310;
+            for (_i1310 = 0; _i1310 < _size1306; ++_i1310)
             {
-              xfer += iprot->readString(this->part_vals[_i1304]);
+              xfer += iprot->readString(this->part_vals[_i1310]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12064,10 +12064,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 _iter1305;
-    for (_iter1305 = this->part_vals.begin(); _iter1305 != this->part_vals.end(); ++_iter1305)
+    std::vector<std::string> ::const_iterator _iter1311;
+    for (_iter1311 = this->part_vals.begin(); _iter1311 != this->part_vals.end(); ++_iter1311)
     {
-      xfer += oprot->writeString((*_iter1305));
+      xfer += oprot->writeString((*_iter1311));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12103,10 +12103,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 _iter1306;
-    for (_iter1306 = (*(this->part_vals)).begin(); _iter1306 != (*(this->part_vals)).end(); ++_iter1306)
+    std::vector<std::string> ::const_iterator _iter1312;
+    for (_iter1312 = (*(this->part_vals)).begin(); _iter1312 != (*(this->part_vals)).end(); ++_iter1312)
     {
-      xfer += oprot->writeString((*_iter1306));
+      xfer += oprot->writeString((*_iter1312));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12909,14 +12909,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1307;
-            ::apache::thrift::protocol::TType _etype1310;
-            xfer += iprot->readListBegin(_etype1310, _size1307);
-            this->part_vals.resize(_size1307);
-            uint32_t _i1311;
-            for (_i1311 = 0; _i1311 < _size1307; ++_i1311)
+            uint32_t _size1313;
+            ::apache::thrift::protocol::TType _etype1316;
+            xfer += iprot->readListBegin(_etype1316, _size1313);
+            this->part_vals.resize(_size1313);
+            uint32_t _i1317;
+            for (_i1317 = 0; _i1317 < _size1313; ++_i1317)
             {
-              xfer += iprot->readString(this->part_vals[_i1311]);
+              xfer += iprot->readString(this->part_vals[_i1317]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12961,10 +12961,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 _iter1312;
-    for (_iter1312 = this->part_vals.begin(); _iter1312 != this->part_vals.end(); ++_iter1312)
+    std::vector<std::string> ::const_iterator _iter1318;
+    for (_iter1318 = this->part_vals.begin(); _iter1318 != this->part_vals.end(); ++_iter1318)
     {
-      xfer += oprot->writeString((*_iter1312));
+      xfer += oprot->writeString((*_iter1318));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13000,10 +13000,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 _iter1313;
-    for (_iter1313 = (*(this->part_vals)).begin(); _iter1313 != (*(this->part_vals)).end(); ++_iter1313)
+    std::vector<std::string> ::const_iterator _iter1319;
+    for (_iter1319 = (*(this->part_vals)).begin(); _iter1319 != (*(this->part_vals)).end(); ++_iter1319)
     {
-      xfer += oprot->writeString((*_iter1313));
+      xfer += oprot->writeString((*_iter1319));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13212,14 +13212,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1314;
-            ::apache::thrift::protocol::TType _etype1317;
-            xfer += iprot->readListBegin(_etype1317, _size1314);
-            this->part_vals.resize(_size1314);
-            uint32_t _i1318;
-            for (_i1318 = 0; _i1318 < _size1314; ++_i1318)
+            uint32_t _size1320;
+            ::apache::thrift::protocol::TType _etype1323;
+            xfer += iprot->readListBegin(_etype1323, _size1320);
+            this->part_vals.resize(_size1320);
+            uint32_t _i1324;
+            for (_i1324 = 0; _i1324 < _size1320; ++_i1324)
             {
-              xfer += iprot->readString(this->part_vals[_i1318]);
+              xfer += iprot->readString(this->part_vals[_i1324]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13272,10 +13272,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 _iter1319;
-    for (_iter1319 = this->part_vals.begin(); _iter1319 != this->part_vals.end(); ++_iter1319)
+    std::vector<std::string> ::const_iterator _iter1325;
+    for (_iter1325 = this->part_vals.begin(); _iter1325 != this->part_vals.end(); ++_iter1325)
     {
-      xfer += oprot->writeString((*_iter1319));
+      xfer += oprot->writeString((*_iter1325));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13315,10 +13315,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 _iter1320;
-    for (_iter1320 = (*(this->part_vals)).begin(); _iter1320 != (*(this->part_vals)).end(); ++_iter1320)
+    std::vector<std::string> ::const_iterator _iter1326;
+    for (_iter1326 = (*(this->part_vals)).begin(); _iter1326 != (*(this->part_vals)).end(); ++_iter1326)
     {
-      xfer += oprot->writeString((*_iter1320));
+      xfer += oprot->writeString((*_iter1326));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14324,14 +14324,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1321;
-            ::apache::thrift::protocol::TType _etype1324;
-            xfer += iprot->readListBegin(_etype1324, _size1321);
-            this->part_vals.resize(_size1321);
-            uint32_t _i1325;
-            for (_i1325 = 0; _i1325 < _size1321; ++_i1325)
+            uint32_t _size1327;
+            ::apache::thrift::protocol::TType _etype1330;
+            xfer += iprot->readListBegin(_etype1330, _size1327);
+            this->part_vals.resize(_size1327);
+            uint32_t _i1331;
+            for (_i1331 = 0; _i1331 < _size1327; ++_i1331)
             {
-              xfer += iprot->readString(this->part_vals[_i1325]);
+              xfer += iprot->readString(this->part_vals[_i1331]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14368,10 +14368,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 _iter1326;
-    for (_iter1326 = this->part_vals.begin(); _iter1326 != this->part_vals.end(); ++_iter1326)
+    std::vector<std::string> ::const_iterator _iter1332;
+    for (_iter1332 = this->part_vals.begin(); _iter1332 != this->part_vals.end(); ++_iter1332)
     {
-      xfer += oprot->writeString((*_iter1326));
+      xfer += oprot->writeString((*_iter1332));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14403,10 +14403,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter1327;
-    for (_iter1327 = (*(this->part_vals)).begin(); _iter1327 != (*(this->part_vals)).end(); ++_iter1327)
+    std::vector<std::string> ::const_iterator _iter1333;
+    for (_iter1333 = (*(this->part_vals)).begin(); _iter1333 != (*(this->part_vals)).end(); ++_iter1333)
     {
-      xfer += oprot->writeString((*_iter1327));
+      xfer += oprot->writeString((*_iter1333));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14595,17 +14595,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size1328;
-            ::apache::thrift::protocol::TType _ktype1329;
-            ::apache::thrift::protocol::TType _vtype1330;
-            xfer += iprot->readMapBegin(_ktype1329, _vtype1330, _size1328);
-            uint32_t _i1332;
-            for (_i1332 = 0; _i1332 < _size1328; ++_i1332)
+            uint32_t _size1334;
+            ::apache::thrift::protocol::TType _ktype1335;
+            ::apache::thrift::protocol::TType _vtype1336;
+            xfer += iprot->readMapBegin(_ktype1335, _vtype1336, _size1334);
+            uint32_t _i1338;
+            for (_i1338 = 0; _i1338 < _size1334; ++_i1338)
             {
-              std::string _key1333;
-              xfer += iprot->readString(_key1333);
-              std::string& _val1334 = this->partitionSpecs[_key1333];
-              xfer += iprot->readString(_val1334);
+              std::string _key1339;
+              xfer += iprot->readString(_key1339);
+              std::string& _val1340 = this->partitionSpecs[_key1339];
+              xfer += iprot->readString(_val1340);
             }
             xfer += iprot->readMapEnd();
           }
@@ -14666,11 +14666,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter1335;
-    for (_iter1335 = this->partitionSpecs.begin(); _iter1335 != this->partitionSpecs.end(); ++_iter1335)
+    std::map<std::string, std::string> ::const_iterator _iter1341;
+    for (_iter1341 = this->partitionSpecs.begin(); _iter1341 != this->partitionSpecs.end(); ++_iter1341)
     {
-      xfer += oprot->writeString(_iter1335->first);
-      xfer += oprot->writeString(_iter1335->second);
+      xfer += oprot->writeString(_iter1341->first);
+      xfer += oprot->writeString(_iter1341->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -14710,11 +14710,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter1336;
-    for (_iter1336 = (*(this->partitionSpecs)).begin(); _iter1336 != (*(this->partitionSpecs)).end(); ++_iter1336)
+    std::map<std::string, std::string> ::const_iterator _iter1342;
+    for (_iter1342 = (*(this->partitionSpecs)).begin(); _iter1342 != (*(this->partitionSpecs)).end(); ++_iter1342)
     {
-      xfer += oprot->writeString(_iter1336->first);
-      xfer += oprot->writeString(_iter1336->second);
+      xfer += oprot->writeString(_iter1342->first);
+      xfer += oprot->writeString(_iter1342->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -14959,17 +14959,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size1337;
-            ::apache::thrift::protocol::TType _ktype1338;
-            ::apache::thrift::protocol::TType _vtype1339;
-            xfer += iprot->readMapBegin(_ktype1338, _vtype1339, _size1337);
-            uint32_t _i1341;
-            for (_i1341 = 0; _i1341 < _size1337; ++_i1341)
+            uint32_t _size1343;
+            ::apache::thrift::protocol::TType _ktype1344;
+            ::apache::thrift::protocol::TType _vtype1345;
+            xfer += iprot->readMapBegin(_ktype1344, _vtype1345, _size1343);
+            uint32_t _i1347;
+            for (_i1347 = 0; _i1347 < _size1343; ++_i1347)
             {
-              std::string _key1342;
-              xfer += iprot->readString(_key1342);
-              std::string& _val1343 = this->partitionSpecs[_key1342];
-              xfer += iprot->readString(_val1343);
+              std::string _key1348;
+              xfer += iprot->readString(_key1348);
+              std::string& _val1349 = this->partitionSpecs[_key1348];
+              xfer += iprot->readString(_val1349);
             }
             xfer += iprot->readMapEnd();
           }
@@ -15030,11 +15030,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter1344;
-    for (_iter1344 = this->partitionSpecs.begin(); _iter1344 != this->partitionSpecs.end(); ++_iter1344)
+    std::map<std::string, std::string> ::const_iterator _iter1350;
+    for (_iter1350 = this->partitionSpecs.begin(); _iter1350 != this->partitionSpecs.end(); ++_iter1350)
     {
-      xfer += oprot->writeString(_iter1344->fir

<TRUNCATED>

[2/8] hive git commit: HIVE-18231 : validate resource plan - part 2 - validate action and trigger expressions (Harish Jaiprakash, reviewed by Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
index 6704d9b..6878ee1 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -23340,6 +23340,10 @@ class WMValidateResourcePlanResponse {
    * @var string[]
    */
   public $errors = null;
+  /**
+   * @var string[]
+   */
+  public $warnings = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -23352,12 +23356,23 @@ class WMValidateResourcePlanResponse {
             'type' => TType::STRING,
             ),
           ),
+        2 => array(
+          'var' => 'warnings',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
         );
     }
     if (is_array($vals)) {
       if (isset($vals['errors'])) {
         $this->errors = $vals['errors'];
       }
+      if (isset($vals['warnings'])) {
+        $this->warnings = $vals['warnings'];
+      }
     }
   }
 
@@ -23397,6 +23412,23 @@ class WMValidateResourcePlanResponse {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 2:
+          if ($ftype == TType::LST) {
+            $this->warnings = array();
+            $_size717 = 0;
+            $_etype720 = 0;
+            $xfer += $input->readListBegin($_etype720, $_size717);
+            for ($_i721 = 0; $_i721 < $_size717; ++$_i721)
+            {
+              $elem722 = null;
+              $xfer += $input->readString($elem722);
+              $this->warnings []= $elem722;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -23418,9 +23450,26 @@ class WMValidateResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRING, count($this->errors));
         {
-          foreach ($this->errors as $iter717)
+          foreach ($this->errors as $iter723)
+          {
+            $xfer += $output->writeString($iter723);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->warnings !== null) {
+      if (!is_array($this->warnings)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('warnings', TType::LST, 2);
+      {
+        $output->writeListBegin(TType::STRING, count($this->warnings));
+        {
+          foreach ($this->warnings as $iter724)
           {
-            $xfer += $output->writeString($iter717);
+            $xfer += $output->writeString($iter724);
           }
         }
         $output->writeListEnd();
@@ -24093,15 +24142,15 @@ class WMGetTriggersForResourePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->triggers = array();
-            $_size718 = 0;
-            $_etype721 = 0;
-            $xfer += $input->readListBegin($_etype721, $_size718);
-            for ($_i722 = 0; $_i722 < $_size718; ++$_i722)
+            $_size725 = 0;
+            $_etype728 = 0;
+            $xfer += $input->readListBegin($_etype728, $_size725);
+            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
             {
-              $elem723 = null;
-              $elem723 = new \metastore\WMTrigger();
-              $xfer += $elem723->read($input);
-              $this->triggers []= $elem723;
+              $elem730 = null;
+              $elem730 = new \metastore\WMTrigger();
+              $xfer += $elem730->read($input);
+              $this->triggers []= $elem730;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24129,9 +24178,9 @@ class WMGetTriggersForResourePlanResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->triggers));
         {
-          foreach ($this->triggers as $iter724)
+          foreach ($this->triggers as $iter731)
           {
-            $xfer += $iter724->write($output);
+            $xfer += $iter731->write($output);
           }
         }
         $output->writeListEnd();

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 13f77bc..988c01a 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -13638,10 +13638,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype725, _size722) = iprot.readListBegin()
-          for _i726 in xrange(_size722):
-            _elem727 = iprot.readString()
-            self.success.append(_elem727)
+          (_etype732, _size729) = iprot.readListBegin()
+          for _i733 in xrange(_size729):
+            _elem734 = iprot.readString()
+            self.success.append(_elem734)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13664,8 +13664,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 iter728 in self.success:
-        oprot.writeString(iter728)
+      for iter735 in self.success:
+        oprot.writeString(iter735)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13770,10 +13770,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype732, _size729) = iprot.readListBegin()
-          for _i733 in xrange(_size729):
-            _elem734 = iprot.readString()
-            self.success.append(_elem734)
+          (_etype739, _size736) = iprot.readListBegin()
+          for _i740 in xrange(_size736):
+            _elem741 = iprot.readString()
+            self.success.append(_elem741)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13796,8 +13796,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 iter735 in self.success:
-        oprot.writeString(iter735)
+      for iter742 in self.success:
+        oprot.writeString(iter742)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14567,12 +14567,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype737, _vtype738, _size736 ) = iprot.readMapBegin()
-          for _i740 in xrange(_size736):
-            _key741 = iprot.readString()
-            _val742 = Type()
-            _val742.read(iprot)
-            self.success[_key741] = _val742
+          (_ktype744, _vtype745, _size743 ) = iprot.readMapBegin()
+          for _i747 in xrange(_size743):
+            _key748 = iprot.readString()
+            _val749 = Type()
+            _val749.read(iprot)
+            self.success[_key748] = _val749
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -14595,9 +14595,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 kiter743,viter744 in self.success.items():
-        oprot.writeString(kiter743)
-        viter744.write(oprot)
+      for kiter750,viter751 in self.success.items():
+        oprot.writeString(kiter750)
+        viter751.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -14740,11 +14740,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype748, _size745) = iprot.readListBegin()
-          for _i749 in xrange(_size745):
-            _elem750 = FieldSchema()
-            _elem750.read(iprot)
-            self.success.append(_elem750)
+          (_etype755, _size752) = iprot.readListBegin()
+          for _i756 in xrange(_size752):
+            _elem757 = FieldSchema()
+            _elem757.read(iprot)
+            self.success.append(_elem757)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14779,8 +14779,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 iter751 in self.success:
-        iter751.write(oprot)
+      for iter758 in self.success:
+        iter758.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14947,11 +14947,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype755, _size752) = iprot.readListBegin()
-          for _i756 in xrange(_size752):
-            _elem757 = FieldSchema()
-            _elem757.read(iprot)
-            self.success.append(_elem757)
+          (_etype762, _size759) = iprot.readListBegin()
+          for _i763 in xrange(_size759):
+            _elem764 = FieldSchema()
+            _elem764.read(iprot)
+            self.success.append(_elem764)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14986,8 +14986,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 iter758 in self.success:
-        iter758.write(oprot)
+      for iter765 in self.success:
+        iter765.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15140,11 +15140,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype762, _size759) = iprot.readListBegin()
-          for _i763 in xrange(_size759):
-            _elem764 = FieldSchema()
-            _elem764.read(iprot)
-            self.success.append(_elem764)
+          (_etype769, _size766) = iprot.readListBegin()
+          for _i770 in xrange(_size766):
+            _elem771 = FieldSchema()
+            _elem771.read(iprot)
+            self.success.append(_elem771)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15179,8 +15179,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 iter765 in self.success:
-        iter765.write(oprot)
+      for iter772 in self.success:
+        iter772.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15347,11 +15347,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype769, _size766) = iprot.readListBegin()
-          for _i770 in xrange(_size766):
-            _elem771 = FieldSchema()
-            _elem771.read(iprot)
-            self.success.append(_elem771)
+          (_etype776, _size773) = iprot.readListBegin()
+          for _i777 in xrange(_size773):
+            _elem778 = FieldSchema()
+            _elem778.read(iprot)
+            self.success.append(_elem778)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15386,8 +15386,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 iter772 in self.success:
-        iter772.write(oprot)
+      for iter779 in self.success:
+        iter779.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15834,44 +15834,44 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype776, _size773) = iprot.readListBegin()
-          for _i777 in xrange(_size773):
-            _elem778 = SQLPrimaryKey()
-            _elem778.read(iprot)
-            self.primaryKeys.append(_elem778)
+          (_etype783, _size780) = iprot.readListBegin()
+          for _i784 in xrange(_size780):
+            _elem785 = SQLPrimaryKey()
+            _elem785.read(iprot)
+            self.primaryKeys.append(_elem785)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype782, _size779) = iprot.readListBegin()
-          for _i783 in xrange(_size779):
-            _elem784 = SQLForeignKey()
-            _elem784.read(iprot)
-            self.foreignKeys.append(_elem784)
+          (_etype789, _size786) = iprot.readListBegin()
+          for _i790 in xrange(_size786):
+            _elem791 = SQLForeignKey()
+            _elem791.read(iprot)
+            self.foreignKeys.append(_elem791)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.uniqueConstraints = []
-          (_etype788, _size785) = iprot.readListBegin()
-          for _i789 in xrange(_size785):
-            _elem790 = SQLUniqueConstraint()
-            _elem790.read(iprot)
-            self.uniqueConstraints.append(_elem790)
+          (_etype795, _size792) = iprot.readListBegin()
+          for _i796 in xrange(_size792):
+            _elem797 = SQLUniqueConstraint()
+            _elem797.read(iprot)
+            self.uniqueConstraints.append(_elem797)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.notNullConstraints = []
-          (_etype794, _size791) = iprot.readListBegin()
-          for _i795 in xrange(_size791):
-            _elem796 = SQLNotNullConstraint()
-            _elem796.read(iprot)
-            self.notNullConstraints.append(_elem796)
+          (_etype801, _size798) = iprot.readListBegin()
+          for _i802 in xrange(_size798):
+            _elem803 = SQLNotNullConstraint()
+            _elem803.read(iprot)
+            self.notNullConstraints.append(_elem803)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15892,29 +15892,29 @@ 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 iter797 in self.primaryKeys:
-        iter797.write(oprot)
+      for iter804 in self.primaryKeys:
+        iter804.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 iter798 in self.foreignKeys:
-        iter798.write(oprot)
+      for iter805 in self.foreignKeys:
+        iter805.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 iter799 in self.uniqueConstraints:
-        iter799.write(oprot)
+      for iter806 in self.uniqueConstraints:
+        iter806.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 iter800 in self.notNullConstraints:
-        iter800.write(oprot)
+      for iter807 in self.notNullConstraints:
+        iter807.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17180,10 +17180,10 @@ class truncate_table_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype804, _size801) = iprot.readListBegin()
-          for _i805 in xrange(_size801):
-            _elem806 = iprot.readString()
-            self.partNames.append(_elem806)
+          (_etype811, _size808) = iprot.readListBegin()
+          for _i812 in xrange(_size808):
+            _elem813 = iprot.readString()
+            self.partNames.append(_elem813)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17208,8 +17208,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 iter807 in self.partNames:
-        oprot.writeString(iter807)
+      for iter814 in self.partNames:
+        oprot.writeString(iter814)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17409,10 +17409,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype811, _size808) = iprot.readListBegin()
-          for _i812 in xrange(_size808):
-            _elem813 = iprot.readString()
-            self.success.append(_elem813)
+          (_etype818, _size815) = iprot.readListBegin()
+          for _i819 in xrange(_size815):
+            _elem820 = iprot.readString()
+            self.success.append(_elem820)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17435,8 +17435,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 iter814 in self.success:
-        oprot.writeString(iter814)
+      for iter821 in self.success:
+        oprot.writeString(iter821)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17586,10 +17586,10 @@ class get_tables_by_type_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype818, _size815) = iprot.readListBegin()
-          for _i819 in xrange(_size815):
-            _elem820 = iprot.readString()
-            self.success.append(_elem820)
+          (_etype825, _size822) = iprot.readListBegin()
+          for _i826 in xrange(_size822):
+            _elem827 = iprot.readString()
+            self.success.append(_elem827)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17612,8 +17612,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 iter821 in self.success:
-        oprot.writeString(iter821)
+      for iter828 in self.success:
+        oprot.writeString(iter828)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17737,10 +17737,10 @@ class get_materialized_views_for_rewriting_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype825, _size822) = iprot.readListBegin()
-          for _i826 in xrange(_size822):
-            _elem827 = iprot.readString()
-            self.success.append(_elem827)
+          (_etype832, _size829) = iprot.readListBegin()
+          for _i833 in xrange(_size829):
+            _elem834 = iprot.readString()
+            self.success.append(_elem834)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17763,8 +17763,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 iter828 in self.success:
-        oprot.writeString(iter828)
+      for iter835 in self.success:
+        oprot.writeString(iter835)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17837,10 +17837,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype832, _size829) = iprot.readListBegin()
-          for _i833 in xrange(_size829):
-            _elem834 = iprot.readString()
-            self.tbl_types.append(_elem834)
+          (_etype839, _size836) = iprot.readListBegin()
+          for _i840 in xrange(_size836):
+            _elem841 = iprot.readString()
+            self.tbl_types.append(_elem841)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17865,8 +17865,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 iter835 in self.tbl_types:
-        oprot.writeString(iter835)
+      for iter842 in self.tbl_types:
+        oprot.writeString(iter842)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17922,11 +17922,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype839, _size836) = iprot.readListBegin()
-          for _i840 in xrange(_size836):
-            _elem841 = TableMeta()
-            _elem841.read(iprot)
-            self.success.append(_elem841)
+          (_etype846, _size843) = iprot.readListBegin()
+          for _i847 in xrange(_size843):
+            _elem848 = TableMeta()
+            _elem848.read(iprot)
+            self.success.append(_elem848)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17949,8 +17949,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 iter842 in self.success:
-        iter842.write(oprot)
+      for iter849 in self.success:
+        iter849.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18074,10 +18074,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype846, _size843) = iprot.readListBegin()
-          for _i847 in xrange(_size843):
-            _elem848 = iprot.readString()
-            self.success.append(_elem848)
+          (_etype853, _size850) = iprot.readListBegin()
+          for _i854 in xrange(_size850):
+            _elem855 = iprot.readString()
+            self.success.append(_elem855)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18100,8 +18100,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 iter849 in self.success:
-        oprot.writeString(iter849)
+      for iter856 in self.success:
+        oprot.writeString(iter856)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18337,10 +18337,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype853, _size850) = iprot.readListBegin()
-          for _i854 in xrange(_size850):
-            _elem855 = iprot.readString()
-            self.tbl_names.append(_elem855)
+          (_etype860, _size857) = iprot.readListBegin()
+          for _i861 in xrange(_size857):
+            _elem862 = iprot.readString()
+            self.tbl_names.append(_elem862)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18361,8 +18361,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 iter856 in self.tbl_names:
-        oprot.writeString(iter856)
+      for iter863 in self.tbl_names:
+        oprot.writeString(iter863)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18414,11 +18414,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype860, _size857) = iprot.readListBegin()
-          for _i861 in xrange(_size857):
-            _elem862 = Table()
-            _elem862.read(iprot)
-            self.success.append(_elem862)
+          (_etype867, _size864) = iprot.readListBegin()
+          for _i868 in xrange(_size864):
+            _elem869 = Table()
+            _elem869.read(iprot)
+            self.success.append(_elem869)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18435,8 +18435,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 iter863 in self.success:
-        iter863.write(oprot)
+      for iter870 in self.success:
+        iter870.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18828,10 +18828,10 @@ class get_materialization_invalidation_info_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype867, _size864) = iprot.readListBegin()
-          for _i868 in xrange(_size864):
-            _elem869 = iprot.readString()
-            self.tbl_names.append(_elem869)
+          (_etype874, _size871) = iprot.readListBegin()
+          for _i875 in xrange(_size871):
+            _elem876 = iprot.readString()
+            self.tbl_names.append(_elem876)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18852,8 +18852,8 @@ class get_materialization_invalidation_info_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter870 in self.tbl_names:
-        oprot.writeString(iter870)
+      for iter877 in self.tbl_names:
+        oprot.writeString(iter877)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18914,12 +18914,12 @@ class get_materialization_invalidation_info_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype872, _vtype873, _size871 ) = iprot.readMapBegin()
-          for _i875 in xrange(_size871):
-            _key876 = iprot.readString()
-            _val877 = Materialization()
-            _val877.read(iprot)
-            self.success[_key876] = _val877
+          (_ktype879, _vtype880, _size878 ) = iprot.readMapBegin()
+          for _i882 in xrange(_size878):
+            _key883 = iprot.readString()
+            _val884 = Materialization()
+            _val884.read(iprot)
+            self.success[_key883] = _val884
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -18954,9 +18954,9 @@ class 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 kiter878,viter879 in self.success.items():
-        oprot.writeString(kiter878)
-        viter879.write(oprot)
+      for kiter885,viter886 in self.success.items():
+        oprot.writeString(kiter885)
+        viter886.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19122,10 +19122,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype883, _size880) = iprot.readListBegin()
-          for _i884 in xrange(_size880):
-            _elem885 = iprot.readString()
-            self.success.append(_elem885)
+          (_etype890, _size887) = iprot.readListBegin()
+          for _i891 in xrange(_size887):
+            _elem892 = iprot.readString()
+            self.success.append(_elem892)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19160,8 +19160,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 iter886 in self.success:
-        oprot.writeString(iter886)
+      for iter893 in self.success:
+        oprot.writeString(iter893)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20131,11 +20131,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype890, _size887) = iprot.readListBegin()
-          for _i891 in xrange(_size887):
-            _elem892 = Partition()
-            _elem892.read(iprot)
-            self.new_parts.append(_elem892)
+          (_etype897, _size894) = iprot.readListBegin()
+          for _i898 in xrange(_size894):
+            _elem899 = Partition()
+            _elem899.read(iprot)
+            self.new_parts.append(_elem899)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20152,8 +20152,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 iter893 in self.new_parts:
-        iter893.write(oprot)
+      for iter900 in self.new_parts:
+        iter900.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20311,11 +20311,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype897, _size894) = iprot.readListBegin()
-          for _i898 in xrange(_size894):
-            _elem899 = PartitionSpec()
-            _elem899.read(iprot)
-            self.new_parts.append(_elem899)
+          (_etype904, _size901) = iprot.readListBegin()
+          for _i905 in xrange(_size901):
+            _elem906 = PartitionSpec()
+            _elem906.read(iprot)
+            self.new_parts.append(_elem906)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20332,8 +20332,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 iter900 in self.new_parts:
-        iter900.write(oprot)
+      for iter907 in self.new_parts:
+        iter907.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20507,10 +20507,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype904, _size901) = iprot.readListBegin()
-          for _i905 in xrange(_size901):
-            _elem906 = iprot.readString()
-            self.part_vals.append(_elem906)
+          (_etype911, _size908) = iprot.readListBegin()
+          for _i912 in xrange(_size908):
+            _elem913 = iprot.readString()
+            self.part_vals.append(_elem913)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20535,8 +20535,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 iter907 in self.part_vals:
-        oprot.writeString(iter907)
+      for iter914 in self.part_vals:
+        oprot.writeString(iter914)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20889,10 +20889,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype911, _size908) = iprot.readListBegin()
-          for _i912 in xrange(_size908):
-            _elem913 = iprot.readString()
-            self.part_vals.append(_elem913)
+          (_etype918, _size915) = iprot.readListBegin()
+          for _i919 in xrange(_size915):
+            _elem920 = iprot.readString()
+            self.part_vals.append(_elem920)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20923,8 +20923,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 iter914 in self.part_vals:
-        oprot.writeString(iter914)
+      for iter921 in self.part_vals:
+        oprot.writeString(iter921)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -21519,10 +21519,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype918, _size915) = iprot.readListBegin()
-          for _i919 in xrange(_size915):
-            _elem920 = iprot.readString()
-            self.part_vals.append(_elem920)
+          (_etype925, _size922) = iprot.readListBegin()
+          for _i926 in xrange(_size922):
+            _elem927 = iprot.readString()
+            self.part_vals.append(_elem927)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21552,8 +21552,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 iter921 in self.part_vals:
-        oprot.writeString(iter921)
+      for iter928 in self.part_vals:
+        oprot.writeString(iter928)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -21726,10 +21726,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype925, _size922) = iprot.readListBegin()
-          for _i926 in xrange(_size922):
-            _elem927 = iprot.readString()
-            self.part_vals.append(_elem927)
+          (_etype932, _size929) = iprot.readListBegin()
+          for _i933 in xrange(_size929):
+            _elem934 = iprot.readString()
+            self.part_vals.append(_elem934)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21765,8 +21765,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 iter928 in self.part_vals:
-        oprot.writeString(iter928)
+      for iter935 in self.part_vals:
+        oprot.writeString(iter935)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -22503,10 +22503,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype932, _size929) = iprot.readListBegin()
-          for _i933 in xrange(_size929):
-            _elem934 = iprot.readString()
-            self.part_vals.append(_elem934)
+          (_etype939, _size936) = iprot.readListBegin()
+          for _i940 in xrange(_size936):
+            _elem941 = iprot.readString()
+            self.part_vals.append(_elem941)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22531,8 +22531,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 iter935 in self.part_vals:
-        oprot.writeString(iter935)
+      for iter942 in self.part_vals:
+        oprot.writeString(iter942)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22691,11 +22691,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype937, _vtype938, _size936 ) = iprot.readMapBegin()
-          for _i940 in xrange(_size936):
-            _key941 = iprot.readString()
-            _val942 = iprot.readString()
-            self.partitionSpecs[_key941] = _val942
+          (_ktype944, _vtype945, _size943 ) = iprot.readMapBegin()
+          for _i947 in xrange(_size943):
+            _key948 = iprot.readString()
+            _val949 = iprot.readString()
+            self.partitionSpecs[_key948] = _val949
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -22732,9 +22732,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 kiter943,viter944 in self.partitionSpecs.items():
-        oprot.writeString(kiter943)
-        oprot.writeString(viter944)
+      for kiter950,viter951 in self.partitionSpecs.items():
+        oprot.writeString(kiter950)
+        oprot.writeString(viter951)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -22939,11 +22939,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype946, _vtype947, _size945 ) = iprot.readMapBegin()
-          for _i949 in xrange(_size945):
-            _key950 = iprot.readString()
-            _val951 = iprot.readString()
-            self.partitionSpecs[_key950] = _val951
+          (_ktype953, _vtype954, _size952 ) = iprot.readMapBegin()
+          for _i956 in xrange(_size952):
+            _key957 = iprot.readString()
+            _val958 = iprot.readString()
+            self.partitionSpecs[_key957] = _val958
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -22980,9 +22980,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 kiter952,viter953 in self.partitionSpecs.items():
-        oprot.writeString(kiter952)
-        oprot.writeString(viter953)
+      for kiter959,viter960 in self.partitionSpecs.items():
+        oprot.writeString(kiter959)
+        oprot.writeString(viter960)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -23065,11 +23065,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype957, _size954) = iprot.readListBegin()
-          for _i958 in xrange(_size954):
-            _elem959 = Partition()
-            _elem959.read(iprot)
-            self.success.append(_elem959)
+          (_etype964, _size961) = iprot.readListBegin()
+          for _i965 in xrange(_size961):
+            _elem966 = Partition()
+            _elem966.read(iprot)
+            self.success.append(_elem966)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23110,8 +23110,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 iter960 in self.success:
-        iter960.write(oprot)
+      for iter967 in self.success:
+        iter967.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23205,10 +23205,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype964, _size961) = iprot.readListBegin()
-          for _i965 in xrange(_size961):
-            _elem966 = iprot.readString()
-            self.part_vals.append(_elem966)
+          (_etype971, _size968) = iprot.readListBegin()
+          for _i972 in xrange(_size968):
+            _elem973 = iprot.readString()
+            self.part_vals.append(_elem973)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23220,10 +23220,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype970, _size967) = iprot.readListBegin()
-          for _i971 in xrange(_size967):
-            _elem972 = iprot.readString()
-            self.group_names.append(_elem972)
+          (_etype977, _size974) = iprot.readListBegin()
+          for _i978 in xrange(_size974):
+            _elem979 = iprot.readString()
+            self.group_names.append(_elem979)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23248,8 +23248,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 iter973 in self.part_vals:
-        oprot.writeString(iter973)
+      for iter980 in self.part_vals:
+        oprot.writeString(iter980)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -23259,8 +23259,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 iter974 in self.group_names:
-        oprot.writeString(iter974)
+      for iter981 in self.group_names:
+        oprot.writeString(iter981)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23689,11 +23689,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype978, _size975) = iprot.readListBegin()
-          for _i979 in xrange(_size975):
-            _elem980 = Partition()
-            _elem980.read(iprot)
-            self.success.append(_elem980)
+          (_etype985, _size982) = iprot.readListBegin()
+          for _i986 in xrange(_size982):
+            _elem987 = Partition()
+            _elem987.read(iprot)
+            self.success.append(_elem987)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23722,8 +23722,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 iter981 in self.success:
-        iter981.write(oprot)
+      for iter988 in self.success:
+        iter988.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23817,10 +23817,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype985, _size982) = iprot.readListBegin()
-          for _i986 in xrange(_size982):
-            _elem987 = iprot.readString()
-            self.group_names.append(_elem987)
+          (_etype992, _size989) = iprot.readListBegin()
+          for _i993 in xrange(_size989):
+            _elem994 = iprot.readString()
+            self.group_names.append(_elem994)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23853,8 +23853,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 iter988 in self.group_names:
-        oprot.writeString(iter988)
+      for iter995 in self.group_names:
+        oprot.writeString(iter995)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23915,11 +23915,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype992, _size989) = iprot.readListBegin()
-          for _i993 in xrange(_size989):
-            _elem994 = Partition()
-            _elem994.read(iprot)
-            self.success.append(_elem994)
+          (_etype999, _size996) = iprot.readListBegin()
+          for _i1000 in xrange(_size996):
+            _elem1001 = Partition()
+            _elem1001.read(iprot)
+            self.success.append(_elem1001)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23948,8 +23948,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 iter995 in self.success:
-        iter995.write(oprot)
+      for iter1002 in self.success:
+        iter1002.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24107,11 +24107,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype999, _size996) = iprot.readListBegin()
-          for _i1000 in xrange(_size996):
-            _elem1001 = PartitionSpec()
-            _elem1001.read(iprot)
-            self.success.append(_elem1001)
+          (_etype1006, _size1003) = iprot.readListBegin()
+          for _i1007 in xrange(_size1003):
+            _elem1008 = PartitionSpec()
+            _elem1008.read(iprot)
+            self.success.append(_elem1008)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24140,8 +24140,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 iter1002 in self.success:
-        iter1002.write(oprot)
+      for iter1009 in self.success:
+        iter1009.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24299,10 +24299,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1006, _size1003) = iprot.readListBegin()
-          for _i1007 in xrange(_size1003):
-            _elem1008 = iprot.readString()
-            self.success.append(_elem1008)
+          (_etype1013, _size1010) = iprot.readListBegin()
+          for _i1014 in xrange(_size1010):
+            _elem1015 = iprot.readString()
+            self.success.append(_elem1015)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24331,8 +24331,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 iter1009 in self.success:
-        oprot.writeString(iter1009)
+      for iter1016 in self.success:
+        oprot.writeString(iter1016)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24572,10 +24572,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1013, _size1010) = iprot.readListBegin()
-          for _i1014 in xrange(_size1010):
-            _elem1015 = iprot.readString()
-            self.part_vals.append(_elem1015)
+          (_etype1020, _size1017) = iprot.readListBegin()
+          for _i1021 in xrange(_size1017):
+            _elem1022 = iprot.readString()
+            self.part_vals.append(_elem1022)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24605,8 +24605,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 iter1016 in self.part_vals:
-        oprot.writeString(iter1016)
+      for iter1023 in self.part_vals:
+        oprot.writeString(iter1023)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -24670,11 +24670,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1020, _size1017) = iprot.readListBegin()
-          for _i1021 in xrange(_size1017):
-            _elem1022 = Partition()
-            _elem1022.read(iprot)
-            self.success.append(_elem1022)
+          (_etype1027, _size1024) = iprot.readListBegin()
+          for _i1028 in xrange(_size1024):
+            _elem1029 = Partition()
+            _elem1029.read(iprot)
+            self.success.append(_elem1029)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24703,8 +24703,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 iter1023 in self.success:
-        iter1023.write(oprot)
+      for iter1030 in self.success:
+        iter1030.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24791,10 +24791,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1027, _size1024) = iprot.readListBegin()
-          for _i1028 in xrange(_size1024):
-            _elem1029 = iprot.readString()
-            self.part_vals.append(_elem1029)
+          (_etype1034, _size1031) = iprot.readListBegin()
+          for _i1035 in xrange(_size1031):
+            _elem1036 = iprot.readString()
+            self.part_vals.append(_elem1036)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24811,10 +24811,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1033, _size1030) = iprot.readListBegin()
-          for _i1034 in xrange(_size1030):
-            _elem1035 = iprot.readString()
-            self.group_names.append(_elem1035)
+          (_etype1040, _size1037) = iprot.readListBegin()
+          for _i1041 in xrange(_size1037):
+            _elem1042 = iprot.readString()
+            self.group_names.append(_elem1042)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24839,8 +24839,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 iter1036 in self.part_vals:
-        oprot.writeString(iter1036)
+      for iter1043 in self.part_vals:
+        oprot.writeString(iter1043)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -24854,8 +24854,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 iter1037 in self.group_names:
-        oprot.writeString(iter1037)
+      for iter1044 in self.group_names:
+        oprot.writeString(iter1044)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -24917,11 +24917,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1041, _size1038) = iprot.readListBegin()
-          for _i1042 in xrange(_size1038):
-            _elem1043 = Partition()
-            _elem1043.read(iprot)
-            self.success.append(_elem1043)
+          (_etype1048, _size1045) = iprot.readListBegin()
+          for _i1049 in xrange(_size1045):
+            _elem1050 = Partition()
+            _elem1050.read(iprot)
+            self.success.append(_elem1050)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24950,8 +24950,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 iter1044 in self.success:
-        iter1044.write(oprot)
+      for iter1051 in self.success:
+        iter1051.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25032,10 +25032,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1048, _size1045) = iprot.readListBegin()
-          for _i1049 in xrange(_size1045):
-            _elem1050 = iprot.readString()
-            self.part_vals.append(_elem1050)
+          (_etype1055, _size1052) = iprot.readListBegin()
+          for _i1056 in xrange(_size1052):
+            _elem1057 = iprot.readString()
+            self.part_vals.append(_elem1057)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25065,8 +25065,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 iter1051 in self.part_vals:
-        oprot.writeString(iter1051)
+      for iter1058 in self.part_vals:
+        oprot.writeString(iter1058)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -25130,10 +25130,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1055, _size1052) = iprot.readListBegin()
-          for _i1056 in xrange(_size1052):
-            _elem1057 = iprot.readString()
-            self.success.append(_elem1057)
+          (_etype1062, _size1059) = iprot.readListBegin()
+          for _i1063 in xrange(_size1059):
+            _elem1064 = iprot.readString()
+            self.success.append(_elem1064)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25162,8 +25162,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 iter1058 in self.success:
-        oprot.writeString(iter1058)
+      for iter1065 in self.success:
+        oprot.writeString(iter1065)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25334,11 +25334,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1062, _size1059) = iprot.readListBegin()
-          for _i1063 in xrange(_size1059):
-            _elem1064 = Partition()
-            _elem1064.read(iprot)
-            self.success.append(_elem1064)
+          (_etype1069, _size1066) = iprot.readListBegin()
+          for _i1070 in xrange(_size1066):
+            _elem1071 = Partition()
+            _elem1071.read(iprot)
+            self.success.append(_elem1071)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25367,8 +25367,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 iter1065 in self.success:
-        iter1065.write(oprot)
+      for iter1072 in self.success:
+        iter1072.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25539,11 +25539,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1069, _size1066) = iprot.readListBegin()
-          for _i1070 in xrange(_size1066):
-            _elem1071 = PartitionSpec()
-            _elem1071.read(iprot)
-            self.success.append(_elem1071)
+          (_etype1076, _size1073) = iprot.readListBegin()
+          for _i1077 in xrange(_size1073):
+            _elem1078 = PartitionSpec()
+            _elem1078.read(iprot)
+            self.success.append(_elem1078)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25572,8 +25572,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 iter1072 in self.success:
-        iter1072.write(oprot)
+      for iter1079 in self.success:
+        iter1079.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25993,10 +25993,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype1076, _size1073) = iprot.readListBegin()
-          for _i1077 in xrange(_size1073):
-            _elem1078 = iprot.readString()
-            self.names.append(_elem1078)
+          (_etype1083, _size1080) = iprot.readListBegin()
+          for _i1084 in xrange(_size1080):
+            _elem1085 = iprot.readString()
+            self.names.append(_elem1085)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26021,8 +26021,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 iter1079 in self.names:
-        oprot.writeString(iter1079)
+      for iter1086 in self.names:
+        oprot.writeString(iter1086)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26081,11 +26081,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1083, _size1080) = iprot.readListBegin()
-          for _i1084 in xrange(_size1080):
-            _elem1085 = Partition()
-            _elem1085.read(iprot)
-            self.success.append(_elem1085)
+          (_etype1090, _size1087) = iprot.readListBegin()
+          for _i1091 in xrange(_size1087):
+            _elem1092 = Partition()
+            _elem1092.read(iprot)
+            self.success.append(_elem1092)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26114,8 +26114,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 iter1086 in self.success:
-        iter1086.write(oprot)
+      for iter1093 in self.success:
+        iter1093.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26365,11 +26365,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1090, _size1087) = iprot.readListBegin()
-          for _i1091 in xrange(_size1087):
-            _elem1092 = Partition()
-            _elem1092.read(iprot)
-            self.new_parts.append(_elem1092)
+          (_etype1097, _size1094) = iprot.readListBegin()
+          for _i1098 in xrange(_size1094):
+            _elem1099 = Partition()
+            _elem1099.read(iprot)
+            self.new_parts.append(_elem1099)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26394,8 +26394,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 iter1093 in self.new_parts:
-        iter1093.write(oprot)
+      for iter1100 in self.new_parts:
+        iter1100.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26548,11 +26548,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1097, _size1094) = iprot.readListBegin()
-          for _i1098 in xrange(_size1094):
-            _elem1099 = Partition()
-            _elem1099.read(iprot)
-            self.new_parts.append(_elem1099)
+          (_etype1104, _size1101) = iprot.readListBegin()
+          for _i1105 in xrange(_size1101):
+            _elem1106 = Partition()
+            _elem1106.read(iprot)
+            self.new_parts.append(_elem1106)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26583,8 +26583,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 iter1100 in self.new_parts:
-        iter1100.write(oprot)
+      for iter1107 in self.new_parts:
+        iter1107.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -26928,10 +26928,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1104, _size1101) = iprot.readListBegin()
-          for _i1105 in xrange(_size1101):
-            _elem1106 = iprot.readString()
-            self.part_vals.append(_elem1106)
+          (_etype1111, _size1108) = iprot.readListBegin()
+          for _i1112 in xrange(_size1108):
+            _elem1113 = iprot.readString()
+            self.part_vals.append(_elem1113)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26962,8 +26962,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 iter1107 in self.part_vals:
-        oprot.writeString(iter1107)
+      for iter1114 in self.part_vals:
+        oprot.writeString(iter1114)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -27105,10 +27105,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1111, _size1108) = iprot.readListBegin()
-          for _i1112 in xrange(_size1108):
-            _elem1113 = iprot.readString()
-            self.part_vals.append(_elem1113)
+          (_etype1118, _size1115) = iprot.readListBegin()
+          for _i1119 in xrange(_size1115):
+            _elem1120 = iprot.readString()
+            self.part_vals.append(_elem1120)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27130,8 +27130,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 iter1114 in self.part_vals:
-        oprot.writeString(iter1114)
+      for iter1121 in self.part_vals:
+        oprot.writeString(iter1121)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -27489,10 +27489,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1118, _size1115) = iprot.readListBegin()
-          for _i1119 in xrange(_size1115):
-            _elem1120 = iprot.readString()
-            self.success.append(_elem1120)
+          (_etype1125, _size1122) = iprot.readListBegin()
+          for _i1126 in xrange(_size1122):
+            _elem1127 = iprot.readString()
+            self.success.append(_elem1127)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27515,8 +27515,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 iter1121 in self.success:
-        oprot.writeString(iter1121)
+      for iter1128 in self.success:
+        oprot.writeString(iter1128)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27640,11 +27640,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype1123, _vtype1124, _size1122 ) = iprot.readMapBegin()
-          for _i1126 in xrange(_size1122):
-            _key1127 = iprot.readString()
-            _val1128 = iprot.readString()
-            self.success[_key1127] = _val1128
+          (_ktype1130, _vtype1131, _size1129 ) = iprot.readMapBegin()
+          for _i1133 in xrange(_size1129):
+            _key1134 = iprot.readString()
+            _val1135 = iprot.readString()
+            self.success[_key1134] = _val1135
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -27667,9 +27667,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 kiter1129,viter1130 in self.success.items():
-        oprot.writeString(kiter1129)
-        oprot.writeString(viter1130)
+      for kiter1136,viter1137 in self.success.items():
+        oprot.writeString(kiter1136)
+        oprot.writeString(viter1137)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27745,11 +27745,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1132, _vtype1133, _size1131 ) = iprot.readMapBegin()
-          for _i1135 in xrange(_size1131):
-            _key1136 = iprot.readString()
-            _val1137 = iprot.readString()
-            self.part_vals[_key1136] = _val1137
+          (_ktype1139, _vtype1140, _size1138 ) = iprot.readMapBegin()
+          for _i1142 in xrange(_size1138):
+            _key1143 = iprot.readString()
+            _val1144 = iprot.readString()
+            self.part_vals[_key1143] = _val1144
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -27779,9 +27779,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 kiter1138,viter1139 in self.part_vals.items():
-        oprot.writeString(kiter1138)
-        oprot.writeString(viter1139)
+      for kiter1145,viter1146 in self.part_vals.items():
+        oprot.writeString(kiter1145)
+        oprot.writeString(viter1146)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -27995,11 +27995,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1141, _vtype1142, _size1140 ) = iprot.readMapBegin()
-          for _i1144 in xrange(_size1140):
-            _key1145 = iprot.readString()
-            _val1146 = iprot.readString()
-            self.part_vals[_key1145] = _val1146
+          (_ktype1148, _vtype1149, _size1147 ) = iprot.readMapBegin()
+          for _i1151 in xrange(_size1147):
+            _key1152 = iprot.readString()
+            _val1153 = iprot.readString()
+            self.part_vals[_key1152] = _val1153
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -28029,9 +28029,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 kiter1147,viter1148 in self.part_vals.items():
-        oprot.writeString(kiter1147)
-        oprot.writeString(viter1148)
+      for kiter1154,viter1155 in self.part_vals.items():
+        oprot.writeString(kiter1154)
+        oprot.writeString(viter1155)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -29086,11 +29086,11 @@ class get_indexes_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1152, _size1149) = iprot.readListBegin()
-          for _i1153 in xrange(_size1149):
-            _elem1154 = Index()
-            _elem1154.read(iprot)
-            self.success.append(_elem1154)
+          (_etype1159, _size1156) = iprot.readListBegin()
+          for _i1160 in xrange(_size1156):
+            _elem1161 = Index()
+            _elem1161.read(iprot)
+            self.success.append(_elem1161)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29119,8 +29119,8 @@ class get_indexes_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1155 in self.success:
-        iter1155.write(oprot)
+      for iter1162 in self.success:
+        iter1162.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29275,10 +29275,10 @@ class get_index_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1159, _size1156) = iprot.readListBegin()
-          for _i1160 in xrange(_size1156):
-            _elem1161 = iprot.readString()
-            self.success.append(_elem1161)
+          (_etype1166, _size1163) = iprot.readListBegin()
+          for _i1167 in xrange(_size1163):
+            _elem1168 = iprot.readString()
+            self.success.append(_elem1168)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29301,8 +29301,8 @@ class get_index_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1162 in self.success:
-        oprot.writeString(iter1162)
+      for iter1169 in self.success:
+        oprot.writeString(iter1169)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -32486,10 +32486,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1166, _size1163) = iprot.readListBegin()
-          for _i1167 in xrange(_size1163):
-            _elem1168 = iprot.readString()
-            self.success.append(_elem1168)
+          (_etype1173, _size1170) = iprot.readListBegin()
+          for _i1174 in xrange(_size1170):
+            _elem1175 = iprot.readString()
+            self.success.append(_elem1175)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -32512,8 +32512,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 iter1169 in self.success:
-        oprot.writeString(iter1169)
+      for iter1176 in self.success:
+        oprot.writeString(iter1176)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -33201,10 +33201,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1173, _size1170) = iprot.readListBegin()
-          for _i1174 in xrange(_size1170):
-            _elem1175 = iprot.readString()
-            self.success.append(_elem1175)
+          (_etype1180, _size1177) = iprot.readListBegin()
+          for _i1181 in xrange(_size1177):
+            _elem1182 = iprot.readString()
+            self.success.append(_elem1182)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -33227,8 +33227,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 iter1176 in self.success:
-        oprot.writeString(iter1176)
+      for iter1183 in self.success:
+        oprot.writeString(iter1183)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -33742,11 +33742,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1180, _size1177) = iprot.readListBegin()
-          for _i1181 in xrange(_size1177):
-            _elem1182 = Role()
-            _elem1182.read(iprot)
-            self.success.append(_elem1182)
+          (_etype1187, _size1184) = iprot.readListBegin()
+          for _i1188 in xrange(_size1184):
+            _elem1189 = Role()
+            _elem1189.read(iprot)
+            self.success.append(_elem1189)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -33769,8 +33769,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 iter1183 in self.success:
-        iter1183.write(oprot)
+      for iter1190 in self.success:
+        iter1190.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -34279,10 +34279,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1187, _size1184) = iprot.readListBegin()
-          for _i1188 in xrange(_size1184):
-            _elem1189 = iprot.readString()
-            self.group_names.append(_elem1189)
+          (_etype1194, _size1191) = iprot.readListBegin()
+          for _i1195 in xrange(_size1191):
+            _elem1196 = iprot.readString()
+            self.group_names.append(_elem1196)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -34307,8 +34307,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 iter1190 in self.group_names:
-        oprot.writeString(iter1190)
+      for iter1197 in self.group_names:
+        oprot.writeString(iter1197)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -34535,11 +34535,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1194, _size1191) = iprot.readListBegin()
-          for _i1195 in xrange(_size1191):
-            _elem1196 = HiveObjectPrivilege()
-            _elem1196.read(iprot)
-            self.success.append(_elem1196)
+          (_etype1201, _size1198) = iprot.readListBegin()
+          for _i1202 in xrange(_size1198):
+            _elem1203 = HiveObjectPrivilege()
+            _elem1203.read(iprot)
+            self.success.append(_elem1203)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -34562,8 +34562,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 iter1197 in self.success:
-        iter1197.write(oprot)
+      for iter1204 in self.success:
+        iter1204.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -35061,10 +35061,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1201, _size1198) = iprot.readListBegin()
-          for _i1202 in xrange(_size1198):
-            _elem1203 = iprot.readString()
-            self.group_names.append(_elem1203)
+          (_etype1208, _size1205) = iprot.readListBegin()
+          for _i1209 in xrange(_size1205):
+            _elem1210 = iprot.readString()
+            self.group_names.append(_elem1210)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -35085,8 +35085,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 iter1204 in self.group_names:
-        oprot.writeString(iter1204)
+      for iter1211 in self.group_names:
+        oprot.writeString(iter1211)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -35141,10 +35141,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1208, _size1205) = iprot.readListBegin()
-          for _i1209 in xrange(_size1205):
-            _elem1210 = iprot.readString()
-            self.success.append(_elem1210)
+          (_etype1215, _size1212) = iprot.readListBegin()
+          for _i1216 in xrange(_size1212):
+            _elem1217 = iprot.readString()
+            self.success.append(_elem1217)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -35167,8 +35167,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 iter1211 in self.success:
-        oprot.writeString(iter1211)
+      for iter1218 in self.success:
+        oprot.writeString(iter1218)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -36100,10 +36100,10 @@ class get_all_token_identifiers_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1215, _size1212) = iprot.readListBegin()
-          for _i1216 in xrange(_size1212):
-            _elem1217 = iprot.readString()
-            self.success.append(_elem1217)
+          (_etype1222, _size1219) = iprot.readListBegin()
+          for _i1223 in xrange(_size1219):
+            _elem1224 = iprot.readString()
+            self.success.append(_elem1224)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36120,8 +36120,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 iter1218 in self.success:
-        oprot.writeString(iter1218)
+      for iter1225 in self.success:
+        oprot.writeString(iter1225)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -36648,10 +36648,10 @@ class get_master_keys_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1222, _size1219) = iprot.readListBegin()
-          for _i1223 in xrange(_size1219):
-            _elem1224 = iprot.readString()
-            self.success.append(_elem1224)
+          (_etype1229, _size1226) = iprot.readListBegin()
+          for _i1230 in xrange(_size1226):
+            _elem1231 = iprot.readString()
+            self.success.append(_elem1231)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36668,8 +36668,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 iter1225 in self.success:
-        oprot.writeString(iter1225)
+      for iter1232 in self.success:
+        oprot.writeString(iter1232)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -38910,20 +38910,20 @@ class get_last_completed_transaction_for_tables_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.db_names = []
-          (_etype1229, _size1226) = iprot.readListBegin()
-          for _i1230 in xrange(_size1226):
-            _elem1231 = iprot.readString()
-            self.db_names.append(_elem1231)
+          (_etype1236, _size1233) = iprot.readListBegin()
+          for _i1237 in xrange(_size1233):
+            _elem1238 = iprot.readString()
+            self.db_names.append(_elem1238)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.table_names = []
-          (_etype1235, _size1232) = iprot.readListBegin()
-          for _i1236 in xrange(_size1232):
-            _elem1237 = iprot.readString()
-            self.table_names.append(_elem1237)
+          (_etype1242, _size1239) = iprot.readListBegin()
+          for _i1243 in xrange(_size1239):
+            _elem1244 = iprot.readString()
+            self.table_names.append(_elem1244)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -38946,15 +38946,15 @@ class get_last_completed_transaction_for_tables_args:
     if self.db_names is not None:
       oprot.writeFieldBegin('db_names', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.db_names))
-      for iter1238 in self.db_names:
-        oprot.writeString(iter1238)
+      for iter1245 in self.db_names:
+        oprot.writeString(iter1245)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.table_names is not None:
       oprot.writeFieldBegin('table_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.table_names))
-      for iter1239 in self.table_names:
-        oprot.writeString(iter1239)
+      for iter1246 in self.table_names:
+        oprot.writeString(iter1246)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.txns_snapshot is not None:
@@ -39011,11 +39011,11 @@ class get_last_completed_transaction_for_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1243, _size1240) = iprot.readListBegin()
-          for _i1244 in xrange(_size1240):
-            _elem1245 = BasicTxnInfo()
-            _elem1245.read(iprot)
-            self.success.append(_elem1245)
+          (_etype1250, _size1247) = iprot.readListBegin()
+          for _i1251 in xrange(_size1247):
+            _elem1252 = BasicTxnInfo()
+            _elem1252.read(iprot)
+            self.success.append(_elem1252)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -39032,8 +39032,8 @@ class get_last_completed_transaction_for_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1246 in self.success:
-        iter1246.write(oprot)
+      for iter1253 in self.success:
+        iter1253.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index c3be4c4..25e9a88 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -16520,15 +16520,18 @@ class WMValidateResourcePlanResponse:
   """
   Attributes:
    - errors
+   - warnings
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.LIST, 'errors', (TType.STRING,None), None, ), # 1
+    (2, TType.LIST, 'warnings', (TType.STRING,None), None, ), # 2
   )
 
-  def __init__(self, errors=None,):
+  def __init__(self, errors=None, warnings=None,):
     self.errors = errors
+    self.warnings = warnings
 
   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:
@@ -16549,6 +16552,16 @@ class WMValidateResourcePlanResponse:
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.LIST:
+          self.warnings = []
+          (_etype717, _size714) = iprot.readListBegin()
+          for _i718 in xrange(_size714):
+            _elem719 = iprot.readString()
+            self.warnings.append(_elem719)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -16562,8 +16575,15 @@ class WMValidateResourcePlanResponse:
     if self.errors is not None:
       oprot.writeFieldBegin('errors', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.errors))
-      for iter714 in self.errors:
-        oprot.writeString(iter714)
+      for iter720 in self.errors:
+        oprot.writeString(iter720)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.warnings is not None:
+      oprot.writeFieldBegin('warnings', TType.LIST, 2)
+      oprot.writeListBegin(TType.STRING, len(self.warnings))
+      for iter721 in self.warnings:
+        oprot.writeString(iter721)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16576,6 +16596,7 @@ class WMValidateResourcePlanResponse:
   def __hash__(self):
     value = 17
     value = (value * 31) ^ hash(self.errors)
+    value = (value * 31) ^ hash(self.warnings)
     return value
 
   def __repr__(self):
@@ -17139,11 +17160,11 @@ class WMGetTriggersForResourePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype718, _size715) = iprot.readListBegin()
-          for _i719 in xrange(_size715):
-            _elem720 = WMTrigger()
-            _elem720.read(iprot)
-            self.triggers.append(_elem720)
+          (_etype725, _size722) = iprot.readListBegin()
+          for _i726 in xrange(_size722):
+            _elem727 = WMTrigger()
+            _elem727.read(iprot)
+            self.triggers.append(_elem727)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17160,8 +17181,8 @@ class WMGetTriggersForResourePlanResponse:
     if self.triggers is not None:
       oprot.writeFieldBegin('triggers', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.triggers))
-      for iter721 in self.triggers:
-        iter721.write(oprot)
+      for iter728 in self.triggers:
+        iter728.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 439553f..3a11a05 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -3729,9 +3729,11 @@ end
 class WMValidateResourcePlanResponse
   include ::Thrift::Struct, ::Thrift::Struct_Union
   ERRORS = 1
+  WARNINGS = 2
 
   FIELDS = {
-    ERRORS => {:type => ::Thrift::Types::LIST, :name => 'errors', :element => {:type => ::Thrift::Types::STRING}, :optional => true}
+    ERRORS => {:type => ::Thrift::Types::LIST, :name => 'errors', :element => {:type => ::Thrift::Types::STRING}, :optional => true},
+    WARNINGS => {:type => ::Thrift::Types::LIST, :name => 'warnings', :element => {:type => ::Thrift::Types::STRING}, :optional => true}
   }
 
   def struct_fields; FIELDS; end


[5/8] hive git commit: HIVE-18231 : validate resource plan - part 2 - validate action and trigger expressions (Harish Jaiprakash, reviewed by Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/90d236af/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index d4d0be3..aaca408 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -34145,13 +34145,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list820 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list820.size);
-                  String _elem821;
-                  for (int _i822 = 0; _i822 < _list820.size; ++_i822)
+                  org.apache.thrift.protocol.TList _list828 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list828.size);
+                  String _elem829;
+                  for (int _i830 = 0; _i830 < _list828.size; ++_i830)
                   {
-                    _elem821 = iprot.readString();
-                    struct.success.add(_elem821);
+                    _elem829 = iprot.readString();
+                    struct.success.add(_elem829);
                   }
                   iprot.readListEnd();
                 }
@@ -34186,9 +34186,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 _iter823 : struct.success)
+            for (String _iter831 : struct.success)
             {
-              oprot.writeString(_iter823);
+              oprot.writeString(_iter831);
             }
             oprot.writeListEnd();
           }
@@ -34227,9 +34227,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter824 : struct.success)
+            for (String _iter832 : struct.success)
             {
-              oprot.writeString(_iter824);
+              oprot.writeString(_iter832);
             }
           }
         }
@@ -34244,13 +34244,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list825 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list825.size);
-            String _elem826;
-            for (int _i827 = 0; _i827 < _list825.size; ++_i827)
+            org.apache.thrift.protocol.TList _list833 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list833.size);
+            String _elem834;
+            for (int _i835 = 0; _i835 < _list833.size; ++_i835)
             {
-              _elem826 = iprot.readString();
-              struct.success.add(_elem826);
+              _elem834 = iprot.readString();
+              struct.success.add(_elem834);
             }
           }
           struct.setSuccessIsSet(true);
@@ -34904,13 +34904,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list828 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list828.size);
-                  String _elem829;
-                  for (int _i830 = 0; _i830 < _list828.size; ++_i830)
+                  org.apache.thrift.protocol.TList _list836 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list836.size);
+                  String _elem837;
+                  for (int _i838 = 0; _i838 < _list836.size; ++_i838)
                   {
-                    _elem829 = iprot.readString();
-                    struct.success.add(_elem829);
+                    _elem837 = iprot.readString();
+                    struct.success.add(_elem837);
                   }
                   iprot.readListEnd();
                 }
@@ -34945,9 +34945,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 _iter831 : struct.success)
+            for (String _iter839 : struct.success)
             {
-              oprot.writeString(_iter831);
+              oprot.writeString(_iter839);
             }
             oprot.writeListEnd();
           }
@@ -34986,9 +34986,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter832 : struct.success)
+            for (String _iter840 : struct.success)
             {
-              oprot.writeString(_iter832);
+              oprot.writeString(_iter840);
             }
           }
         }
@@ -35003,13 +35003,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list833 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list833.size);
-            String _elem834;
-            for (int _i835 = 0; _i835 < _list833.size; ++_i835)
+            org.apache.thrift.protocol.TList _list841 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list841.size);
+            String _elem842;
+            for (int _i843 = 0; _i843 < _list841.size; ++_i843)
             {
-              _elem834 = iprot.readString();
-              struct.success.add(_elem834);
+              _elem842 = iprot.readString();
+              struct.success.add(_elem842);
             }
           }
           struct.setSuccessIsSet(true);
@@ -39616,16 +39616,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map836 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map836.size);
-                  String _key837;
-                  Type _val838;
-                  for (int _i839 = 0; _i839 < _map836.size; ++_i839)
+                  org.apache.thrift.protocol.TMap _map844 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map844.size);
+                  String _key845;
+                  Type _val846;
+                  for (int _i847 = 0; _i847 < _map844.size; ++_i847)
                   {
-                    _key837 = iprot.readString();
-                    _val838 = new Type();
-                    _val838.read(iprot);
-                    struct.success.put(_key837, _val838);
+                    _key845 = iprot.readString();
+                    _val846 = new Type();
+                    _val846.read(iprot);
+                    struct.success.put(_key845, _val846);
                   }
                   iprot.readMapEnd();
                 }
@@ -39660,10 +39660,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> _iter840 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter848 : struct.success.entrySet())
             {
-              oprot.writeString(_iter840.getKey());
-              _iter840.getValue().write(oprot);
+              oprot.writeString(_iter848.getKey());
+              _iter848.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -39702,10 +39702,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter841 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter849 : struct.success.entrySet())
             {
-              oprot.writeString(_iter841.getKey());
-              _iter841.getValue().write(oprot);
+              oprot.writeString(_iter849.getKey());
+              _iter849.getValue().write(oprot);
             }
           }
         }
@@ -39720,16 +39720,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map842 = 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*_map842.size);
-            String _key843;
-            Type _val844;
-            for (int _i845 = 0; _i845 < _map842.size; ++_i845)
+            org.apache.thrift.protocol.TMap _map850 = 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*_map850.size);
+            String _key851;
+            Type _val852;
+            for (int _i853 = 0; _i853 < _map850.size; ++_i853)
             {
-              _key843 = iprot.readString();
-              _val844 = new Type();
-              _val844.read(iprot);
-              struct.success.put(_key843, _val844);
+              _key851 = iprot.readString();
+              _val852 = new Type();
+              _val852.read(iprot);
+              struct.success.put(_key851, _val852);
             }
           }
           struct.setSuccessIsSet(true);
@@ -40764,14 +40764,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list846 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list846.size);
-                  FieldSchema _elem847;
-                  for (int _i848 = 0; _i848 < _list846.size; ++_i848)
+                  org.apache.thrift.protocol.TList _list854 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list854.size);
+                  FieldSchema _elem855;
+                  for (int _i856 = 0; _i856 < _list854.size; ++_i856)
                   {
-                    _elem847 = new FieldSchema();
-                    _elem847.read(iprot);
-                    struct.success.add(_elem847);
+                    _elem855 = new FieldSchema();
+                    _elem855.read(iprot);
+                    struct.success.add(_elem855);
                   }
                   iprot.readListEnd();
                 }
@@ -40824,9 +40824,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 _iter849 : struct.success)
+            for (FieldSchema _iter857 : struct.success)
             {
-              _iter849.write(oprot);
+              _iter857.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -40881,9 +40881,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter850 : struct.success)
+            for (FieldSchema _iter858 : struct.success)
             {
-              _iter850.write(oprot);
+              _iter858.write(oprot);
             }
           }
         }
@@ -40904,14 +40904,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list851 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list851.size);
-            FieldSchema _elem852;
-            for (int _i853 = 0; _i853 < _list851.size; ++_i853)
+            org.apache.thrift.protocol.TList _list859 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list859.size);
+            FieldSchema _elem860;
+            for (int _i861 = 0; _i861 < _list859.size; ++_i861)
             {
-              _elem852 = new FieldSchema();
-              _elem852.read(iprot);
-              struct.success.add(_elem852);
+              _elem860 = new FieldSchema();
+              _elem860.read(iprot);
+              struct.success.add(_elem860);
             }
           }
           struct.setSuccessIsSet(true);
@@ -42065,14 +42065,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list854 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list854.size);
-                  FieldSchema _elem855;
-                  for (int _i856 = 0; _i856 < _list854.size; ++_i856)
+                  org.apache.thrift.protocol.TList _list862 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list862.size);
+                  FieldSchema _elem863;
+                  for (int _i864 = 0; _i864 < _list862.size; ++_i864)
                   {
-                    _elem855 = new FieldSchema();
-                    _elem855.read(iprot);
-                    struct.success.add(_elem855);
+                    _elem863 = new FieldSchema();
+                    _elem863.read(iprot);
+                    struct.success.add(_elem863);
                   }
                   iprot.readListEnd();
                 }
@@ -42125,9 +42125,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 _iter857 : struct.success)
+            for (FieldSchema _iter865 : struct.success)
             {
-              _iter857.write(oprot);
+              _iter865.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -42182,9 +42182,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter858 : struct.success)
+            for (FieldSchema _iter866 : struct.success)
             {
-              _iter858.write(oprot);
+              _iter866.write(oprot);
             }
           }
         }
@@ -42205,14 +42205,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list859 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list859.size);
-            FieldSchema _elem860;
-            for (int _i861 = 0; _i861 < _list859.size; ++_i861)
+            org.apache.thrift.protocol.TList _list867 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list867.size);
+            FieldSchema _elem868;
+            for (int _i869 = 0; _i869 < _list867.size; ++_i869)
             {
-              _elem860 = new FieldSchema();
-              _elem860.read(iprot);
-              struct.success.add(_elem860);
+              _elem868 = new FieldSchema();
+              _elem868.read(iprot);
+              struct.success.add(_elem868);
             }
           }
           struct.setSuccessIsSet(true);
@@ -43257,14 +43257,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list862 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list862.size);
-                  FieldSchema _elem863;
-                  for (int _i864 = 0; _i864 < _list862.size; ++_i864)
+                  org.apache.thrift.protocol.TList _list870 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list870.size);
+                  FieldSchema _elem871;
+                  for (int _i872 = 0; _i872 < _list870.size; ++_i872)
                   {
-                    _elem863 = new FieldSchema();
-                    _elem863.read(iprot);
-                    struct.success.add(_elem863);
+                    _elem871 = new FieldSchema();
+                    _elem871.read(iprot);
+                    struct.success.add(_elem871);
                   }
                   iprot.readListEnd();
                 }
@@ -43317,9 +43317,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 _iter865 : struct.success)
+            for (FieldSchema _iter873 : struct.success)
             {
-              _iter865.write(oprot);
+              _iter873.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -43374,9 +43374,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter866 : struct.success)
+            for (FieldSchema _iter874 : struct.success)
             {
-              _iter866.write(oprot);
+              _iter874.write(oprot);
             }
           }
         }
@@ -43397,14 +43397,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list867 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list867.size);
-            FieldSchema _elem868;
-            for (int _i869 = 0; _i869 < _list867.size; ++_i869)
+            org.apache.thrift.protocol.TList _list875 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list875.size);
+            FieldSchema _elem876;
+            for (int _i877 = 0; _i877 < _list875.size; ++_i877)
             {
-              _elem868 = new FieldSchema();
-              _elem868.read(iprot);
-              struct.success.add(_elem868);
+              _elem876 = new FieldSchema();
+              _elem876.read(iprot);
+              struct.success.add(_elem876);
             }
           }
           struct.setSuccessIsSet(true);
@@ -44558,14 +44558,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list870 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list870.size);
-                  FieldSchema _elem871;
-                  for (int _i872 = 0; _i872 < _list870.size; ++_i872)
+                  org.apache.thrift.protocol.TList _list878 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list878.size);
+                  FieldSchema _elem879;
+                  for (int _i880 = 0; _i880 < _list878.size; ++_i880)
                   {
-                    _elem871 = new FieldSchema();
-                    _elem871.read(iprot);
-                    struct.success.add(_elem871);
+                    _elem879 = new FieldSchema();
+                    _elem879.read(iprot);
+                    struct.success.add(_elem879);
                   }
                   iprot.readListEnd();
                 }
@@ -44618,9 +44618,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 _iter873 : struct.success)
+            for (FieldSchema _iter881 : struct.success)
             {
-              _iter873.write(oprot);
+              _iter881.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -44675,9 +44675,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter874 : struct.success)
+            for (FieldSchema _iter882 : struct.success)
             {
-              _iter874.write(oprot);
+              _iter882.write(oprot);
             }
           }
         }
@@ -44698,14 +44698,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list875 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list875.size);
-            FieldSchema _elem876;
-            for (int _i877 = 0; _i877 < _list875.size; ++_i877)
+            org.apache.thrift.protocol.TList _list883 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list883.size);
+            FieldSchema _elem884;
+            for (int _i885 = 0; _i885 < _list883.size; ++_i885)
             {
-              _elem876 = new FieldSchema();
-              _elem876.read(iprot);
-              struct.success.add(_elem876);
+              _elem884 = new FieldSchema();
+              _elem884.read(iprot);
+              struct.success.add(_elem884);
             }
           }
           struct.setSuccessIsSet(true);
@@ -47632,14 +47632,14 @@ import org.slf4j.LoggerFactory;
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list878 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list878.size);
-                  SQLPrimaryKey _elem879;
-                  for (int _i880 = 0; _i880 < _list878.size; ++_i880)
+                  org.apache.thrift.protocol.TList _list886 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list886.size);
+                  SQLPrimaryKey _elem887;
+                  for (int _i888 = 0; _i888 < _list886.size; ++_i888)
                   {
-                    _elem879 = new SQLPrimaryKey();
-                    _elem879.read(iprot);
-                    struct.primaryKeys.add(_elem879);
+                    _elem887 = new SQLPrimaryKey();
+                    _elem887.read(iprot);
+                    struct.primaryKeys.add(_elem887);
                   }
                   iprot.readListEnd();
                 }
@@ -47651,14 +47651,14 @@ import org.slf4j.LoggerFactory;
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list881 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list881.size);
-                  SQLForeignKey _elem882;
-                  for (int _i883 = 0; _i883 < _list881.size; ++_i883)
+                  org.apache.thrift.protocol.TList _list889 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list889.size);
+                  SQLForeignKey _elem890;
+                  for (int _i891 = 0; _i891 < _list889.size; ++_i891)
                   {
-                    _elem882 = new SQLForeignKey();
-                    _elem882.read(iprot);
-                    struct.foreignKeys.add(_elem882);
+                    _elem890 = new SQLForeignKey();
+                    _elem890.read(iprot);
+                    struct.foreignKeys.add(_elem890);
                   }
                   iprot.readListEnd();
                 }
@@ -47670,14 +47670,14 @@ import org.slf4j.LoggerFactory;
             case 4: // UNIQUE_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list884 = iprot.readListBegin();
-                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list884.size);
-                  SQLUniqueConstraint _elem885;
-                  for (int _i886 = 0; _i886 < _list884.size; ++_i886)
+                  org.apache.thrift.protocol.TList _list892 = iprot.readListBegin();
+                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list892.size);
+                  SQLUniqueConstraint _elem893;
+                  for (int _i894 = 0; _i894 < _list892.size; ++_i894)
                   {
-                    _elem885 = new SQLUniqueConstraint();
-                    _elem885.read(iprot);
-                    struct.uniqueConstraints.add(_elem885);
+                    _elem893 = new SQLUniqueConstraint();
+                    _elem893.read(iprot);
+                    struct.uniqueConstraints.add(_elem893);
                   }
                   iprot.readListEnd();
                 }
@@ -47689,14 +47689,14 @@ import org.slf4j.LoggerFactory;
             case 5: // NOT_NULL_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list887 = iprot.readListBegin();
-                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list887.size);
-                  SQLNotNullConstraint _elem888;
-                  for (int _i889 = 0; _i889 < _list887.size; ++_i889)
+                  org.apache.thrift.protocol.TList _list895 = iprot.readListBegin();
+                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list895.size);
+                  SQLNotNullConstraint _elem896;
+                  for (int _i897 = 0; _i897 < _list895.size; ++_i897)
                   {
-                    _elem888 = new SQLNotNullConstraint();
-                    _elem888.read(iprot);
-                    struct.notNullConstraints.add(_elem888);
+                    _elem896 = new SQLNotNullConstraint();
+                    _elem896.read(iprot);
+                    struct.notNullConstraints.add(_elem896);
                   }
                   iprot.readListEnd();
                 }
@@ -47727,9 +47727,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 _iter890 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter898 : struct.primaryKeys)
             {
-              _iter890.write(oprot);
+              _iter898.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -47739,9 +47739,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 _iter891 : struct.foreignKeys)
+            for (SQLForeignKey _iter899 : struct.foreignKeys)
             {
-              _iter891.write(oprot);
+              _iter899.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -47751,9 +47751,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 _iter892 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter900 : struct.uniqueConstraints)
             {
-              _iter892.write(oprot);
+              _iter900.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -47763,9 +47763,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 _iter893 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter901 : struct.notNullConstraints)
             {
-              _iter893.write(oprot);
+              _iter901.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -47811,36 +47811,36 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter894 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter902 : struct.primaryKeys)
             {
-              _iter894.write(oprot);
+              _iter902.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter895 : struct.foreignKeys)
+            for (SQLForeignKey _iter903 : struct.foreignKeys)
             {
-              _iter895.write(oprot);
+              _iter903.write(oprot);
             }
           }
         }
         if (struct.isSetUniqueConstraints()) {
           {
             oprot.writeI32(struct.uniqueConstraints.size());
-            for (SQLUniqueConstraint _iter896 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter904 : struct.uniqueConstraints)
             {
-              _iter896.write(oprot);
+              _iter904.write(oprot);
             }
           }
         }
         if (struct.isSetNotNullConstraints()) {
           {
             oprot.writeI32(struct.notNullConstraints.size());
-            for (SQLNotNullConstraint _iter897 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter905 : struct.notNullConstraints)
             {
-              _iter897.write(oprot);
+              _iter905.write(oprot);
             }
           }
         }
@@ -47857,56 +47857,56 @@ import org.slf4j.LoggerFactory;
         }
         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.primaryKeys = new ArrayList<SQLPrimaryKey>(_list898.size);
-            SQLPrimaryKey _elem899;
-            for (int _i900 = 0; _i900 < _list898.size; ++_i900)
+            org.apache.thrift.protocol.TList _list906 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list906.size);
+            SQLPrimaryKey _elem907;
+            for (int _i908 = 0; _i908 < _list906.size; ++_i908)
             {
-              _elem899 = new SQLPrimaryKey();
-              _elem899.read(iprot);
-              struct.primaryKeys.add(_elem899);
+              _elem907 = new SQLPrimaryKey();
+              _elem907.read(iprot);
+              struct.primaryKeys.add(_elem907);
             }
           }
           struct.setPrimaryKeysIsSet(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.foreignKeys = new ArrayList<SQLForeignKey>(_list901.size);
-            SQLForeignKey _elem902;
-            for (int _i903 = 0; _i903 < _list901.size; ++_i903)
+            org.apache.thrift.protocol.TList _list909 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list909.size);
+            SQLForeignKey _elem910;
+            for (int _i911 = 0; _i911 < _list909.size; ++_i911)
             {
-              _elem902 = new SQLForeignKey();
-              _elem902.read(iprot);
-              struct.foreignKeys.add(_elem902);
+              _elem910 = new SQLForeignKey();
+              _elem910.read(iprot);
+              struct.foreignKeys.add(_elem910);
             }
           }
           struct.setForeignKeysIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list904 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list904.size);
-            SQLUniqueConstraint _elem905;
-            for (int _i906 = 0; _i906 < _list904.size; ++_i906)
+            org.apache.thrift.protocol.TList _list912 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list912.size);
+            SQLUniqueConstraint _elem913;
+            for (int _i914 = 0; _i914 < _list912.size; ++_i914)
             {
-              _elem905 = new SQLUniqueConstraint();
-              _elem905.read(iprot);
-              struct.uniqueConstraints.add(_elem905);
+              _elem913 = new SQLUniqueConstraint();
+              _elem913.read(iprot);
+              struct.uniqueConstraints.add(_elem913);
             }
           }
           struct.setUniqueConstraintsIsSet(true);
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list907 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list907.size);
-            SQLNotNullConstraint _elem908;
-            for (int _i909 = 0; _i909 < _list907.size; ++_i909)
+            org.apache.thrift.protocol.TList _list915 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list915.size);
+            SQLNotNullConstraint _elem916;
+            for (int _i917 = 0; _i917 < _list915.size; ++_i917)
             {
-              _elem908 = new SQLNotNullConstraint();
-              _elem908.read(iprot);
-              struct.notNullConstraints.add(_elem908);
+              _elem916 = new SQLNotNullConstraint();
+              _elem916.read(iprot);
+              struct.notNullConstraints.add(_elem916);
             }
           }
           struct.setNotNullConstraintsIsSet(true);
@@ -55398,13 +55398,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list910 = iprot.readListBegin();
-                  struct.partNames = new ArrayList<String>(_list910.size);
-                  String _elem911;
-                  for (int _i912 = 0; _i912 < _list910.size; ++_i912)
+                  org.apache.thrift.protocol.TList _list918 = iprot.readListBegin();
+                  struct.partNames = new ArrayList<String>(_list918.size);
+                  String _elem919;
+                  for (int _i920 = 0; _i920 < _list918.size; ++_i920)
                   {
-                    _elem911 = iprot.readString();
-                    struct.partNames.add(_elem911);
+                    _elem919 = iprot.readString();
+                    struct.partNames.add(_elem919);
                   }
                   iprot.readListEnd();
                 }
@@ -55440,9 +55440,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 _iter913 : struct.partNames)
+            for (String _iter921 : struct.partNames)
             {
-              oprot.writeString(_iter913);
+              oprot.writeString(_iter921);
             }
             oprot.writeListEnd();
           }
@@ -55485,9 +55485,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPartNames()) {
           {
             oprot.writeI32(struct.partNames.size());
-            for (String _iter914 : struct.partNames)
+            for (String _iter922 : struct.partNames)
             {
-              oprot.writeString(_iter914);
+              oprot.writeString(_iter922);
             }
           }
         }
@@ -55507,13 +55507,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list915 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partNames = new ArrayList<String>(_list915.size);
-            String _elem916;
-            for (int _i917 = 0; _i917 < _list915.size; ++_i917)
+            org.apache.thrift.protocol.TList _list923 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partNames = new ArrayList<String>(_list923.size);
+            String _elem924;
+            for (int _i925 = 0; _i925 < _list923.size; ++_i925)
             {
-              _elem916 = iprot.readString();
-              struct.partNames.add(_elem916);
+              _elem924 = iprot.readString();
+              struct.partNames.add(_elem924);
             }
           }
           struct.setPartNamesIsSet(true);
@@ -56738,13 +56738,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list918 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list918.size);
-                  String _elem919;
-                  for (int _i920 = 0; _i920 < _list918.size; ++_i920)
+                  org.apache.thrift.protocol.TList _list926 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list926.size);
+                  String _elem927;
+                  for (int _i928 = 0; _i928 < _list926.size; ++_i928)
                   {
-                    _elem919 = iprot.readString();
-                    struct.success.add(_elem919);
+                    _elem927 = iprot.readString();
+                    struct.success.add(_elem927);
                   }
                   iprot.readListEnd();
                 }
@@ -56779,9 +56779,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 _iter921 : struct.success)
+            for (String _iter929 : struct.success)
             {
-              oprot.writeString(_iter921);
+              oprot.writeString(_iter929);
             }
             oprot.writeListEnd();
           }
@@ -56820,9 +56820,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter922 : struct.success)
+            for (String _iter930 : struct.success)
             {
-              oprot.writeString(_iter922);
+              oprot.writeString(_iter930);
             }
           }
         }
@@ -56837,13 +56837,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list923 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list923.size);
-            String _elem924;
-            for (int _i925 = 0; _i925 < _list923.size; ++_i925)
+            org.apache.thrift.protocol.TList _list931 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list931.size);
+            String _elem932;
+            for (int _i933 = 0; _i933 < _list931.size; ++_i933)
             {
-              _elem924 = iprot.readString();
-              struct.success.add(_elem924);
+              _elem932 = iprot.readString();
+              struct.success.add(_elem932);
             }
           }
           struct.setSuccessIsSet(true);
@@ -57817,13 +57817,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list926 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list926.size);
-                  String _elem927;
-                  for (int _i928 = 0; _i928 < _list926.size; ++_i928)
+                  org.apache.thrift.protocol.TList _list934 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list934.size);
+                  String _elem935;
+                  for (int _i936 = 0; _i936 < _list934.size; ++_i936)
                   {
-                    _elem927 = iprot.readString();
-                    struct.success.add(_elem927);
+                    _elem935 = iprot.readString();
+                    struct.success.add(_elem935);
                   }
                   iprot.readListEnd();
                 }
@@ -57858,9 +57858,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 _iter929 : struct.success)
+            for (String _iter937 : struct.success)
             {
-              oprot.writeString(_iter929);
+              oprot.writeString(_iter937);
             }
             oprot.writeListEnd();
           }
@@ -57899,9 +57899,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter930 : struct.success)
+            for (String _iter938 : struct.success)
             {
-              oprot.writeString(_iter930);
+              oprot.writeString(_iter938);
             }
           }
         }
@@ -57916,13 +57916,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list931 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list931.size);
-            String _elem932;
-            for (int _i933 = 0; _i933 < _list931.size; ++_i933)
+            org.apache.thrift.protocol.TList _list939 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list939.size);
+            String _elem940;
+            for (int _i941 = 0; _i941 < _list939.size; ++_i941)
             {
-              _elem932 = iprot.readString();
-              struct.success.add(_elem932);
+              _elem940 = iprot.readString();
+              struct.success.add(_elem940);
             }
           }
           struct.setSuccessIsSet(true);
@@ -58688,13 +58688,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list934 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list934.size);
-                  String _elem935;
-                  for (int _i936 = 0; _i936 < _list934.size; ++_i936)
+                  org.apache.thrift.protocol.TList _list942 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list942.size);
+                  String _elem943;
+                  for (int _i944 = 0; _i944 < _list942.size; ++_i944)
                   {
-                    _elem935 = iprot.readString();
-                    struct.success.add(_elem935);
+                    _elem943 = iprot.readString();
+                    struct.success.add(_elem943);
                   }
                   iprot.readListEnd();
                 }
@@ -58729,9 +58729,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 _iter937 : struct.success)
+            for (String _iter945 : struct.success)
             {
-              oprot.writeString(_iter937);
+              oprot.writeString(_iter945);
             }
             oprot.writeListEnd();
           }
@@ -58770,9 +58770,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter938 : struct.success)
+            for (String _iter946 : struct.success)
             {
-              oprot.writeString(_iter938);
+              oprot.writeString(_iter946);
             }
           }
         }
@@ -58787,13 +58787,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list939 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list939.size);
-            String _elem940;
-            for (int _i941 = 0; _i941 < _list939.size; ++_i941)
+            org.apache.thrift.protocol.TList _list947 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list947.size);
+            String _elem948;
+            for (int _i949 = 0; _i949 < _list947.size; ++_i949)
             {
-              _elem940 = iprot.readString();
-              struct.success.add(_elem940);
+              _elem948 = iprot.readString();
+              struct.success.add(_elem948);
             }
           }
           struct.setSuccessIsSet(true);
@@ -59298,13 +59298,13 @@ import org.slf4j.LoggerFactory;
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list942 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list942.size);
-                  String _elem943;
-                  for (int _i944 = 0; _i944 < _list942.size; ++_i944)
+                  org.apache.thrift.protocol.TList _list950 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list950.size);
+                  String _elem951;
+                  for (int _i952 = 0; _i952 < _list950.size; ++_i952)
                   {
-                    _elem943 = iprot.readString();
-                    struct.tbl_types.add(_elem943);
+                    _elem951 = iprot.readString();
+                    struct.tbl_types.add(_elem951);
                   }
                   iprot.readListEnd();
                 }
@@ -59340,9 +59340,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 _iter945 : struct.tbl_types)
+            for (String _iter953 : struct.tbl_types)
             {
-              oprot.writeString(_iter945);
+              oprot.writeString(_iter953);
             }
             oprot.writeListEnd();
           }
@@ -59385,9 +59385,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter946 : struct.tbl_types)
+            for (String _iter954 : struct.tbl_types)
             {
-              oprot.writeString(_iter946);
+              oprot.writeString(_iter954);
             }
           }
         }
@@ -59407,13 +59407,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list947 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list947.size);
-            String _elem948;
-            for (int _i949 = 0; _i949 < _list947.size; ++_i949)
+            org.apache.thrift.protocol.TList _list955 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list955.size);
+            String _elem956;
+            for (int _i957 = 0; _i957 < _list955.size; ++_i957)
             {
-              _elem948 = iprot.readString();
-              struct.tbl_types.add(_elem948);
+              _elem956 = iprot.readString();
+              struct.tbl_types.add(_elem956);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -59819,14 +59819,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list950 = iprot.readListBegin();
-                  struct.success = new ArrayList<TableMeta>(_list950.size);
-                  TableMeta _elem951;
-                  for (int _i952 = 0; _i952 < _list950.size; ++_i952)
+                  org.apache.thrift.protocol.TList _list958 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list958.size);
+                  TableMeta _elem959;
+                  for (int _i960 = 0; _i960 < _list958.size; ++_i960)
                   {
-                    _elem951 = new TableMeta();
-                    _elem951.read(iprot);
-                    struct.success.add(_elem951);
+                    _elem959 = new TableMeta();
+                    _elem959.read(iprot);
+                    struct.success.add(_elem959);
                   }
                   iprot.readListEnd();
                 }
@@ -59861,9 +59861,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 _iter953 : struct.success)
+            for (TableMeta _iter961 : struct.success)
             {
-              _iter953.write(oprot);
+              _iter961.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -59902,9 +59902,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter954 : struct.success)
+            for (TableMeta _iter962 : struct.success)
             {
-              _iter954.write(oprot);
+              _iter962.write(oprot);
             }
           }
         }
@@ -59919,14 +59919,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list955 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TableMeta>(_list955.size);
-            TableMeta _elem956;
-            for (int _i957 = 0; _i957 < _list955.size; ++_i957)
+            org.apache.thrift.protocol.TList _list963 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list963.size);
+            TableMeta _elem964;
+            for (int _i965 = 0; _i965 < _list963.size; ++_i965)
             {
-              _elem956 = new TableMeta();
-              _elem956.read(iprot);
-              struct.success.add(_elem956);
+              _elem964 = new TableMeta();
+              _elem964.read(iprot);
+              struct.success.add(_elem964);
             }
           }
           struct.setSuccessIsSet(true);
@@ -60692,13 +60692,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list958 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list958.size);
-                  String _elem959;
-                  for (int _i960 = 0; _i960 < _list958.size; ++_i960)
+                  org.apache.thrift.protocol.TList _list966 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list966.size);
+                  String _elem967;
+                  for (int _i968 = 0; _i968 < _list966.size; ++_i968)
                   {
-                    _elem959 = iprot.readString();
-                    struct.success.add(_elem959);
+                    _elem967 = iprot.readString();
+                    struct.success.add(_elem967);
                   }
                   iprot.readListEnd();
                 }
@@ -60733,9 +60733,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 _iter961 : struct.success)
+            for (String _iter969 : struct.success)
             {
-              oprot.writeString(_iter961);
+              oprot.writeString(_iter969);
             }
             oprot.writeListEnd();
           }
@@ -60774,9 +60774,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter962 : struct.success)
+            for (String _iter970 : struct.success)
             {
-              oprot.writeString(_iter962);
+              oprot.writeString(_iter970);
             }
           }
         }
@@ -60791,13 +60791,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list963 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list963.size);
-            String _elem964;
-            for (int _i965 = 0; _i965 < _list963.size; ++_i965)
+            org.apache.thrift.protocol.TList _list971 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list971.size);
+            String _elem972;
+            for (int _i973 = 0; _i973 < _list971.size; ++_i973)
             {
-              _elem964 = iprot.readString();
-              struct.success.add(_elem964);
+              _elem972 = iprot.readString();
+              struct.success.add(_elem972);
             }
           }
           struct.setSuccessIsSet(true);
@@ -62250,13 +62250,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list966 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list966.size);
-                  String _elem967;
-                  for (int _i968 = 0; _i968 < _list966.size; ++_i968)
+                  org.apache.thrift.protocol.TList _list974 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list974.size);
+                  String _elem975;
+                  for (int _i976 = 0; _i976 < _list974.size; ++_i976)
                   {
-                    _elem967 = iprot.readString();
-                    struct.tbl_names.add(_elem967);
+                    _elem975 = iprot.readString();
+                    struct.tbl_names.add(_elem975);
                   }
                   iprot.readListEnd();
                 }
@@ -62287,9 +62287,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 _iter969 : struct.tbl_names)
+            for (String _iter977 : struct.tbl_names)
             {
-              oprot.writeString(_iter969);
+              oprot.writeString(_iter977);
             }
             oprot.writeListEnd();
           }
@@ -62326,9 +62326,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter970 : struct.tbl_names)
+            for (String _iter978 : struct.tbl_names)
             {
-              oprot.writeString(_iter970);
+              oprot.writeString(_iter978);
             }
           }
         }
@@ -62344,13 +62344,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list971 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list971.size);
-            String _elem972;
-            for (int _i973 = 0; _i973 < _list971.size; ++_i973)
+            org.apache.thrift.protocol.TList _list979 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list979.size);
+            String _elem980;
+            for (int _i981 = 0; _i981 < _list979.size; ++_i981)
             {
-              _elem972 = iprot.readString();
-              struct.tbl_names.add(_elem972);
+              _elem980 = iprot.readString();
+              struct.tbl_names.add(_elem980);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -62675,14 +62675,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list974 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list974.size);
-                  Table _elem975;
-                  for (int _i976 = 0; _i976 < _list974.size; ++_i976)
+                  org.apache.thrift.protocol.TList _list982 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list982.size);
+                  Table _elem983;
+                  for (int _i984 = 0; _i984 < _list982.size; ++_i984)
                   {
-                    _elem975 = new Table();
-                    _elem975.read(iprot);
-                    struct.success.add(_elem975);
+                    _elem983 = new Table();
+                    _elem983.read(iprot);
+                    struct.success.add(_elem983);
                   }
                   iprot.readListEnd();
                 }
@@ -62708,9 +62708,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 _iter977 : struct.success)
+            for (Table _iter985 : struct.success)
             {
-              _iter977.write(oprot);
+              _iter985.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -62741,9 +62741,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter978 : struct.success)
+            for (Table _iter986 : struct.success)
             {
-              _iter978.write(oprot);
+              _iter986.write(oprot);
             }
           }
         }
@@ -62755,14 +62755,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list979 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list979.size);
-            Table _elem980;
-            for (int _i981 = 0; _i981 < _list979.size; ++_i981)
+            org.apache.thrift.protocol.TList _list987 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list987.size);
+            Table _elem988;
+            for (int _i989 = 0; _i989 < _list987.size; ++_i989)
             {
-              _elem980 = new Table();
-              _elem980.read(iprot);
-              struct.success.add(_elem980);
+              _elem988 = new Table();
+              _elem988.read(iprot);
+              struct.success.add(_elem988);
             }
           }
           struct.setSuccessIsSet(true);
@@ -65155,13 +65155,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list982 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list982.size);
-                  String _elem983;
-                  for (int _i984 = 0; _i984 < _list982.size; ++_i984)
+                  org.apache.thrift.protocol.TList _list990 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list990.size);
+                  String _elem991;
+                  for (int _i992 = 0; _i992 < _list990.size; ++_i992)
                   {
-                    _elem983 = iprot.readString();
-                    struct.tbl_names.add(_elem983);
+                    _elem991 = iprot.readString();
+                    struct.tbl_names.add(_elem991);
                   }
                   iprot.readListEnd();
                 }
@@ -65192,9 +65192,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 _iter985 : struct.tbl_names)
+            for (String _iter993 : struct.tbl_names)
             {
-              oprot.writeString(_iter985);
+              oprot.writeString(_iter993);
             }
             oprot.writeListEnd();
           }
@@ -65231,9 +65231,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter986 : struct.tbl_names)
+            for (String _iter994 : struct.tbl_names)
             {
-              oprot.writeString(_iter986);
+              oprot.writeString(_iter994);
             }
           }
         }
@@ -65249,13 +65249,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list987 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list987.size);
-            String _elem988;
-            for (int _i989 = 0; _i989 < _list987.size; ++_i989)
+            org.apache.thrift.protocol.TList _list995 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list995.size);
+            String _elem996;
+            for (int _i997 = 0; _i997 < _list995.size; ++_i997)
             {
-              _elem988 = iprot.readString();
-              struct.tbl_names.add(_elem988);
+              _elem996 = iprot.readString();
+              struct.tbl_names.add(_elem996);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -65828,16 +65828,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map990 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Materialization>(2*_map990.size);
-                  String _key991;
-                  Materialization _val992;
-                  for (int _i993 = 0; _i993 < _map990.size; ++_i993)
+                  org.apache.thrift.protocol.TMap _map998 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Materialization>(2*_map998.size);
+                  String _key999;
+                  Materialization _val1000;
+                  for (int _i1001 = 0; _i1001 < _map998.size; ++_i1001)
                   {
-                    _key991 = iprot.readString();
-                    _val992 = new Materialization();
-                    _val992.read(iprot);
-                    struct.success.put(_key991, _val992);
+                    _key999 = iprot.readString();
+                    _val1000 = new Materialization();
+                    _val1000.read(iprot);
+                    struct.success.put(_key999, _val1000);
                   }
                   iprot.readMapEnd();
                 }
@@ -65890,10 +65890,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, Materialization> _iter994 : struct.success.entrySet())
+            for (Map.Entry<String, Materialization> _iter1002 : struct.success.entrySet())
             {
-              oprot.writeString(_iter994.getKey());
-              _iter994.getValue().write(oprot);
+              oprot.writeString(_iter1002.getKey());
+              _iter1002.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -65948,10 +65948,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Materialization> _iter995 : struct.success.entrySet())
+            for (Map.Entry<String, Materialization> _iter1003 : struct.success.entrySet())
             {
-              oprot.writeString(_iter995.getKey());
-              _iter995.getValue().write(oprot);
+              oprot.writeString(_iter1003.getKey());
+              _iter1003.getValue().write(oprot);
             }
           }
         }
@@ -65972,16 +65972,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map996 = 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,Materialization>(2*_map996.size);
-            String _key997;
-            Materialization _val998;
-            for (int _i999 = 0; _i999 < _map996.size; ++_i999)
+            org.apache.thrift.protocol.TMap _map1004 = 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,Materialization>(2*_map1004.size);
+            String _key1005;
+            Materialization _val1006;
+            for (int _i1007 = 0; _i1007 < _map1004.size; ++_i1007)
             {
-              _key997 = iprot.readString();
-              _val998 = new Materialization();
-              _val998.read(iprot);
-              struct.success.put(_key997, _val998);
+              _key1005 = iprot.readString();
+              _val1006 = new Materialization();
+              _val1006.read(iprot);
+              struct.success.put(_key1005, _val1006);
             }
           }
           struct.setSuccessIsSet(true);
@@ -67127,13 +67127,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1000 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1000.size);
-                  String _elem1001;
-                  for (int _i1002 = 0; _i1002 < _list1000.size; ++_i1002)
+                  org.apache.thrift.protocol.TList _list1008 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1008.size);
+                  String _elem1009;
+                  for (int _i1010 = 0; _i1010 < _list1008.size; ++_i1010)
                   {
-                    _elem1001 = iprot.readString();
-                    struct.success.add(_elem1001);
+                    _elem1009 = iprot.readString();
+                    struct.success.add(_elem1009);
                   }
                   iprot.readListEnd();
                 }
@@ -67186,9 +67186,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 _iter1003 : struct.success)
+            for (String _iter1011 : struct.success)
             {
-              oprot.writeString(_iter1003);
+              oprot.writeString(_iter1011);
             }
             oprot.writeListEnd();
           }
@@ -67243,9 +67243,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1004 : struct.success)
+            for (String _iter1012 : struct.success)
             {
-              oprot.writeString(_iter1004);
+              oprot.writeString(_iter1012);
             }
           }
         }
@@ -67266,13 +67266,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1005 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1005.size);
-            String _elem1006;
-            for (int _i1007 = 0; _i1007 < _list1005.size; ++_i1007)
+            org.apache.thrift.protocol.TList _list1013 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1013.size);
+            String _elem1014;
+            for (int _i1015 = 0; _i1015 < _list1013.size; ++_i1015)
             {
-              _elem1006 = iprot.readString();
-              struct.success.add(_elem1006);
+              _elem1014 = iprot.readString();
+              struct.success.add(_elem1014);
             }
           }
           struct.setSuccessIsSet(true);
@@ -73131,14 +73131,14 @@ import org.slf4j.LoggerFactory;
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1008 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list1008.size);
-                  Partition _elem1009;
-                  for (int _i1010 = 0; _i1010 < _list1008.size; ++_i1010)
+                  org.apache.thrift.protocol.TList _list1016 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list1016.size);
+                  Partition _elem1017;
+                  for (int _i1018 = 0; _i1018 < _list1016.size; ++_i1018)
                   {
-                    _elem1009 = new Partition();
-                    _elem1009.read(iprot);
-                    struct.new_parts.add(_elem1009);
+                    _elem1017 = new Partition();
+                    _elem1017.read(iprot);
+                    struct.new_parts.add(_elem1017);
                   }
                   iprot.readListEnd();
                 }
@@ -73164,9 +73164,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (Partition _iter1011 : struct.new_parts)
+            for (Partition _iter1019 : struct.new_parts)
             {
-              _iter1011.write(oprot);
+              _iter1019.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -73197,9 +73197,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter1012 : struct.new_parts)
+            for (Partition _iter1020 : struct.new_parts)
             {
-              _iter1012.write(oprot);
+              _iter1020.write(oprot);
             }
           }
         }
@@ -73211,14 +73211,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1013 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list1013.size);
-            Partition _elem1014;
-            for (int _i1015 = 0; _i1015 < _list1013.size; ++_i1015)
+            org.apache.thrift.protocol.TList _list1021 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list1021.size);
+            Partition _elem1022;
+            for (int _i1023 = 0; _i1023 < _list1021.size; ++_i1023)
             {
-              _elem1014 = new Partition();
-              _elem1014.read(iprot);
-              struct.new_parts.add(_elem1014);
+              _elem1022 = new Partition();
+              _elem1022.read(iprot);
+              struct.new_parts.add(_elem1022);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -74219,14 +74219,14 @@ import org.slf4j.LoggerFactory;
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1016 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<PartitionSpec>(_list1016.size);
-                  PartitionSpec _elem1017;
-                  for (int _i1018 = 0; _i1018 < _list1016.size; ++_i1018)
+                  org.apache.thrift.protocol.TList _list1024 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<PartitionSpec>(_list1024.size);
+                  PartitionSpec _elem1025;
+                  for (int _i1026 = 0; _i1026 < _list1024.size; ++_i1026)
                   {
-                    _elem1017 = new PartitionSpec();
-                    _elem1017.read(iprot);
-                    struct.new_parts.add(_elem1017);
+                    _elem1025 = new PartitionSpec();
+                    _elem1025.read(iprot);
+                    struct.new_parts.add(_elem1025);
                   }
                   iprot.readListEnd();
                 }
@@ -74252,9 +74252,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (PartitionSpec _iter1019 : struct.new_parts)
+            for (PartitionSpec _iter1027 : struct.new_parts)
             {
-              _iter1019.write(oprot);
+              _iter1027.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -74285,9 +74285,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (PartitionSpec _iter1020 : struct.new_parts)
+            for (PartitionSpec _iter1028 : struct.new_parts)
             {
-              _iter1020.write(oprot);
+              _iter1028.write(oprot);
             }
           }
         }
@@ -74299,14 +74299,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1021 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<PartitionSpec>(_list1021.size);
-            PartitionSpec _elem1022;
-            for (int _i1023 = 0; _i1023 < _list1021.size; ++_i1023)
+            org.apache.thrift.protocol.TList _list1029 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<PartitionSpec>(_list1029.size);
+            PartitionSpec _elem1030;
+            for (int _i1031 = 0; _i1031 < _list1029.size; ++_i1031)
             {
-              _elem1022 = new PartitionSpec();
-              _elem1022.read(iprot);
-              struct.new_parts.add(_elem1022);
+              _elem1030 = new PartitionSpec();
+              _elem1030.read(iprot);
+              struct.new_parts.add(_elem1030);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -75482,13 +75482,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1024 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1024.size);
-                  String _elem1025;
-                  for (int _i1026 = 0; _i1026 < _list1024.size; ++_i1026)
+                  org.apache.thrift.protocol.TList _list1032 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1032.size);
+                  String _elem1033;
+                  for (int _i1034 = 0; _i1034 < _list1032.size; ++_i1034)
                   {
-                    _elem1025 = iprot.readString();
-                    struct.part_vals.add(_elem1025);
+                    _elem1033 = iprot.readString();
+                    struct.part_vals.add(_elem1033);
                   }
                   iprot.readListEnd();
                 }
@@ -75524,9 +75524,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter1027 : struct.part_vals)
+            for (String _iter1035 : struct.part_vals)
             {
-              oprot.writeString(_iter1027);
+              oprot.writeString(_iter1035);
             }
             oprot.writeListEnd();
           }
@@ -75569,9 +75569,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1028 : struct.part_vals)
+            for (String _iter1036 : struct.part_vals)
             {
-              oprot.writeString(_iter1028);
+              oprot.writeString(_iter1036);
             }
           }
         }
@@ -75591,13 +75591,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1029 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1029.size);
-            String _elem1030;
-            for (int _i1031 = 0; _i1031 < _list1029.size; ++_i1031)
+            org.apache.thrift.protocol.TList _list1037 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1037.size);
+            String _elem1038;
+            for (int _i1039 = 0; _i1039 < _list1037.size; ++_i1039)
             {
-              _elem1030 = iprot.readString();
-              struct.part_vals.add(_elem1030);
+              _elem1038 = iprot.readString();
+              struct.part_vals.add(_elem1038);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -77906,13 +77906,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1032 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1032.size);
-                  String _elem1033;
-                  for (int _i1034 = 0; _i1034 < _list1032.size; ++_i1034)
+                  org.apache.thrift.protocol.TList _list1040 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1040.size);
+                  String _elem1041;
+                  for (int _i1042 = 0; _i1042 < _list1040.size; ++_i1042)
                   {
-                    _elem1033 = iprot.readString();
-                    struct.part_vals.add(_elem1033);
+                    _elem1041 = iprot.readString();
+                    struct.part_vals.add(_elem1041);
                   }
                   iprot.readListEnd();
                 }
@@ -77957,9 +77957,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter1035 : struct.part_vals)
+            for (String _iter1043 : struct.part_vals)
             {
-              oprot.writeString(_iter1035);
+              oprot.writeString(_iter1043);
             }
             oprot.writeListEnd();
           }
@@ -78010,9 +78010,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1036 : struct.part_vals)
+            for (String _iter1044 : struct.part_vals)
             {
-              oprot.writeString(_iter1036);
+              oprot.writeString(_iter1044);
             }
           }
         }
@@ -78035,13 +78035,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1037 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1037.size);
-            String _elem1038;
-            for (int _i1039 = 0; _i1039 < _list1037.size; ++_i1039)
+            org.apache.thrift.protocol.TList _list1045 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1045.size);
+            String _elem1046;
+            for (int _i1047 = 0; _i1047 < _list1045.size; ++_i1047)
             {
-              _elem1038 = iprot.readString();
-              struct.part_vals.add(_elem1038);
+              _elem1046 = iprot.readString();
+              struct.part_vals.add(_elem1046);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -81911,13 +81911,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1040 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1040.size);
-                  String _elem1041;
-                  for (int _i1042 = 0; _i1042 < _list1040.size; ++_i1042)
+                  org.apache.thrift.protocol.TList _list1048 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1048.size);
+                  String _elem1049;
+                  for (int _i1050 = 0; _i1050 < _list1048.size; ++_i1050)
                   {
-                    _elem1041 = iprot.readString();
-                    struct.part_vals.add(_elem1041);
+                    _elem1049 = iprot.readString();
+                    struct.part_vals.add(_elem1049);
                   }
                   iprot.readListEnd();
                 }
@@ -81961,9 +81961,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter1043 : struct.part_vals)
+            for (String _iter1051 : struct.part_vals)
             {
-              oprot.writeString(_iter1043);
+              oprot.writeString(_iter1051);
             }
             oprot.writeListEnd();
           }
@@ -82012,9 +82012,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1044 : struct.part_vals)
+            for (String _iter1052 : struct.part_vals)
             {
-              oprot.writeString(_iter1044);
+              oprot.writeString(_iter1052);
             }
           }
         }
@@ -82037,13 +82037,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1045 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1045.size);
-            String _elem1046;
-            for (int _i1047 = 0; _i1047 < _list1045.size; ++_i1047)
+      

<TRUNCATED>