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/18 21:29:05 UTC

[1/6] hive git commit: HIVE-18457 : improve show plan output (triggers, mappings) (Sergey Shelukhin, reviewed by Harish Jaiprakash, Prasanth Jayachandran)

Repository: hive
Updated Branches:
  refs/heads/master 9816cfb44 -> 900da8291


HIVE-18457 : improve show plan output (triggers, mappings) (Sergey Shelukhin, reviewed by Harish Jaiprakash, Prasanth Jayachandran)


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

Branch: refs/heads/master
Commit: 572eaccba316f9ae911b6fda01c09547ccbb1157
Parents: 9816cfb
Author: sergey <se...@apache.org>
Authored: Thu Jan 18 12:48:08 2018 -0800
Committer: sergey <se...@apache.org>
Committed: Thu Jan 18 12:48:08 2018 -0800

----------------------------------------------------------------------
 .../hive/ql/exec/tez/UserPoolMapping.java       |   1 -
 .../formatting/JsonMetaDataFormatter.java       |  78 ++++++---
 .../formatting/MetaDataFormatUtils.java         | 158 +++++++++++++++----
 .../formatting/TextMetaDataFormatter.java       | 121 +++++++++++---
 .../formatting/TestJsonRPFormatter.java         |  15 ++
 .../test/queries/clientpositive/resourceplan.q  |   3 +
 .../clientpositive/llap/resourceplan.q.out      |  54 ++++++-
 7 files changed, 346 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/572eaccb/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/UserPoolMapping.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/UserPoolMapping.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/UserPoolMapping.java
index 8f28b62..df6bd22 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/UserPoolMapping.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/UserPoolMapping.java
@@ -34,7 +34,6 @@ class UserPoolMapping {
     USER, GROUP, APPLICATION
   }
 
-  // TODO# create commands for app mappings
   private final Map<String, Mapping> userMappings = new HashMap<>(),
       groupMappings = new HashMap<>(), appMappings = new HashMap<>();
   private final String defaultPoolPath;

http://git-wip-us.apache.org/repos/asf/hive/blob/572eaccb/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 035c5fc..efc6742 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
@@ -50,6 +50,7 @@ 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;
 
@@ -472,7 +473,6 @@ public class JsonMetaDataFormatter implements MetaDataFormatter {
    */
   private static class JsonRPFormatter implements MetaDataFormatUtils.RPFormatter, Closeable {
     private final JsonGenerator generator;
-    private boolean inPool = false;
 
     JsonRPFormatter(DataOutputStream out) throws IOException {
       generator = new ObjectMapper().getJsonFactory().createJsonGenerator(out);
@@ -489,49 +489,81 @@ public class JsonMetaDataFormatter implements MetaDataFormatter {
     }
 
     @Override
-    public void formatRP(String rpName, Object ... kvPairs) throws IOException {
+    public void startRP(String rpName, Object ... kvPairs) throws IOException {
       generator.writeStartObject();
       writeNameAndFields(rpName, kvPairs);
+    }
+
+    @Override
+    public void endRP() throws IOException {
+      // End the root rp object.
+      generator.writeEndObject();
+    }
+
+    @Override
+    public void startPools() throws IOException {
       generator.writeArrayFieldStart("pools");
     }
 
     @Override
-    public void formatPool(String poolName, int indentLevel, Object ... kvPairs)
-        throws IOException {
-      if (inPool) {
-        // End the triggers array.
-        generator.writeEndArray();
-        // End the pool object.
-        generator.writeEndObject();
-      } else {
-        inPool = true;
-      }
+    public void endPools() throws IOException {
+      // End the pools array.
+      generator.writeEndArray();
+    }
+
+    @Override
+    public void startPool(String poolName, Object ... kvPairs) throws IOException {
       generator.writeStartObject();
       writeNameAndFields(poolName, kvPairs);
+    }
+
+    @Override
+    public void startTriggers() throws IOException {
       generator.writeArrayFieldStart("triggers");
-      // triggers array and pool object left to be ended.
     }
 
     @Override
-    public void formatTrigger(String triggerName, String actionExpression, String triggerExpression,
-        int indentLevel) throws IOException {
+    public void endTriggers() throws IOException {
+      generator.writeEndArray();
+    }
+
+    @Override
+    public void startMappings() throws IOException {
+      generator.writeArrayFieldStart("mappings");
+    }
+
+    @Override
+    public void endMappings() throws IOException {
+      generator.writeEndArray();
+    }
+
+    @Override
+    public void endPool() throws IOException {
+      generator.writeEndObject();
+    }
+
+    @Override
+    public void formatTrigger(String triggerName, String actionExpression,
+        String triggerExpression) throws IOException {
       generator.writeStartObject();
       writeNameAndFields(triggerName, "action", actionExpression, "trigger", triggerExpression);
       generator.writeEndObject();
     }
 
     @Override
-    public void close() throws IOException {
-      if (inPool) {
-        // end the triggers within pool object.
-        generator.writeEndArray();
-        // End the last pool object.
-        generator.writeEndObject();
+    public void formatMappingType(String type, List<String> names) throws IOException {
+      generator.writeStartObject();
+      generator.writeStringField("type", type);
+      generator.writeArrayFieldStart("values");
+      for (String name : names) {
+        generator.writeString(name);
       }
-      // End the pools array.
       generator.writeEndArray();
-      // End the root rp object.
       generator.writeEndObject();
+    }
+
+    @Override
+    public void close() throws IOException {
       generator.close();
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/572eaccb/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 489842e..1c83f69 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
@@ -37,6 +37,7 @@ import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.api.WMPoolTrigger;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
@@ -67,6 +68,7 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.Date;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -729,46 +731,79 @@ public final class MetaDataFormatUtils {
    * Interface to implement actual conversion to text or json of a resource plan.
    */
   public interface RPFormatter {
-    void formatRP(String rpName, Object ... kvPairs) throws IOException;
-    void formatPool(String poolName, int indentLevel, Object ...kvPairs) throws IOException;
-    void formatTrigger(String triggerName, String actionExpression, String triggerExpression,
-        int indentLevel) throws IOException;
+    void startRP(String rpName, Object ... kvPairs) throws IOException;
+    void endRP() throws IOException;
+    void startPools() throws IOException;
+    void startPool(String poolName, Object ...kvPairs) throws IOException;
+    void endPool() throws IOException;
+    void endPools() throws IOException;
+    void startTriggers() throws IOException;
+    void formatTrigger(String triggerName,
+        String actionExpression, String triggerExpression) throws IOException;
+    void endTriggers() throws IOException;
+    void startMappings() throws IOException;
+    void formatMappingType(String type, List<String> names) throws IOException;
+    void endMappings() throws IOException;
   }
 
   /**
    * A n-ary tree for the pools, each node contains a pool and its children.
    */
   private static class PoolTreeNode {
+    private String nonPoolName;
     private WMPool pool;
     private final List<PoolTreeNode> children = new ArrayList<>();
     private final List<WMTrigger> triggers = new ArrayList<>();
+    private final HashMap<String, List<String>> mappings = new HashMap<>();
+    private boolean isDefault;
 
     private PoolTreeNode() {}
 
-    private void writePoolTreeNode(RPFormatter rpFormatter, int indentLevel) throws IOException {
-      String path = pool.getPoolPath();
-      int idx = path.lastIndexOf('.');
-      if (idx != -1) {
-        path = path.substring(idx + 1);
+    private void writePoolTreeNode(RPFormatter rpFormatter) throws IOException {
+      if (pool != null) {
+        String path = pool.getPoolPath();
+        int idx = path.lastIndexOf('.');
+        if (idx != -1) {
+          path = path.substring(idx + 1);
+        }
+        Double allocFraction = pool.getAllocFraction();
+        String schedulingPolicy = pool.isSetSchedulingPolicy() ? pool.getSchedulingPolicy() : null;
+        Integer parallelism = pool.getQueryParallelism();
+        rpFormatter.startPool(path, "allocFraction", allocFraction,
+            "schedulingPolicy", schedulingPolicy, "parallelism", parallelism);
+      } else {
+        rpFormatter.startPool(nonPoolName);
       }
-      Double allocFraction = pool.getAllocFraction();
-      String schedulingPolicy = pool.isSetSchedulingPolicy() ? pool.getSchedulingPolicy() : null;
-      Integer parallelism = pool.getQueryParallelism();
-
-      rpFormatter.formatPool(path, indentLevel, "allocFraction", allocFraction,
-          "schedulingPolicy", schedulingPolicy, "parallelism", parallelism);
+      rpFormatter.startTriggers();
       for (WMTrigger trigger : triggers) {
         rpFormatter.formatTrigger(trigger.getTriggerName(), trigger.getActionExpression(),
-            trigger.getTriggerExpression(), indentLevel);
+            trigger.getTriggerExpression());
+      }
+      rpFormatter.endTriggers();
+      rpFormatter.startMappings();
+      for (Map.Entry<String, List<String>> mappingsOfType : mappings.entrySet()) {
+        rpFormatter.formatMappingType(mappingsOfType.getKey(), mappingsOfType.getValue());
+      }
+      if (isDefault) {
+        rpFormatter.formatMappingType("default", Lists.<String>newArrayList());
       }
+      rpFormatter.endMappings();
+      rpFormatter.startPools();
       for (PoolTreeNode node : children) {
-        node.writePoolTreeNode(rpFormatter, indentLevel + 1);
+        node.writePoolTreeNode(rpFormatter);
       }
+      rpFormatter.endPools();
+      rpFormatter.endPool();
     }
 
     private void sortChildren() {
-      children.sort((PoolTreeNode p1, PoolTreeNode p2) ->
-          Double.compare(p2.pool.getAllocFraction(), p1.pool.getAllocFraction()));
+      children.sort((PoolTreeNode p1, PoolTreeNode p2) -> {
+        if (p2.pool == null) {
+          return (p1.pool == null) ? 0 : -1;
+        }
+        if (p1.pool == null) return 1;
+        return Double.compare(p2.pool.getAllocFraction(), p1.pool.getAllocFraction());
+      });
       for (PoolTreeNode child : children) {
         child.sortChildren();
       }
@@ -786,6 +821,10 @@ public final class MetaDataFormatUtils {
           poolMap.put(path, curr);
         }
         curr.pool = pool;
+        if (fullRp.getPlan().isSetDefaultPoolPath()
+            && fullRp.getPlan().getDefaultPoolPath().equals(path)) {
+          curr.isDefault = true;
+        }
 
         // Add this node to the parent node.
         int ind = path.lastIndexOf('.');
@@ -803,12 +842,19 @@ public final class MetaDataFormatUtils {
         parent.children.add(curr);
       }
       Map<String, WMTrigger> triggerMap = new HashMap<>();
-      if (fullRp.getTriggers() != null) {
+      List<WMTrigger> unmanagedTriggers = new ArrayList<>();
+      HashSet<WMTrigger> unusedTriggers = new HashSet<>();
+      if (fullRp.isSetTriggers()) {
         for (WMTrigger trigger : fullRp.getTriggers()) {
           triggerMap.put(trigger.getTriggerName(), trigger);
+          if (trigger.isIsInUnmanaged()) {
+            unmanagedTriggers.add(trigger);
+          } else {
+            unusedTriggers.add(trigger);
+          }
         }
       }
-      if (fullRp.getPoolTriggers() != null) {
+      if (fullRp.isSetPoolTriggers()) {
         for (WMPoolTrigger pool2Trigger : fullRp.getPoolTriggers()) {
           PoolTreeNode node = poolMap.get(pool2Trigger.getPool());
           WMTrigger trigger = triggerMap.get(pool2Trigger.getTrigger());
@@ -816,31 +862,83 @@ public final class MetaDataFormatUtils {
             throw new IllegalStateException("Invalid trigger to pool: " + pool2Trigger.getPool() +
                 ", " + pool2Trigger.getTrigger());
           }
+          unusedTriggers.remove(trigger);
           node.triggers.add(trigger);
         }
       }
+      HashMap<String, List<String>> unmanagedMappings = new HashMap<>();
+      HashMap<String, List<String>> invalidMappings = new HashMap<>();
+      if (fullRp.isSetMappings()) {
+        for (WMMapping mapping : fullRp.getMappings()) {
+          if (mapping.isSetPoolPath()) {
+            PoolTreeNode destNode = poolMap.get(mapping.getPoolPath());
+            addMappingToMap((destNode == null) ? invalidMappings : destNode.mappings, mapping);
+          } else {
+            addMappingToMap(unmanagedMappings, mapping);
+          }
+        }
+      }
+
+      if (!unmanagedTriggers.isEmpty() || !unmanagedMappings.isEmpty()) {
+        PoolTreeNode curr = createNonPoolNode(poolMap, "unmanaged queries", root);
+        curr.triggers.addAll(unmanagedTriggers);
+        curr.mappings.putAll(unmanagedMappings);
+      }
+      // TODO: perhaps we should also summarize the triggers pointing to invalid pools.
+      if (!unusedTriggers.isEmpty()) {
+        PoolTreeNode curr = createNonPoolNode(poolMap, "unused triggers", root);
+        curr.triggers.addAll(unusedTriggers);
+      }
+      if (!invalidMappings.isEmpty()) {
+        PoolTreeNode curr = createNonPoolNode(poolMap, "invalid mappings", root);
+        curr.mappings.putAll(invalidMappings);
+      }
       return root;
     }
-  }
 
-  private static void writeRPLine(RPFormatter rpFormatter, WMResourcePlan plan)
-      throws IOException {
-    Integer parallelism = plan.isSetQueryParallelism() ? plan.getQueryParallelism() : null;
-    String defaultPool = plan.isSetDefaultPoolPath() ? plan.getDefaultPoolPath() : null;
-    rpFormatter.formatRP(plan.getName(), "status", plan.getStatus().toString(),
-         "parallelism", parallelism, "defaultPool", defaultPool);
+    private static PoolTreeNode createNonPoolNode(
+        Map<String, PoolTreeNode> poolMap, String name, PoolTreeNode root) {
+      PoolTreeNode result;
+      do {
+        name = "<" + name + ">";
+        result = poolMap.get(name);
+        // We expect this to never happen in practice. Can pool paths even have angled braces?
+      } while (result != null);
+      result = new PoolTreeNode();
+      result.nonPoolName = name;
+      poolMap.put(name, result);
+      root.children.add(result);
+      return result;
+    }
+
+    private static void addMappingToMap(HashMap<String, List<String>> map, WMMapping mapping) {
+      List<String> list = map.get(mapping.getEntityType());
+      if (list == null) {
+        list = new ArrayList<String>();
+        map.put(mapping.getEntityType(), list);
+      }
+      list.add(mapping.getEntityName());
+    }
   }
 
   public static void formatFullRP(RPFormatter rpFormatter, WMFullResourcePlan fullRp)
       throws HiveException {
     try {
       WMResourcePlan plan = fullRp.getPlan();
-      writeRPLine(rpFormatter, plan);
+      Integer parallelism = plan.isSetQueryParallelism() ? plan.getQueryParallelism() : null;
+      String defaultPool = plan.isSetDefaultPoolPath() ? plan.getDefaultPoolPath() : null;
+      rpFormatter.startRP(plan.getName(), "status", plan.getStatus().toString(),
+           "parallelism", parallelism, "defaultPool", defaultPool);
+      rpFormatter.startPools();
+
       PoolTreeNode root = PoolTreeNode.makePoolTree(fullRp);
       root.sortChildren();
       for (PoolTreeNode pool : root.children) {
-        pool.writePoolTreeNode(rpFormatter, 1);
+        pool.writePoolTreeNode(rpFormatter);
       }
+
+      rpFormatter.endPools();
+      rpFormatter.endRP();
     } catch (IOException e) {
       throw new HiveException(e);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/572eaccb/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 0f1e893..932fc5f 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
@@ -154,7 +154,6 @@ class TextMetaDataFormatter implements MetaDataFormatter {
 
       if (colPath.equals(tableName)) {
         if ((partCols != null) && !partCols.isEmpty() && showPartColsSeparately) {
-
           mdt = new TextMetaDataTable();
           output += MetaDataFormatUtils.LINE_DELIM + "# Partition Information" + MetaDataFormatUtils.LINE_DELIM + "# ";
           mdt.addRow(MetaDataFormatUtils.getColumnsHeader(null));
@@ -164,7 +163,6 @@ class TextMetaDataFormatter implements MetaDataFormatter {
           output += mdt.renderTable(isOutputPadded);
         }
       } else {
-
         String statsState;
         if (tbl.getParameters() != null && (statsState = tbl.getParameters().get(StatsSetupConst.COLUMN_STATS_ACCURATE)) != null) {
           StringBuilder str = new StringBuilder();
@@ -559,13 +557,13 @@ class TextMetaDataFormatter implements MetaDataFormatter {
         if (plan.isSetQueryParallelism()) {
           out.write(Integer.toString(plan.getQueryParallelism()).getBytes(UTF_8));
         } else {
-          out.write("null".getBytes(UTF_8));
+          write(out, "null");
         }
         out.write(separator);
         if (plan.isSetDefaultPoolPath()) {
           out.write(plan.getDefaultPoolPath().getBytes(UTF_8));
         } else {
-          out.write("null".getBytes(UTF_8));
+          write(out, "null");
         }
         out.write(terminator);
       }
@@ -582,32 +580,66 @@ class TextMetaDataFormatter implements MetaDataFormatter {
    *       > <trigger_name>: if(<triggerExpression>){<actionExpression>}
    */
   private static class TextRPFormatter implements MetaDataFormatUtils.RPFormatter {
+    private static final byte[] INDENT = str("    ");
+    private static final byte[] INDENT2 = str(" |  ");
+    private static final byte[] INDENT_BRANCH = str(" +  ");
+
     private final DataOutputStream out;
+    private int indentLevel = 0;
 
     TextRPFormatter(DataOutputStream out) {
       this.out = out;
     }
 
     @Override
-    public void formatRP(String rpName, Object ... kvPairs) throws IOException {
-      out.write(rpName.getBytes(UTF_8));
+    public void startRP(String rpName, Object ... kvPairs) throws IOException {
+      write(out, rpName);
       writeFields(kvPairs);
       out.write(terminator);
     }
 
-    private static final byte[] INDENT = "    ".getBytes(UTF_8);
+    @Override
+    public void endRP() throws IOException {
+    }
 
     @Override
-    public void formatPool(String poolName, int indentLevel, Object ... kvPairs)
-        throws IOException {
-      for (int i = 0; i < indentLevel; ++i) {
-        out.write(INDENT);
-      }
-      out.write(poolName.getBytes(UTF_8));
+    public void startPools() throws IOException {
+    }
+
+    @Override
+    public void endPools() throws IOException {
+    }
+
+    @Override
+    public void startPool(String poolName, Object ... kvPairs) throws IOException {
+      ++indentLevel;
+      writeIndent(true);
+      write(out, poolName);
       writeFields(kvPairs);
       out.write(terminator);
     }
 
+    @Override
+    public void endPool() throws IOException {
+      --indentLevel;
+    }
+
+    @Override
+    public void startTriggers() throws IOException {
+    }
+
+    @Override
+    public void startMappings() throws IOException {
+    }
+
+    @Override
+    public void endTriggers() throws IOException {
+    }
+
+    @Override
+    public void endMappings() throws IOException {
+    }
+
     private void writeFields(Object ... kvPairs)
         throws IOException {
       if (kvPairs.length % 2 != 0) {
@@ -630,20 +662,53 @@ class TextMetaDataFormatter implements MetaDataFormatter {
     }
 
     @Override
-    public void formatTrigger(String triggerName, String actionExpression, String triggerExpression,
-        int indentLevel) throws IOException {
-      for (int i = 0; i < indentLevel; ++i) {
-        out.write(INDENT);
+    public void formatTrigger(
+        String triggerName, String actionExpression, String triggerExpression) throws IOException {
+      writeIndent(false);
+      write(out, "trigger ");
+      write(out, triggerName);
+      write(out, ": if (");
+      write(out, triggerExpression);
+      write(out, ") { ");
+      write(out, actionExpression);
+      write(out, " }");
+      out.write(terminator);
+    }
+
+    @Override
+    public void formatMappingType(String type, List<String> names) throws IOException {
+      final int maxList = 5;
+      writeIndent(false);
+      write(out, "mapped for ");
+      out.write(type.toLowerCase().getBytes(UTF_8));
+      if (!names.isEmpty()) {
+        write(out, "s: ");
+        int count = Math.min(maxList, names.size());
+        for (int i = 0; i < count; ++i) {
+          if (i != 0) {
+            write(out, ", ");
+          }
+          out.write(names.get(i).getBytes(UTF_8));
+        }
+        int remaining = names.size() - count;
+        if (remaining > 0) {
+          out.write((" and " + remaining + " others").getBytes(UTF_8));
+        }
       }
-      out.write("  > ".getBytes(UTF_8));
-      out.write(triggerName.getBytes(UTF_8));
-      out.write(": if(".getBytes(UTF_8));
-      out.write(triggerExpression.getBytes(UTF_8));
-      out.write("){".getBytes(UTF_8));
-      out.write(actionExpression.getBytes(UTF_8));
-      out.write('}');
       out.write(terminator);
     }
+
+    private void writeIndent(boolean isPool) throws IOException {
+      for (int i = 0; i < indentLevel - 1; ++i) {
+        out.write(INDENT);
+      }
+      if (isPool) {
+        out.write(INDENT_BRANCH);
+      } else {
+        out.write(INDENT);
+        out.write(INDENT2);
+      }
+    }
   }
 
   public void showFullResourcePlan(DataOutputStream out, WMFullResourcePlan fullResourcePlan)
@@ -651,6 +716,14 @@ class TextMetaDataFormatter implements MetaDataFormatter {
     MetaDataFormatUtils.formatFullRP(new TextRPFormatter(out), fullResourcePlan);
   }
 
+  private static byte[] str(String str) {
+    return str.getBytes(UTF_8);
+  }
+
+  private static void write(DataOutputStream out, String val) throws IOException {
+    out.write(str(val));
+  }
+
   public void showErrors(DataOutputStream out, List<String> errors) throws HiveException {
     try {
       for (String error : errors) {

http://git-wip-us.apache.org/repos/asf/hive/blob/572eaccb/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 f2b6890..17605c8 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
@@ -27,6 +27,7 @@ import java.io.DataOutputStream;
 import java.util.ArrayList;
 
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.api.WMPoolTrigger;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
@@ -94,6 +95,12 @@ public class TestJsonRPFormatter {
     fullRp.addToPoolTriggers(pool2Trigger);
   }
 
+  private void addMapping(WMFullResourcePlan fullRp, String type, String name, String poolName) {
+    WMMapping mapping = new WMMapping(fullRp.getPlan().getName(), type, name);
+    mapping.setPoolPath(poolName);
+    fullRp.addToMappings(mapping);
+  }
+
   @Test
   public void testJsonEmptyRPFormatter() throws Exception {
     WMFullResourcePlan fullRp = createRP("test_rp_1", null, null);
@@ -118,6 +125,8 @@ public class TestJsonRPFormatter {
     addPool(fullRp, "pool1", 0.3, 3, "fair");
     addTrigger(fullRp, "trigger1", "KILL", "BYTES > 2", "pool1");
     addPool(fullRp, "pool2", 0.7, 7, "fcfs");
+    addMapping(fullRp, "user", "foo", "pool2");
+    addMapping(fullRp, "user", "bar", "pool2");
     formatter.showFullResourcePlan(out, fullRp);
     out.flush();
 
@@ -139,6 +148,12 @@ public class TestJsonRPFormatter {
     assertEquals(0.7, pool2.get("allocFraction").asDouble(), 0.00001);
     assertTrue(pool2.get("triggers").isArray());
     assertEquals(0, pool2.get("triggers").size());
+    assertTrue(pool2.get("mappings").isArray());
+    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());
 
     JsonNode pool1 = jsonTree.get("pools").get(1);
     assertEquals("pool1", pool1.get("name").asText());

http://git-wip-us.apache.org/repos/asf/hive/blob/572eaccb/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 b46c1ee..c616c22 100644
--- a/ql/src/test/queries/clientpositive/resourceplan.q
+++ b/ql/src/test/queries/clientpositive/resourceplan.q
@@ -281,6 +281,8 @@ ALTER POOL `table`.`table`.pool.child2 ADD TRIGGER `trigger2`;
 ALTER TRIGGER `table`.`trigger1` ADD TO UNMANAGED; 
 SELECT * FROM SYS.WM_POOLS_TO_TRIGGERS;
 
+SHOW RESOURCE PLAN `table`;
+
 ALTER TRIGGER `table`.`trigger1` DROP FROM POOL `table`.pool.child2;
 ALTER TRIGGER `table`.`trigger1` DROP FROM UNMANAGED; 
 SELECT * FROM SYS.WM_POOLS_TO_TRIGGERS;
@@ -321,6 +323,7 @@ CREATE GROUP MAPPING 'group2' IN plan_2 TO def.c2 WITH ORDER 1;
 CREATE GROUP MAPPING 'group3' IN plan_2 UNMANAGED WITH ORDER 1;
 ALTER USER MAPPING "user1" IN plan_2 UNMANAGED;
 
+SHOW RESOURCE PLAN plan_2;
 
 SELECT * FROM SYS.WM_MAPPINGS;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/572eaccb/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 72a78b1..22883bc 100644
--- a/ql/src/test/results/clientpositive/llap/resourceplan.q.out
+++ b/ql/src/test/results/clientpositive/llap/resourceplan.q.out
@@ -3199,7 +3199,8 @@ PREHOOK: type: SHOW RESOURCEPLAN
 POSTHOOK: query: SHOW RESOURCE PLAN plan_1
 POSTHOOK: type: SHOW RESOURCEPLAN
 plan_1[status=DISABLED,parallelism=null,defaultPool=default]
-    default[allocFraction=1.0,schedulingPolicy=null,parallelism=4]
+ +  default[allocFraction=1.0,schedulingPolicy=null,parallelism=4]
+     |  mapped for default
 PREHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@wm_resourceplans
@@ -3224,7 +3225,8 @@ 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]
+ +  default[allocFraction=1.0,schedulingPolicy=null,parallelism=4]
+     |  mapped for default
 PREHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@wm_resourceplans
@@ -3997,6 +3999,25 @@ table	table.pool.child1	table
 table	table.pool.child1	trigger1
 table	table.pool.child2	trigger1
 table	table.pool.child2	trigger2
+PREHOOK: query: SHOW RESOURCE PLAN `table`
+PREHOOK: type: SHOW RESOURCEPLAN
+POSTHOOK: query: SHOW RESOURCE PLAN `table`
+POSTHOOK: type: SHOW RESOURCEPLAN
+table[status=DISABLED,parallelism=1,defaultPool=table.pool]
+ +  table[allocFraction=0.0,schedulingPolicy=fifo,parallelism=1]
+     |  trigger table: if (BYTES_WRITTEN > 100K) { MOVE TO table }
+     +  pool[allocFraction=0.9,schedulingPolicy=fair,parallelism=3]
+         |  mapped for default
+         +  child2[allocFraction=0.7,schedulingPolicy=fair,parallelism=3]
+             |  trigger trigger2: if (BYTES_READ > 100) { KILL }
+             |  trigger trigger1: if (ELAPSED_TIME > 10) { KILL }
+         +  child1[allocFraction=0.3,schedulingPolicy=fair,parallelism=1]
+             |  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 }
 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
@@ -4092,6 +4113,23 @@ PREHOOK: query: ALTER USER MAPPING "user1" IN plan_2 UNMANAGED
 PREHOOK: type: ALTER MAPPING
 POSTHOOK: query: ALTER USER MAPPING "user1" IN plan_2 UNMANAGED
 POSTHOOK: type: ALTER MAPPING
+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]
+     |  mapped for users: user2
+     |  mapped for default
+     +  c2[allocFraction=0.7,schedulingPolicy=fair,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
+         |  mapped for applications: app1
+ +  <unmanaged queries>
+     |  mapped for users: user1
+     |  mapped for groups: group3
 PREHOOK: query: SELECT * FROM SYS.WM_MAPPINGS
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@wm_mappings
@@ -4148,10 +4186,14 @@ 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]
-        c2[allocFraction=0.7,schedulingPolicy=fair,parallelism=1]
-          > trigger_1: if(BYTES_READ = 0){MOVE TO null_pool}
-        c1[allocFraction=0.3,schedulingPolicy=fair,parallelism=3]
+ +  def[allocFraction=1.0,schedulingPolicy=null,parallelism=4]
+     |  mapped for default
+     +  c2[allocFraction=0.7,schedulingPolicy=fair,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>
+     |  mapped for users: user1
 PREHOOK: query: DROP RESOURCE PLAN plan_2
 PREHOOK: type: DROP RESOURCEPLAN
 POSTHOOK: query: DROP RESOURCE PLAN plan_2


[2/6] hive git commit: HIVE-18438 : WM RP: it's impossible to unset things (Sergey Shelukhin, reviewed by Harish Jaiprakash, Prasanth Jayachandran)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 e5f3e6c..c3be4c4 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
@@ -14907,6 +14907,139 @@ class WMResourcePlan:
   def __ne__(self, other):
     return not (self == other)
 
+class WMNullableResourcePlan:
+  """
+  Attributes:
+   - name
+   - status
+   - queryParallelism
+   - isSetQueryParallelism
+   - defaultPoolPath
+   - isSetDefaultPoolPath
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'name', None, None, ), # 1
+    (2, TType.I32, 'status', None, None, ), # 2
+    None, # 3
+    (4, TType.I32, 'queryParallelism', None, None, ), # 4
+    (5, TType.BOOL, 'isSetQueryParallelism', None, None, ), # 5
+    (6, TType.STRING, 'defaultPoolPath', None, None, ), # 6
+    (7, TType.BOOL, 'isSetDefaultPoolPath', None, None, ), # 7
+  )
+
+  def __init__(self, name=None, status=None, queryParallelism=None, isSetQueryParallelism=None, defaultPoolPath=None, isSetDefaultPoolPath=None,):
+    self.name = name
+    self.status = status
+    self.queryParallelism = queryParallelism
+    self.isSetQueryParallelism = isSetQueryParallelism
+    self.defaultPoolPath = defaultPoolPath
+    self.isSetDefaultPoolPath = isSetDefaultPoolPath
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.status = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.I32:
+          self.queryParallelism = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.BOOL:
+          self.isSetQueryParallelism = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.STRING:
+          self.defaultPoolPath = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.BOOL:
+          self.isSetDefaultPoolPath = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('WMNullableResourcePlan')
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 1)
+      oprot.writeString(self.name)
+      oprot.writeFieldEnd()
+    if self.status is not None:
+      oprot.writeFieldBegin('status', TType.I32, 2)
+      oprot.writeI32(self.status)
+      oprot.writeFieldEnd()
+    if self.queryParallelism is not None:
+      oprot.writeFieldBegin('queryParallelism', TType.I32, 4)
+      oprot.writeI32(self.queryParallelism)
+      oprot.writeFieldEnd()
+    if self.isSetQueryParallelism is not None:
+      oprot.writeFieldBegin('isSetQueryParallelism', TType.BOOL, 5)
+      oprot.writeBool(self.isSetQueryParallelism)
+      oprot.writeFieldEnd()
+    if self.defaultPoolPath is not None:
+      oprot.writeFieldBegin('defaultPoolPath', TType.STRING, 6)
+      oprot.writeString(self.defaultPoolPath)
+      oprot.writeFieldEnd()
+    if self.isSetDefaultPoolPath is not None:
+      oprot.writeFieldBegin('isSetDefaultPoolPath', TType.BOOL, 7)
+      oprot.writeBool(self.isSetDefaultPoolPath)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.name is None:
+      raise TProtocol.TProtocolException(message='Required field name is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.status)
+    value = (value * 31) ^ hash(self.queryParallelism)
+    value = (value * 31) ^ hash(self.isSetQueryParallelism)
+    value = (value * 31) ^ hash(self.defaultPoolPath)
+    value = (value * 31) ^ hash(self.isSetDefaultPoolPath)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class WMPool:
   """
   Attributes:
@@ -15028,6 +15161,140 @@ class WMPool:
   def __ne__(self, other):
     return not (self == other)
 
+class WMNullablePool:
+  """
+  Attributes:
+   - resourcePlanName
+   - poolPath
+   - allocFraction
+   - queryParallelism
+   - schedulingPolicy
+   - isSetSchedulingPolicy
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'resourcePlanName', None, None, ), # 1
+    (2, TType.STRING, 'poolPath', None, None, ), # 2
+    (3, TType.DOUBLE, 'allocFraction', None, None, ), # 3
+    (4, TType.I32, 'queryParallelism', None, None, ), # 4
+    (5, TType.STRING, 'schedulingPolicy', None, None, ), # 5
+    (6, TType.BOOL, 'isSetSchedulingPolicy', None, None, ), # 6
+  )
+
+  def __init__(self, resourcePlanName=None, poolPath=None, allocFraction=None, queryParallelism=None, schedulingPolicy=None, isSetSchedulingPolicy=None,):
+    self.resourcePlanName = resourcePlanName
+    self.poolPath = poolPath
+    self.allocFraction = allocFraction
+    self.queryParallelism = queryParallelism
+    self.schedulingPolicy = schedulingPolicy
+    self.isSetSchedulingPolicy = isSetSchedulingPolicy
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.resourcePlanName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.poolPath = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.DOUBLE:
+          self.allocFraction = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.I32:
+          self.queryParallelism = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.schedulingPolicy = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.BOOL:
+          self.isSetSchedulingPolicy = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('WMNullablePool')
+    if self.resourcePlanName is not None:
+      oprot.writeFieldBegin('resourcePlanName', TType.STRING, 1)
+      oprot.writeString(self.resourcePlanName)
+      oprot.writeFieldEnd()
+    if self.poolPath is not None:
+      oprot.writeFieldBegin('poolPath', TType.STRING, 2)
+      oprot.writeString(self.poolPath)
+      oprot.writeFieldEnd()
+    if self.allocFraction is not None:
+      oprot.writeFieldBegin('allocFraction', TType.DOUBLE, 3)
+      oprot.writeDouble(self.allocFraction)
+      oprot.writeFieldEnd()
+    if self.queryParallelism is not None:
+      oprot.writeFieldBegin('queryParallelism', TType.I32, 4)
+      oprot.writeI32(self.queryParallelism)
+      oprot.writeFieldEnd()
+    if self.schedulingPolicy is not None:
+      oprot.writeFieldBegin('schedulingPolicy', TType.STRING, 5)
+      oprot.writeString(self.schedulingPolicy)
+      oprot.writeFieldEnd()
+    if self.isSetSchedulingPolicy is not None:
+      oprot.writeFieldBegin('isSetSchedulingPolicy', TType.BOOL, 6)
+      oprot.writeBool(self.isSetSchedulingPolicy)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.resourcePlanName is None:
+      raise TProtocol.TProtocolException(message='Required field resourcePlanName is unset!')
+    if self.poolPath is None:
+      raise TProtocol.TProtocolException(message='Required field poolPath is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.resourcePlanName)
+    value = (value * 31) ^ hash(self.poolPath)
+    value = (value * 31) ^ hash(self.allocFraction)
+    value = (value * 31) ^ hash(self.queryParallelism)
+    value = (value * 31) ^ hash(self.schedulingPolicy)
+    value = (value * 31) ^ hash(self.isSetSchedulingPolicy)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class WMTrigger:
   """
   Attributes:
@@ -16013,7 +16280,7 @@ class WMAlterResourcePlanRequest:
   thrift_spec = (
     None, # 0
     (1, TType.STRING, 'resourcePlanName', None, None, ), # 1
-    (2, TType.STRUCT, 'resourcePlan', (WMResourcePlan, WMResourcePlan.thrift_spec), None, ), # 2
+    (2, TType.STRUCT, 'resourcePlan', (WMNullableResourcePlan, WMNullableResourcePlan.thrift_spec), None, ), # 2
     (3, TType.BOOL, 'isEnableAndActivate', None, None, ), # 3
     (4, TType.BOOL, 'isForceDeactivate', None, None, ), # 4
     (5, TType.BOOL, 'isReplace', None, None, ), # 5
@@ -16042,7 +16309,7 @@ class WMAlterResourcePlanRequest:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.STRUCT:
-          self.resourcePlan = WMResourcePlan()
+          self.resourcePlan = WMNullableResourcePlan()
           self.resourcePlan.read(iprot)
         else:
           iprot.skip(ftype)
@@ -17041,7 +17308,7 @@ class WMAlterPoolRequest:
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRUCT, 'pool', (WMPool, WMPool.thrift_spec), None, ), # 1
+    (1, TType.STRUCT, 'pool', (WMNullablePool, WMNullablePool.thrift_spec), None, ), # 1
     (2, TType.STRING, 'poolPath', None, None, ), # 2
   )
 
@@ -17060,7 +17327,7 @@ class WMAlterPoolRequest:
         break
       if fid == 1:
         if ftype == TType.STRUCT:
-          self.pool = WMPool()
+          self.pool = WMNullablePool()
           self.pool.read(iprot)
         else:
           iprot.skip(ftype)

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 afcec9e..439553f 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
@@ -3360,6 +3360,36 @@ class WMResourcePlan
   ::Thrift::Struct.generate_accessors self
 end
 
+class WMNullableResourcePlan
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  NAME = 1
+  STATUS = 2
+  QUERYPARALLELISM = 4
+  ISSETQUERYPARALLELISM = 5
+  DEFAULTPOOLPATH = 6
+  ISSETDEFAULTPOOLPATH = 7
+
+  FIELDS = {
+    NAME => {:type => ::Thrift::Types::STRING, :name => 'name'},
+    STATUS => {:type => ::Thrift::Types::I32, :name => 'status', :optional => true, :enum_class => ::WMResourcePlanStatus},
+    QUERYPARALLELISM => {:type => ::Thrift::Types::I32, :name => 'queryParallelism', :optional => true},
+    ISSETQUERYPARALLELISM => {:type => ::Thrift::Types::BOOL, :name => 'isSetQueryParallelism', :optional => true},
+    DEFAULTPOOLPATH => {:type => ::Thrift::Types::STRING, :name => 'defaultPoolPath', :optional => true},
+    ISSETDEFAULTPOOLPATH => {:type => ::Thrift::Types::BOOL, :name => 'isSetDefaultPoolPath', :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field name is unset!') unless @name
+    unless @status.nil? || ::WMResourcePlanStatus::VALID_VALUES.include?(@status)
+      raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field status!')
+    end
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class WMPool
   include ::Thrift::Struct, ::Thrift::Struct_Union
   RESOURCEPLANNAME = 1
@@ -3386,6 +3416,34 @@ class WMPool
   ::Thrift::Struct.generate_accessors self
 end
 
+class WMNullablePool
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  RESOURCEPLANNAME = 1
+  POOLPATH = 2
+  ALLOCFRACTION = 3
+  QUERYPARALLELISM = 4
+  SCHEDULINGPOLICY = 5
+  ISSETSCHEDULINGPOLICY = 6
+
+  FIELDS = {
+    RESOURCEPLANNAME => {:type => ::Thrift::Types::STRING, :name => 'resourcePlanName'},
+    POOLPATH => {:type => ::Thrift::Types::STRING, :name => 'poolPath'},
+    ALLOCFRACTION => {:type => ::Thrift::Types::DOUBLE, :name => 'allocFraction', :optional => true},
+    QUERYPARALLELISM => {:type => ::Thrift::Types::I32, :name => 'queryParallelism', :optional => true},
+    SCHEDULINGPOLICY => {:type => ::Thrift::Types::STRING, :name => 'schedulingPolicy', :optional => true},
+    ISSETSCHEDULINGPOLICY => {:type => ::Thrift::Types::BOOL, :name => 'isSetSchedulingPolicy', :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field resourcePlanName is unset!') unless @resourcePlanName
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field poolPath is unset!') unless @poolPath
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class WMTrigger
   include ::Thrift::Struct, ::Thrift::Struct_Union
   RESOURCEPLANNAME = 1
@@ -3622,7 +3680,7 @@ class WMAlterResourcePlanRequest
 
   FIELDS = {
     RESOURCEPLANNAME => {:type => ::Thrift::Types::STRING, :name => 'resourcePlanName', :optional => true},
-    RESOURCEPLAN => {:type => ::Thrift::Types::STRUCT, :name => 'resourcePlan', :class => ::WMResourcePlan, :optional => true},
+    RESOURCEPLAN => {:type => ::Thrift::Types::STRUCT, :name => 'resourcePlan', :class => ::WMNullableResourcePlan, :optional => true},
     ISENABLEANDACTIVATE => {:type => ::Thrift::Types::BOOL, :name => 'isEnableAndActivate', :optional => true},
     ISFORCEDEACTIVATE => {:type => ::Thrift::Types::BOOL, :name => 'isForceDeactivate', :optional => true},
     ISREPLACE => {:type => ::Thrift::Types::BOOL, :name => 'isReplace', :optional => true}
@@ -3879,7 +3937,7 @@ class WMAlterPoolRequest
   POOLPATH = 2
 
   FIELDS = {
-    POOL => {:type => ::Thrift::Types::STRUCT, :name => 'pool', :class => ::WMPool, :optional => true},
+    POOL => {:type => ::Thrift::Types::STRUCT, :name => 'pool', :class => ::WMNullablePool, :optional => true},
     POOLPATH => {:type => ::Thrift::Types::STRING, :name => 'poolPath', :optional => true}
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 53a8669..c44099f 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
@@ -2676,7 +2676,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   @Override
-  public WMFullResourcePlan alterResourcePlan(String resourcePlanName, WMResourcePlan resourcePlan,
+  public WMFullResourcePlan alterResourcePlan(String resourcePlanName, WMNullableResourcePlan resourcePlan,
       boolean canActivateDisabled, boolean isForceDeactivate, boolean isReplace)
       throws NoSuchObjectException, InvalidObjectException, MetaException, TException {
     WMAlterResourcePlanRequest request = new WMAlterResourcePlanRequest();
@@ -2744,7 +2744,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   @Override
-  public void alterWMPool(WMPool pool, String poolPath)
+  public void alterWMPool(WMNullablePool pool, String poolPath)
       throws NoSuchObjectException, InvalidObjectException, MetaException, TException {
     WMAlterPoolRequest request = new WMAlterPoolRequest();
     request.setPool(pool);

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 3261405..238c5ed 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
@@ -105,6 +105,8 @@ import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
+import org.apache.hadoop.hive.metastore.api.WMNullablePool;
+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;
@@ -1819,7 +1821,7 @@ public interface IMetaStoreClient {
   void dropResourcePlan(String resourcePlanName)
       throws NoSuchObjectException, MetaException, TException;
 
-  WMFullResourcePlan alterResourcePlan(String resourcePlanName, WMResourcePlan resourcePlan,
+  WMFullResourcePlan alterResourcePlan(String resourcePlanName, WMNullableResourcePlan resourcePlan,
       boolean canActivateDisabled, boolean isForceDeactivate, boolean isReplace)
       throws NoSuchObjectException, InvalidObjectException, MetaException, TException;
 
@@ -1843,7 +1845,7 @@ public interface IMetaStoreClient {
   void createWMPool(WMPool pool)
       throws NoSuchObjectException, InvalidObjectException, MetaException, TException;
 
-  void alterWMPool(WMPool pool, String poolPath)
+  void alterWMPool(WMNullablePool pool, String poolPath)
       throws NoSuchObjectException, InvalidObjectException, MetaException, TException;
 
   void dropWMPool(String resourcePlanName, String poolPath)

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 b9fecc7..6f5d8a6 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
@@ -133,6 +133,8 @@ import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
+import org.apache.hadoop.hive.metastore.api.WMNullablePool;
+import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.api.WMPoolTrigger;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
@@ -9904,7 +9906,7 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public WMFullResourcePlan alterResourcePlan(String name, WMResourcePlan changes,
+  public WMFullResourcePlan alterResourcePlan(String name, WMNullableResourcePlan changes,
       boolean canActivateDisabled, boolean canDeactivate, boolean isReplace)
     throws AlreadyExistsException, NoSuchObjectException, InvalidOperationException, MetaException {
     name = name == null ? null : normalizeIdentifier(name);
@@ -9935,14 +9937,15 @@ public class ObjectStore implements RawStore, Configurable {
     }
   }
 
-  private WMFullResourcePlan handleSimpleAlter(String name, WMResourcePlan changes,
+  private WMFullResourcePlan handleSimpleAlter(String name, WMNullableResourcePlan changes,
       boolean canActivateDisabled, boolean canDeactivate)
           throws InvalidOperationException, NoSuchObjectException, MetaException {
     MWMResourcePlan plan = name == null ? getActiveMWMResourcePlan()
         : getMWMResourcePlan(name, !changes.isSetStatus());
     boolean hasNameChange = changes.isSetName() && !changes.getName().equals(name);
     // Verify that field changes are consistent with what Hive does. Note: we could handle this.
-    if (changes.isSetQueryParallelism() || changes.isSetDefaultPoolPath() || hasNameChange) {
+    if (changes.isSetIsSetQueryParallelism()
+        || changes.isSetIsSetDefaultPoolPath() || hasNameChange) {
       if (changes.isSetStatus()) {
         throw new InvalidOperationException("Cannot change values during status switch.");
       } else if (plan.getStatus() != MWMResourcePlan.Status.DISABLED) {
@@ -9960,15 +9963,23 @@ public class ObjectStore implements RawStore, Configurable {
         plan.setName(newName);
       }
     }
-    if (changes.isSetQueryParallelism()) {
-      if (changes.getQueryParallelism() <= 0) {
-        throw new InvalidOperationException("queryParallelism should be positive.");
+    if (changes.isSetIsSetQueryParallelism() && changes.isIsSetQueryParallelism()) {
+      if (changes.isSetQueryParallelism()) {
+        if (changes.getQueryParallelism() <= 0) {
+          throw new InvalidOperationException("queryParallelism should be positive.");
+        }
+        plan.setQueryParallelism(changes.getQueryParallelism());
+      } else {
+        plan.setQueryParallelism(null);
       }
-      plan.setQueryParallelism(changes.getQueryParallelism());
     }
-    if (changes.isSetDefaultPoolPath()) {
-      MWMPool pool = getPool(plan, changes.getDefaultPoolPath());
-      plan.setDefaultPool(pool);
+    if (changes.isSetIsSetDefaultPoolPath() && changes.isIsSetDefaultPoolPath()) {
+      if (changes.isSetDefaultPoolPath()) {
+        MWMPool pool = getPool(plan, changes.getDefaultPoolPath());
+        plan.setDefaultPool(pool);
+      } else {
+        plan.setDefaultPool(null);
+      }
     }
 
     // Handle the status change.
@@ -9979,7 +9990,7 @@ public class ObjectStore implements RawStore, Configurable {
     return null;
   }
 
-  private WMFullResourcePlan handleAlterReplace(String name, WMResourcePlan changes)
+  private WMFullResourcePlan handleAlterReplace(String name, WMNullableResourcePlan changes)
           throws InvalidOperationException, NoSuchObjectException, MetaException {
     // Verify that field changes are consistent with what Hive does. Note: we could handle this.
     if (changes.isSetQueryParallelism() || changes.isSetDefaultPoolPath()) {
@@ -10432,7 +10443,7 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public void alterPool(WMPool pool, String poolPath) throws AlreadyExistsException,
+  public void alterPool(WMNullablePool pool, String poolPath) throws AlreadyExistsException,
       NoSuchObjectException, InvalidOperationException, MetaException {
     boolean commited = false;
     try {
@@ -10446,17 +10457,22 @@ public class ObjectStore implements RawStore, Configurable {
       if (pool.isSetQueryParallelism()) {
         mPool.setQueryParallelism(pool.getQueryParallelism());
       }
-      if (pool.isSetSchedulingPolicy()) {
-        String policy = pool.getSchedulingPolicy();
-        if (!MetaStoreUtils.isValidSchedulingPolicy(policy)) {
-          throw new InvalidOperationException("Invalid scheduling policy " + policy);
+      if (pool.isSetIsSetSchedulingPolicy() && pool.isIsSetSchedulingPolicy()) {
+        if (pool.isSetSchedulingPolicy()) {
+          String policy = pool.getSchedulingPolicy();
+          if (!MetaStoreUtils.isValidSchedulingPolicy(policy)) {
+            throw new InvalidOperationException("Invalid scheduling policy " + policy);
+          }
+          mPool.setSchedulingPolicy(pool.getSchedulingPolicy());
+        } else {
+          mPool.setSchedulingPolicy(null);
         }
-        mPool.setSchedulingPolicy(pool.getSchedulingPolicy());
       }
       if (pool.isSetPoolPath() && !pool.getPoolPath().equals(mPool.getPath())) {
         moveDescendents(resourcePlan, mPool.getPath(), pool.getPoolPath());
         mPool.setPath(pool.getPoolPath());
       }
+
       commited = commitTransaction();
     } finally {
       rollbackAndCleanup(commited, (Query)null);

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 e5092ba..f4eff4c 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
@@ -58,6 +58,8 @@ import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
 import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
+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.Role;
@@ -762,7 +764,7 @@ public interface RawStore extends Configurable {
 
   List<WMResourcePlan> getAllResourcePlans() throws MetaException;
 
-  WMFullResourcePlan alterResourcePlan(String name, WMResourcePlan resourcePlan,
+  WMFullResourcePlan alterResourcePlan(String name, WMNullableResourcePlan resourcePlan,
       boolean canActivateDisabled, boolean canDeactivate, boolean isReplace)
       throws AlreadyExistsException, NoSuchObjectException, InvalidOperationException,
           MetaException;
@@ -790,7 +792,7 @@ public interface RawStore extends Configurable {
   void createPool(WMPool pool) throws AlreadyExistsException, NoSuchObjectException,
       InvalidOperationException, MetaException;
 
-  void alterPool(WMPool pool, String poolPath) throws AlreadyExistsException,
+  void alterPool(WMNullablePool pool, String poolPath) throws AlreadyExistsException,
       NoSuchObjectException, InvalidOperationException, MetaException;
 
   void dropWMPool(String resourcePlanName, String poolPath)

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 e28e44a..5598540 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
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.metastore.cache;
 
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
+
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -34,6 +35,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.StatsSetupConst;
@@ -74,6 +76,8 @@ import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
 import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
+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.cache.SharedCache.StatsType;
@@ -105,6 +109,7 @@ import org.apache.hadoop.hive.metastore.utils.StringUtils;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import com.google.common.annotations.VisibleForTesting;
 
 // TODO filter->expr
@@ -2615,7 +2620,7 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public WMFullResourcePlan alterResourcePlan(String name, WMResourcePlan resourcePlan,
+  public WMFullResourcePlan alterResourcePlan(String name, WMNullableResourcePlan resourcePlan,
     boolean canActivateDisabled, boolean canDeactivate, boolean isReplace)
       throws AlreadyExistsException, NoSuchObjectException, InvalidOperationException,
           MetaException {
@@ -2671,7 +2676,7 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public void alterPool(WMPool pool, String poolPath) throws AlreadyExistsException,
+  public void alterPool(WMNullablePool pool, String poolPath) throws AlreadyExistsException,
       NoSuchObjectException, InvalidOperationException, MetaException {
     rawStore.alterPool(pool, poolPath);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 ff7636f..b77afac 100644
--- a/standalone-metastore/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/src/main/thrift/hive_metastore.thrift
@@ -1073,6 +1073,15 @@ struct WMResourcePlan {
   4: optional string defaultPoolPath;
 }
 
+struct WMNullableResourcePlan {
+  1: required string name;
+  2: optional WMResourcePlanStatus status;
+  4: optional i32 queryParallelism;
+  5: optional bool isSetQueryParallelism;
+  6: optional string defaultPoolPath;
+  7: optional bool isSetDefaultPoolPath;
+}
+
 struct WMPool {
   1: required string resourcePlanName;
   2: required string poolPath;
@@ -1081,6 +1090,16 @@ struct WMPool {
   5: optional string schedulingPolicy;
 }
 
+
+struct WMNullablePool {
+  1: required string resourcePlanName;
+  2: required string poolPath;
+  3: optional double allocFraction;
+  4: optional i32 queryParallelism;
+  5: optional string schedulingPolicy;
+  6: optional bool isSetSchedulingPolicy;
+}
+
 struct WMTrigger {
   1: required string resourcePlanName;
   2: required string triggerName;
@@ -1144,7 +1163,7 @@ struct WMGetAllResourcePlanResponse {
 
 struct WMAlterResourcePlanRequest {
   1: optional string resourcePlanName;
-  2: optional WMResourcePlan resourcePlan;
+  2: optional WMNullableResourcePlan resourcePlan;
   3: optional bool isEnableAndActivate;
   4: optional bool isForceDeactivate;
   5: optional bool isReplace;
@@ -1207,7 +1226,7 @@ struct WMCreatePoolResponse {
 }
 
 struct WMAlterPoolRequest {
-  1: optional WMPool pool;
+  1: optional WMNullablePool pool;
   2: optional string poolPath;
 }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 877899c..e0a67bc 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
@@ -55,6 +55,8 @@ import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
 import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
+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.Role;
@@ -958,7 +960,7 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public WMFullResourcePlan alterResourcePlan(String name, WMResourcePlan resourcePlan,
+  public WMFullResourcePlan alterResourcePlan(String name, WMNullableResourcePlan resourcePlan,
       boolean canActivateDisabled, boolean canDeactivate, boolean isReplace)
       throws AlreadyExistsException, NoSuchObjectException, InvalidOperationException,
           MetaException {
@@ -1014,7 +1016,7 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public void alterPool(WMPool pool, String poolPath) throws AlreadyExistsException,
+  public void alterPool(WMNullablePool pool, String poolPath) throws AlreadyExistsException,
       NoSuchObjectException, InvalidOperationException, MetaException {
     objectStore.alterPool(pool, poolPath);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 6aa5d95..aad2914 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
@@ -53,6 +53,8 @@ import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
 import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
+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.Role;
@@ -970,7 +972,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
 
   @Override
   public WMFullResourcePlan alterResourcePlan(
-      String name, WMResourcePlan resourcePlan, boolean canActivateDisabled, boolean canDeactivate,
+      String name, WMNullableResourcePlan resourcePlan, boolean canActivateDisabled, boolean canDeactivate,
       boolean isReplace)
       throws NoSuchObjectException, InvalidOperationException, MetaException {
     return null;
@@ -1017,7 +1019,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public void alterPool(WMPool pool, String poolPath) throws AlreadyExistsException,
+  public void alterPool(WMNullablePool pool, String poolPath) throws AlreadyExistsException,
       NoSuchObjectException, InvalidOperationException, MetaException {
   }
 


[3/6] hive git commit: HIVE-18438 : WM RP: it's impossible to unset things (Sergey Shelukhin, reviewed by Harish Jaiprakash, Prasanth Jayachandran)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMAlterPoolRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMAlterPoolRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMAlterPoolRequest.java
index 2ea6cf0..ecebb5e 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMAlterPoolRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMAlterPoolRequest.java
@@ -47,7 +47,7 @@ import org.slf4j.LoggerFactory;
     schemes.put(TupleScheme.class, new WMAlterPoolRequestTupleSchemeFactory());
   }
 
-  private WMPool pool; // optional
+  private WMNullablePool pool; // optional
   private String poolPath; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -117,7 +117,7 @@ import org.slf4j.LoggerFactory;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
     tmpMap.put(_Fields.POOL, new org.apache.thrift.meta_data.FieldMetaData("pool", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
-        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WMPool.class)));
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WMNullablePool.class)));
     tmpMap.put(_Fields.POOL_PATH, new org.apache.thrift.meta_data.FieldMetaData("poolPath", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
@@ -132,7 +132,7 @@ import org.slf4j.LoggerFactory;
    */
   public WMAlterPoolRequest(WMAlterPoolRequest other) {
     if (other.isSetPool()) {
-      this.pool = new WMPool(other.pool);
+      this.pool = new WMNullablePool(other.pool);
     }
     if (other.isSetPoolPath()) {
       this.poolPath = other.poolPath;
@@ -149,11 +149,11 @@ import org.slf4j.LoggerFactory;
     this.poolPath = null;
   }
 
-  public WMPool getPool() {
+  public WMNullablePool getPool() {
     return this.pool;
   }
 
-  public void setPool(WMPool pool) {
+  public void setPool(WMNullablePool pool) {
     this.pool = pool;
   }
 
@@ -201,7 +201,7 @@ import org.slf4j.LoggerFactory;
       if (value == null) {
         unsetPool();
       } else {
-        setPool((WMPool)value);
+        setPool((WMNullablePool)value);
       }
       break;
 
@@ -409,7 +409,7 @@ import org.slf4j.LoggerFactory;
         switch (schemeField.id) {
           case 1: // POOL
             if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-              struct.pool = new WMPool();
+              struct.pool = new WMNullablePool();
               struct.pool.read(iprot);
               struct.setPoolIsSet(true);
             } else { 
@@ -489,7 +489,7 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol iprot = (TTupleProtocol) prot;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
-        struct.pool = new WMPool();
+        struct.pool = new WMNullablePool();
         struct.pool.read(iprot);
         struct.setPoolIsSet(true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMAlterResourcePlanRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMAlterResourcePlanRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMAlterResourcePlanRequest.java
index cc85d0a..4124001 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMAlterResourcePlanRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMAlterResourcePlanRequest.java
@@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory;
   }
 
   private String resourcePlanName; // optional
-  private WMResourcePlan resourcePlan; // optional
+  private WMNullableResourcePlan resourcePlan; // optional
   private boolean isEnableAndActivate; // optional
   private boolean isForceDeactivate; // optional
   private boolean isReplace; // optional
@@ -138,7 +138,7 @@ import org.slf4j.LoggerFactory;
     tmpMap.put(_Fields.RESOURCE_PLAN_NAME, new org.apache.thrift.meta_data.FieldMetaData("resourcePlanName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.RESOURCE_PLAN, new org.apache.thrift.meta_data.FieldMetaData("resourcePlan", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
-        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WMResourcePlan.class)));
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WMNullableResourcePlan.class)));
     tmpMap.put(_Fields.IS_ENABLE_AND_ACTIVATE, new org.apache.thrift.meta_data.FieldMetaData("isEnableAndActivate", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     tmpMap.put(_Fields.IS_FORCE_DEACTIVATE, new org.apache.thrift.meta_data.FieldMetaData("isForceDeactivate", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
@@ -161,7 +161,7 @@ import org.slf4j.LoggerFactory;
       this.resourcePlanName = other.resourcePlanName;
     }
     if (other.isSetResourcePlan()) {
-      this.resourcePlan = new WMResourcePlan(other.resourcePlan);
+      this.resourcePlan = new WMNullableResourcePlan(other.resourcePlan);
     }
     this.isEnableAndActivate = other.isEnableAndActivate;
     this.isForceDeactivate = other.isForceDeactivate;
@@ -207,11 +207,11 @@ import org.slf4j.LoggerFactory;
     }
   }
 
-  public WMResourcePlan getResourcePlan() {
+  public WMNullableResourcePlan getResourcePlan() {
     return this.resourcePlan;
   }
 
-  public void setResourcePlan(WMResourcePlan resourcePlan) {
+  public void setResourcePlan(WMNullableResourcePlan resourcePlan) {
     this.resourcePlan = resourcePlan;
   }
 
@@ -310,7 +310,7 @@ import org.slf4j.LoggerFactory;
       if (value == null) {
         unsetResourcePlan();
       } else {
-        setResourcePlan((WMResourcePlan)value);
+        setResourcePlan((WMNullableResourcePlan)value);
       }
       break;
 
@@ -649,7 +649,7 @@ import org.slf4j.LoggerFactory;
             break;
           case 2: // RESOURCE_PLAN
             if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-              struct.resourcePlan = new WMResourcePlan();
+              struct.resourcePlan = new WMNullableResourcePlan();
               struct.resourcePlan.read(iprot);
               struct.setResourcePlanIsSet(true);
             } else { 
@@ -782,7 +782,7 @@ import org.slf4j.LoggerFactory;
         struct.setResourcePlanNameIsSet(true);
       }
       if (incoming.get(1)) {
-        struct.resourcePlan = new WMResourcePlan();
+        struct.resourcePlan = new WMNullableResourcePlan();
         struct.resourcePlan.read(iprot);
         struct.setResourcePlanIsSet(true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMNullablePool.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMNullablePool.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMNullablePool.java
new file mode 100644
index 0000000..5e3e403
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMNullablePool.java
@@ -0,0 +1,901 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class WMNullablePool implements org.apache.thrift.TBase<WMNullablePool, WMNullablePool._Fields>, java.io.Serializable, Cloneable, Comparable<WMNullablePool> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WMNullablePool");
+
+  private static final org.apache.thrift.protocol.TField RESOURCE_PLAN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("resourcePlanName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField POOL_PATH_FIELD_DESC = new org.apache.thrift.protocol.TField("poolPath", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField ALLOC_FRACTION_FIELD_DESC = new org.apache.thrift.protocol.TField("allocFraction", org.apache.thrift.protocol.TType.DOUBLE, (short)3);
+  private static final org.apache.thrift.protocol.TField QUERY_PARALLELISM_FIELD_DESC = new org.apache.thrift.protocol.TField("queryParallelism", org.apache.thrift.protocol.TType.I32, (short)4);
+  private static final org.apache.thrift.protocol.TField SCHEDULING_POLICY_FIELD_DESC = new org.apache.thrift.protocol.TField("schedulingPolicy", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField IS_SET_SCHEDULING_POLICY_FIELD_DESC = new org.apache.thrift.protocol.TField("isSetSchedulingPolicy", org.apache.thrift.protocol.TType.BOOL, (short)6);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new WMNullablePoolStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new WMNullablePoolTupleSchemeFactory());
+  }
+
+  private String resourcePlanName; // required
+  private String poolPath; // required
+  private double allocFraction; // optional
+  private int queryParallelism; // optional
+  private String schedulingPolicy; // optional
+  private boolean isSetSchedulingPolicy; // 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 {
+    RESOURCE_PLAN_NAME((short)1, "resourcePlanName"),
+    POOL_PATH((short)2, "poolPath"),
+    ALLOC_FRACTION((short)3, "allocFraction"),
+    QUERY_PARALLELISM((short)4, "queryParallelism"),
+    SCHEDULING_POLICY((short)5, "schedulingPolicy"),
+    IS_SET_SCHEDULING_POLICY((short)6, "isSetSchedulingPolicy");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // RESOURCE_PLAN_NAME
+          return RESOURCE_PLAN_NAME;
+        case 2: // POOL_PATH
+          return POOL_PATH;
+        case 3: // ALLOC_FRACTION
+          return ALLOC_FRACTION;
+        case 4: // QUERY_PARALLELISM
+          return QUERY_PARALLELISM;
+        case 5: // SCHEDULING_POLICY
+          return SCHEDULING_POLICY;
+        case 6: // IS_SET_SCHEDULING_POLICY
+          return IS_SET_SCHEDULING_POLICY;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __ALLOCFRACTION_ISSET_ID = 0;
+  private static final int __QUERYPARALLELISM_ISSET_ID = 1;
+  private static final int __ISSETSCHEDULINGPOLICY_ISSET_ID = 2;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.ALLOC_FRACTION,_Fields.QUERY_PARALLELISM,_Fields.SCHEDULING_POLICY,_Fields.IS_SET_SCHEDULING_POLICY};
+  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.RESOURCE_PLAN_NAME, new org.apache.thrift.meta_data.FieldMetaData("resourcePlanName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.POOL_PATH, new org.apache.thrift.meta_data.FieldMetaData("poolPath", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ALLOC_FRACTION, new org.apache.thrift.meta_data.FieldMetaData("allocFraction", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.QUERY_PARALLELISM, new org.apache.thrift.meta_data.FieldMetaData("queryParallelism", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.SCHEDULING_POLICY, new org.apache.thrift.meta_data.FieldMetaData("schedulingPolicy", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.IS_SET_SCHEDULING_POLICY, new org.apache.thrift.meta_data.FieldMetaData("isSetSchedulingPolicy", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WMNullablePool.class, metaDataMap);
+  }
+
+  public WMNullablePool() {
+  }
+
+  public WMNullablePool(
+    String resourcePlanName,
+    String poolPath)
+  {
+    this();
+    this.resourcePlanName = resourcePlanName;
+    this.poolPath = poolPath;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public WMNullablePool(WMNullablePool other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetResourcePlanName()) {
+      this.resourcePlanName = other.resourcePlanName;
+    }
+    if (other.isSetPoolPath()) {
+      this.poolPath = other.poolPath;
+    }
+    this.allocFraction = other.allocFraction;
+    this.queryParallelism = other.queryParallelism;
+    if (other.isSetSchedulingPolicy()) {
+      this.schedulingPolicy = other.schedulingPolicy;
+    }
+    this.isSetSchedulingPolicy = other.isSetSchedulingPolicy;
+  }
+
+  public WMNullablePool deepCopy() {
+    return new WMNullablePool(this);
+  }
+
+  @Override
+  public void clear() {
+    this.resourcePlanName = null;
+    this.poolPath = null;
+    setAllocFractionIsSet(false);
+    this.allocFraction = 0.0;
+    setQueryParallelismIsSet(false);
+    this.queryParallelism = 0;
+    this.schedulingPolicy = null;
+    setIsSetSchedulingPolicyIsSet(false);
+    this.isSetSchedulingPolicy = false;
+  }
+
+  public String getResourcePlanName() {
+    return this.resourcePlanName;
+  }
+
+  public void setResourcePlanName(String resourcePlanName) {
+    this.resourcePlanName = resourcePlanName;
+  }
+
+  public void unsetResourcePlanName() {
+    this.resourcePlanName = null;
+  }
+
+  /** Returns true if field resourcePlanName is set (has been assigned a value) and false otherwise */
+  public boolean isSetResourcePlanName() {
+    return this.resourcePlanName != null;
+  }
+
+  public void setResourcePlanNameIsSet(boolean value) {
+    if (!value) {
+      this.resourcePlanName = null;
+    }
+  }
+
+  public String getPoolPath() {
+    return this.poolPath;
+  }
+
+  public void setPoolPath(String poolPath) {
+    this.poolPath = poolPath;
+  }
+
+  public void unsetPoolPath() {
+    this.poolPath = null;
+  }
+
+  /** Returns true if field poolPath is set (has been assigned a value) and false otherwise */
+  public boolean isSetPoolPath() {
+    return this.poolPath != null;
+  }
+
+  public void setPoolPathIsSet(boolean value) {
+    if (!value) {
+      this.poolPath = null;
+    }
+  }
+
+  public double getAllocFraction() {
+    return this.allocFraction;
+  }
+
+  public void setAllocFraction(double allocFraction) {
+    this.allocFraction = allocFraction;
+    setAllocFractionIsSet(true);
+  }
+
+  public void unsetAllocFraction() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ALLOCFRACTION_ISSET_ID);
+  }
+
+  /** Returns true if field allocFraction is set (has been assigned a value) and false otherwise */
+  public boolean isSetAllocFraction() {
+    return EncodingUtils.testBit(__isset_bitfield, __ALLOCFRACTION_ISSET_ID);
+  }
+
+  public void setAllocFractionIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ALLOCFRACTION_ISSET_ID, value);
+  }
+
+  public int getQueryParallelism() {
+    return this.queryParallelism;
+  }
+
+  public void setQueryParallelism(int queryParallelism) {
+    this.queryParallelism = queryParallelism;
+    setQueryParallelismIsSet(true);
+  }
+
+  public void unsetQueryParallelism() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __QUERYPARALLELISM_ISSET_ID);
+  }
+
+  /** Returns true if field queryParallelism is set (has been assigned a value) and false otherwise */
+  public boolean isSetQueryParallelism() {
+    return EncodingUtils.testBit(__isset_bitfield, __QUERYPARALLELISM_ISSET_ID);
+  }
+
+  public void setQueryParallelismIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __QUERYPARALLELISM_ISSET_ID, value);
+  }
+
+  public String getSchedulingPolicy() {
+    return this.schedulingPolicy;
+  }
+
+  public void setSchedulingPolicy(String schedulingPolicy) {
+    this.schedulingPolicy = schedulingPolicy;
+  }
+
+  public void unsetSchedulingPolicy() {
+    this.schedulingPolicy = null;
+  }
+
+  /** Returns true if field schedulingPolicy is set (has been assigned a value) and false otherwise */
+  public boolean isSetSchedulingPolicy() {
+    return this.schedulingPolicy != null;
+  }
+
+  public void setSchedulingPolicyIsSet(boolean value) {
+    if (!value) {
+      this.schedulingPolicy = null;
+    }
+  }
+
+  public boolean isIsSetSchedulingPolicy() {
+    return this.isSetSchedulingPolicy;
+  }
+
+  public void setIsSetSchedulingPolicy(boolean isSetSchedulingPolicy) {
+    this.isSetSchedulingPolicy = isSetSchedulingPolicy;
+    setIsSetSchedulingPolicyIsSet(true);
+  }
+
+  public void unsetIsSetSchedulingPolicy() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ISSETSCHEDULINGPOLICY_ISSET_ID);
+  }
+
+  /** Returns true if field isSetSchedulingPolicy is set (has been assigned a value) and false otherwise */
+  public boolean isSetIsSetSchedulingPolicy() {
+    return EncodingUtils.testBit(__isset_bitfield, __ISSETSCHEDULINGPOLICY_ISSET_ID);
+  }
+
+  public void setIsSetSchedulingPolicyIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISSETSCHEDULINGPOLICY_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case RESOURCE_PLAN_NAME:
+      if (value == null) {
+        unsetResourcePlanName();
+      } else {
+        setResourcePlanName((String)value);
+      }
+      break;
+
+    case POOL_PATH:
+      if (value == null) {
+        unsetPoolPath();
+      } else {
+        setPoolPath((String)value);
+      }
+      break;
+
+    case ALLOC_FRACTION:
+      if (value == null) {
+        unsetAllocFraction();
+      } else {
+        setAllocFraction((Double)value);
+      }
+      break;
+
+    case QUERY_PARALLELISM:
+      if (value == null) {
+        unsetQueryParallelism();
+      } else {
+        setQueryParallelism((Integer)value);
+      }
+      break;
+
+    case SCHEDULING_POLICY:
+      if (value == null) {
+        unsetSchedulingPolicy();
+      } else {
+        setSchedulingPolicy((String)value);
+      }
+      break;
+
+    case IS_SET_SCHEDULING_POLICY:
+      if (value == null) {
+        unsetIsSetSchedulingPolicy();
+      } else {
+        setIsSetSchedulingPolicy((Boolean)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case RESOURCE_PLAN_NAME:
+      return getResourcePlanName();
+
+    case POOL_PATH:
+      return getPoolPath();
+
+    case ALLOC_FRACTION:
+      return getAllocFraction();
+
+    case QUERY_PARALLELISM:
+      return getQueryParallelism();
+
+    case SCHEDULING_POLICY:
+      return getSchedulingPolicy();
+
+    case IS_SET_SCHEDULING_POLICY:
+      return isIsSetSchedulingPolicy();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case RESOURCE_PLAN_NAME:
+      return isSetResourcePlanName();
+    case POOL_PATH:
+      return isSetPoolPath();
+    case ALLOC_FRACTION:
+      return isSetAllocFraction();
+    case QUERY_PARALLELISM:
+      return isSetQueryParallelism();
+    case SCHEDULING_POLICY:
+      return isSetSchedulingPolicy();
+    case IS_SET_SCHEDULING_POLICY:
+      return isSetIsSetSchedulingPolicy();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof WMNullablePool)
+      return this.equals((WMNullablePool)that);
+    return false;
+  }
+
+  public boolean equals(WMNullablePool that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_resourcePlanName = true && this.isSetResourcePlanName();
+    boolean that_present_resourcePlanName = true && that.isSetResourcePlanName();
+    if (this_present_resourcePlanName || that_present_resourcePlanName) {
+      if (!(this_present_resourcePlanName && that_present_resourcePlanName))
+        return false;
+      if (!this.resourcePlanName.equals(that.resourcePlanName))
+        return false;
+    }
+
+    boolean this_present_poolPath = true && this.isSetPoolPath();
+    boolean that_present_poolPath = true && that.isSetPoolPath();
+    if (this_present_poolPath || that_present_poolPath) {
+      if (!(this_present_poolPath && that_present_poolPath))
+        return false;
+      if (!this.poolPath.equals(that.poolPath))
+        return false;
+    }
+
+    boolean this_present_allocFraction = true && this.isSetAllocFraction();
+    boolean that_present_allocFraction = true && that.isSetAllocFraction();
+    if (this_present_allocFraction || that_present_allocFraction) {
+      if (!(this_present_allocFraction && that_present_allocFraction))
+        return false;
+      if (this.allocFraction != that.allocFraction)
+        return false;
+    }
+
+    boolean this_present_queryParallelism = true && this.isSetQueryParallelism();
+    boolean that_present_queryParallelism = true && that.isSetQueryParallelism();
+    if (this_present_queryParallelism || that_present_queryParallelism) {
+      if (!(this_present_queryParallelism && that_present_queryParallelism))
+        return false;
+      if (this.queryParallelism != that.queryParallelism)
+        return false;
+    }
+
+    boolean this_present_schedulingPolicy = true && this.isSetSchedulingPolicy();
+    boolean that_present_schedulingPolicy = true && that.isSetSchedulingPolicy();
+    if (this_present_schedulingPolicy || that_present_schedulingPolicy) {
+      if (!(this_present_schedulingPolicy && that_present_schedulingPolicy))
+        return false;
+      if (!this.schedulingPolicy.equals(that.schedulingPolicy))
+        return false;
+    }
+
+    boolean this_present_isSetSchedulingPolicy = true && this.isSetIsSetSchedulingPolicy();
+    boolean that_present_isSetSchedulingPolicy = true && that.isSetIsSetSchedulingPolicy();
+    if (this_present_isSetSchedulingPolicy || that_present_isSetSchedulingPolicy) {
+      if (!(this_present_isSetSchedulingPolicy && that_present_isSetSchedulingPolicy))
+        return false;
+      if (this.isSetSchedulingPolicy != that.isSetSchedulingPolicy)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_resourcePlanName = true && (isSetResourcePlanName());
+    list.add(present_resourcePlanName);
+    if (present_resourcePlanName)
+      list.add(resourcePlanName);
+
+    boolean present_poolPath = true && (isSetPoolPath());
+    list.add(present_poolPath);
+    if (present_poolPath)
+      list.add(poolPath);
+
+    boolean present_allocFraction = true && (isSetAllocFraction());
+    list.add(present_allocFraction);
+    if (present_allocFraction)
+      list.add(allocFraction);
+
+    boolean present_queryParallelism = true && (isSetQueryParallelism());
+    list.add(present_queryParallelism);
+    if (present_queryParallelism)
+      list.add(queryParallelism);
+
+    boolean present_schedulingPolicy = true && (isSetSchedulingPolicy());
+    list.add(present_schedulingPolicy);
+    if (present_schedulingPolicy)
+      list.add(schedulingPolicy);
+
+    boolean present_isSetSchedulingPolicy = true && (isSetIsSetSchedulingPolicy());
+    list.add(present_isSetSchedulingPolicy);
+    if (present_isSetSchedulingPolicy)
+      list.add(isSetSchedulingPolicy);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(WMNullablePool other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetResourcePlanName()).compareTo(other.isSetResourcePlanName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetResourcePlanName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.resourcePlanName, other.resourcePlanName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetPoolPath()).compareTo(other.isSetPoolPath());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPoolPath()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.poolPath, other.poolPath);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetAllocFraction()).compareTo(other.isSetAllocFraction());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetAllocFraction()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.allocFraction, other.allocFraction);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetQueryParallelism()).compareTo(other.isSetQueryParallelism());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetQueryParallelism()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.queryParallelism, other.queryParallelism);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetSchedulingPolicy()).compareTo(other.isSetSchedulingPolicy());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSchedulingPolicy()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schedulingPolicy, other.schedulingPolicy);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetIsSetSchedulingPolicy()).compareTo(other.isSetIsSetSchedulingPolicy());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetIsSetSchedulingPolicy()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isSetSchedulingPolicy, other.isSetSchedulingPolicy);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("WMNullablePool(");
+    boolean first = true;
+
+    sb.append("resourcePlanName:");
+    if (this.resourcePlanName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.resourcePlanName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("poolPath:");
+    if (this.poolPath == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.poolPath);
+    }
+    first = false;
+    if (isSetAllocFraction()) {
+      if (!first) sb.append(", ");
+      sb.append("allocFraction:");
+      sb.append(this.allocFraction);
+      first = false;
+    }
+    if (isSetQueryParallelism()) {
+      if (!first) sb.append(", ");
+      sb.append("queryParallelism:");
+      sb.append(this.queryParallelism);
+      first = false;
+    }
+    if (isSetSchedulingPolicy()) {
+      if (!first) sb.append(", ");
+      sb.append("schedulingPolicy:");
+      if (this.schedulingPolicy == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.schedulingPolicy);
+      }
+      first = false;
+    }
+    if (isSetIsSetSchedulingPolicy()) {
+      if (!first) sb.append(", ");
+      sb.append("isSetSchedulingPolicy:");
+      sb.append(this.isSetSchedulingPolicy);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetResourcePlanName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'resourcePlanName' is unset! Struct:" + toString());
+    }
+
+    if (!isSetPoolPath()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'poolPath' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class WMNullablePoolStandardSchemeFactory implements SchemeFactory {
+    public WMNullablePoolStandardScheme getScheme() {
+      return new WMNullablePoolStandardScheme();
+    }
+  }
+
+  private static class WMNullablePoolStandardScheme extends StandardScheme<WMNullablePool> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, WMNullablePool struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // RESOURCE_PLAN_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.resourcePlanName = iprot.readString();
+              struct.setResourcePlanNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // POOL_PATH
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.poolPath = iprot.readString();
+              struct.setPoolPathIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // ALLOC_FRACTION
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.allocFraction = iprot.readDouble();
+              struct.setAllocFractionIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // QUERY_PARALLELISM
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.queryParallelism = iprot.readI32();
+              struct.setQueryParallelismIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // SCHEDULING_POLICY
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.schedulingPolicy = iprot.readString();
+              struct.setSchedulingPolicyIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // IS_SET_SCHEDULING_POLICY
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.isSetSchedulingPolicy = iprot.readBool();
+              struct.setIsSetSchedulingPolicyIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, WMNullablePool struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.resourcePlanName != null) {
+        oprot.writeFieldBegin(RESOURCE_PLAN_NAME_FIELD_DESC);
+        oprot.writeString(struct.resourcePlanName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.poolPath != null) {
+        oprot.writeFieldBegin(POOL_PATH_FIELD_DESC);
+        oprot.writeString(struct.poolPath);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetAllocFraction()) {
+        oprot.writeFieldBegin(ALLOC_FRACTION_FIELD_DESC);
+        oprot.writeDouble(struct.allocFraction);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetQueryParallelism()) {
+        oprot.writeFieldBegin(QUERY_PARALLELISM_FIELD_DESC);
+        oprot.writeI32(struct.queryParallelism);
+        oprot.writeFieldEnd();
+      }
+      if (struct.schedulingPolicy != null) {
+        if (struct.isSetSchedulingPolicy()) {
+          oprot.writeFieldBegin(SCHEDULING_POLICY_FIELD_DESC);
+          oprot.writeString(struct.schedulingPolicy);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.isSetIsSetSchedulingPolicy()) {
+        oprot.writeFieldBegin(IS_SET_SCHEDULING_POLICY_FIELD_DESC);
+        oprot.writeBool(struct.isSetSchedulingPolicy);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class WMNullablePoolTupleSchemeFactory implements SchemeFactory {
+    public WMNullablePoolTupleScheme getScheme() {
+      return new WMNullablePoolTupleScheme();
+    }
+  }
+
+  private static class WMNullablePoolTupleScheme extends TupleScheme<WMNullablePool> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, WMNullablePool struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.resourcePlanName);
+      oprot.writeString(struct.poolPath);
+      BitSet optionals = new BitSet();
+      if (struct.isSetAllocFraction()) {
+        optionals.set(0);
+      }
+      if (struct.isSetQueryParallelism()) {
+        optionals.set(1);
+      }
+      if (struct.isSetSchedulingPolicy()) {
+        optionals.set(2);
+      }
+      if (struct.isSetIsSetSchedulingPolicy()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
+      if (struct.isSetAllocFraction()) {
+        oprot.writeDouble(struct.allocFraction);
+      }
+      if (struct.isSetQueryParallelism()) {
+        oprot.writeI32(struct.queryParallelism);
+      }
+      if (struct.isSetSchedulingPolicy()) {
+        oprot.writeString(struct.schedulingPolicy);
+      }
+      if (struct.isSetIsSetSchedulingPolicy()) {
+        oprot.writeBool(struct.isSetSchedulingPolicy);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, WMNullablePool struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.resourcePlanName = iprot.readString();
+      struct.setResourcePlanNameIsSet(true);
+      struct.poolPath = iprot.readString();
+      struct.setPoolPathIsSet(true);
+      BitSet incoming = iprot.readBitSet(4);
+      if (incoming.get(0)) {
+        struct.allocFraction = iprot.readDouble();
+        struct.setAllocFractionIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.queryParallelism = iprot.readI32();
+        struct.setQueryParallelismIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.schedulingPolicy = iprot.readString();
+        struct.setSchedulingPolicyIsSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.isSetSchedulingPolicy = iprot.readBool();
+        struct.setIsSetSchedulingPolicyIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMNullableResourcePlan.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMNullableResourcePlan.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMNullableResourcePlan.java
new file mode 100644
index 0000000..7b7cefc
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMNullableResourcePlan.java
@@ -0,0 +1,918 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class WMNullableResourcePlan implements org.apache.thrift.TBase<WMNullableResourcePlan, WMNullableResourcePlan._Fields>, java.io.Serializable, Cloneable, Comparable<WMNullableResourcePlan> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WMNullableResourcePlan");
+
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField QUERY_PARALLELISM_FIELD_DESC = new org.apache.thrift.protocol.TField("queryParallelism", org.apache.thrift.protocol.TType.I32, (short)4);
+  private static final org.apache.thrift.protocol.TField IS_SET_QUERY_PARALLELISM_FIELD_DESC = new org.apache.thrift.protocol.TField("isSetQueryParallelism", org.apache.thrift.protocol.TType.BOOL, (short)5);
+  private static final org.apache.thrift.protocol.TField DEFAULT_POOL_PATH_FIELD_DESC = new org.apache.thrift.protocol.TField("defaultPoolPath", org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField IS_SET_DEFAULT_POOL_PATH_FIELD_DESC = new org.apache.thrift.protocol.TField("isSetDefaultPoolPath", org.apache.thrift.protocol.TType.BOOL, (short)7);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new WMNullableResourcePlanStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new WMNullableResourcePlanTupleSchemeFactory());
+  }
+
+  private String name; // required
+  private WMResourcePlanStatus status; // optional
+  private int queryParallelism; // optional
+  private boolean isSetQueryParallelism; // optional
+  private String defaultPoolPath; // optional
+  private boolean isSetDefaultPoolPath; // 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 {
+    NAME((short)1, "name"),
+    /**
+     * 
+     * @see WMResourcePlanStatus
+     */
+    STATUS((short)2, "status"),
+    QUERY_PARALLELISM((short)4, "queryParallelism"),
+    IS_SET_QUERY_PARALLELISM((short)5, "isSetQueryParallelism"),
+    DEFAULT_POOL_PATH((short)6, "defaultPoolPath"),
+    IS_SET_DEFAULT_POOL_PATH((short)7, "isSetDefaultPoolPath");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NAME
+          return NAME;
+        case 2: // STATUS
+          return STATUS;
+        case 4: // QUERY_PARALLELISM
+          return QUERY_PARALLELISM;
+        case 5: // IS_SET_QUERY_PARALLELISM
+          return IS_SET_QUERY_PARALLELISM;
+        case 6: // DEFAULT_POOL_PATH
+          return DEFAULT_POOL_PATH;
+        case 7: // IS_SET_DEFAULT_POOL_PATH
+          return IS_SET_DEFAULT_POOL_PATH;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __QUERYPARALLELISM_ISSET_ID = 0;
+  private static final int __ISSETQUERYPARALLELISM_ISSET_ID = 1;
+  private static final int __ISSETDEFAULTPOOLPATH_ISSET_ID = 2;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.STATUS,_Fields.QUERY_PARALLELISM,_Fields.IS_SET_QUERY_PARALLELISM,_Fields.DEFAULT_POOL_PATH,_Fields.IS_SET_DEFAULT_POOL_PATH};
+  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.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, WMResourcePlanStatus.class)));
+    tmpMap.put(_Fields.QUERY_PARALLELISM, new org.apache.thrift.meta_data.FieldMetaData("queryParallelism", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.IS_SET_QUERY_PARALLELISM, new org.apache.thrift.meta_data.FieldMetaData("isSetQueryParallelism", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.DEFAULT_POOL_PATH, new org.apache.thrift.meta_data.FieldMetaData("defaultPoolPath", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.IS_SET_DEFAULT_POOL_PATH, new org.apache.thrift.meta_data.FieldMetaData("isSetDefaultPoolPath", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WMNullableResourcePlan.class, metaDataMap);
+  }
+
+  public WMNullableResourcePlan() {
+  }
+
+  public WMNullableResourcePlan(
+    String name)
+  {
+    this();
+    this.name = name;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public WMNullableResourcePlan(WMNullableResourcePlan other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetName()) {
+      this.name = other.name;
+    }
+    if (other.isSetStatus()) {
+      this.status = other.status;
+    }
+    this.queryParallelism = other.queryParallelism;
+    this.isSetQueryParallelism = other.isSetQueryParallelism;
+    if (other.isSetDefaultPoolPath()) {
+      this.defaultPoolPath = other.defaultPoolPath;
+    }
+    this.isSetDefaultPoolPath = other.isSetDefaultPoolPath;
+  }
+
+  public WMNullableResourcePlan deepCopy() {
+    return new WMNullableResourcePlan(this);
+  }
+
+  @Override
+  public void clear() {
+    this.name = null;
+    this.status = null;
+    setQueryParallelismIsSet(false);
+    this.queryParallelism = 0;
+    setIsSetQueryParallelismIsSet(false);
+    this.isSetQueryParallelism = false;
+    this.defaultPoolPath = null;
+    setIsSetDefaultPoolPathIsSet(false);
+    this.isSetDefaultPoolPath = false;
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public void unsetName() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean isSetName() {
+    return this.name != null;
+  }
+
+  public void setNameIsSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  /**
+   * 
+   * @see WMResourcePlanStatus
+   */
+  public WMResourcePlanStatus getStatus() {
+    return this.status;
+  }
+
+  /**
+   * 
+   * @see WMResourcePlanStatus
+   */
+  public void setStatus(WMResourcePlanStatus status) {
+    this.status = status;
+  }
+
+  public void unsetStatus() {
+    this.status = null;
+  }
+
+  /** Returns true if field status is set (has been assigned a value) and false otherwise */
+  public boolean isSetStatus() {
+    return this.status != null;
+  }
+
+  public void setStatusIsSet(boolean value) {
+    if (!value) {
+      this.status = null;
+    }
+  }
+
+  public int getQueryParallelism() {
+    return this.queryParallelism;
+  }
+
+  public void setQueryParallelism(int queryParallelism) {
+    this.queryParallelism = queryParallelism;
+    setQueryParallelismIsSet(true);
+  }
+
+  public void unsetQueryParallelism() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __QUERYPARALLELISM_ISSET_ID);
+  }
+
+  /** Returns true if field queryParallelism is set (has been assigned a value) and false otherwise */
+  public boolean isSetQueryParallelism() {
+    return EncodingUtils.testBit(__isset_bitfield, __QUERYPARALLELISM_ISSET_ID);
+  }
+
+  public void setQueryParallelismIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __QUERYPARALLELISM_ISSET_ID, value);
+  }
+
+  public boolean isIsSetQueryParallelism() {
+    return this.isSetQueryParallelism;
+  }
+
+  public void setIsSetQueryParallelism(boolean isSetQueryParallelism) {
+    this.isSetQueryParallelism = isSetQueryParallelism;
+    setIsSetQueryParallelismIsSet(true);
+  }
+
+  public void unsetIsSetQueryParallelism() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ISSETQUERYPARALLELISM_ISSET_ID);
+  }
+
+  /** Returns true if field isSetQueryParallelism is set (has been assigned a value) and false otherwise */
+  public boolean isSetIsSetQueryParallelism() {
+    return EncodingUtils.testBit(__isset_bitfield, __ISSETQUERYPARALLELISM_ISSET_ID);
+  }
+
+  public void setIsSetQueryParallelismIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISSETQUERYPARALLELISM_ISSET_ID, value);
+  }
+
+  public String getDefaultPoolPath() {
+    return this.defaultPoolPath;
+  }
+
+  public void setDefaultPoolPath(String defaultPoolPath) {
+    this.defaultPoolPath = defaultPoolPath;
+  }
+
+  public void unsetDefaultPoolPath() {
+    this.defaultPoolPath = null;
+  }
+
+  /** Returns true if field defaultPoolPath is set (has been assigned a value) and false otherwise */
+  public boolean isSetDefaultPoolPath() {
+    return this.defaultPoolPath != null;
+  }
+
+  public void setDefaultPoolPathIsSet(boolean value) {
+    if (!value) {
+      this.defaultPoolPath = null;
+    }
+  }
+
+  public boolean isIsSetDefaultPoolPath() {
+    return this.isSetDefaultPoolPath;
+  }
+
+  public void setIsSetDefaultPoolPath(boolean isSetDefaultPoolPath) {
+    this.isSetDefaultPoolPath = isSetDefaultPoolPath;
+    setIsSetDefaultPoolPathIsSet(true);
+  }
+
+  public void unsetIsSetDefaultPoolPath() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ISSETDEFAULTPOOLPATH_ISSET_ID);
+  }
+
+  /** Returns true if field isSetDefaultPoolPath is set (has been assigned a value) and false otherwise */
+  public boolean isSetIsSetDefaultPoolPath() {
+    return EncodingUtils.testBit(__isset_bitfield, __ISSETDEFAULTPOOLPATH_ISSET_ID);
+  }
+
+  public void setIsSetDefaultPoolPathIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISSETDEFAULTPOOLPATH_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NAME:
+      if (value == null) {
+        unsetName();
+      } else {
+        setName((String)value);
+      }
+      break;
+
+    case STATUS:
+      if (value == null) {
+        unsetStatus();
+      } else {
+        setStatus((WMResourcePlanStatus)value);
+      }
+      break;
+
+    case QUERY_PARALLELISM:
+      if (value == null) {
+        unsetQueryParallelism();
+      } else {
+        setQueryParallelism((Integer)value);
+      }
+      break;
+
+    case IS_SET_QUERY_PARALLELISM:
+      if (value == null) {
+        unsetIsSetQueryParallelism();
+      } else {
+        setIsSetQueryParallelism((Boolean)value);
+      }
+      break;
+
+    case DEFAULT_POOL_PATH:
+      if (value == null) {
+        unsetDefaultPoolPath();
+      } else {
+        setDefaultPoolPath((String)value);
+      }
+      break;
+
+    case IS_SET_DEFAULT_POOL_PATH:
+      if (value == null) {
+        unsetIsSetDefaultPoolPath();
+      } else {
+        setIsSetDefaultPoolPath((Boolean)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NAME:
+      return getName();
+
+    case STATUS:
+      return getStatus();
+
+    case QUERY_PARALLELISM:
+      return getQueryParallelism();
+
+    case IS_SET_QUERY_PARALLELISM:
+      return isIsSetQueryParallelism();
+
+    case DEFAULT_POOL_PATH:
+      return getDefaultPoolPath();
+
+    case IS_SET_DEFAULT_POOL_PATH:
+      return isIsSetDefaultPoolPath();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NAME:
+      return isSetName();
+    case STATUS:
+      return isSetStatus();
+    case QUERY_PARALLELISM:
+      return isSetQueryParallelism();
+    case IS_SET_QUERY_PARALLELISM:
+      return isSetIsSetQueryParallelism();
+    case DEFAULT_POOL_PATH:
+      return isSetDefaultPoolPath();
+    case IS_SET_DEFAULT_POOL_PATH:
+      return isSetIsSetDefaultPoolPath();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof WMNullableResourcePlan)
+      return this.equals((WMNullableResourcePlan)that);
+    return false;
+  }
+
+  public boolean equals(WMNullableResourcePlan that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_name = true && this.isSetName();
+    boolean that_present_name = true && that.isSetName();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_status = true && this.isSetStatus();
+    boolean that_present_status = true && that.isSetStatus();
+    if (this_present_status || that_present_status) {
+      if (!(this_present_status && that_present_status))
+        return false;
+      if (!this.status.equals(that.status))
+        return false;
+    }
+
+    boolean this_present_queryParallelism = true && this.isSetQueryParallelism();
+    boolean that_present_queryParallelism = true && that.isSetQueryParallelism();
+    if (this_present_queryParallelism || that_present_queryParallelism) {
+      if (!(this_present_queryParallelism && that_present_queryParallelism))
+        return false;
+      if (this.queryParallelism != that.queryParallelism)
+        return false;
+    }
+
+    boolean this_present_isSetQueryParallelism = true && this.isSetIsSetQueryParallelism();
+    boolean that_present_isSetQueryParallelism = true && that.isSetIsSetQueryParallelism();
+    if (this_present_isSetQueryParallelism || that_present_isSetQueryParallelism) {
+      if (!(this_present_isSetQueryParallelism && that_present_isSetQueryParallelism))
+        return false;
+      if (this.isSetQueryParallelism != that.isSetQueryParallelism)
+        return false;
+    }
+
+    boolean this_present_defaultPoolPath = true && this.isSetDefaultPoolPath();
+    boolean that_present_defaultPoolPath = true && that.isSetDefaultPoolPath();
+    if (this_present_defaultPoolPath || that_present_defaultPoolPath) {
+      if (!(this_present_defaultPoolPath && that_present_defaultPoolPath))
+        return false;
+      if (!this.defaultPoolPath.equals(that.defaultPoolPath))
+        return false;
+    }
+
+    boolean this_present_isSetDefaultPoolPath = true && this.isSetIsSetDefaultPoolPath();
+    boolean that_present_isSetDefaultPoolPath = true && that.isSetIsSetDefaultPoolPath();
+    if (this_present_isSetDefaultPoolPath || that_present_isSetDefaultPoolPath) {
+      if (!(this_present_isSetDefaultPoolPath && that_present_isSetDefaultPoolPath))
+        return false;
+      if (this.isSetDefaultPoolPath != that.isSetDefaultPoolPath)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_name = true && (isSetName());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_status = true && (isSetStatus());
+    list.add(present_status);
+    if (present_status)
+      list.add(status.getValue());
+
+    boolean present_queryParallelism = true && (isSetQueryParallelism());
+    list.add(present_queryParallelism);
+    if (present_queryParallelism)
+      list.add(queryParallelism);
+
+    boolean present_isSetQueryParallelism = true && (isSetIsSetQueryParallelism());
+    list.add(present_isSetQueryParallelism);
+    if (present_isSetQueryParallelism)
+      list.add(isSetQueryParallelism);
+
+    boolean present_defaultPoolPath = true && (isSetDefaultPoolPath());
+    list.add(present_defaultPoolPath);
+    if (present_defaultPoolPath)
+      list.add(defaultPoolPath);
+
+    boolean present_isSetDefaultPoolPath = true && (isSetIsSetDefaultPoolPath());
+    list.add(present_isSetDefaultPoolPath);
+    if (present_isSetDefaultPoolPath)
+      list.add(isSetDefaultPoolPath);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(WMNullableResourcePlan other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetName()).compareTo(other.isSetName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetStatus()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetQueryParallelism()).compareTo(other.isSetQueryParallelism());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetQueryParallelism()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.queryParallelism, other.queryParallelism);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetIsSetQueryParallelism()).compareTo(other.isSetIsSetQueryParallelism());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetIsSetQueryParallelism()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isSetQueryParallelism, other.isSetQueryParallelism);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDefaultPoolPath()).compareTo(other.isSetDefaultPoolPath());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDefaultPoolPath()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.defaultPoolPath, other.defaultPoolPath);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetIsSetDefaultPoolPath()).compareTo(other.isSetIsSetDefaultPoolPath());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetIsSetDefaultPoolPath()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isSetDefaultPoolPath, other.isSetDefaultPoolPath);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("WMNullableResourcePlan(");
+    boolean first = true;
+
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    if (isSetStatus()) {
+      if (!first) sb.append(", ");
+      sb.append("status:");
+      if (this.status == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.status);
+      }
+      first = false;
+    }
+    if (isSetQueryParallelism()) {
+      if (!first) sb.append(", ");
+      sb.append("queryParallelism:");
+      sb.append(this.queryParallelism);
+      first = false;
+    }
+    if (isSetIsSetQueryParallelism()) {
+      if (!first) sb.append(", ");
+      sb.append("isSetQueryParallelism:");
+      sb.append(this.isSetQueryParallelism);
+      first = false;
+    }
+    if (isSetDefaultPoolPath()) {
+      if (!first) sb.append(", ");
+      sb.append("defaultPoolPath:");
+      if (this.defaultPoolPath == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.defaultPoolPath);
+      }
+      first = false;
+    }
+    if (isSetIsSetDefaultPoolPath()) {
+      if (!first) sb.append(", ");
+      sb.append("isSetDefaultPoolPath:");
+      sb.append(this.isSetDefaultPoolPath);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class WMNullableResourcePlanStandardSchemeFactory implements SchemeFactory {
+    public WMNullableResourcePlanStandardScheme getScheme() {
+      return new WMNullableResourcePlanStandardScheme();
+    }
+  }
+
+  private static class WMNullableResourcePlanStandardScheme extends StandardScheme<WMNullableResourcePlan> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, WMNullableResourcePlan struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.setNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.status = org.apache.hadoop.hive.metastore.api.WMResourcePlanStatus.findByValue(iprot.readI32());
+              struct.setStatusIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // QUERY_PARALLELISM
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.queryParallelism = iprot.readI32();
+              struct.setQueryParallelismIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // IS_SET_QUERY_PARALLELISM
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.isSetQueryParallelism = iprot.readBool();
+              struct.setIsSetQueryParallelismIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // DEFAULT_POOL_PATH
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.defaultPoolPath = iprot.readString();
+              struct.setDefaultPoolPathIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // IS_SET_DEFAULT_POOL_PATH
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.isSetDefaultPoolPath = iprot.readBool();
+              struct.setIsSetDefaultPoolPathIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, WMNullableResourcePlan struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(struct.name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.status != null) {
+        if (struct.isSetStatus()) {
+          oprot.writeFieldBegin(STATUS_FIELD_DESC);
+          oprot.writeI32(struct.status.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.isSetQueryParallelism()) {
+        oprot.writeFieldBegin(QUERY_PARALLELISM_FIELD_DESC);
+        oprot.writeI32(struct.queryParallelism);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetIsSetQueryParallelism()) {
+        oprot.writeFieldBegin(IS_SET_QUERY_PARALLELISM_FIELD_DESC);
+        oprot.writeBool(struct.isSetQueryParallelism);
+        oprot.writeFieldEnd();
+      }
+      if (struct.defaultPoolPath != null) {
+        if (struct.isSetDefaultPoolPath()) {
+          oprot.writeFieldBegin(DEFAULT_POOL_PATH_FIELD_DESC);
+          oprot.writeString(struct.defaultPoolPath);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.isSetIsSetDefaultPoolPath()) {
+        oprot.writeFieldBegin(IS_SET_DEFAULT_POOL_PATH_FIELD_DESC);
+        oprot.writeBool(struct.isSetDefaultPoolPath);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class WMNullableResourcePlanTupleSchemeFactory implements SchemeFactory {
+    public WMNullableResourcePlanTupleScheme getScheme() {
+      return new WMNullableResourcePlanTupleScheme();
+    }
+  }
+
+  private static class WMNullableResourcePlanTupleScheme extends TupleScheme<WMNullableResourcePlan> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, WMNullableResourcePlan struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.name);
+      BitSet optionals = new BitSet();
+      if (struct.isSetStatus()) {
+        optionals.set(0);
+      }
+      if (struct.isSetQueryParallelism()) {
+        optionals.set(1);
+      }
+      if (struct.isSetIsSetQueryParallelism()) {
+        optionals.set(2);
+      }
+      if (struct.isSetDefaultPoolPath()) {
+        optionals.set(3);
+      }
+      if (struct.isSetIsSetDefaultPoolPath()) {
+        optionals.set(4);
+      }
+      oprot.writeBitSet(optionals, 5);
+      if (struct.isSetStatus()) {
+        oprot.writeI32(struct.status.getValue());
+      }
+      if (struct.isSetQueryParallelism()) {
+        oprot.writeI32(struct.queryParallelism);
+      }
+      if (struct.isSetIsSetQueryParallelism()) {
+        oprot.writeBool(struct.isSetQueryParallelism);
+      }
+      if (struct.isSetDefaultPoolPath()) {
+        oprot.writeString(struct.defaultPoolPath);
+      }
+      if (struct.isSetIsSetDefaultPoolPath()) {
+        oprot.writeBool(struct.isSetDefaultPoolPath);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, WMNullableResourcePlan struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.name = iprot.readString();
+      struct.setNameIsSet(true);
+      BitSet incoming = iprot.readBitSet(5);
+      if (incoming.get(0)) {
+        struct.status = org.apache.hadoop.hive.metastore.api.WMResourcePlanStatus.findByValue(iprot.readI32());
+        struct.setStatusIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.queryParallelism = iprot.readI32();
+        struct.setQueryParallelismIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.isSetQueryParallelism = iprot.readBool();
+        struct.setIsSetQueryParallelismIsSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.defaultPoolPath = iprot.readString();
+        struct.setDefaultPoolPathIsSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.isSetDefaultPoolPath = iprot.readBool();
+        struct.setIsSetDefaultPoolPathIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 e98e3f2..6704d9b 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -21152,6 +21152,196 @@ class WMResourcePlan {
 
 }
 
+class WMNullableResourcePlan {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $name = null;
+  /**
+   * @var int
+   */
+  public $status = null;
+  /**
+   * @var int
+   */
+  public $queryParallelism = null;
+  /**
+   * @var bool
+   */
+  public $isSetQueryParallelism = null;
+  /**
+   * @var string
+   */
+  public $defaultPoolPath = null;
+  /**
+   * @var bool
+   */
+  public $isSetDefaultPoolPath = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'name',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'status',
+          'type' => TType::I32,
+          ),
+        4 => array(
+          'var' => 'queryParallelism',
+          'type' => TType::I32,
+          ),
+        5 => array(
+          'var' => 'isSetQueryParallelism',
+          'type' => TType::BOOL,
+          ),
+        6 => array(
+          'var' => 'defaultPoolPath',
+          'type' => TType::STRING,
+          ),
+        7 => array(
+          'var' => 'isSetDefaultPoolPath',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+      if (isset($vals['status'])) {
+        $this->status = $vals['status'];
+      }
+      if (isset($vals['queryParallelism'])) {
+        $this->queryParallelism = $vals['queryParallelism'];
+      }
+      if (isset($vals['isSetQueryParallelism'])) {
+        $this->isSetQueryParallelism = $vals['isSetQueryParallelism'];
+      }
+      if (isset($vals['defaultPoolPath'])) {
+        $this->defaultPoolPath = $vals['defaultPoolPath'];
+      }
+      if (isset($vals['isSetDefaultPoolPath'])) {
+        $this->isSetDefaultPoolPath = $vals['isSetDefaultPoolPath'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'WMNullableResourcePlan';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->status);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->queryParallelism);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->isSetQueryParallelism);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->defaultPoolPath);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->isSetDefaultPoolPath);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('WMNullableResourcePlan');
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
+      $xfer += $output->writeString($this->name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->status !== null) {
+      $xfer += $output->writeFieldBegin('status', TType::I32, 2);
+      $xfer += $output->writeI32($this->status);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->queryParallelism !== null) {
+      $xfer += $output->writeFieldBegin('queryParallelism', TType::I32, 4);
+      $xfer += $output->writeI32($this->queryParallelism);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->isSetQueryParallelism !== null) {
+      $xfer += $output->writeFieldBegin('isSetQueryParallelism', TType::BOOL, 5);
+      $xfer += $output->writeBool($this->isSetQueryParallelism);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->defaultPoolPath !== null) {
+      $xfer += $output->writeFieldBegin('defaultPoolPath', TType::STRING, 6);
+      $xfer += $output->writeString($this->defaultPoolPath);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->isSetDefaultPoolPath !== null) {
+      $xfer += $output->writeFieldBegin('isSetDefaultPoolPath', TType::BOOL, 7);
+      $xfer += $output->writeBool($this->isSetDefaultPoolPath);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class WMPool {
   static $_TSPEC;
 
@@ -21319,6 +21509,196 @@ class WMPool {
 
 }
 
+class WMNullablePool {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $resourcePlanName = null;
+  /**
+   * @var string
+   */
+  public $poolPath = null;
+  /**
+   * @var double
+   */
+  public $allocFraction = null;
+  /**
+   * @var int
+   */
+  public $queryParallelism = null;
+  /**
+   * @var string
+   */
+  public $schedulingPolicy = null;
+  /**
+   * @var bool
+   */
+  public $isSetSchedulingPolicy = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'resourcePlanName',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'poolPath',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'allocFraction',
+          'type' => TType::DOUBLE,
+          ),
+        4 => array(
+          'var' => 'queryParallelism',
+          'type' => TType::I32,
+          ),
+        5 => array(
+          'var' => 'schedulingPolicy',
+          'type' => TType::STRING,
+          ),
+        6 => array(
+          'var' => 'isSetSchedulingPolicy',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['resourcePlanName'])) {
+        $this->resourcePlanName = $vals['resourcePlanName'];
+      }
+      if (isset($vals['poolPath'])) {
+        $this->poolPath = $vals['poolPath'];
+      }
+      if (isset($vals['allocFraction'])) {
+        $this->allocFraction = $vals['allocFraction'];
+      }
+      if (isset($vals['queryParallelism'])) {
+        $this->queryParallelism = $vals['queryParallelism'];
+      }
+      if (isset($vals['schedulingPolicy'])) {
+        $this->schedulingPolicy = $vals['schedulingPolicy'];
+      }
+      if (isset($vals['isSetSchedulingPolicy'])) {
+        $this->isSetSchedulingPolicy = $vals['isSetSchedulingPolicy'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'WMNullablePool';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->resourcePlanName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->poolPath);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::DOUBLE) {
+            $xfer += $input->readDouble($this->allocFraction);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->queryParallelism);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->schedulingPolicy);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->isSetSchedulingPolicy);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('WMNullablePool');
+    if ($this->resourcePlanName !== null) {
+      $xfer += $output->writeFieldBegin('resourcePlanName', TType::STRING, 1);
+      $xfer += $output->writeString($this->resourcePlanName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->poolPath !== null) {
+      $xfer += $output->writeFieldBegin('poolPath', TType::STRING, 2);
+      $xfer += $output->writeString($this->poolPath);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->allocFraction !== null) {
+      $xfer += $output->writeFieldBegin('allocFraction', TType::DOUBLE, 3);
+      $xfer += $output->writeDouble($this->allocFraction);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->queryParallelism !== null) {
+      $xfer += $output->writeFieldBegin('queryParallelism', TType::I32, 4);
+      $xfer += $output->writeI32($this->queryParallelism);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->schedulingPolicy !== null) {
+      $xfer += $output->writeFieldBegin('schedulingPolicy', TType::STRING, 5);
+      $xfer += $output->writeString($this->schedulingPolicy);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->isSetSchedulingPolicy !== null) {
+      $xfer += $output->writeFieldBegin('isSetSchedulingPolicy', TType::BOOL, 6);
+      $xfer += $output->writeBool($this->isSetSchedulingPolicy);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class WMTrigger {
   static $_TSPEC;
 
@@ -22634,7 +23014,7 @@ class WMAlterResourcePlanRequest {
    */
   public $resourcePlanName = null;
   /**
-   * @var \metastore\WMResourcePlan
+   * @var \metastore\WMNullableResourcePlan
    */
   public $resourcePlan = null;
   /**
@@ -22660,7 +23040,7 @@ class WMAlterResourcePlanRequest {
         2 => array(
           'var' => 'resourcePlan',
           'type' => TType::STRUCT,
-          'class' => '\metastore\WMResourcePlan',
+          'class' => '\metastore\WMNullableResourcePlan',
           ),
         3 => array(
           'var' => 'isEnableAndActivate',
@@ -22723,7 +23103,7 @@ class WMAlterResourcePlanRequest {
           break;
         case 2:
           if ($ftype == TType::STRUCT) {
-            $this->resourcePlan = new \metastore\WMResourcePlan();
+            $this->resourcePlan = new \metastore\WMNullableResourcePlan();
             $xfer += $this->resourcePlan->read($input);
           } else {
             $xfer += $input->skip($ftype);
@@ -23899,7 +24279,7 @@ class WMAlterPoolRequest {
   static $_TSPEC;
 
   /**
-   * @var \metastore\WMPool
+   * @var \metastore\WMNullablePool
    */
   public $pool = null;
   /**
@@ -23913,7 +24293,7 @@ class WMAlterPoolRequest {
         1 => array(
           'var' => 'pool',
           'type' => TType::STRUCT,
-          'class' => '\metastore\WMPool',
+          'class' => '\metastore\WMNullablePool',
           ),
         2 => array(
           'var' => 'poolPath',
@@ -23952,7 +24332,7 @@ class WMAlterPoolRequest {
       {
         case 1:
           if ($ftype == TType::STRUCT) {
-            $this->pool = new \metastore\WMPool();
+            $this->pool = new \metastore\WMNullablePool();
             $xfer += $this->pool->read($input);
           } else {
             $xfer += $input->skip($ftype);


[6/6] hive git commit: HIVE-18438 : WM RP: it's impossible to unset things (Sergey Shelukhin, reviewed by Harish Jaiprakash, Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-18438 : WM RP: it's impossible to unset things (Sergey Shelukhin, reviewed by Harish Jaiprakash, Prasanth Jayachandran)


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

Branch: refs/heads/master
Commit: 900da82915209630ec7c194619028e5fa1b87d28
Parents: 572eacc
Author: sergey <se...@apache.org>
Authored: Thu Jan 18 13:28:54 2018 -0800
Committer: sergey <se...@apache.org>
Committed: Thu Jan 18 13:28:54 2018 -0800

----------------------------------------------------------------------
 .../listener/DummyRawStoreFailEvent.java        |    7 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |    7 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   13 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   41 +-
 .../hadoop/hive/ql/parse/ResourcePlanParser.g   |   17 +
 .../hive/ql/plan/AlterResourcePlanDesc.java     |    9 +-
 .../hive/ql/plan/CreateOrAlterWMPoolDesc.java   |   32 +-
 .../test/queries/clientpositive/resourceplan.q  |   13 +
 .../clientpositive/llap/resourceplan.q.out      |  124 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2326 +++++++++---------
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 1295 ++++++----
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  179 +-
 .../hive/metastore/api/WMAlterPoolRequest.java  |   16 +-
 .../api/WMAlterResourcePlanRequest.java         |   16 +-
 .../hive/metastore/api/WMNullablePool.java      |  901 +++++++
 .../metastore/api/WMNullableResourcePlan.java   |  918 +++++++
 .../src/gen/thrift/gen-php/metastore/Types.php  |  392 ++-
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  275 ++-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   62 +-
 .../hive/metastore/HiveMetaStoreClient.java     |    4 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |    6 +-
 .../hadoop/hive/metastore/ObjectStore.java      |   50 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |    6 +-
 .../hive/metastore/cache/CachedStore.java       |    9 +-
 .../src/main/thrift/hive_metastore.thrift       |   23 +-
 .../DummyRawStoreControlledCommit.java          |    6 +-
 .../DummyRawStoreForJdoConnection.java          |    6 +-
 27 files changed, 5018 insertions(+), 1735 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 bc9ef62..140b566 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
@@ -58,6 +58,7 @@ import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 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.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
@@ -73,6 +74,7 @@ import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMPool;
+import org.apache.hadoop.hive.metastore.api.WMNullablePool;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.ColStatsObjWithSourceInfo;
 import org.apache.thrift.TException;
@@ -997,7 +999,8 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
  
   @Override
-  public WMFullResourcePlan alterResourcePlan(String name, WMResourcePlan resourcePlan, boolean canActivateDisabled, boolean canDeactivate, boolean isReplace)
+  public WMFullResourcePlan alterResourcePlan(String name, WMNullableResourcePlan resourcePlan,
+      boolean canActivateDisabled, boolean canDeactivate, boolean isReplace)
       throws AlreadyExistsException, NoSuchObjectException, InvalidOperationException, MetaException {
     return objectStore.alterResourcePlan(name, resourcePlan, canActivateDisabled, canDeactivate, isReplace);
   }
@@ -1050,7 +1053,7 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public void alterPool(WMPool pool, String poolPath) throws AlreadyExistsException,
+  public void alterPool(WMNullablePool pool, String poolPath) throws AlreadyExistsException,
       NoSuchObjectException, InvalidOperationException, MetaException {
     objectStore.alterPool(pool, poolPath);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 81e1dd9..aa66a39 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
@@ -105,6 +105,7 @@ import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.TxnInfo;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
+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;
@@ -725,7 +726,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       return 0;
     }
 
-    WMResourcePlan resourcePlan = desc.getResourcePlan();
+    WMNullableResourcePlan resourcePlan = desc.getResourcePlan();
     final WorkloadManager wm = WorkloadManager.getInstance();
     final TezSessionPoolManager pm = TezSessionPoolManager.getInstance();
     boolean isActivate = false, isInTest = HiveConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST);
@@ -818,9 +819,9 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
   private int createOrAlterWMPool(Hive db, CreateOrAlterWMPoolDesc desc) throws HiveException {
     if (desc.isUpdate()) {
-      db.alterWMPool(desc.getPool(), desc.getPoolPath());
+      db.alterWMPool(desc.getAlterPool(), desc.getPoolPath());
     } else {
-      db.createWMPool(desc.getPool());
+      db.createWMPool(desc.getCreatePool());
     }
     return 0;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 aa9bf9a..16861bc 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
@@ -132,6 +132,8 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.UniqueConstraintsRequest;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
+import org.apache.hadoop.hive.metastore.api.WMNullablePool;
+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;
@@ -4857,12 +4859,11 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
   }
 
-  public WMFullResourcePlan alterResourcePlan(String rpName, WMResourcePlan resourcePlan,
-      boolean canActivateDisabled, boolean isForceDeactivate, boolean isReplace)
-          throws HiveException {
+  public WMFullResourcePlan alterResourcePlan(String rpName, WMNullableResourcePlan resourcePlan,
+      boolean canActivateDisabled, boolean isForceDeactivate, boolean isReplace) throws HiveException {
     try {
-      return getMSC().alterResourcePlan(
-          rpName, resourcePlan, canActivateDisabled, isForceDeactivate, isReplace);
+      return getMSC().alterResourcePlan(rpName, resourcePlan, canActivateDisabled,
+          isForceDeactivate, isReplace);
     } catch (Exception e) {
       throw new HiveException(e);
     }
@@ -4916,7 +4917,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
   }
 
-  public void alterWMPool(WMPool pool, String poolPath) throws HiveException {
+  public void alterWMPool(WMNullablePool pool, String poolPath) throws HiveException {
     try {
       getMSC().alterWMPool(pool, poolPath);
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index db86320..a731579 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -62,6 +62,8 @@ import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
+import org.apache.hadoop.hive.metastore.api.WMNullablePool;
+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.WMResourcePlanStatus;
@@ -951,7 +953,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       // This command exists solely to output this message. TODO: can we do it w/o an error?
       throw new SemanticException("Activate a resource plan to enable workload management");
     case HiveParser.TOK_DISABLE:
-      WMResourcePlan anyRp = new WMResourcePlan();
+      WMNullableResourcePlan anyRp = new WMNullableResourcePlan();
       anyRp.setStatus(WMResourcePlanStatus.ENABLED);
       AlterResourcePlanDesc desc = new AlterResourcePlanDesc(
           anyRp, null, false, false, true, false);
@@ -963,7 +965,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       throw new SemanticException("Invalid syntax for ALTER RESOURCE PLAN statement");
     }
     String rpName = unescapeIdentifier(ast.getChild(0).getText());
-    WMResourcePlan resourcePlan = new WMResourcePlan();
+    WMNullableResourcePlan resourcePlan = new WMNullableResourcePlan();
     boolean isEnableActivate = false, isReplace = false;
     boolean validate = false;
     for (int i = 1; i < ast.getChildCount(); ++i) {
@@ -1008,18 +1010,32 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           resourcePlan.setStatus(WMResourcePlanStatus.ACTIVE);
         }
         break;
-      case HiveParser.TOK_QUERY_PARALLELISM:
+      case HiveParser.TOK_QUERY_PARALLELISM: {
         if (child.getChildCount() != 1) {
           throw new SemanticException("Expected one argument");
         }
-        resourcePlan.setQueryParallelism(Integer.parseInt(child.getChild(0).getText()));
+        Tree val = child.getChild(0);
+        resourcePlan.setIsSetQueryParallelism(true);
+        if (val.getType() == HiveParser.TOK_NULL) {
+          resourcePlan.unsetQueryParallelism();
+        } else {
+          resourcePlan.setQueryParallelism(Integer.parseInt(val.getText()));
+        }
         break;
-      case HiveParser.TOK_DEFAULT_POOL:
+      }
+      case HiveParser.TOK_DEFAULT_POOL: {
         if (child.getChildCount() != 1) {
           throw new SemanticException("Expected one argument");
         }
-        resourcePlan.setDefaultPoolPath(poolPath(child.getChild(0)));
+        Tree val = child.getChild(0);
+        resourcePlan.setIsSetDefaultPoolPath(true);
+        if (val.getType() == HiveParser.TOK_NULL) {
+          resourcePlan.unsetDefaultPoolPath();
+        } else {
+          resourcePlan.setDefaultPoolPath(poolPath(child.getChild(0)));
+        }
         break;
+      }
       case HiveParser.TOK_RENAME:
         if (child.getChildCount() != 1) {
           throw new SemanticException("Expected one argument");
@@ -1179,7 +1195,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
   private void analyzeAlterPool(ASTNode ast) throws SemanticException {
     if (ast.getChildCount() < 3) {
-      throw new SemanticException("Invalid syntax for alter pool.");
+      throw new SemanticException("Invalid syntax for alter pool: " + ast.toStringTree());
     }
     String rpName = unescapeIdentifier(ast.getChild(0).getText());
     Tree poolTarget = ast.getChild(1);
@@ -1192,9 +1208,11 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       poolPath = poolPath(ast.getChild(1));
     }
 
-    WMPool poolChanges = null;
+    WMNullablePool poolChanges = null;
     for (int i = 2; i < ast.getChildCount(); ++i) {
       Tree child = ast.getChild(i);
+                    LOG.error("TODO# got2 " + child.toStringTree());
+
       if (child.getChildCount() != 1) {
         throw new SemanticException("Invalid syntax in alter pool expected parameter.");
       }
@@ -1211,7 +1229,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           throw new SemanticException("Cannot alter the unmanaged pool");
         }
         if (poolChanges == null) {
-          poolChanges = new WMPool(rpName, null);
+          poolChanges = new WMNullablePool(rpName, null);
         }
         switch (child.getType()) {
         case HiveParser.TOK_ALLOC_FRACTION:
@@ -1221,7 +1239,10 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           poolChanges.setQueryParallelism(Integer.parseInt(param.getText()));
           break;
         case HiveParser.TOK_SCHEDULING_POLICY:
-          poolChanges.setSchedulingPolicy(PlanUtils.stripQuotes(param.getText()));
+          poolChanges.setIsSetSchedulingPolicy(true);
+          if (param.getType() != HiveParser.TOK_NULL) {
+            poolChanges.setSchedulingPolicy(PlanUtils.stripQuotes(param.getText()));
+          }
           break;
         case HiveParser.TOK_PATH:
           poolChanges.setPoolPath(poolPath(param));

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 b47f7b3..e3ea1f2 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
@@ -56,6 +56,21 @@ rpAssignList
   : rpAssign (COMMA rpAssign)* -> rpAssign+
   ;
 
+rpUnassign
+@init { gParent.pushMsg("rpAssign", state); }
+@after { gParent.popMsg(state); }
+  : (
+      (KW_QUERY_PARALLELISM) -> ^(TOK_QUERY_PARALLELISM TOK_NULL)
+    | (KW_DEFAULT KW_POOL) -> ^(TOK_DEFAULT_POOL TOK_NULL)
+    )
+  ;
+
+rpUnassignList
+@init { gParent.pushMsg("rpAssignList", state); }
+@after { gParent.popMsg(state); }
+  : rpUnassign (COMMA rpUnassign)* -> rpUnassign+
+  ;
+
 createResourcePlanStatement
 @init { gParent.pushMsg("create resource plan statement", state); }
 @after { gParent.popMsg(state); }
@@ -80,6 +95,7 @@ alterResourcePlanStatement
           (KW_VALIDATE -> ^(TOK_ALTER_RP $name TOK_VALIDATE))
         | (KW_DISABLE -> ^(TOK_ALTER_RP $name TOK_DISABLE))
         | (KW_SET rpAssignList -> ^(TOK_ALTER_RP $name rpAssignList))
+        | (KW_UNSET rpUnassignList -> ^(TOK_ALTER_RP $name rpUnassignList))
         | (KW_RENAME KW_TO newName=identifier -> ^(TOK_ALTER_RP $name ^(TOK_RENAME $newName)))
         | ((activate enable? | enable activate?) -> ^(TOK_ALTER_RP $name activate? enable?))
       )
@@ -219,6 +235,7 @@ alterPoolStatement
 @after { gParent.popMsg(state); }
     : KW_ALTER KW_POOL rpName=identifier DOT poolPath (
         (KW_SET poolAssignList -> ^(TOK_ALTER_POOL $rpName poolPath poolAssignList))
+        | (KW_UNSET KW_SCHEDULING_POLICY -> ^(TOK_ALTER_POOL $rpName poolPath ^(TOK_SCHEDULING_POLICY TOK_NULL)))
         | (KW_ADD KW_TRIGGER triggerName=identifier
             -> ^(TOK_ALTER_POOL $rpName poolPath ^(TOK_ADD_TRIGGER $triggerName)))
         | (KW_DROP KW_TRIGGER triggerName=identifier

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterResourcePlanDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterResourcePlanDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterResourcePlanDesc.java
index 7339520..dd2eace 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterResourcePlanDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterResourcePlanDesc.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
+import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
 
@@ -28,7 +29,7 @@ import org.apache.hadoop.hive.ql.plan.Explain.Level;
 public class AlterResourcePlanDesc extends DDLDesc implements Serializable {
   private static final long serialVersionUID = -3514685833183437279L;
 
-  private WMResourcePlan resourcePlan;
+  private WMNullableResourcePlan resourcePlan;
   private String rpName;
   private boolean validate;
   private boolean isEnableActivate, isForceDeactivate, isReplace;
@@ -36,7 +37,7 @@ public class AlterResourcePlanDesc extends DDLDesc implements Serializable {
 
   public AlterResourcePlanDesc() {}
 
-  public AlterResourcePlanDesc(WMResourcePlan resourcePlan, String rpName, boolean validate,
+  public AlterResourcePlanDesc(WMNullableResourcePlan resourcePlan, String rpName, boolean validate,
       boolean isEnableActivate, boolean isForceDeactivate, boolean isReplace) {
     this.resourcePlan = resourcePlan;
     this.rpName = rpName;
@@ -48,11 +49,11 @@ public class AlterResourcePlanDesc extends DDLDesc implements Serializable {
 
   @Explain(displayName="Resource plan changed fields",
       explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
-  public WMResourcePlan getResourcePlan() {
+  public WMNullableResourcePlan getResourcePlan() {
     return resourcePlan;
   }
 
-  public void setResourcePlan(WMResourcePlan resourcePlan) {
+  public void setResourcePlan(WMNullableResourcePlan resourcePlan) {
     this.resourcePlan = resourcePlan;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateOrAlterWMPoolDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateOrAlterWMPoolDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateOrAlterWMPoolDesc.java
index 73806c5..6835faa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateOrAlterWMPoolDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateOrAlterWMPoolDesc.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
+import org.apache.hadoop.hive.metastore.api.WMNullablePool;
 import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
 
@@ -26,25 +27,44 @@ import org.apache.hadoop.hive.ql.plan.Explain.Level;
 public class CreateOrAlterWMPoolDesc extends DDLDesc implements Serializable {
   private static final long serialVersionUID = 4872940135771213510L;
 
-  private WMPool pool;
+  private WMPool createPool;
+  private WMNullablePool alterPool;
   private String poolPath;
   private boolean update;
 
   public CreateOrAlterWMPoolDesc() {}
 
   public CreateOrAlterWMPoolDesc(WMPool pool, String poolPath, boolean update) {
-    this.pool = pool;
+    this.createPool = pool;
+    this.poolPath = poolPath;
+    this.update = update;
+  }
+
+  public CreateOrAlterWMPoolDesc(WMNullablePool pool, String poolPath, boolean update) {
+    this.alterPool = pool;
     this.poolPath = poolPath;
     this.update = update;
   }
 
   @Explain(displayName="pool", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
-  public WMPool getPool() {
-    return pool;
+  public Object getPool() {
+    return createPool == null ? alterPool : createPool;
+  }
+
+  public WMPool getCreatePool() {
+    return createPool;
+  }
+
+  public WMNullablePool getAlterPool() {
+    return alterPool;
+  }
+
+  public void setCreatePool(WMPool pool) {
+    this.createPool = pool;
   }
 
-  public void setPool(WMPool pool) {
-    this.pool = pool;
+  public void setAlterPool(WMNullablePool pool) {
+    this.alterPool = pool;
   }
 
   @Explain(displayName="poolPath", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 c616c22..d2aec73 100644
--- a/ql/src/test/queries/clientpositive/resourceplan.q
+++ b/ql/src/test/queries/clientpositive/resourceplan.q
@@ -51,6 +51,11 @@ SELECT * FROM SYS.WM_RESOURCEPLANS;
 ALTER RESOURCE PLAN plan_3 SET QUERY_PARALLELISM = 4;
 SELECT * FROM SYS.WM_RESOURCEPLANS;
 
+-- Change query parallelism, success.
+ALTER RESOURCE PLAN plan_3 UNSET QUERY_PARALLELISM;
+SELECT * FROM SYS.WM_RESOURCEPLANS;
+
+
 -- Will fail for now; there are no pools.
 ALTER RESOURCE PLAN plan_3 SET QUERY_PARALLELISM = 30, DEFAULT POOL = default1;
 SELECT * FROM SYS.WM_RESOURCEPLANS;
@@ -214,6 +219,10 @@ 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 SCHEDULING_POLICY='fair';
+SELECT * FROM SYS.WM_POOLS;
+ALTER POOL plan_2.default.c2 UNSET SCHEDULING_POLICY;
+SELECT * FROM SYS.WM_POOLS;
 
 -- Now we can activate.
 ALTER RESOURCE PLAN plan_2 VALIDATE;
@@ -262,6 +271,10 @@ ALTER RESOURCE PLAN `table` SET DEFAULT POOL = `table`.pool;
 DROP POOL `table`.default;
 SELECT * FROM SYS.WM_POOLS;
 
+-- Change query parallelism, success.
+ALTER RESOURCE PLAN `table` UNSET DEFAULT POOL;
+SELECT * FROM SYS.WM_RESOURCEPLANS;
+
 --
 -- Pool to trigger mappings.
 --

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 22883bc..68508e9 100644
--- a/ql/src/test/results/clientpositive/llap/resourceplan.q.out
+++ b/ql/src/test/results/clientpositive/llap/resourceplan.q.out
@@ -3279,6 +3279,20 @@ POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	DISABLED	4	default
 plan_3	DISABLED	4	default
+PREHOOK: query: ALTER RESOURCE PLAN plan_3 UNSET QUERY_PARALLELISM
+PREHOOK: type: ALTER RESOURCEPLAN
+POSTHOOK: query: ALTER RESOURCE PLAN plan_3 UNSET QUERY_PARALLELISM
+POSTHOOK: type: ALTER RESOURCEPLAN
+PREHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
+PREHOOK: type: QUERY
+PREHOOK: Input: sys@wm_resourceplans
+#### A masked pattern was here ####
+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_3	DISABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 SET QUERY_PARALLELISM = 30, DEFAULT POOL = default1
 PREHOOK: type: ALTER RESOURCEPLAN
 FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. NoSuchObjectException(message:Cannot find pool: default1)
@@ -3291,7 +3305,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	DISABLED	4	default
-plan_3	DISABLED	4	default
+plan_3	DISABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 ENABLE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_3 ENABLE
@@ -3315,7 +3329,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	DISABLED	4	default
-plan_3	DISABLED	4	default
+plan_3	DISABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 ACTIVATE
 PREHOOK: type: ALTER RESOURCEPLAN
 FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidOperationException(message:Resource plan plan_3 is disabled and should be enabled before activation (or in the same command))
@@ -3328,7 +3342,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	DISABLED	4	default
-plan_3	DISABLED	4	default
+plan_3	DISABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 DISABLE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_3 DISABLE
@@ -3342,7 +3356,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	DISABLED	4	default
-plan_3	DISABLED	4	default
+plan_3	DISABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 ENABLE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_3 ENABLE
@@ -3356,7 +3370,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	DISABLED	4	default
-plan_3	ENABLED	4	default
+plan_3	ENABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 ACTIVATE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_3 ACTIVATE
@@ -3370,7 +3384,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	DISABLED	4	default
-plan_3	ACTIVE	4	default
+plan_3	ACTIVE	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 ACTIVATE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_3 ACTIVATE
@@ -3384,7 +3398,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	DISABLED	4	default
-plan_3	ACTIVE	4	default
+plan_3	ACTIVE	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 ENABLE
 PREHOOK: type: ALTER RESOURCEPLAN
 FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidOperationException(message:Resource plan plan_3 is active; activate another plan first, or disable workload management.)
@@ -3397,7 +3411,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	DISABLED	4	default
-plan_3	ACTIVE	4	default
+plan_3	ACTIVE	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 DISABLE
 PREHOOK: type: ALTER RESOURCEPLAN
 FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidOperationException(message:Resource plan plan_3 is active; activate another plan first, or disable workload management.)
@@ -3410,7 +3424,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	DISABLED	4	default
-plan_3	ACTIVE	4	default
+plan_3	ACTIVE	NULL	default
 PREHOOK: query: DISABLE WORKLOAD MANAGEMENT
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: DISABLE WORKLOAD MANAGEMENT
@@ -3424,7 +3438,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	DISABLED	4	default
-plan_3	ENABLED	4	default
+plan_3	ENABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 DISABLE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_3 DISABLE
@@ -3442,7 +3456,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	DISABLED	4	default
-plan_3	ACTIVE	4	default
+plan_3	ACTIVE	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_2 ENABLE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_2 ENABLE
@@ -3456,7 +3470,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	ENABLED	4	default
-plan_3	ACTIVE	4	default
+plan_3	ACTIVE	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_2 ACTIVATE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_2 ACTIVATE
@@ -3470,7 +3484,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	ACTIVE	4	default
-plan_3	ENABLED	4	default
+plan_3	ENABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 ENABLE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_3 ENABLE
@@ -3484,7 +3498,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	ACTIVE	4	default
-plan_3	ENABLED	4	default
+plan_3	ENABLED	NULL	default
 PREHOOK: query: ALTER RESOURCE PLAN plan_3 DISABLE
 PREHOOK: type: ALTER RESOURCEPLAN
 POSTHOOK: query: ALTER RESOURCE PLAN plan_3 DISABLE
@@ -3498,7 +3512,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@wm_resourceplans
 #### A masked pattern was here ####
 plan_2	ACTIVE	4	default
-plan_3	DISABLED	4	default
+plan_3	DISABLED	NULL	default
 PREHOOK: query: DROP RESOURCE PLAN plan_2
 PREHOOK: type: DROP RESOURCEPLAN
 FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:Cannot drop an active resource plan)
@@ -3757,6 +3771,40 @@ PREHOOK: query: ALTER POOL plan_2.default.c2 SET ALLOC_FRACTION = 0.7, QUERY_PAR
 PREHOOK: type: ALTER POOL
 POSTHOOK: query: ALTER POOL plan_2.default.c2 SET ALLOC_FRACTION = 0.7, QUERY_PARALLELISM = 1
 POSTHOOK: type: ALTER POOL
+PREHOOK: query: ALTER POOL plan_2.default.c2 SET SCHEDULING_POLICY='fair'
+PREHOOK: type: ALTER POOL
+POSTHOOK: query: ALTER POOL plan_2.default.c2 SET SCHEDULING_POLICY='fair'
+POSTHOOK: type: ALTER POOL
+PREHOOK: query: SELECT * FROM SYS.WM_POOLS
+PREHOOK: type: QUERY
+PREHOOK: Input: sys@wm_pools
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM SYS.WM_POOLS
+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.c1	0.3	3	fair
+plan_2	default.c2	0.7	1	fair
+table	default	1.0	4	NULL
+PREHOOK: query: ALTER POOL plan_2.default.c2 UNSET SCHEDULING_POLICY
+PREHOOK: type: ALTER POOL
+POSTHOOK: query: ALTER POOL plan_2.default.c2 UNSET SCHEDULING_POLICY
+POSTHOOK: type: ALTER POOL
+PREHOOK: query: SELECT * FROM SYS.WM_POOLS
+PREHOOK: type: QUERY
+PREHOOK: Input: sys@wm_pools
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM SYS.WM_POOLS
+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.c1	0.3	3	fair
+plan_2	default.c2	0.7	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
@@ -3788,7 +3836,7 @@ POSTHOOK: Input: sys@wm_pools
 plan_1	default	1.0	4	NULL
 plan_2	def	1.0	4	NULL
 plan_2	def.c1	0.3	3	fair
-plan_2	def.c2	0.7	1	fair
+plan_2	def.c2	0.7	1	NULL
 table	default	1.0	4	NULL
 PREHOOK: query: DROP POOL plan_2.default
 PREHOOK: type: DROP POOL
@@ -3804,7 +3852,7 @@ POSTHOOK: Input: sys@wm_pools
 plan_1	default	1.0	4	NULL
 plan_2	def	1.0	4	NULL
 plan_2	def.c1	0.3	3	fair
-plan_2	def.c2	0.7	1	fair
+plan_2	def.c2	0.7	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
@@ -3849,7 +3897,7 @@ POSTHOOK: Input: sys@wm_pools
 plan_1	default	1.0	4	NULL
 plan_2	def	1.0	4	NULL
 plan_2	def.c1	0.3	3	fair
-plan_2	def.c2	0.7	1	fair
+plan_2	def.c2	0.7	1	NULL
 table	default	1.0	4	NULL
 table	table	0.0	1	fifo
 table	table.pool1	0.9	3	fair
@@ -3870,7 +3918,7 @@ POSTHOOK: Input: sys@wm_pools
 plan_1	default	1.0	4	NULL
 plan_2	def	1.0	4	NULL
 plan_2	def.c1	0.3	3	fair
-plan_2	def.c2	0.7	1	fair
+plan_2	def.c2	0.7	1	NULL
 table	default	1.0	4	NULL
 table	table	0.0	1	fifo
 table	table.pool	0.9	3	fair
@@ -3890,7 +3938,7 @@ POSTHOOK: Input: sys@wm_pools
 plan_1	default	1.0	4	NULL
 plan_2	def	1.0	4	NULL
 plan_2	def.c1	0.3	3	fair
-plan_2	def.c2	0.7	1	fair
+plan_2	def.c2	0.7	1	NULL
 table	default	1.0	4	NULL
 table	table	0.0	1	fifo
 table	table.pool	0.9	3	fair
@@ -3910,7 +3958,7 @@ POSTHOOK: Input: sys@wm_pools
 plan_1	default	1.0	4	NULL
 plan_2	def	1.0	4	NULL
 plan_2	def.c1	0.3	3	fair
-plan_2	def.c2	0.7	1	fair
+plan_2	def.c2	0.7	1	NULL
 table	default	1.0	4	NULL
 table	table	0.0	1	fifo
 table	table.pool	0.9	3	fair
@@ -3946,11 +3994,26 @@ POSTHOOK: Input: sys@wm_pools
 plan_1	default	1.0	4	NULL
 plan_2	def	1.0	4	NULL
 plan_2	def.c1	0.3	3	fair
-plan_2	def.c2	0.7	1	fair
+plan_2	def.c2	0.7	1	NULL
 table	table	0.0	1	fifo
 table	table.pool	0.9	3	fair
 table	table.pool.child1	0.3	1	fair
 table	table.pool.child2	0.7	3	fair
+PREHOOK: query: ALTER RESOURCE PLAN `table` UNSET DEFAULT POOL
+PREHOOK: type: ALTER RESOURCEPLAN
+POSTHOOK: query: ALTER RESOURCE PLAN `table` UNSET DEFAULT POOL
+POSTHOOK: type: ALTER RESOURCEPLAN
+PREHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
+PREHOOK: type: QUERY
+PREHOOK: Input: sys@wm_resourceplans
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM SYS.WM_RESOURCEPLANS
+POSTHOOK: type: QUERY
+POSTHOOK: Input: sys@wm_resourceplans
+#### A masked pattern was here ####
+plan_1	ACTIVE	NULL	default
+plan_2	DISABLED	4	def
+table	DISABLED	1	NULL
 PREHOOK: query: ALTER POOL plan_2.def.c1 ADD TRIGGER trigger_1
 PREHOOK: type: ALTER POOL
 POSTHOOK: query: ALTER POOL plan_2.def.c1 ADD TRIGGER trigger_1
@@ -4003,11 +4066,10 @@ PREHOOK: query: SHOW RESOURCE PLAN `table`
 PREHOOK: type: SHOW RESOURCEPLAN
 POSTHOOK: query: SHOW RESOURCE PLAN `table`
 POSTHOOK: type: SHOW RESOURCEPLAN
-table[status=DISABLED,parallelism=1,defaultPool=table.pool]
+table[status=DISABLED,parallelism=1,defaultPool=null]
  +  table[allocFraction=0.0,schedulingPolicy=fifo,parallelism=1]
      |  trigger table: if (BYTES_WRITTEN > 100K) { MOVE TO table }
      +  pool[allocFraction=0.9,schedulingPolicy=fair,parallelism=3]
-         |  mapped for default
          +  child2[allocFraction=0.7,schedulingPolicy=fair,parallelism=3]
              |  trigger trigger2: if (BYTES_READ > 100) { KILL }
              |  trigger trigger1: if (ELAPSED_TIME > 10) { KILL }
@@ -4121,7 +4183,7 @@ plan_2[status=DISABLED,parallelism=4,defaultPool=def]
  +  def[allocFraction=1.0,schedulingPolicy=null,parallelism=4]
      |  mapped for users: user2
      |  mapped for default
-     +  c2[allocFraction=0.7,schedulingPolicy=fair,parallelism=1]
+     +  c2[allocFraction=0.7,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]
@@ -4188,7 +4250,7 @@ POSTHOOK: type: SHOW RESOURCEPLAN
 plan_2[status=DISABLED,parallelism=4,defaultPool=def]
  +  def[allocFraction=1.0,schedulingPolicy=null,parallelism=4]
      |  mapped for default
-     +  c2[allocFraction=0.7,schedulingPolicy=fair,parallelism=1]
+     +  c2[allocFraction=0.7,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
@@ -4213,7 +4275,7 @@ POSTHOOK: Input: sys@wm_resourceplans
 plan_1	ENABLED	NULL	default
 plan_2	DISABLED	NULL	default
 plan_4	ACTIVE	NULL	default
-table	DISABLED	1	table.pool
+table	DISABLED	1	NULL
 PREHOOK: query: SELECT * FROM SYS.WM_POOLS
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@wm_pools
@@ -4302,7 +4364,7 @@ plan_2	DISABLED	NULL	default
 plan_4	ACTIVE	NULL	default
 plan_4a	DISABLED	NULL	default
 plan_4b	DISABLED	NULL	default
-table	DISABLED	1	table.pool
+table	DISABLED	1	NULL
 PREHOOK: query: SELECT * FROM SYS.WM_POOLS
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@wm_pools
@@ -4376,7 +4438,7 @@ plan_2	DISABLED	NULL	default
 plan_4	ACTIVE	NULL	default
 plan_4a	DISABLED	NULL	default
 plan_4a-old-0	DISABLED	NULL	default
-table	DISABLED	1	table.pool
+table	DISABLED	1	NULL
 PREHOOK: query: SELECT * FROM SYS.WM_POOLS
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@wm_pools
@@ -4412,7 +4474,7 @@ plan_2	DISABLED	NULL	default
 plan_4	ACTIVE	NULL	default
 plan_4-old-0	DISABLED	NULL	default
 plan_4a-old-0	DISABLED	NULL	default
-table	DISABLED	1	table.pool
+table	DISABLED	1	NULL
 PREHOOK: query: CREATE RESOURCE PLAN plan_4a LIKE plan_4
 PREHOOK: type: CREATE RESOURCEPLAN
 POSTHOOK: query: CREATE RESOURCE PLAN plan_4a LIKE plan_4
@@ -4439,7 +4501,7 @@ plan_4	ACTIVE	NULL	default
 plan_4-old-0	DISABLED	NULL	default
 plan_4-old-1	DISABLED	NULL	default
 plan_4a-old-0	DISABLED	NULL	default
-table	DISABLED	1	table.pool
+table	DISABLED	1	NULL
 PREHOOK: query: SELECT * FROM SYS.WM_POOLS
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@wm_pools


[5/6] hive git commit: HIVE-18438 : WM RP: it's impossible to unset things (Sergey Shelukhin, reviewed by Harish Jaiprakash, Prasanth Jayachandran)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 fc80011..49e99d3 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 _size1045;
-            ::apache::thrift::protocol::TType _etype1048;
-            xfer += iprot->readListBegin(_etype1048, _size1045);
-            this->success.resize(_size1045);
-            uint32_t _i1049;
-            for (_i1049 = 0; _i1049 < _size1045; ++_i1049)
+            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)
             {
-              xfer += iprot->readString(this->success[_i1049]);
+              xfer += iprot->readString(this->success[_i1054]);
             }
             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 _iter1050;
-      for (_iter1050 = this->success.begin(); _iter1050 != this->success.end(); ++_iter1050)
+      std::vector<std::string> ::const_iterator _iter1055;
+      for (_iter1055 = this->success.begin(); _iter1055 != this->success.end(); ++_iter1055)
       {
-        xfer += oprot->writeString((*_iter1050));
+        xfer += oprot->writeString((*_iter1055));
       }
       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 _size1051;
-            ::apache::thrift::protocol::TType _etype1054;
-            xfer += iprot->readListBegin(_etype1054, _size1051);
-            (*(this->success)).resize(_size1051);
-            uint32_t _i1055;
-            for (_i1055 = 0; _i1055 < _size1051; ++_i1055)
+            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))[_i1055]);
+              xfer += iprot->readString((*(this->success))[_i1060]);
             }
             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 _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 _size1061;
+            ::apache::thrift::protocol::TType _etype1064;
+            xfer += iprot->readListBegin(_etype1064, _size1061);
+            this->success.resize(_size1061);
+            uint32_t _i1065;
+            for (_i1065 = 0; _i1065 < _size1061; ++_i1065)
             {
-              xfer += iprot->readString(this->success[_i1060]);
+              xfer += iprot->readString(this->success[_i1065]);
             }
             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 _iter1061;
-      for (_iter1061 = this->success.begin(); _iter1061 != this->success.end(); ++_iter1061)
+      std::vector<std::string> ::const_iterator _iter1066;
+      for (_iter1066 = this->success.begin(); _iter1066 != this->success.end(); ++_iter1066)
       {
-        xfer += oprot->writeString((*_iter1061));
+        xfer += oprot->writeString((*_iter1066));
       }
       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 _size1062;
-            ::apache::thrift::protocol::TType _etype1065;
-            xfer += iprot->readListBegin(_etype1065, _size1062);
-            (*(this->success)).resize(_size1062);
-            uint32_t _i1066;
-            for (_i1066 = 0; _i1066 < _size1062; ++_i1066)
+            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))[_i1066]);
+              xfer += iprot->readString((*(this->success))[_i1071]);
             }
             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 _size1067;
-            ::apache::thrift::protocol::TType _ktype1068;
-            ::apache::thrift::protocol::TType _vtype1069;
-            xfer += iprot->readMapBegin(_ktype1068, _vtype1069, _size1067);
-            uint32_t _i1071;
-            for (_i1071 = 0; _i1071 < _size1067; ++_i1071)
+            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)
             {
-              std::string _key1072;
-              xfer += iprot->readString(_key1072);
-              Type& _val1073 = this->success[_key1072];
-              xfer += _val1073.read(iprot);
+              std::string _key1077;
+              xfer += iprot->readString(_key1077);
+              Type& _val1078 = this->success[_key1077];
+              xfer += _val1078.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 _iter1074;
-      for (_iter1074 = this->success.begin(); _iter1074 != this->success.end(); ++_iter1074)
+      std::map<std::string, Type> ::const_iterator _iter1079;
+      for (_iter1079 = this->success.begin(); _iter1079 != this->success.end(); ++_iter1079)
       {
-        xfer += oprot->writeString(_iter1074->first);
-        xfer += _iter1074->second.write(oprot);
+        xfer += oprot->writeString(_iter1079->first);
+        xfer += _iter1079->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 _size1075;
-            ::apache::thrift::protocol::TType _ktype1076;
-            ::apache::thrift::protocol::TType _vtype1077;
-            xfer += iprot->readMapBegin(_ktype1076, _vtype1077, _size1075);
-            uint32_t _i1079;
-            for (_i1079 = 0; _i1079 < _size1075; ++_i1079)
+            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)
             {
-              std::string _key1080;
-              xfer += iprot->readString(_key1080);
-              Type& _val1081 = (*(this->success))[_key1080];
-              xfer += _val1081.read(iprot);
+              std::string _key1085;
+              xfer += iprot->readString(_key1085);
+              Type& _val1086 = (*(this->success))[_key1085];
+              xfer += _val1086.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 _size1082;
-            ::apache::thrift::protocol::TType _etype1085;
-            xfer += iprot->readListBegin(_etype1085, _size1082);
-            this->success.resize(_size1082);
-            uint32_t _i1086;
-            for (_i1086 = 0; _i1086 < _size1082; ++_i1086)
+            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)
             {
-              xfer += this->success[_i1086].read(iprot);
+              xfer += this->success[_i1091].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 _iter1087;
-      for (_iter1087 = this->success.begin(); _iter1087 != this->success.end(); ++_iter1087)
+      std::vector<FieldSchema> ::const_iterator _iter1092;
+      for (_iter1092 = this->success.begin(); _iter1092 != this->success.end(); ++_iter1092)
       {
-        xfer += (*_iter1087).write(oprot);
+        xfer += (*_iter1092).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 _size1088;
-            ::apache::thrift::protocol::TType _etype1091;
-            xfer += iprot->readListBegin(_etype1091, _size1088);
-            (*(this->success)).resize(_size1088);
-            uint32_t _i1092;
-            for (_i1092 = 0; _i1092 < _size1088; ++_i1092)
+            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))[_i1092].read(iprot);
+              xfer += (*(this->success))[_i1097].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 _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 _size1098;
+            ::apache::thrift::protocol::TType _etype1101;
+            xfer += iprot->readListBegin(_etype1101, _size1098);
+            this->success.resize(_size1098);
+            uint32_t _i1102;
+            for (_i1102 = 0; _i1102 < _size1098; ++_i1102)
             {
-              xfer += this->success[_i1097].read(iprot);
+              xfer += this->success[_i1102].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 _iter1098;
-      for (_iter1098 = this->success.begin(); _iter1098 != this->success.end(); ++_iter1098)
+      std::vector<FieldSchema> ::const_iterator _iter1103;
+      for (_iter1103 = this->success.begin(); _iter1103 != this->success.end(); ++_iter1103)
       {
-        xfer += (*_iter1098).write(oprot);
+        xfer += (*_iter1103).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 _size1099;
-            ::apache::thrift::protocol::TType _etype1102;
-            xfer += iprot->readListBegin(_etype1102, _size1099);
-            (*(this->success)).resize(_size1099);
-            uint32_t _i1103;
-            for (_i1103 = 0; _i1103 < _size1099; ++_i1103)
+            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))[_i1103].read(iprot);
+              xfer += (*(this->success))[_i1108].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 _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 _size1109;
+            ::apache::thrift::protocol::TType _etype1112;
+            xfer += iprot->readListBegin(_etype1112, _size1109);
+            this->success.resize(_size1109);
+            uint32_t _i1113;
+            for (_i1113 = 0; _i1113 < _size1109; ++_i1113)
             {
-              xfer += this->success[_i1108].read(iprot);
+              xfer += this->success[_i1113].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 _iter1109;
-      for (_iter1109 = this->success.begin(); _iter1109 != this->success.end(); ++_iter1109)
+      std::vector<FieldSchema> ::const_iterator _iter1114;
+      for (_iter1114 = this->success.begin(); _iter1114 != this->success.end(); ++_iter1114)
       {
-        xfer += (*_iter1109).write(oprot);
+        xfer += (*_iter1114).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 _size1110;
-            ::apache::thrift::protocol::TType _etype1113;
-            xfer += iprot->readListBegin(_etype1113, _size1110);
-            (*(this->success)).resize(_size1110);
-            uint32_t _i1114;
-            for (_i1114 = 0; _i1114 < _size1110; ++_i1114)
+            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))[_i1114].read(iprot);
+              xfer += (*(this->success))[_i1119].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 _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 _size1120;
+            ::apache::thrift::protocol::TType _etype1123;
+            xfer += iprot->readListBegin(_etype1123, _size1120);
+            this->success.resize(_size1120);
+            uint32_t _i1124;
+            for (_i1124 = 0; _i1124 < _size1120; ++_i1124)
             {
-              xfer += this->success[_i1119].read(iprot);
+              xfer += this->success[_i1124].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 _iter1120;
-      for (_iter1120 = this->success.begin(); _iter1120 != this->success.end(); ++_iter1120)
+      std::vector<FieldSchema> ::const_iterator _iter1125;
+      for (_iter1125 = this->success.begin(); _iter1125 != this->success.end(); ++_iter1125)
       {
-        xfer += (*_iter1120).write(oprot);
+        xfer += (*_iter1125).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 _size1121;
-            ::apache::thrift::protocol::TType _etype1124;
-            xfer += iprot->readListBegin(_etype1124, _size1121);
-            (*(this->success)).resize(_size1121);
-            uint32_t _i1125;
-            for (_i1125 = 0; _i1125 < _size1121; ++_i1125)
+            uint32_t _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))[_i1125].read(iprot);
+              xfer += (*(this->success))[_i1130].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 _size1126;
-            ::apache::thrift::protocol::TType _etype1129;
-            xfer += iprot->readListBegin(_etype1129, _size1126);
-            this->primaryKeys.resize(_size1126);
-            uint32_t _i1130;
-            for (_i1130 = 0; _i1130 < _size1126; ++_i1130)
+            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)
             {
-              xfer += this->primaryKeys[_i1130].read(iprot);
+              xfer += this->primaryKeys[_i1135].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 _size1131;
-            ::apache::thrift::protocol::TType _etype1134;
-            xfer += iprot->readListBegin(_etype1134, _size1131);
-            this->foreignKeys.resize(_size1131);
-            uint32_t _i1135;
-            for (_i1135 = 0; _i1135 < _size1131; ++_i1135)
+            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)
             {
-              xfer += this->foreignKeys[_i1135].read(iprot);
+              xfer += this->foreignKeys[_i1140].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 _size1136;
-            ::apache::thrift::protocol::TType _etype1139;
-            xfer += iprot->readListBegin(_etype1139, _size1136);
-            this->uniqueConstraints.resize(_size1136);
-            uint32_t _i1140;
-            for (_i1140 = 0; _i1140 < _size1136; ++_i1140)
+            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)
             {
-              xfer += this->uniqueConstraints[_i1140].read(iprot);
+              xfer += this->uniqueConstraints[_i1145].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 _size1141;
-            ::apache::thrift::protocol::TType _etype1144;
-            xfer += iprot->readListBegin(_etype1144, _size1141);
-            this->notNullConstraints.resize(_size1141);
-            uint32_t _i1145;
-            for (_i1145 = 0; _i1145 < _size1141; ++_i1145)
+            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)
             {
-              xfer += this->notNullConstraints[_i1145].read(iprot);
+              xfer += this->notNullConstraints[_i1150].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 _iter1146;
-    for (_iter1146 = this->primaryKeys.begin(); _iter1146 != this->primaryKeys.end(); ++_iter1146)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1151;
+    for (_iter1151 = this->primaryKeys.begin(); _iter1151 != this->primaryKeys.end(); ++_iter1151)
     {
-      xfer += (*_iter1146).write(oprot);
+      xfer += (*_iter1151).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 _iter1147;
-    for (_iter1147 = this->foreignKeys.begin(); _iter1147 != this->foreignKeys.end(); ++_iter1147)
+    std::vector<SQLForeignKey> ::const_iterator _iter1152;
+    for (_iter1152 = this->foreignKeys.begin(); _iter1152 != this->foreignKeys.end(); ++_iter1152)
     {
-      xfer += (*_iter1147).write(oprot);
+      xfer += (*_iter1152).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 _iter1148;
-    for (_iter1148 = this->uniqueConstraints.begin(); _iter1148 != this->uniqueConstraints.end(); ++_iter1148)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1153;
+    for (_iter1153 = this->uniqueConstraints.begin(); _iter1153 != this->uniqueConstraints.end(); ++_iter1153)
     {
-      xfer += (*_iter1148).write(oprot);
+      xfer += (*_iter1153).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 _iter1149;
-    for (_iter1149 = this->notNullConstraints.begin(); _iter1149 != this->notNullConstraints.end(); ++_iter1149)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1154;
+    for (_iter1154 = this->notNullConstraints.begin(); _iter1154 != this->notNullConstraints.end(); ++_iter1154)
     {
-      xfer += (*_iter1149).write(oprot);
+      xfer += (*_iter1154).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 _iter1150;
-    for (_iter1150 = (*(this->primaryKeys)).begin(); _iter1150 != (*(this->primaryKeys)).end(); ++_iter1150)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1155;
+    for (_iter1155 = (*(this->primaryKeys)).begin(); _iter1155 != (*(this->primaryKeys)).end(); ++_iter1155)
     {
-      xfer += (*_iter1150).write(oprot);
+      xfer += (*_iter1155).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 _iter1151;
-    for (_iter1151 = (*(this->foreignKeys)).begin(); _iter1151 != (*(this->foreignKeys)).end(); ++_iter1151)
+    std::vector<SQLForeignKey> ::const_iterator _iter1156;
+    for (_iter1156 = (*(this->foreignKeys)).begin(); _iter1156 != (*(this->foreignKeys)).end(); ++_iter1156)
     {
-      xfer += (*_iter1151).write(oprot);
+      xfer += (*_iter1156).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 _iter1152;
-    for (_iter1152 = (*(this->uniqueConstraints)).begin(); _iter1152 != (*(this->uniqueConstraints)).end(); ++_iter1152)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1157;
+    for (_iter1157 = (*(this->uniqueConstraints)).begin(); _iter1157 != (*(this->uniqueConstraints)).end(); ++_iter1157)
     {
-      xfer += (*_iter1152).write(oprot);
+      xfer += (*_iter1157).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 _iter1153;
-    for (_iter1153 = (*(this->notNullConstraints)).begin(); _iter1153 != (*(this->notNullConstraints)).end(); ++_iter1153)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1158;
+    for (_iter1158 = (*(this->notNullConstraints)).begin(); _iter1158 != (*(this->notNullConstraints)).end(); ++_iter1158)
     {
-      xfer += (*_iter1153).write(oprot);
+      xfer += (*_iter1158).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 _size1154;
-            ::apache::thrift::protocol::TType _etype1157;
-            xfer += iprot->readListBegin(_etype1157, _size1154);
-            this->partNames.resize(_size1154);
-            uint32_t _i1158;
-            for (_i1158 = 0; _i1158 < _size1154; ++_i1158)
+            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)
             {
-              xfer += iprot->readString(this->partNames[_i1158]);
+              xfer += iprot->readString(this->partNames[_i1163]);
             }
             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 _iter1159;
-    for (_iter1159 = this->partNames.begin(); _iter1159 != this->partNames.end(); ++_iter1159)
+    std::vector<std::string> ::const_iterator _iter1164;
+    for (_iter1164 = this->partNames.begin(); _iter1164 != this->partNames.end(); ++_iter1164)
     {
-      xfer += oprot->writeString((*_iter1159));
+      xfer += oprot->writeString((*_iter1164));
     }
     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 _iter1160;
-    for (_iter1160 = (*(this->partNames)).begin(); _iter1160 != (*(this->partNames)).end(); ++_iter1160)
+    std::vector<std::string> ::const_iterator _iter1165;
+    for (_iter1165 = (*(this->partNames)).begin(); _iter1165 != (*(this->partNames)).end(); ++_iter1165)
     {
-      xfer += oprot->writeString((*_iter1160));
+      xfer += oprot->writeString((*_iter1165));
     }
     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 _size1161;
-            ::apache::thrift::protocol::TType _etype1164;
-            xfer += iprot->readListBegin(_etype1164, _size1161);
-            this->success.resize(_size1161);
-            uint32_t _i1165;
-            for (_i1165 = 0; _i1165 < _size1161; ++_i1165)
+            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)
             {
-              xfer += iprot->readString(this->success[_i1165]);
+              xfer += iprot->readString(this->success[_i1170]);
             }
             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 _iter1166;
-      for (_iter1166 = this->success.begin(); _iter1166 != this->success.end(); ++_iter1166)
+      std::vector<std::string> ::const_iterator _iter1171;
+      for (_iter1171 = this->success.begin(); _iter1171 != this->success.end(); ++_iter1171)
       {
-        xfer += oprot->writeString((*_iter1166));
+        xfer += oprot->writeString((*_iter1171));
       }
       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 _size1167;
-            ::apache::thrift::protocol::TType _etype1170;
-            xfer += iprot->readListBegin(_etype1170, _size1167);
-            (*(this->success)).resize(_size1167);
-            uint32_t _i1171;
-            for (_i1171 = 0; _i1171 < _size1167; ++_i1171)
+            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))[_i1171]);
+              xfer += iprot->readString((*(this->success))[_i1176]);
             }
             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 _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 _size1177;
+            ::apache::thrift::protocol::TType _etype1180;
+            xfer += iprot->readListBegin(_etype1180, _size1177);
+            this->success.resize(_size1177);
+            uint32_t _i1181;
+            for (_i1181 = 0; _i1181 < _size1177; ++_i1181)
             {
-              xfer += iprot->readString(this->success[_i1176]);
+              xfer += iprot->readString(this->success[_i1181]);
             }
             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 _iter1177;
-      for (_iter1177 = this->success.begin(); _iter1177 != this->success.end(); ++_iter1177)
+      std::vector<std::string> ::const_iterator _iter1182;
+      for (_iter1182 = this->success.begin(); _iter1182 != this->success.end(); ++_iter1182)
       {
-        xfer += oprot->writeString((*_iter1177));
+        xfer += oprot->writeString((*_iter1182));
       }
       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 _size1178;
-            ::apache::thrift::protocol::TType _etype1181;
-            xfer += iprot->readListBegin(_etype1181, _size1178);
-            (*(this->success)).resize(_size1178);
-            uint32_t _i1182;
-            for (_i1182 = 0; _i1182 < _size1178; ++_i1182)
+            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))[_i1182]);
+              xfer += iprot->readString((*(this->success))[_i1187]);
             }
             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 _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 _size1188;
+            ::apache::thrift::protocol::TType _etype1191;
+            xfer += iprot->readListBegin(_etype1191, _size1188);
+            this->success.resize(_size1188);
+            uint32_t _i1192;
+            for (_i1192 = 0; _i1192 < _size1188; ++_i1192)
             {
-              xfer += iprot->readString(this->success[_i1187]);
+              xfer += iprot->readString(this->success[_i1192]);
             }
             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 _iter1188;
-      for (_iter1188 = this->success.begin(); _iter1188 != this->success.end(); ++_iter1188)
+      std::vector<std::string> ::const_iterator _iter1193;
+      for (_iter1193 = this->success.begin(); _iter1193 != this->success.end(); ++_iter1193)
       {
-        xfer += oprot->writeString((*_iter1188));
+        xfer += oprot->writeString((*_iter1193));
       }
       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 _size1189;
-            ::apache::thrift::protocol::TType _etype1192;
-            xfer += iprot->readListBegin(_etype1192, _size1189);
-            (*(this->success)).resize(_size1189);
-            uint32_t _i1193;
-            for (_i1193 = 0; _i1193 < _size1189; ++_i1193)
+            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))[_i1193]);
+              xfer += iprot->readString((*(this->success))[_i1198]);
             }
             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 _size1194;
-            ::apache::thrift::protocol::TType _etype1197;
-            xfer += iprot->readListBegin(_etype1197, _size1194);
-            this->tbl_types.resize(_size1194);
-            uint32_t _i1198;
-            for (_i1198 = 0; _i1198 < _size1194; ++_i1198)
+            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)
             {
-              xfer += iprot->readString(this->tbl_types[_i1198]);
+              xfer += iprot->readString(this->tbl_types[_i1203]);
             }
             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 _iter1199;
-    for (_iter1199 = this->tbl_types.begin(); _iter1199 != this->tbl_types.end(); ++_iter1199)
+    std::vector<std::string> ::const_iterator _iter1204;
+    for (_iter1204 = this->tbl_types.begin(); _iter1204 != this->tbl_types.end(); ++_iter1204)
     {
-      xfer += oprot->writeString((*_iter1199));
+      xfer += oprot->writeString((*_iter1204));
     }
     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 _iter1200;
-    for (_iter1200 = (*(this->tbl_types)).begin(); _iter1200 != (*(this->tbl_types)).end(); ++_iter1200)
+    std::vector<std::string> ::const_iterator _iter1205;
+    for (_iter1205 = (*(this->tbl_types)).begin(); _iter1205 != (*(this->tbl_types)).end(); ++_iter1205)
     {
-      xfer += oprot->writeString((*_iter1200));
+      xfer += oprot->writeString((*_iter1205));
     }
     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 _size1201;
-            ::apache::thrift::protocol::TType _etype1204;
-            xfer += iprot->readListBegin(_etype1204, _size1201);
-            this->success.resize(_size1201);
-            uint32_t _i1205;
-            for (_i1205 = 0; _i1205 < _size1201; ++_i1205)
+            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)
             {
-              xfer += this->success[_i1205].read(iprot);
+              xfer += this->success[_i1210].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 _iter1206;
-      for (_iter1206 = this->success.begin(); _iter1206 != this->success.end(); ++_iter1206)
+      std::vector<TableMeta> ::const_iterator _iter1211;
+      for (_iter1211 = this->success.begin(); _iter1211 != this->success.end(); ++_iter1211)
       {
-        xfer += (*_iter1206).write(oprot);
+        xfer += (*_iter1211).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 _size1207;
-            ::apache::thrift::protocol::TType _etype1210;
-            xfer += iprot->readListBegin(_etype1210, _size1207);
-            (*(this->success)).resize(_size1207);
-            uint32_t _i1211;
-            for (_i1211 = 0; _i1211 < _size1207; ++_i1211)
+            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))[_i1211].read(iprot);
+              xfer += (*(this->success))[_i1216].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 _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 _size1217;
+            ::apache::thrift::protocol::TType _etype1220;
+            xfer += iprot->readListBegin(_etype1220, _size1217);
+            this->success.resize(_size1217);
+            uint32_t _i1221;
+            for (_i1221 = 0; _i1221 < _size1217; ++_i1221)
             {
-              xfer += iprot->readString(this->success[_i1216]);
+              xfer += iprot->readString(this->success[_i1221]);
             }
             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 _iter1217;
-      for (_iter1217 = this->success.begin(); _iter1217 != this->success.end(); ++_iter1217)
+      std::vector<std::string> ::const_iterator _iter1222;
+      for (_iter1222 = this->success.begin(); _iter1222 != this->success.end(); ++_iter1222)
       {
-        xfer += oprot->writeString((*_iter1217));
+        xfer += oprot->writeString((*_iter1222));
       }
       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 _size1218;
-            ::apache::thrift::protocol::TType _etype1221;
-            xfer += iprot->readListBegin(_etype1221, _size1218);
-            (*(this->success)).resize(_size1218);
-            uint32_t _i1222;
-            for (_i1222 = 0; _i1222 < _size1218; ++_i1222)
+            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))[_i1222]);
+              xfer += iprot->readString((*(this->success))[_i1227]);
             }
             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 _size1223;
-            ::apache::thrift::protocol::TType _etype1226;
-            xfer += iprot->readListBegin(_etype1226, _size1223);
-            this->tbl_names.resize(_size1223);
-            uint32_t _i1227;
-            for (_i1227 = 0; _i1227 < _size1223; ++_i1227)
+            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)
             {
-              xfer += iprot->readString(this->tbl_names[_i1227]);
+              xfer += iprot->readString(this->tbl_names[_i1232]);
             }
             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 _iter1228;
-    for (_iter1228 = this->tbl_names.begin(); _iter1228 != this->tbl_names.end(); ++_iter1228)
+    std::vector<std::string> ::const_iterator _iter1233;
+    for (_iter1233 = this->tbl_names.begin(); _iter1233 != this->tbl_names.end(); ++_iter1233)
     {
-      xfer += oprot->writeString((*_iter1228));
+      xfer += oprot->writeString((*_iter1233));
     }
     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 _iter1229;
-    for (_iter1229 = (*(this->tbl_names)).begin(); _iter1229 != (*(this->tbl_names)).end(); ++_iter1229)
+    std::vector<std::string> ::const_iterator _iter1234;
+    for (_iter1234 = (*(this->tbl_names)).begin(); _iter1234 != (*(this->tbl_names)).end(); ++_iter1234)
     {
-      xfer += oprot->writeString((*_iter1229));
+      xfer += oprot->writeString((*_iter1234));
     }
     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 _size1230;
-            ::apache::thrift::protocol::TType _etype1233;
-            xfer += iprot->readListBegin(_etype1233, _size1230);
-            this->success.resize(_size1230);
-            uint32_t _i1234;
-            for (_i1234 = 0; _i1234 < _size1230; ++_i1234)
+            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)
             {
-              xfer += this->success[_i1234].read(iprot);
+              xfer += this->success[_i1239].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 _iter1235;
-      for (_iter1235 = this->success.begin(); _iter1235 != this->success.end(); ++_iter1235)
+      std::vector<Table> ::const_iterator _iter1240;
+      for (_iter1240 = this->success.begin(); _iter1240 != this->success.end(); ++_iter1240)
       {
-        xfer += (*_iter1235).write(oprot);
+        xfer += (*_iter1240).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 _size1236;
-            ::apache::thrift::protocol::TType _etype1239;
-            xfer += iprot->readListBegin(_etype1239, _size1236);
-            (*(this->success)).resize(_size1236);
-            uint32_t _i1240;
-            for (_i1240 = 0; _i1240 < _size1236; ++_i1240)
+            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))[_i1240].read(iprot);
+              xfer += (*(this->success))[_i1245].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 _size1241;
-            ::apache::thrift::protocol::TType _etype1244;
-            xfer += iprot->readListBegin(_etype1244, _size1241);
-            this->tbl_names.resize(_size1241);
-            uint32_t _i1245;
-            for (_i1245 = 0; _i1245 < _size1241; ++_i1245)
+            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)
             {
-              xfer += iprot->readString(this->tbl_names[_i1245]);
+              xfer += iprot->readString(this->tbl_names[_i1250]);
             }
             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 _iter1246;
-    for (_iter1246 = this->tbl_names.begin(); _iter1246 != this->tbl_names.end(); ++_iter1246)
+    std::vector<std::string> ::const_iterator _iter1251;
+    for (_iter1251 = this->tbl_names.begin(); _iter1251 != this->tbl_names.end(); ++_iter1251)
     {
-      xfer += oprot->writeString((*_iter1246));
+      xfer += oprot->writeString((*_iter1251));
     }
     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 _iter1247;
-    for (_iter1247 = (*(this->tbl_names)).begin(); _iter1247 != (*(this->tbl_names)).end(); ++_iter1247)
+    std::vector<std::string> ::const_iterator _iter1252;
+    for (_iter1252 = (*(this->tbl_names)).begin(); _iter1252 != (*(this->tbl_names)).end(); ++_iter1252)
     {
-      xfer += oprot->writeString((*_iter1247));
+      xfer += oprot->writeString((*_iter1252));
     }
     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 _size1248;
-            ::apache::thrift::protocol::TType _ktype1249;
-            ::apache::thrift::protocol::TType _vtype1250;
-            xfer += iprot->readMapBegin(_ktype1249, _vtype1250, _size1248);
-            uint32_t _i1252;
-            for (_i1252 = 0; _i1252 < _size1248; ++_i1252)
+            uint32_t _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)
             {
-              std::string _key1253;
-              xfer += iprot->readString(_key1253);
-              Materialization& _val1254 = this->success[_key1253];
-              xfer += _val1254.read(iprot);
+              std::string _key1258;
+              xfer += iprot->readString(_key1258);
+              Materialization& _val1259 = this->success[_key1258];
+              xfer += _val1259.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 _iter1255;
-      for (_iter1255 = this->success.begin(); _iter1255 != this->success.end(); ++_iter1255)
+      std::map<std::string, Materialization> ::const_iterator _iter1260;
+      for (_iter1260 = this->success.begin(); _iter1260 != this->success.end(); ++_iter1260)
       {
-        xfer += oprot->writeString(_iter1255->first);
-        xfer += _iter1255->second.write(oprot);
+        xfer += oprot->writeString(_iter1260->first);
+        xfer += _iter1260->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 _size1256;
-            ::apache::thrift::protocol::TType _ktype1257;
-            ::apache::thrift::protocol::TType _vtype1258;
-            xfer += iprot->readMapBegin(_ktype1257, _vtype1258, _size1256);
-            uint32_t _i1260;
-            for (_i1260 = 0; _i1260 < _size1256; ++_i1260)
+            uint32_t _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)
             {
-              std::string _key1261;
-              xfer += iprot->readString(_key1261);
-              Materialization& _val1262 = (*(this->success))[_key1261];
-              xfer += _val1262.read(iprot);
+              std::string _key1266;
+              xfer += iprot->readString(_key1266);
+              Materialization& _val1267 = (*(this->success))[_key1266];
+              xfer += _val1267.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 _size1263;
-            ::apache::thrift::protocol::TType _etype1266;
-            xfer += iprot->readListBegin(_etype1266, _size1263);
-            this->success.resize(_size1263);
-            uint32_t _i1267;
-            for (_i1267 = 0; _i1267 < _size1263; ++_i1267)
+            uint32_t _size1268;
+            ::apache::thrift::protocol::TType _etype1271;
+            xfer += iprot->readListBegin(_etype1271, _size1268);
+            this->success.resize(_size1268);
+            uint32_t _i1272;
+            for (_i1272 = 0; _i1272 < _size1268; ++_i1272)
             {
-              xfer += iprot->readString(this->success[_i1267]);
+              xfer += iprot->readString(this->success[_i1272]);
             }
             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 _iter1268;
-      for (_iter1268 = this->success.begin(); _iter1268 != this->success.end(); ++_iter1268)
+      std::vector<std::string> ::const_iterator _iter1273;
+      for (_iter1273 = this->success.begin(); _iter1273 != this->success.end(); ++_iter1273)
       {
-        xfer += oprot->writeString((*_iter1268));
+        xfer += oprot->writeString((*_iter1273));
       }
       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 _size1269;
-            ::apache::thrift::protocol::TType _etype1272;
-            xfer += iprot->readListBegin(_etype1272, _size1269);
-            (*(this->success)).resize(_size1269);
-            uint32_t _i1273;
-            for (_i1273 = 0; _i1273 < _size1269; ++_i1273)
+            uint32_t _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))[_i1273]);
+              xfer += iprot->readString((*(this->success))[_i1278]);
             }
             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 _size1274;
-            ::apache::thrift::protocol::TType _etype1277;
-            xfer += iprot->readListBegin(_etype1277, _size1274);
-            this->new_parts.resize(_size1274);
-            uint32_t _i1278;
-            for (_i1278 = 0; _i1278 < _size1274; ++_i1278)
+            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)
             {
-              xfer += this->new_parts[_i1278].read(iprot);
+              xfer += this->new_parts[_i1283].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 _iter1279;
-    for (_iter1279 = this->new_parts.begin(); _iter1279 != this->new_parts.end(); ++_iter1279)
+    std::vector<Partition> ::const_iterator _iter1284;
+    for (_iter1284 = this->new_parts.begin(); _iter1284 != this->new_parts.end(); ++_iter1284)
     {
-      xfer += (*_iter1279).write(oprot);
+      xfer += (*_iter1284).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 _iter1280;
-    for (_iter1280 = (*(this->new_parts)).begin(); _iter1280 != (*(this->new_parts)).end(); ++_iter1280)
+    std::vector<Partition> ::const_iterator _iter1285;
+    for (_iter1285 = (*(this->new_parts)).begin(); _iter1285 != (*(this->new_parts)).end(); ++_iter1285)
     {
-      xfer += (*_iter1280).write(oprot);
+      xfer += (*_iter1285).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 _size1281;
-            ::apache::thrift::protocol::TType _etype1284;
-            xfer += iprot->readListBegin(_etype1284, _size1281);
-            this->new_parts.resize(_size1281);
-            uint32_t _i1285;
-            for (_i1285 = 0; _i1285 < _size1281; ++_i1285)
+            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)
             {
-              xfer += this->new_parts[_i1285].read(iprot);
+              xfer += this->new_parts[_i1290].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 _iter1286;
-    for (_iter1286 = this->new_parts.begin(); _iter1286 != this->new_parts.end(); ++_iter1286)
+    std::vector<PartitionSpec> ::const_iterator _iter1291;
+    for (_iter1291 = this->new_parts.begin(); _iter1291 != this->new_parts.end(); ++_iter1291)
     {
-      xfer += (*_iter1286).write(oprot);
+      xfer += (*_iter1291).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 _iter1287;
-    for (_iter1287 = (*(this->new_parts)).begin(); _iter1287 != (*(this->new_parts)).end(); ++_iter1287)
+    std::vector<PartitionSpec> ::const_iterator _iter1292;
+    for (_iter1292 = (*(this->new_parts)).begin(); _iter1292 != (*(this->new_parts)).end(); ++_iter1292)
     {
-      xfer += (*_iter1287).write(oprot);
+      xfer += (*_iter1292).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 _size1288;
-            ::apache::thrift::protocol::TType _etype1291;
-            xfer += iprot->readListBegin(_etype1291, _size1288);
-            this->part_vals.resize(_size1288);
-            uint32_t _i1292;
-            for (_i1292 = 0; _i1292 < _size1288; ++_i1292)
+            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)
             {
-              xfer += iprot->readString(this->part_vals[_i1292]);
+              xfer += iprot->readString(this->part_vals[_i1297]);
             }
             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 _iter1293;
-    for (_iter1293 = this->part_vals.begin(); _iter1293 != this->part_vals.end(); ++_iter1293)
+    std::vector<std::string> ::const_iterator _iter1298;
+    for (_iter1298 = this->part_vals.begin(); _iter1298 != this->part_vals.end(); ++_iter1298)
     {
-      xfer += oprot->writeString((*_iter1293));
+      xfer += oprot->writeString((*_iter1298));
     }
     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 _iter1294;
-    for (_iter1294 = (*(this->part_vals)).begin(); _iter1294 != (*(this->part_vals)).end(); ++_iter1294)
+    std::vector<std::string> ::const_iterator _iter1299;
+    for (_iter1299 = (*(this->part_vals)).begin(); _iter1299 != (*(this->part_vals)).end(); ++_iter1299)
     {
-      xfer += oprot->writeString((*_iter1294));
+      xfer += oprot->writeString((*_iter1299));
     }
     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 _size1295;
-            ::apache::thrift::protocol::TType _etype1298;
-            xfer += iprot->readListBegin(_etype1298, _size1295);
-            this->part_vals.resize(_size1295);
-            uint32_t _i1299;
-            for (_i1299 = 0; _i1299 < _size1295; ++_i1299)
+            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)
             {
-              xfer += iprot->readString(this->part_vals[_i1299]);
+              xfer += iprot->readString(this->part_vals[_i1304]);
             }
             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 _iter1300;
-    for (_iter1300 = this->part_vals.begin(); _iter1300 != this->part_vals.end(); ++_iter1300)
+    std::vector<std::string> ::const_iterator _iter1305;
+    for (_iter1305 = this->part_vals.begin(); _iter1305 != this->part_vals.end(); ++_iter1305)
     {
-      xfer += oprot->writeString((*_iter1300));
+      xfer += oprot->writeString((*_iter1305));
     }
     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 _iter1301;
-    for (_iter1301 = (*(this->part_vals)).begin(); _iter1301 != (*(this->part_vals)).end(); ++_iter1301)
+    std::vector<std::string> ::const_iterator _iter1306;
+    for (_iter1306 = (*(this->part_vals)).begin(); _iter1306 != (*(this->part_vals)).end(); ++_iter1306)
     {
-      xfer += oprot->writeString((*_iter1301));
+      xfer += oprot->writeString((*_iter1306));
     }
     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 _size1302;
-            ::apache::thrift::protocol::TType _etype1305;
-            xfer += iprot->readListBegin(_etype1305, _size1302);
-            this->part_vals.resize(_size1302);
-            uint32_t _i1306;
-            for (_i1306 = 0; _i1306 < _size1302; ++_i1306)
+            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)
             {
-              xfer += iprot->readString(this->part_vals[_i1306]);
+              xfer += iprot->readString(this->part_vals[_i1311]);
             }
             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 _iter1307;
-    for (_iter1307 = this->part_vals.begin(); _iter1307 != this->part_vals.end(); ++_iter1307)
+    std::vector<std::string> ::const_iterator _iter1312;
+    for (_iter1312 = this->part_vals.begin(); _iter1312 != this->part_vals.end(); ++_iter1312)
     {
-      xfer += oprot->writeString((*_iter1307));
+      xfer += oprot->writeString((*_iter1312));
     }
     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 _iter1308;
-    for (_iter1308 = (*(this->part_vals)).begin(); _iter1308 != (*(this->part_vals)).end(); ++_iter1308)
+    std::vector<std::string> ::const_iterator _iter1313;
+    for (_iter1313 = (*(this->part_vals)).begin(); _iter1313 != (*(this->part_vals)).end(); ++_iter1313)
     {
-      xfer += oprot->writeString((*_iter1308));
+      xfer += oprot->writeString((*_iter1313));
     }
     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 _size1309;
-            ::apache::thrift::protocol::TType _etype1312;
-            xfer += iprot->readListBegin(_etype1312, _size1309);
-            this->part_vals.resize(_size1309);
-            uint32_t _i1313;
-            for (_i1313 = 0; _i1313 < _size1309; ++_i1313)
+            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)
             {
-              xfer += iprot->readString(this->part_vals[_i1313]);
+              xfer += iprot->readString(this->part_vals[_i1318]);
             }
             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 _iter1314;
-    for (_iter1314 = this->part_vals.begin(); _iter1314 != this->part_vals.end(); ++_iter1314)
+    std::vector<std::string> ::const_iterator _iter1319;
+    for (_iter1319 = this->part_vals.begin(); _iter1319 != this->part_vals.end(); ++_iter1319)
     {
-      xfer += oprot->writeString((*_iter1314));
+      xfer += oprot->writeString((*_iter1319));
     }
     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 _iter1315;
-    for (_iter1315 = (*(this->part_vals)).begin(); _iter1315 != (*(this->part_vals)).end(); ++_iter1315)
+    std::vector<std::string> ::const_iterator _iter1320;
+    for (_iter1320 = (*(this->part_vals)).begin(); _iter1320 != (*(this->part_vals)).end(); ++_iter1320)
     {
-      xfer += oprot->writeString((*_iter1315));
+      xfer += oprot->writeString((*_iter1320));
     }
     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 _size1316;
-            ::apache::thrift::protocol::TType _etype1319;
-            xfer += iprot->readListBegin(_etype1319, _size1316);
-            this->part_vals.resize(_size1316);
-            uint32_t _i1320;
-            for (_i1320 = 0; _i1320 < _size1316; ++_i1320)
+            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)
             {
-              xfer += iprot->readString(this->part_vals[_i1320]);
+              xfer += iprot->readString(this->part_vals[_i1325]);
             }
             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 _iter1321;
-    for (_iter1321 = this->part_vals.begin(); _iter1321 != this->part_vals.end(); ++_iter1321)
+    std::vector<std::string> ::const_iterator _iter1326;
+    for (_iter1326 = this->part_vals.begin(); _iter1326 != this->part_vals.end(); ++_iter1326)
     {
-      xfer += oprot->writeString((*_iter1321));
+      xfer += oprot->writeString((*_iter1326));
     }
     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 _iter1322;
-    for (_iter1322 = (*(this->part_vals)).begin(); _iter1322 != (*(this->part_vals)).end(); ++_iter1322)
+    std::vector<std::string> ::const_iterator _iter1327;
+    for (_iter1327 = (*(this->part_vals)).begin(); _iter1327 != (*(this->part_vals)).end(); ++_iter1327)
     {
-      xfer += oprot->writeString((*_iter1322));
+      xfer += oprot->writeString((*_iter1327));
     }
     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 _size1323;
-            ::apache::thrift::protocol::TType _ktype1324;
-            ::apache::thrift::protocol::TType _vtype1325;
-            xfer += iprot->readMapBegin(_ktype1324, _vtype1325, _size1323);
-            uint32_t _i1327;
-            for (_i1327 = 0; _i1327 < _size1323; ++_i1327)
+            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)
             {
-              std::string _key1328;
-              xfer += iprot->readString(_key1328);
-              std::string& _val1329 = this->partitionSpecs[_key1328];
-              xfer += iprot->readString(_val1329);
+              std::string _key1333;
+              xfer += iprot->readString(_key1333);
+              std::string& _val1334 = this->partitionSpecs[_key1333];
+              xfer += iprot->readString(_val1334);
             }
             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 _iter1330;
-    for (_iter1330 = this->partitionSpecs.begin(); _iter1330 != this->partitionSpecs.end(); ++_iter1330)
+    std::map<std::string, std::string> ::const_iterator _iter1335;
+    for (_iter1335 = this->partitionSpecs.begin(); _iter1335 != this->partitionSpecs.end(); ++_iter1335)
     {
-      xfer += oprot->writeString(_iter1330->first);
-      xfer += oprot->writeString(_iter1330->second);
+      xfer += oprot->writeString(_iter1335->first);
+      xfer += oprot->writeString(_iter1335->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 _iter1331;
-    for (_iter1331 = (*(this->partitionSpecs)).begin(); _iter1331 != (*(this->partitionSpecs)).end(); ++_iter1331)
+    std::map<std::string, std::string> ::const_iterator _iter1336;
+    for (_iter1336 = (*(this->partitionSpecs)).begin(); _iter1336 != (*(this->partitionSpecs)).end(); ++_iter1336)
     {
-      xfer += oprot->writeString(_iter1331->first);
-      xfer += oprot->writeString(_iter1331->second);
+      xfer += oprot->writeString(_iter1336->first);
+      xfer += oprot->writeString(_iter1336->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 _size1332;
-            ::apache::thrift::protocol::TType _ktype1333;
-            ::apache::thrift::protocol::TType _vtype1334;
-            xfer += iprot->readMapBegin(_ktype1333, _vtype1334, _size1332);
-            uint32_t _i1336;
-            for (_i1336 = 0; _i1336 < _size1332; ++_i1336)
+            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)
             {
-              std::string _key1337;
-              xfer += iprot->readString(_key1337);
-              std::string& _val1338 = this->partitionSpecs[_key1337];
-              xfer += iprot->readString(_val1338);
+              std::string _key1342;
+              xfer += iprot->readString(_key1342);
+              std::string& _val1343 = this->partitionSpecs[_key1342];
+              xfer += iprot->readString(_val1343);
             }
             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 _iter1339;
-    for (_iter1339 = this->partitionSpecs.begin(); _iter1339 != this->partitionSpecs.end(); ++_iter1339)
+    std::map<std::string, std::string> ::const_iterator _iter1344;
+    for (_iter1344 = this->partitionSpecs.begin(); _iter1344 != this->partitionSpecs.end(); ++_iter1344)
     {
-      xfer += oprot->writeString(_iter1339->fir

<TRUNCATED>

[4/6] hive git commit: HIVE-18438 : WM RP: it's impossible to unset things (Sergey Shelukhin, reviewed by Harish Jaiprakash, Prasanth Jayachandran)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 bf1c5a1..11e9447 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
@@ -21490,34 +21490,418 @@ void WMResourcePlan::printTo(std::ostream& out) const {
 }
 
 
+WMNullableResourcePlan::~WMNullableResourcePlan() throw() {
+}
+
+
+void WMNullableResourcePlan::__set_name(const std::string& val) {
+  this->name = val;
+}
+
+void WMNullableResourcePlan::__set_status(const WMResourcePlanStatus::type val) {
+  this->status = val;
+__isset.status = true;
+}
+
+void WMNullableResourcePlan::__set_queryParallelism(const int32_t val) {
+  this->queryParallelism = val;
+__isset.queryParallelism = true;
+}
+
+void WMNullableResourcePlan::__set_isSetQueryParallelism(const bool val) {
+  this->isSetQueryParallelism = val;
+__isset.isSetQueryParallelism = true;
+}
+
+void WMNullableResourcePlan::__set_defaultPoolPath(const std::string& val) {
+  this->defaultPoolPath = val;
+__isset.defaultPoolPath = true;
+}
+
+void WMNullableResourcePlan::__set_isSetDefaultPoolPath(const bool val) {
+  this->isSetDefaultPoolPath = val;
+__isset.isSetDefaultPoolPath = true;
+}
+
+uint32_t WMNullableResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_name = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->name);
+          isset_name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast893;
+          xfer += iprot->readI32(ecast893);
+          this->status = (WMResourcePlanStatus::type)ecast893;
+          this->__isset.status = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          xfer += iprot->readI32(this->queryParallelism);
+          this->__isset.queryParallelism = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->isSetQueryParallelism);
+          this->__isset.isSetQueryParallelism = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->defaultPoolPath);
+          this->__isset.defaultPoolPath = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 7:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->isSetDefaultPoolPath);
+          this->__isset.isSetDefaultPoolPath = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_name)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t WMNullableResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("WMNullableResourcePlan");
+
+  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->name);
+  xfer += oprot->writeFieldEnd();
+
+  if (this->__isset.status) {
+    xfer += oprot->writeFieldBegin("status", ::apache::thrift::protocol::T_I32, 2);
+    xfer += oprot->writeI32((int32_t)this->status);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.queryParallelism) {
+    xfer += oprot->writeFieldBegin("queryParallelism", ::apache::thrift::protocol::T_I32, 4);
+    xfer += oprot->writeI32(this->queryParallelism);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.isSetQueryParallelism) {
+    xfer += oprot->writeFieldBegin("isSetQueryParallelism", ::apache::thrift::protocol::T_BOOL, 5);
+    xfer += oprot->writeBool(this->isSetQueryParallelism);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.defaultPoolPath) {
+    xfer += oprot->writeFieldBegin("defaultPoolPath", ::apache::thrift::protocol::T_STRING, 6);
+    xfer += oprot->writeString(this->defaultPoolPath);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.isSetDefaultPoolPath) {
+    xfer += oprot->writeFieldBegin("isSetDefaultPoolPath", ::apache::thrift::protocol::T_BOOL, 7);
+    xfer += oprot->writeBool(this->isSetDefaultPoolPath);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(WMNullableResourcePlan &a, WMNullableResourcePlan &b) {
+  using ::std::swap;
+  swap(a.name, b.name);
+  swap(a.status, b.status);
+  swap(a.queryParallelism, b.queryParallelism);
+  swap(a.isSetQueryParallelism, b.isSetQueryParallelism);
+  swap(a.defaultPoolPath, b.defaultPoolPath);
+  swap(a.isSetDefaultPoolPath, b.isSetDefaultPoolPath);
+  swap(a.__isset, b.__isset);
+}
+
+WMNullableResourcePlan::WMNullableResourcePlan(const WMNullableResourcePlan& other894) {
+  name = other894.name;
+  status = other894.status;
+  queryParallelism = other894.queryParallelism;
+  isSetQueryParallelism = other894.isSetQueryParallelism;
+  defaultPoolPath = other894.defaultPoolPath;
+  isSetDefaultPoolPath = other894.isSetDefaultPoolPath;
+  __isset = other894.__isset;
+}
+WMNullableResourcePlan& WMNullableResourcePlan::operator=(const WMNullableResourcePlan& other895) {
+  name = other895.name;
+  status = other895.status;
+  queryParallelism = other895.queryParallelism;
+  isSetQueryParallelism = other895.isSetQueryParallelism;
+  defaultPoolPath = other895.defaultPoolPath;
+  isSetDefaultPoolPath = other895.isSetDefaultPoolPath;
+  __isset = other895.__isset;
+  return *this;
+}
+void WMNullableResourcePlan::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "WMNullableResourcePlan(";
+  out << "name=" << to_string(name);
+  out << ", " << "status="; (__isset.status ? (out << to_string(status)) : (out << "<null>"));
+  out << ", " << "queryParallelism="; (__isset.queryParallelism ? (out << to_string(queryParallelism)) : (out << "<null>"));
+  out << ", " << "isSetQueryParallelism="; (__isset.isSetQueryParallelism ? (out << to_string(isSetQueryParallelism)) : (out << "<null>"));
+  out << ", " << "defaultPoolPath="; (__isset.defaultPoolPath ? (out << to_string(defaultPoolPath)) : (out << "<null>"));
+  out << ", " << "isSetDefaultPoolPath="; (__isset.isSetDefaultPoolPath ? (out << to_string(isSetDefaultPoolPath)) : (out << "<null>"));
+  out << ")";
+}
+
+
 WMPool::~WMPool() throw() {
 }
 
 
-void WMPool::__set_resourcePlanName(const std::string& val) {
+void WMPool::__set_resourcePlanName(const std::string& val) {
+  this->resourcePlanName = val;
+}
+
+void WMPool::__set_poolPath(const std::string& val) {
+  this->poolPath = val;
+}
+
+void WMPool::__set_allocFraction(const double val) {
+  this->allocFraction = val;
+__isset.allocFraction = true;
+}
+
+void WMPool::__set_queryParallelism(const int32_t val) {
+  this->queryParallelism = val;
+__isset.queryParallelism = true;
+}
+
+void WMPool::__set_schedulingPolicy(const std::string& val) {
+  this->schedulingPolicy = val;
+__isset.schedulingPolicy = true;
+}
+
+uint32_t WMPool::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_resourcePlanName = false;
+  bool isset_poolPath = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->resourcePlanName);
+          isset_resourcePlanName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->poolPath);
+          isset_poolPath = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_DOUBLE) {
+          xfer += iprot->readDouble(this->allocFraction);
+          this->__isset.allocFraction = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          xfer += iprot->readI32(this->queryParallelism);
+          this->__isset.queryParallelism = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->schedulingPolicy);
+          this->__isset.schedulingPolicy = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_resourcePlanName)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_poolPath)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t WMPool::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("WMPool");
+
+  xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->resourcePlanName);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("poolPath", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->poolPath);
+  xfer += oprot->writeFieldEnd();
+
+  if (this->__isset.allocFraction) {
+    xfer += oprot->writeFieldBegin("allocFraction", ::apache::thrift::protocol::T_DOUBLE, 3);
+    xfer += oprot->writeDouble(this->allocFraction);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.queryParallelism) {
+    xfer += oprot->writeFieldBegin("queryParallelism", ::apache::thrift::protocol::T_I32, 4);
+    xfer += oprot->writeI32(this->queryParallelism);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.schedulingPolicy) {
+    xfer += oprot->writeFieldBegin("schedulingPolicy", ::apache::thrift::protocol::T_STRING, 5);
+    xfer += oprot->writeString(this->schedulingPolicy);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(WMPool &a, WMPool &b) {
+  using ::std::swap;
+  swap(a.resourcePlanName, b.resourcePlanName);
+  swap(a.poolPath, b.poolPath);
+  swap(a.allocFraction, b.allocFraction);
+  swap(a.queryParallelism, b.queryParallelism);
+  swap(a.schedulingPolicy, b.schedulingPolicy);
+  swap(a.__isset, b.__isset);
+}
+
+WMPool::WMPool(const WMPool& other896) {
+  resourcePlanName = other896.resourcePlanName;
+  poolPath = other896.poolPath;
+  allocFraction = other896.allocFraction;
+  queryParallelism = other896.queryParallelism;
+  schedulingPolicy = other896.schedulingPolicy;
+  __isset = other896.__isset;
+}
+WMPool& WMPool::operator=(const WMPool& other897) {
+  resourcePlanName = other897.resourcePlanName;
+  poolPath = other897.poolPath;
+  allocFraction = other897.allocFraction;
+  queryParallelism = other897.queryParallelism;
+  schedulingPolicy = other897.schedulingPolicy;
+  __isset = other897.__isset;
+  return *this;
+}
+void WMPool::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "WMPool(";
+  out << "resourcePlanName=" << to_string(resourcePlanName);
+  out << ", " << "poolPath=" << to_string(poolPath);
+  out << ", " << "allocFraction="; (__isset.allocFraction ? (out << to_string(allocFraction)) : (out << "<null>"));
+  out << ", " << "queryParallelism="; (__isset.queryParallelism ? (out << to_string(queryParallelism)) : (out << "<null>"));
+  out << ", " << "schedulingPolicy="; (__isset.schedulingPolicy ? (out << to_string(schedulingPolicy)) : (out << "<null>"));
+  out << ")";
+}
+
+
+WMNullablePool::~WMNullablePool() throw() {
+}
+
+
+void WMNullablePool::__set_resourcePlanName(const std::string& val) {
   this->resourcePlanName = val;
 }
 
-void WMPool::__set_poolPath(const std::string& val) {
+void WMNullablePool::__set_poolPath(const std::string& val) {
   this->poolPath = val;
 }
 
-void WMPool::__set_allocFraction(const double val) {
+void WMNullablePool::__set_allocFraction(const double val) {
   this->allocFraction = val;
 __isset.allocFraction = true;
 }
 
-void WMPool::__set_queryParallelism(const int32_t val) {
+void WMNullablePool::__set_queryParallelism(const int32_t val) {
   this->queryParallelism = val;
 __isset.queryParallelism = true;
 }
 
-void WMPool::__set_schedulingPolicy(const std::string& val) {
+void WMNullablePool::__set_schedulingPolicy(const std::string& val) {
   this->schedulingPolicy = val;
 __isset.schedulingPolicy = true;
 }
 
-uint32_t WMPool::read(::apache::thrift::protocol::TProtocol* iprot) {
+void WMNullablePool::__set_isSetSchedulingPolicy(const bool val) {
+  this->isSetSchedulingPolicy = val;
+__isset.isSetSchedulingPolicy = true;
+}
+
+uint32_t WMNullablePool::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -21580,6 +21964,14 @@ uint32_t WMPool::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->isSetSchedulingPolicy);
+          this->__isset.isSetSchedulingPolicy = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -21596,10 +21988,10 @@ uint32_t WMPool::read(::apache::thrift::protocol::TProtocol* iprot) {
   return xfer;
 }
 
-uint32_t WMPool::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t WMNullablePool::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("WMPool");
+  xfer += oprot->writeStructBegin("WMNullablePool");
 
   xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString(this->resourcePlanName);
@@ -21624,46 +22016,55 @@ uint32_t WMPool::write(::apache::thrift::protocol::TProtocol* oprot) const {
     xfer += oprot->writeString(this->schedulingPolicy);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.isSetSchedulingPolicy) {
+    xfer += oprot->writeFieldBegin("isSetSchedulingPolicy", ::apache::thrift::protocol::T_BOOL, 6);
+    xfer += oprot->writeBool(this->isSetSchedulingPolicy);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
-void swap(WMPool &a, WMPool &b) {
+void swap(WMNullablePool &a, WMNullablePool &b) {
   using ::std::swap;
   swap(a.resourcePlanName, b.resourcePlanName);
   swap(a.poolPath, b.poolPath);
   swap(a.allocFraction, b.allocFraction);
   swap(a.queryParallelism, b.queryParallelism);
   swap(a.schedulingPolicy, b.schedulingPolicy);
+  swap(a.isSetSchedulingPolicy, b.isSetSchedulingPolicy);
   swap(a.__isset, b.__isset);
 }
 
-WMPool::WMPool(const WMPool& other893) {
-  resourcePlanName = other893.resourcePlanName;
-  poolPath = other893.poolPath;
-  allocFraction = other893.allocFraction;
-  queryParallelism = other893.queryParallelism;
-  schedulingPolicy = other893.schedulingPolicy;
-  __isset = other893.__isset;
+WMNullablePool::WMNullablePool(const WMNullablePool& other898) {
+  resourcePlanName = other898.resourcePlanName;
+  poolPath = other898.poolPath;
+  allocFraction = other898.allocFraction;
+  queryParallelism = other898.queryParallelism;
+  schedulingPolicy = other898.schedulingPolicy;
+  isSetSchedulingPolicy = other898.isSetSchedulingPolicy;
+  __isset = other898.__isset;
 }
-WMPool& WMPool::operator=(const WMPool& other894) {
-  resourcePlanName = other894.resourcePlanName;
-  poolPath = other894.poolPath;
-  allocFraction = other894.allocFraction;
-  queryParallelism = other894.queryParallelism;
-  schedulingPolicy = other894.schedulingPolicy;
-  __isset = other894.__isset;
+WMNullablePool& WMNullablePool::operator=(const WMNullablePool& other899) {
+  resourcePlanName = other899.resourcePlanName;
+  poolPath = other899.poolPath;
+  allocFraction = other899.allocFraction;
+  queryParallelism = other899.queryParallelism;
+  schedulingPolicy = other899.schedulingPolicy;
+  isSetSchedulingPolicy = other899.isSetSchedulingPolicy;
+  __isset = other899.__isset;
   return *this;
 }
-void WMPool::printTo(std::ostream& out) const {
+void WMNullablePool::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "WMPool(";
+  out << "WMNullablePool(";
   out << "resourcePlanName=" << to_string(resourcePlanName);
   out << ", " << "poolPath=" << to_string(poolPath);
   out << ", " << "allocFraction="; (__isset.allocFraction ? (out << to_string(allocFraction)) : (out << "<null>"));
   out << ", " << "queryParallelism="; (__isset.queryParallelism ? (out << to_string(queryParallelism)) : (out << "<null>"));
   out << ", " << "schedulingPolicy="; (__isset.schedulingPolicy ? (out << to_string(schedulingPolicy)) : (out << "<null>"));
+  out << ", " << "isSetSchedulingPolicy="; (__isset.isSetSchedulingPolicy ? (out << to_string(isSetSchedulingPolicy)) : (out << "<null>"));
   out << ")";
 }
 
@@ -21817,21 +22218,21 @@ void swap(WMTrigger &a, WMTrigger &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMTrigger::WMTrigger(const WMTrigger& other895) {
-  resourcePlanName = other895.resourcePlanName;
-  triggerName = other895.triggerName;
-  triggerExpression = other895.triggerExpression;
-  actionExpression = other895.actionExpression;
-  isInUnmanaged = other895.isInUnmanaged;
-  __isset = other895.__isset;
-}
-WMTrigger& WMTrigger::operator=(const WMTrigger& other896) {
-  resourcePlanName = other896.resourcePlanName;
-  triggerName = other896.triggerName;
-  triggerExpression = other896.triggerExpression;
-  actionExpression = other896.actionExpression;
-  isInUnmanaged = other896.isInUnmanaged;
-  __isset = other896.__isset;
+WMTrigger::WMTrigger(const WMTrigger& other900) {
+  resourcePlanName = other900.resourcePlanName;
+  triggerName = other900.triggerName;
+  triggerExpression = other900.triggerExpression;
+  actionExpression = other900.actionExpression;
+  isInUnmanaged = other900.isInUnmanaged;
+  __isset = other900.__isset;
+}
+WMTrigger& WMTrigger::operator=(const WMTrigger& other901) {
+  resourcePlanName = other901.resourcePlanName;
+  triggerName = other901.triggerName;
+  triggerExpression = other901.triggerExpression;
+  actionExpression = other901.actionExpression;
+  isInUnmanaged = other901.isInUnmanaged;
+  __isset = other901.__isset;
   return *this;
 }
 void WMTrigger::printTo(std::ostream& out) const {
@@ -21996,21 +22397,21 @@ void swap(WMMapping &a, WMMapping &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMMapping::WMMapping(const WMMapping& other897) {
-  resourcePlanName = other897.resourcePlanName;
-  entityType = other897.entityType;
-  entityName = other897.entityName;
-  poolPath = other897.poolPath;
-  ordering = other897.ordering;
-  __isset = other897.__isset;
-}
-WMMapping& WMMapping::operator=(const WMMapping& other898) {
-  resourcePlanName = other898.resourcePlanName;
-  entityType = other898.entityType;
-  entityName = other898.entityName;
-  poolPath = other898.poolPath;
-  ordering = other898.ordering;
-  __isset = other898.__isset;
+WMMapping::WMMapping(const WMMapping& other902) {
+  resourcePlanName = other902.resourcePlanName;
+  entityType = other902.entityType;
+  entityName = other902.entityName;
+  poolPath = other902.poolPath;
+  ordering = other902.ordering;
+  __isset = other902.__isset;
+}
+WMMapping& WMMapping::operator=(const WMMapping& other903) {
+  resourcePlanName = other903.resourcePlanName;
+  entityType = other903.entityType;
+  entityName = other903.entityName;
+  poolPath = other903.poolPath;
+  ordering = other903.ordering;
+  __isset = other903.__isset;
   return *this;
 }
 void WMMapping::printTo(std::ostream& out) const {
@@ -22116,13 +22517,13 @@ void swap(WMPoolTrigger &a, WMPoolTrigger &b) {
   swap(a.trigger, b.trigger);
 }
 
-WMPoolTrigger::WMPoolTrigger(const WMPoolTrigger& other899) {
-  pool = other899.pool;
-  trigger = other899.trigger;
+WMPoolTrigger::WMPoolTrigger(const WMPoolTrigger& other904) {
+  pool = other904.pool;
+  trigger = other904.trigger;
 }
-WMPoolTrigger& WMPoolTrigger::operator=(const WMPoolTrigger& other900) {
-  pool = other900.pool;
-  trigger = other900.trigger;
+WMPoolTrigger& WMPoolTrigger::operator=(const WMPoolTrigger& other905) {
+  pool = other905.pool;
+  trigger = other905.trigger;
   return *this;
 }
 void WMPoolTrigger::printTo(std::ostream& out) const {
@@ -22196,14 +22597,14 @@ uint32_t WMFullResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->pools.clear();
-            uint32_t _size901;
-            ::apache::thrift::protocol::TType _etype904;
-            xfer += iprot->readListBegin(_etype904, _size901);
-            this->pools.resize(_size901);
-            uint32_t _i905;
-            for (_i905 = 0; _i905 < _size901; ++_i905)
+            uint32_t _size906;
+            ::apache::thrift::protocol::TType _etype909;
+            xfer += iprot->readListBegin(_etype909, _size906);
+            this->pools.resize(_size906);
+            uint32_t _i910;
+            for (_i910 = 0; _i910 < _size906; ++_i910)
             {
-              xfer += this->pools[_i905].read(iprot);
+              xfer += this->pools[_i910].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22216,14 +22617,14 @@ uint32_t WMFullResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->mappings.clear();
-            uint32_t _size906;
-            ::apache::thrift::protocol::TType _etype909;
-            xfer += iprot->readListBegin(_etype909, _size906);
-            this->mappings.resize(_size906);
-            uint32_t _i910;
-            for (_i910 = 0; _i910 < _size906; ++_i910)
+            uint32_t _size911;
+            ::apache::thrift::protocol::TType _etype914;
+            xfer += iprot->readListBegin(_etype914, _size911);
+            this->mappings.resize(_size911);
+            uint32_t _i915;
+            for (_i915 = 0; _i915 < _size911; ++_i915)
             {
-              xfer += this->mappings[_i910].read(iprot);
+              xfer += this->mappings[_i915].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22236,14 +22637,14 @@ uint32_t WMFullResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->triggers.clear();
-            uint32_t _size911;
-            ::apache::thrift::protocol::TType _etype914;
-            xfer += iprot->readListBegin(_etype914, _size911);
-            this->triggers.resize(_size911);
-            uint32_t _i915;
-            for (_i915 = 0; _i915 < _size911; ++_i915)
+            uint32_t _size916;
+            ::apache::thrift::protocol::TType _etype919;
+            xfer += iprot->readListBegin(_etype919, _size916);
+            this->triggers.resize(_size916);
+            uint32_t _i920;
+            for (_i920 = 0; _i920 < _size916; ++_i920)
             {
-              xfer += this->triggers[_i915].read(iprot);
+              xfer += this->triggers[_i920].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22256,14 +22657,14 @@ uint32_t WMFullResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->poolTriggers.clear();
-            uint32_t _size916;
-            ::apache::thrift::protocol::TType _etype919;
-            xfer += iprot->readListBegin(_etype919, _size916);
-            this->poolTriggers.resize(_size916);
-            uint32_t _i920;
-            for (_i920 = 0; _i920 < _size916; ++_i920)
+            uint32_t _size921;
+            ::apache::thrift::protocol::TType _etype924;
+            xfer += iprot->readListBegin(_etype924, _size921);
+            this->poolTriggers.resize(_size921);
+            uint32_t _i925;
+            for (_i925 = 0; _i925 < _size921; ++_i925)
             {
-              xfer += this->poolTriggers[_i920].read(iprot);
+              xfer += this->poolTriggers[_i925].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22300,10 +22701,10 @@ uint32_t WMFullResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("pools", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->pools.size()));
-    std::vector<WMPool> ::const_iterator _iter921;
-    for (_iter921 = this->pools.begin(); _iter921 != this->pools.end(); ++_iter921)
+    std::vector<WMPool> ::const_iterator _iter926;
+    for (_iter926 = this->pools.begin(); _iter926 != this->pools.end(); ++_iter926)
     {
-      xfer += (*_iter921).write(oprot);
+      xfer += (*_iter926).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -22313,10 +22714,10 @@ uint32_t WMFullResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot)
     xfer += oprot->writeFieldBegin("mappings", ::apache::thrift::protocol::T_LIST, 3);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->mappings.size()));
-      std::vector<WMMapping> ::const_iterator _iter922;
-      for (_iter922 = this->mappings.begin(); _iter922 != this->mappings.end(); ++_iter922)
+      std::vector<WMMapping> ::const_iterator _iter927;
+      for (_iter927 = this->mappings.begin(); _iter927 != this->mappings.end(); ++_iter927)
       {
-        xfer += (*_iter922).write(oprot);
+        xfer += (*_iter927).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -22326,10 +22727,10 @@ uint32_t WMFullResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot)
     xfer += oprot->writeFieldBegin("triggers", ::apache::thrift::protocol::T_LIST, 4);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->triggers.size()));
-      std::vector<WMTrigger> ::const_iterator _iter923;
-      for (_iter923 = this->triggers.begin(); _iter923 != this->triggers.end(); ++_iter923)
+      std::vector<WMTrigger> ::const_iterator _iter928;
+      for (_iter928 = this->triggers.begin(); _iter928 != this->triggers.end(); ++_iter928)
       {
-        xfer += (*_iter923).write(oprot);
+        xfer += (*_iter928).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -22339,10 +22740,10 @@ uint32_t WMFullResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot)
     xfer += oprot->writeFieldBegin("poolTriggers", ::apache::thrift::protocol::T_LIST, 5);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->poolTriggers.size()));
-      std::vector<WMPoolTrigger> ::const_iterator _iter924;
-      for (_iter924 = this->poolTriggers.begin(); _iter924 != this->poolTriggers.end(); ++_iter924)
+      std::vector<WMPoolTrigger> ::const_iterator _iter929;
+      for (_iter929 = this->poolTriggers.begin(); _iter929 != this->poolTriggers.end(); ++_iter929)
       {
-        xfer += (*_iter924).write(oprot);
+        xfer += (*_iter929).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -22363,21 +22764,21 @@ void swap(WMFullResourcePlan &a, WMFullResourcePlan &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMFullResourcePlan::WMFullResourcePlan(const WMFullResourcePlan& other925) {
-  plan = other925.plan;
-  pools = other925.pools;
-  mappings = other925.mappings;
-  triggers = other925.triggers;
-  poolTriggers = other925.poolTriggers;
-  __isset = other925.__isset;
-}
-WMFullResourcePlan& WMFullResourcePlan::operator=(const WMFullResourcePlan& other926) {
-  plan = other926.plan;
-  pools = other926.pools;
-  mappings = other926.mappings;
-  triggers = other926.triggers;
-  poolTriggers = other926.poolTriggers;
-  __isset = other926.__isset;
+WMFullResourcePlan::WMFullResourcePlan(const WMFullResourcePlan& other930) {
+  plan = other930.plan;
+  pools = other930.pools;
+  mappings = other930.mappings;
+  triggers = other930.triggers;
+  poolTriggers = other930.poolTriggers;
+  __isset = other930.__isset;
+}
+WMFullResourcePlan& WMFullResourcePlan::operator=(const WMFullResourcePlan& other931) {
+  plan = other931.plan;
+  pools = other931.pools;
+  mappings = other931.mappings;
+  triggers = other931.triggers;
+  poolTriggers = other931.poolTriggers;
+  __isset = other931.__isset;
   return *this;
 }
 void WMFullResourcePlan::printTo(std::ostream& out) const {
@@ -22482,15 +22883,15 @@ void swap(WMCreateResourcePlanRequest &a, WMCreateResourcePlanRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMCreateResourcePlanRequest::WMCreateResourcePlanRequest(const WMCreateResourcePlanRequest& other927) {
-  resourcePlan = other927.resourcePlan;
-  copyFrom = other927.copyFrom;
-  __isset = other927.__isset;
+WMCreateResourcePlanRequest::WMCreateResourcePlanRequest(const WMCreateResourcePlanRequest& other932) {
+  resourcePlan = other932.resourcePlan;
+  copyFrom = other932.copyFrom;
+  __isset = other932.__isset;
 }
-WMCreateResourcePlanRequest& WMCreateResourcePlanRequest::operator=(const WMCreateResourcePlanRequest& other928) {
-  resourcePlan = other928.resourcePlan;
-  copyFrom = other928.copyFrom;
-  __isset = other928.__isset;
+WMCreateResourcePlanRequest& WMCreateResourcePlanRequest::operator=(const WMCreateResourcePlanRequest& other933) {
+  resourcePlan = other933.resourcePlan;
+  copyFrom = other933.copyFrom;
+  __isset = other933.__isset;
   return *this;
 }
 void WMCreateResourcePlanRequest::printTo(std::ostream& out) const {
@@ -22550,11 +22951,11 @@ void swap(WMCreateResourcePlanResponse &a, WMCreateResourcePlanResponse &b) {
   (void) b;
 }
 
-WMCreateResourcePlanResponse::WMCreateResourcePlanResponse(const WMCreateResourcePlanResponse& other929) {
-  (void) other929;
+WMCreateResourcePlanResponse::WMCreateResourcePlanResponse(const WMCreateResourcePlanResponse& other934) {
+  (void) other934;
 }
-WMCreateResourcePlanResponse& WMCreateResourcePlanResponse::operator=(const WMCreateResourcePlanResponse& other930) {
-  (void) other930;
+WMCreateResourcePlanResponse& WMCreateResourcePlanResponse::operator=(const WMCreateResourcePlanResponse& other935) {
+  (void) other935;
   return *this;
 }
 void WMCreateResourcePlanResponse::printTo(std::ostream& out) const {
@@ -22612,11 +23013,11 @@ void swap(WMGetActiveResourcePlanRequest &a, WMGetActiveResourcePlanRequest &b)
   (void) b;
 }
 
-WMGetActiveResourcePlanRequest::WMGetActiveResourcePlanRequest(const WMGetActiveResourcePlanRequest& other931) {
-  (void) other931;
+WMGetActiveResourcePlanRequest::WMGetActiveResourcePlanRequest(const WMGetActiveResourcePlanRequest& other936) {
+  (void) other936;
 }
-WMGetActiveResourcePlanRequest& WMGetActiveResourcePlanRequest::operator=(const WMGetActiveResourcePlanRequest& other932) {
-  (void) other932;
+WMGetActiveResourcePlanRequest& WMGetActiveResourcePlanRequest::operator=(const WMGetActiveResourcePlanRequest& other937) {
+  (void) other937;
   return *this;
 }
 void WMGetActiveResourcePlanRequest::printTo(std::ostream& out) const {
@@ -22697,13 +23098,13 @@ void swap(WMGetActiveResourcePlanResponse &a, WMGetActiveResourcePlanResponse &b
   swap(a.__isset, b.__isset);
 }
 
-WMGetActiveResourcePlanResponse::WMGetActiveResourcePlanResponse(const WMGetActiveResourcePlanResponse& other933) {
-  resourcePlan = other933.resourcePlan;
-  __isset = other933.__isset;
+WMGetActiveResourcePlanResponse::WMGetActiveResourcePlanResponse(const WMGetActiveResourcePlanResponse& other938) {
+  resourcePlan = other938.resourcePlan;
+  __isset = other938.__isset;
 }
-WMGetActiveResourcePlanResponse& WMGetActiveResourcePlanResponse::operator=(const WMGetActiveResourcePlanResponse& other934) {
-  resourcePlan = other934.resourcePlan;
-  __isset = other934.__isset;
+WMGetActiveResourcePlanResponse& WMGetActiveResourcePlanResponse::operator=(const WMGetActiveResourcePlanResponse& other939) {
+  resourcePlan = other939.resourcePlan;
+  __isset = other939.__isset;
   return *this;
 }
 void WMGetActiveResourcePlanResponse::printTo(std::ostream& out) const {
@@ -22785,13 +23186,13 @@ void swap(WMGetResourcePlanRequest &a, WMGetResourcePlanRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMGetResourcePlanRequest::WMGetResourcePlanRequest(const WMGetResourcePlanRequest& other935) {
-  resourcePlanName = other935.resourcePlanName;
-  __isset = other935.__isset;
+WMGetResourcePlanRequest::WMGetResourcePlanRequest(const WMGetResourcePlanRequest& other940) {
+  resourcePlanName = other940.resourcePlanName;
+  __isset = other940.__isset;
 }
-WMGetResourcePlanRequest& WMGetResourcePlanRequest::operator=(const WMGetResourcePlanRequest& other936) {
-  resourcePlanName = other936.resourcePlanName;
-  __isset = other936.__isset;
+WMGetResourcePlanRequest& WMGetResourcePlanRequest::operator=(const WMGetResourcePlanRequest& other941) {
+  resourcePlanName = other941.resourcePlanName;
+  __isset = other941.__isset;
   return *this;
 }
 void WMGetResourcePlanRequest::printTo(std::ostream& out) const {
@@ -22873,13 +23274,13 @@ void swap(WMGetResourcePlanResponse &a, WMGetResourcePlanResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMGetResourcePlanResponse::WMGetResourcePlanResponse(const WMGetResourcePlanResponse& other937) {
-  resourcePlan = other937.resourcePlan;
-  __isset = other937.__isset;
+WMGetResourcePlanResponse::WMGetResourcePlanResponse(const WMGetResourcePlanResponse& other942) {
+  resourcePlan = other942.resourcePlan;
+  __isset = other942.__isset;
 }
-WMGetResourcePlanResponse& WMGetResourcePlanResponse::operator=(const WMGetResourcePlanResponse& other938) {
-  resourcePlan = other938.resourcePlan;
-  __isset = other938.__isset;
+WMGetResourcePlanResponse& WMGetResourcePlanResponse::operator=(const WMGetResourcePlanResponse& other943) {
+  resourcePlan = other943.resourcePlan;
+  __isset = other943.__isset;
   return *this;
 }
 void WMGetResourcePlanResponse::printTo(std::ostream& out) const {
@@ -22938,11 +23339,11 @@ void swap(WMGetAllResourcePlanRequest &a, WMGetAllResourcePlanRequest &b) {
   (void) b;
 }
 
-WMGetAllResourcePlanRequest::WMGetAllResourcePlanRequest(const WMGetAllResourcePlanRequest& other939) {
-  (void) other939;
+WMGetAllResourcePlanRequest::WMGetAllResourcePlanRequest(const WMGetAllResourcePlanRequest& other944) {
+  (void) other944;
 }
-WMGetAllResourcePlanRequest& WMGetAllResourcePlanRequest::operator=(const WMGetAllResourcePlanRequest& other940) {
-  (void) other940;
+WMGetAllResourcePlanRequest& WMGetAllResourcePlanRequest::operator=(const WMGetAllResourcePlanRequest& other945) {
+  (void) other945;
   return *this;
 }
 void WMGetAllResourcePlanRequest::printTo(std::ostream& out) const {
@@ -22986,14 +23387,14 @@ uint32_t WMGetAllResourcePlanResponse::read(::apache::thrift::protocol::TProtoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->resourcePlans.clear();
-            uint32_t _size941;
-            ::apache::thrift::protocol::TType _etype944;
-            xfer += iprot->readListBegin(_etype944, _size941);
-            this->resourcePlans.resize(_size941);
-            uint32_t _i945;
-            for (_i945 = 0; _i945 < _size941; ++_i945)
+            uint32_t _size946;
+            ::apache::thrift::protocol::TType _etype949;
+            xfer += iprot->readListBegin(_etype949, _size946);
+            this->resourcePlans.resize(_size946);
+            uint32_t _i950;
+            for (_i950 = 0; _i950 < _size946; ++_i950)
             {
-              xfer += this->resourcePlans[_i945].read(iprot);
+              xfer += this->resourcePlans[_i950].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -23023,10 +23424,10 @@ uint32_t WMGetAllResourcePlanResponse::write(::apache::thrift::protocol::TProtoc
     xfer += oprot->writeFieldBegin("resourcePlans", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->resourcePlans.size()));
-      std::vector<WMResourcePlan> ::const_iterator _iter946;
-      for (_iter946 = this->resourcePlans.begin(); _iter946 != this->resourcePlans.end(); ++_iter946)
+      std::vector<WMResourcePlan> ::const_iterator _iter951;
+      for (_iter951 = this->resourcePlans.begin(); _iter951 != this->resourcePlans.end(); ++_iter951)
       {
-        xfer += (*_iter946).write(oprot);
+        xfer += (*_iter951).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -23043,13 +23444,13 @@ void swap(WMGetAllResourcePlanResponse &a, WMGetAllResourcePlanResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMGetAllResourcePlanResponse::WMGetAllResourcePlanResponse(const WMGetAllResourcePlanResponse& other947) {
-  resourcePlans = other947.resourcePlans;
-  __isset = other947.__isset;
+WMGetAllResourcePlanResponse::WMGetAllResourcePlanResponse(const WMGetAllResourcePlanResponse& other952) {
+  resourcePlans = other952.resourcePlans;
+  __isset = other952.__isset;
 }
-WMGetAllResourcePlanResponse& WMGetAllResourcePlanResponse::operator=(const WMGetAllResourcePlanResponse& other948) {
-  resourcePlans = other948.resourcePlans;
-  __isset = other948.__isset;
+WMGetAllResourcePlanResponse& WMGetAllResourcePlanResponse::operator=(const WMGetAllResourcePlanResponse& other953) {
+  resourcePlans = other953.resourcePlans;
+  __isset = other953.__isset;
   return *this;
 }
 void WMGetAllResourcePlanResponse::printTo(std::ostream& out) const {
@@ -23069,7 +23470,7 @@ void WMAlterResourcePlanRequest::__set_resourcePlanName(const std::string& val)
 __isset.resourcePlanName = true;
 }
 
-void WMAlterResourcePlanRequest::__set_resourcePlan(const WMResourcePlan& val) {
+void WMAlterResourcePlanRequest::__set_resourcePlan(const WMNullableResourcePlan& val) {
   this->resourcePlan = val;
 __isset.resourcePlan = true;
 }
@@ -23207,21 +23608,21 @@ void swap(WMAlterResourcePlanRequest &a, WMAlterResourcePlanRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMAlterResourcePlanRequest::WMAlterResourcePlanRequest(const WMAlterResourcePlanRequest& other949) {
-  resourcePlanName = other949.resourcePlanName;
-  resourcePlan = other949.resourcePlan;
-  isEnableAndActivate = other949.isEnableAndActivate;
-  isForceDeactivate = other949.isForceDeactivate;
-  isReplace = other949.isReplace;
-  __isset = other949.__isset;
-}
-WMAlterResourcePlanRequest& WMAlterResourcePlanRequest::operator=(const WMAlterResourcePlanRequest& other950) {
-  resourcePlanName = other950.resourcePlanName;
-  resourcePlan = other950.resourcePlan;
-  isEnableAndActivate = other950.isEnableAndActivate;
-  isForceDeactivate = other950.isForceDeactivate;
-  isReplace = other950.isReplace;
-  __isset = other950.__isset;
+WMAlterResourcePlanRequest::WMAlterResourcePlanRequest(const WMAlterResourcePlanRequest& other954) {
+  resourcePlanName = other954.resourcePlanName;
+  resourcePlan = other954.resourcePlan;
+  isEnableAndActivate = other954.isEnableAndActivate;
+  isForceDeactivate = other954.isForceDeactivate;
+  isReplace = other954.isReplace;
+  __isset = other954.__isset;
+}
+WMAlterResourcePlanRequest& WMAlterResourcePlanRequest::operator=(const WMAlterResourcePlanRequest& other955) {
+  resourcePlanName = other955.resourcePlanName;
+  resourcePlan = other955.resourcePlan;
+  isEnableAndActivate = other955.isEnableAndActivate;
+  isForceDeactivate = other955.isForceDeactivate;
+  isReplace = other955.isReplace;
+  __isset = other955.__isset;
   return *this;
 }
 void WMAlterResourcePlanRequest::printTo(std::ostream& out) const {
@@ -23307,13 +23708,13 @@ void swap(WMAlterResourcePlanResponse &a, WMAlterResourcePlanResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMAlterResourcePlanResponse::WMAlterResourcePlanResponse(const WMAlterResourcePlanResponse& other951) {
-  fullResourcePlan = other951.fullResourcePlan;
-  __isset = other951.__isset;
+WMAlterResourcePlanResponse::WMAlterResourcePlanResponse(const WMAlterResourcePlanResponse& other956) {
+  fullResourcePlan = other956.fullResourcePlan;
+  __isset = other956.__isset;
 }
-WMAlterResourcePlanResponse& WMAlterResourcePlanResponse::operator=(const WMAlterResourcePlanResponse& other952) {
-  fullResourcePlan = other952.fullResourcePlan;
-  __isset = other952.__isset;
+WMAlterResourcePlanResponse& WMAlterResourcePlanResponse::operator=(const WMAlterResourcePlanResponse& other957) {
+  fullResourcePlan = other957.fullResourcePlan;
+  __isset = other957.__isset;
   return *this;
 }
 void WMAlterResourcePlanResponse::printTo(std::ostream& out) const {
@@ -23395,13 +23796,13 @@ void swap(WMValidateResourcePlanRequest &a, WMValidateResourcePlanRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMValidateResourcePlanRequest::WMValidateResourcePlanRequest(const WMValidateResourcePlanRequest& other953) {
-  resourcePlanName = other953.resourcePlanName;
-  __isset = other953.__isset;
+WMValidateResourcePlanRequest::WMValidateResourcePlanRequest(const WMValidateResourcePlanRequest& other958) {
+  resourcePlanName = other958.resourcePlanName;
+  __isset = other958.__isset;
 }
-WMValidateResourcePlanRequest& WMValidateResourcePlanRequest::operator=(const WMValidateResourcePlanRequest& other954) {
-  resourcePlanName = other954.resourcePlanName;
-  __isset = other954.__isset;
+WMValidateResourcePlanRequest& WMValidateResourcePlanRequest::operator=(const WMValidateResourcePlanRequest& other959) {
+  resourcePlanName = other959.resourcePlanName;
+  __isset = other959.__isset;
   return *this;
 }
 void WMValidateResourcePlanRequest::printTo(std::ostream& out) const {
@@ -23446,14 +23847,14 @@ uint32_t WMValidateResourcePlanResponse::read(::apache::thrift::protocol::TProto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->errors.clear();
-            uint32_t _size955;
-            ::apache::thrift::protocol::TType _etype958;
-            xfer += iprot->readListBegin(_etype958, _size955);
-            this->errors.resize(_size955);
-            uint32_t _i959;
-            for (_i959 = 0; _i959 < _size955; ++_i959)
+            uint32_t _size960;
+            ::apache::thrift::protocol::TType _etype963;
+            xfer += iprot->readListBegin(_etype963, _size960);
+            this->errors.resize(_size960);
+            uint32_t _i964;
+            for (_i964 = 0; _i964 < _size960; ++_i964)
             {
-              xfer += iprot->readString(this->errors[_i959]);
+              xfer += iprot->readString(this->errors[_i964]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23483,10 +23884,10 @@ uint32_t WMValidateResourcePlanResponse::write(::apache::thrift::protocol::TProt
     xfer += oprot->writeFieldBegin("errors", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->errors.size()));
-      std::vector<std::string> ::const_iterator _iter960;
-      for (_iter960 = this->errors.begin(); _iter960 != this->errors.end(); ++_iter960)
+      std::vector<std::string> ::const_iterator _iter965;
+      for (_iter965 = this->errors.begin(); _iter965 != this->errors.end(); ++_iter965)
       {
-        xfer += oprot->writeString((*_iter960));
+        xfer += oprot->writeString((*_iter965));
       }
       xfer += oprot->writeListEnd();
     }
@@ -23503,13 +23904,13 @@ void swap(WMValidateResourcePlanResponse &a, WMValidateResourcePlanResponse &b)
   swap(a.__isset, b.__isset);
 }
 
-WMValidateResourcePlanResponse::WMValidateResourcePlanResponse(const WMValidateResourcePlanResponse& other961) {
-  errors = other961.errors;
-  __isset = other961.__isset;
+WMValidateResourcePlanResponse::WMValidateResourcePlanResponse(const WMValidateResourcePlanResponse& other966) {
+  errors = other966.errors;
+  __isset = other966.__isset;
 }
-WMValidateResourcePlanResponse& WMValidateResourcePlanResponse::operator=(const WMValidateResourcePlanResponse& other962) {
-  errors = other962.errors;
-  __isset = other962.__isset;
+WMValidateResourcePlanResponse& WMValidateResourcePlanResponse::operator=(const WMValidateResourcePlanResponse& other967) {
+  errors = other967.errors;
+  __isset = other967.__isset;
   return *this;
 }
 void WMValidateResourcePlanResponse::printTo(std::ostream& out) const {
@@ -23591,13 +23992,13 @@ void swap(WMDropResourcePlanRequest &a, WMDropResourcePlanRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMDropResourcePlanRequest::WMDropResourcePlanRequest(const WMDropResourcePlanRequest& other963) {
-  resourcePlanName = other963.resourcePlanName;
-  __isset = other963.__isset;
+WMDropResourcePlanRequest::WMDropResourcePlanRequest(const WMDropResourcePlanRequest& other968) {
+  resourcePlanName = other968.resourcePlanName;
+  __isset = other968.__isset;
 }
-WMDropResourcePlanRequest& WMDropResourcePlanRequest::operator=(const WMDropResourcePlanRequest& other964) {
-  resourcePlanName = other964.resourcePlanName;
-  __isset = other964.__isset;
+WMDropResourcePlanRequest& WMDropResourcePlanRequest::operator=(const WMDropResourcePlanRequest& other969) {
+  resourcePlanName = other969.resourcePlanName;
+  __isset = other969.__isset;
   return *this;
 }
 void WMDropResourcePlanRequest::printTo(std::ostream& out) const {
@@ -23656,11 +24057,11 @@ void swap(WMDropResourcePlanResponse &a, WMDropResourcePlanResponse &b) {
   (void) b;
 }
 
-WMDropResourcePlanResponse::WMDropResourcePlanResponse(const WMDropResourcePlanResponse& other965) {
-  (void) other965;
+WMDropResourcePlanResponse::WMDropResourcePlanResponse(const WMDropResourcePlanResponse& other970) {
+  (void) other970;
 }
-WMDropResourcePlanResponse& WMDropResourcePlanResponse::operator=(const WMDropResourcePlanResponse& other966) {
-  (void) other966;
+WMDropResourcePlanResponse& WMDropResourcePlanResponse::operator=(const WMDropResourcePlanResponse& other971) {
+  (void) other971;
   return *this;
 }
 void WMDropResourcePlanResponse::printTo(std::ostream& out) const {
@@ -23741,13 +24142,13 @@ void swap(WMCreateTriggerRequest &a, WMCreateTriggerRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMCreateTriggerRequest::WMCreateTriggerRequest(const WMCreateTriggerRequest& other967) {
-  trigger = other967.trigger;
-  __isset = other967.__isset;
+WMCreateTriggerRequest::WMCreateTriggerRequest(const WMCreateTriggerRequest& other972) {
+  trigger = other972.trigger;
+  __isset = other972.__isset;
 }
-WMCreateTriggerRequest& WMCreateTriggerRequest::operator=(const WMCreateTriggerRequest& other968) {
-  trigger = other968.trigger;
-  __isset = other968.__isset;
+WMCreateTriggerRequest& WMCreateTriggerRequest::operator=(const WMCreateTriggerRequest& other973) {
+  trigger = other973.trigger;
+  __isset = other973.__isset;
   return *this;
 }
 void WMCreateTriggerRequest::printTo(std::ostream& out) const {
@@ -23806,11 +24207,11 @@ void swap(WMCreateTriggerResponse &a, WMCreateTriggerResponse &b) {
   (void) b;
 }
 
-WMCreateTriggerResponse::WMCreateTriggerResponse(const WMCreateTriggerResponse& other969) {
-  (void) other969;
+WMCreateTriggerResponse::WMCreateTriggerResponse(const WMCreateTriggerResponse& other974) {
+  (void) other974;
 }
-WMCreateTriggerResponse& WMCreateTriggerResponse::operator=(const WMCreateTriggerResponse& other970) {
-  (void) other970;
+WMCreateTriggerResponse& WMCreateTriggerResponse::operator=(const WMCreateTriggerResponse& other975) {
+  (void) other975;
   return *this;
 }
 void WMCreateTriggerResponse::printTo(std::ostream& out) const {
@@ -23891,13 +24292,13 @@ void swap(WMAlterTriggerRequest &a, WMAlterTriggerRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMAlterTriggerRequest::WMAlterTriggerRequest(const WMAlterTriggerRequest& other971) {
-  trigger = other971.trigger;
-  __isset = other971.__isset;
+WMAlterTriggerRequest::WMAlterTriggerRequest(const WMAlterTriggerRequest& other976) {
+  trigger = other976.trigger;
+  __isset = other976.__isset;
 }
-WMAlterTriggerRequest& WMAlterTriggerRequest::operator=(const WMAlterTriggerRequest& other972) {
-  trigger = other972.trigger;
-  __isset = other972.__isset;
+WMAlterTriggerRequest& WMAlterTriggerRequest::operator=(const WMAlterTriggerRequest& other977) {
+  trigger = other977.trigger;
+  __isset = other977.__isset;
   return *this;
 }
 void WMAlterTriggerRequest::printTo(std::ostream& out) const {
@@ -23956,11 +24357,11 @@ void swap(WMAlterTriggerResponse &a, WMAlterTriggerResponse &b) {
   (void) b;
 }
 
-WMAlterTriggerResponse::WMAlterTriggerResponse(const WMAlterTriggerResponse& other973) {
-  (void) other973;
+WMAlterTriggerResponse::WMAlterTriggerResponse(const WMAlterTriggerResponse& other978) {
+  (void) other978;
 }
-WMAlterTriggerResponse& WMAlterTriggerResponse::operator=(const WMAlterTriggerResponse& other974) {
-  (void) other974;
+WMAlterTriggerResponse& WMAlterTriggerResponse::operator=(const WMAlterTriggerResponse& other979) {
+  (void) other979;
   return *this;
 }
 void WMAlterTriggerResponse::printTo(std::ostream& out) const {
@@ -24060,15 +24461,15 @@ void swap(WMDropTriggerRequest &a, WMDropTriggerRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMDropTriggerRequest::WMDropTriggerRequest(const WMDropTriggerRequest& other975) {
-  resourcePlanName = other975.resourcePlanName;
-  triggerName = other975.triggerName;
-  __isset = other975.__isset;
+WMDropTriggerRequest::WMDropTriggerRequest(const WMDropTriggerRequest& other980) {
+  resourcePlanName = other980.resourcePlanName;
+  triggerName = other980.triggerName;
+  __isset = other980.__isset;
 }
-WMDropTriggerRequest& WMDropTriggerRequest::operator=(const WMDropTriggerRequest& other976) {
-  resourcePlanName = other976.resourcePlanName;
-  triggerName = other976.triggerName;
-  __isset = other976.__isset;
+WMDropTriggerRequest& WMDropTriggerRequest::operator=(const WMDropTriggerRequest& other981) {
+  resourcePlanName = other981.resourcePlanName;
+  triggerName = other981.triggerName;
+  __isset = other981.__isset;
   return *this;
 }
 void WMDropTriggerRequest::printTo(std::ostream& out) const {
@@ -24128,11 +24529,11 @@ void swap(WMDropTriggerResponse &a, WMDropTriggerResponse &b) {
   (void) b;
 }
 
-WMDropTriggerResponse::WMDropTriggerResponse(const WMDropTriggerResponse& other977) {
-  (void) other977;
+WMDropTriggerResponse::WMDropTriggerResponse(const WMDropTriggerResponse& other982) {
+  (void) other982;
 }
-WMDropTriggerResponse& WMDropTriggerResponse::operator=(const WMDropTriggerResponse& other978) {
-  (void) other978;
+WMDropTriggerResponse& WMDropTriggerResponse::operator=(const WMDropTriggerResponse& other983) {
+  (void) other983;
   return *this;
 }
 void WMDropTriggerResponse::printTo(std::ostream& out) const {
@@ -24213,13 +24614,13 @@ void swap(WMGetTriggersForResourePlanRequest &a, WMGetTriggersForResourePlanRequ
   swap(a.__isset, b.__isset);
 }
 
-WMGetTriggersForResourePlanRequest::WMGetTriggersForResourePlanRequest(const WMGetTriggersForResourePlanRequest& other979) {
-  resourcePlanName = other979.resourcePlanName;
-  __isset = other979.__isset;
+WMGetTriggersForResourePlanRequest::WMGetTriggersForResourePlanRequest(const WMGetTriggersForResourePlanRequest& other984) {
+  resourcePlanName = other984.resourcePlanName;
+  __isset = other984.__isset;
 }
-WMGetTriggersForResourePlanRequest& WMGetTriggersForResourePlanRequest::operator=(const WMGetTriggersForResourePlanRequest& other980) {
-  resourcePlanName = other980.resourcePlanName;
-  __isset = other980.__isset;
+WMGetTriggersForResourePlanRequest& WMGetTriggersForResourePlanRequest::operator=(const WMGetTriggersForResourePlanRequest& other985) {
+  resourcePlanName = other985.resourcePlanName;
+  __isset = other985.__isset;
   return *this;
 }
 void WMGetTriggersForResourePlanRequest::printTo(std::ostream& out) const {
@@ -24264,14 +24665,14 @@ uint32_t WMGetTriggersForResourePlanResponse::read(::apache::thrift::protocol::T
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->triggers.clear();
-            uint32_t _size981;
-            ::apache::thrift::protocol::TType _etype984;
-            xfer += iprot->readListBegin(_etype984, _size981);
-            this->triggers.resize(_size981);
-            uint32_t _i985;
-            for (_i985 = 0; _i985 < _size981; ++_i985)
+            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)
             {
-              xfer += this->triggers[_i985].read(iprot);
+              xfer += this->triggers[_i990].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -24301,10 +24702,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 _iter986;
-      for (_iter986 = this->triggers.begin(); _iter986 != this->triggers.end(); ++_iter986)
+      std::vector<WMTrigger> ::const_iterator _iter991;
+      for (_iter991 = this->triggers.begin(); _iter991 != this->triggers.end(); ++_iter991)
       {
-        xfer += (*_iter986).write(oprot);
+        xfer += (*_iter991).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -24321,13 +24722,13 @@ void swap(WMGetTriggersForResourePlanResponse &a, WMGetTriggersForResourePlanRes
   swap(a.__isset, b.__isset);
 }
 
-WMGetTriggersForResourePlanResponse::WMGetTriggersForResourePlanResponse(const WMGetTriggersForResourePlanResponse& other987) {
-  triggers = other987.triggers;
-  __isset = other987.__isset;
+WMGetTriggersForResourePlanResponse::WMGetTriggersForResourePlanResponse(const WMGetTriggersForResourePlanResponse& other992) {
+  triggers = other992.triggers;
+  __isset = other992.__isset;
 }
-WMGetTriggersForResourePlanResponse& WMGetTriggersForResourePlanResponse::operator=(const WMGetTriggersForResourePlanResponse& other988) {
-  triggers = other988.triggers;
-  __isset = other988.__isset;
+WMGetTriggersForResourePlanResponse& WMGetTriggersForResourePlanResponse::operator=(const WMGetTriggersForResourePlanResponse& other993) {
+  triggers = other993.triggers;
+  __isset = other993.__isset;
   return *this;
 }
 void WMGetTriggersForResourePlanResponse::printTo(std::ostream& out) const {
@@ -24409,13 +24810,13 @@ void swap(WMCreatePoolRequest &a, WMCreatePoolRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMCreatePoolRequest::WMCreatePoolRequest(const WMCreatePoolRequest& other989) {
-  pool = other989.pool;
-  __isset = other989.__isset;
+WMCreatePoolRequest::WMCreatePoolRequest(const WMCreatePoolRequest& other994) {
+  pool = other994.pool;
+  __isset = other994.__isset;
 }
-WMCreatePoolRequest& WMCreatePoolRequest::operator=(const WMCreatePoolRequest& other990) {
-  pool = other990.pool;
-  __isset = other990.__isset;
+WMCreatePoolRequest& WMCreatePoolRequest::operator=(const WMCreatePoolRequest& other995) {
+  pool = other995.pool;
+  __isset = other995.__isset;
   return *this;
 }
 void WMCreatePoolRequest::printTo(std::ostream& out) const {
@@ -24474,11 +24875,11 @@ void swap(WMCreatePoolResponse &a, WMCreatePoolResponse &b) {
   (void) b;
 }
 
-WMCreatePoolResponse::WMCreatePoolResponse(const WMCreatePoolResponse& other991) {
-  (void) other991;
+WMCreatePoolResponse::WMCreatePoolResponse(const WMCreatePoolResponse& other996) {
+  (void) other996;
 }
-WMCreatePoolResponse& WMCreatePoolResponse::operator=(const WMCreatePoolResponse& other992) {
-  (void) other992;
+WMCreatePoolResponse& WMCreatePoolResponse::operator=(const WMCreatePoolResponse& other997) {
+  (void) other997;
   return *this;
 }
 void WMCreatePoolResponse::printTo(std::ostream& out) const {
@@ -24492,7 +24893,7 @@ WMAlterPoolRequest::~WMAlterPoolRequest() throw() {
 }
 
 
-void WMAlterPoolRequest::__set_pool(const WMPool& val) {
+void WMAlterPoolRequest::__set_pool(const WMNullablePool& val) {
   this->pool = val;
 __isset.pool = true;
 }
@@ -24578,15 +24979,15 @@ void swap(WMAlterPoolRequest &a, WMAlterPoolRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMAlterPoolRequest::WMAlterPoolRequest(const WMAlterPoolRequest& other993) {
-  pool = other993.pool;
-  poolPath = other993.poolPath;
-  __isset = other993.__isset;
+WMAlterPoolRequest::WMAlterPoolRequest(const WMAlterPoolRequest& other998) {
+  pool = other998.pool;
+  poolPath = other998.poolPath;
+  __isset = other998.__isset;
 }
-WMAlterPoolRequest& WMAlterPoolRequest::operator=(const WMAlterPoolRequest& other994) {
-  pool = other994.pool;
-  poolPath = other994.poolPath;
-  __isset = other994.__isset;
+WMAlterPoolRequest& WMAlterPoolRequest::operator=(const WMAlterPoolRequest& other999) {
+  pool = other999.pool;
+  poolPath = other999.poolPath;
+  __isset = other999.__isset;
   return *this;
 }
 void WMAlterPoolRequest::printTo(std::ostream& out) const {
@@ -24646,11 +25047,11 @@ void swap(WMAlterPoolResponse &a, WMAlterPoolResponse &b) {
   (void) b;
 }
 
-WMAlterPoolResponse::WMAlterPoolResponse(const WMAlterPoolResponse& other995) {
-  (void) other995;
+WMAlterPoolResponse::WMAlterPoolResponse(const WMAlterPoolResponse& other1000) {
+  (void) other1000;
 }
-WMAlterPoolResponse& WMAlterPoolResponse::operator=(const WMAlterPoolResponse& other996) {
-  (void) other996;
+WMAlterPoolResponse& WMAlterPoolResponse::operator=(const WMAlterPoolResponse& other1001) {
+  (void) other1001;
   return *this;
 }
 void WMAlterPoolResponse::printTo(std::ostream& out) const {
@@ -24750,15 +25151,15 @@ void swap(WMDropPoolRequest &a, WMDropPoolRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMDropPoolRequest::WMDropPoolRequest(const WMDropPoolRequest& other997) {
-  resourcePlanName = other997.resourcePlanName;
-  poolPath = other997.poolPath;
-  __isset = other997.__isset;
+WMDropPoolRequest::WMDropPoolRequest(const WMDropPoolRequest& other1002) {
+  resourcePlanName = other1002.resourcePlanName;
+  poolPath = other1002.poolPath;
+  __isset = other1002.__isset;
 }
-WMDropPoolRequest& WMDropPoolRequest::operator=(const WMDropPoolRequest& other998) {
-  resourcePlanName = other998.resourcePlanName;
-  poolPath = other998.poolPath;
-  __isset = other998.__isset;
+WMDropPoolRequest& WMDropPoolRequest::operator=(const WMDropPoolRequest& other1003) {
+  resourcePlanName = other1003.resourcePlanName;
+  poolPath = other1003.poolPath;
+  __isset = other1003.__isset;
   return *this;
 }
 void WMDropPoolRequest::printTo(std::ostream& out) const {
@@ -24818,11 +25219,11 @@ void swap(WMDropPoolResponse &a, WMDropPoolResponse &b) {
   (void) b;
 }
 
-WMDropPoolResponse::WMDropPoolResponse(const WMDropPoolResponse& other999) {
-  (void) other999;
+WMDropPoolResponse::WMDropPoolResponse(const WMDropPoolResponse& other1004) {
+  (void) other1004;
 }
-WMDropPoolResponse& WMDropPoolResponse::operator=(const WMDropPoolResponse& other1000) {
-  (void) other1000;
+WMDropPoolResponse& WMDropPoolResponse::operator=(const WMDropPoolResponse& other1005) {
+  (void) other1005;
   return *this;
 }
 void WMDropPoolResponse::printTo(std::ostream& out) const {
@@ -24922,15 +25323,15 @@ void swap(WMCreateOrUpdateMappingRequest &a, WMCreateOrUpdateMappingRequest &b)
   swap(a.__isset, b.__isset);
 }
 
-WMCreateOrUpdateMappingRequest::WMCreateOrUpdateMappingRequest(const WMCreateOrUpdateMappingRequest& other1001) {
-  mapping = other1001.mapping;
-  update = other1001.update;
-  __isset = other1001.__isset;
+WMCreateOrUpdateMappingRequest::WMCreateOrUpdateMappingRequest(const WMCreateOrUpdateMappingRequest& other1006) {
+  mapping = other1006.mapping;
+  update = other1006.update;
+  __isset = other1006.__isset;
 }
-WMCreateOrUpdateMappingRequest& WMCreateOrUpdateMappingRequest::operator=(const WMCreateOrUpdateMappingRequest& other1002) {
-  mapping = other1002.mapping;
-  update = other1002.update;
-  __isset = other1002.__isset;
+WMCreateOrUpdateMappingRequest& WMCreateOrUpdateMappingRequest::operator=(const WMCreateOrUpdateMappingRequest& other1007) {
+  mapping = other1007.mapping;
+  update = other1007.update;
+  __isset = other1007.__isset;
   return *this;
 }
 void WMCreateOrUpdateMappingRequest::printTo(std::ostream& out) const {
@@ -24990,11 +25391,11 @@ void swap(WMCreateOrUpdateMappingResponse &a, WMCreateOrUpdateMappingResponse &b
   (void) b;
 }
 
-WMCreateOrUpdateMappingResponse::WMCreateOrUpdateMappingResponse(const WMCreateOrUpdateMappingResponse& other1003) {
-  (void) other1003;
+WMCreateOrUpdateMappingResponse::WMCreateOrUpdateMappingResponse(const WMCreateOrUpdateMappingResponse& other1008) {
+  (void) other1008;
 }
-WMCreateOrUpdateMappingResponse& WMCreateOrUpdateMappingResponse::operator=(const WMCreateOrUpdateMappingResponse& other1004) {
-  (void) other1004;
+WMCreateOrUpdateMappingResponse& WMCreateOrUpdateMappingResponse::operator=(const WMCreateOrUpdateMappingResponse& other1009) {
+  (void) other1009;
   return *this;
 }
 void WMCreateOrUpdateMappingResponse::printTo(std::ostream& out) const {
@@ -25075,13 +25476,13 @@ void swap(WMDropMappingRequest &a, WMDropMappingRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-WMDropMappingRequest::WMDropMappingRequest(const WMDropMappingRequest& other1005) {
-  mapping = other1005.mapping;
-  __isset = other1005.__isset;
+WMDropMappingRequest::WMDropMappingRequest(const WMDropMappingRequest& other1010) {
+  mapping = other1010.mapping;
+  __isset = other1010.__isset;
 }
-WMDropMappingRequest& WMDropMappingRequest::operator=(const WMDropMappingRequest& other1006) {
-  mapping = other1006.mapping;
-  __isset = other1006.__isset;
+WMDropMappingRequest& WMDropMappingRequest::operator=(const WMDropMappingRequest& other1011) {
+  mapping = other1011.mapping;
+  __isset = other1011.__isset;
   return *this;
 }
 void WMDropMappingRequest::printTo(std::ostream& out) const {
@@ -25140,11 +25541,11 @@ void swap(WMDropMappingResponse &a, WMDropMappingResponse &b) {
   (void) b;
 }
 
-WMDropMappingResponse::WMDropMappingResponse(const WMDropMappingResponse& other1007) {
-  (void) other1007;
+WMDropMappingResponse::WMDropMappingResponse(const WMDropMappingResponse& other1012) {
+  (void) other1012;
 }
-WMDropMappingResponse& WMDropMappingResponse::operator=(const WMDropMappingResponse& other1008) {
-  (void) other1008;
+WMDropMappingResponse& WMDropMappingResponse::operator=(const WMDropMappingResponse& other1013) {
+  (void) other1013;
   return *this;
 }
 void WMDropMappingResponse::printTo(std::ostream& out) const {
@@ -25282,19 +25683,19 @@ void swap(WMCreateOrDropTriggerToPoolMappingRequest &a, WMCreateOrDropTriggerToP
   swap(a.__isset, b.__isset);
 }
 
-WMCreateOrDropTriggerToPoolMappingRequest::WMCreateOrDropTriggerToPoolMappingRequest(const WMCreateOrDropTriggerToPoolMappingRequest& other1009) {
-  resourcePlanName = other1009.resourcePlanName;
-  triggerName = other1009.triggerName;
-  poolPath = other1009.poolPath;
-  drop = other1009.drop;
-  __isset = other1009.__isset;
+WMCreateOrDropTriggerToPoolMappingRequest::WMCreateOrDropTriggerToPoolMappingRequest(const WMCreateOrDropTriggerToPoolMappingRequest& other1014) {
+  resourcePlanName = other1014.resourcePlanName;
+  triggerName = other1014.triggerName;
+  poolPath = other1014.poolPath;
+  drop = other1014.drop;
+  __isset = other1014.__isset;
 }
-WMCreateOrDropTriggerToPoolMappingRequest& WMCreateOrDropTriggerToPoolMappingRequest::operator=(const WMCreateOrDropTriggerToPoolMappingRequest& other1010) {
-  resourcePlanName = other1010.resourcePlanName;
-  triggerName = other1010.triggerName;
-  poolPath = other1010.poolPath;
-  drop = other1010.drop;
-  __isset = other1010.__isset;
+WMCreateOrDropTriggerToPoolMappingRequest& WMCreateOrDropTriggerToPoolMappingRequest::operator=(const WMCreateOrDropTriggerToPoolMappingRequest& other1015) {
+  resourcePlanName = other1015.resourcePlanName;
+  triggerName = other1015.triggerName;
+  poolPath = other1015.poolPath;
+  drop = other1015.drop;
+  __isset = other1015.__isset;
   return *this;
 }
 void WMCreateOrDropTriggerToPoolMappingRequest::printTo(std::ostream& out) const {
@@ -25356,11 +25757,11 @@ void swap(WMCreateOrDropTriggerToPoolMappingResponse &a, WMCreateOrDropTriggerTo
   (void) b;
 }
 
-WMCreateOrDropTriggerToPoolMappingResponse::WMCreateOrDropTriggerToPoolMappingResponse(const WMCreateOrDropTriggerToPoolMappingResponse& other1011) {
-  (void) other1011;
+WMCreateOrDropTriggerToPoolMappingResponse::WMCreateOrDropTriggerToPoolMappingResponse(const WMCreateOrDropTriggerToPoolMappingResponse& other1016) {
+  (void) other1016;
 }
-WMCreateOrDropTriggerToPoolMappingResponse& WMCreateOrDropTriggerToPoolMappingResponse::operator=(const WMCreateOrDropTriggerToPoolMappingResponse& other1012) {
-  (void) other1012;
+WMCreateOrDropTriggerToPoolMappingResponse& WMCreateOrDropTriggerToPoolMappingResponse::operator=(const WMCreateOrDropTriggerToPoolMappingResponse& other1017) {
+  (void) other1017;
   return *this;
 }
 void WMCreateOrDropTriggerToPoolMappingResponse::printTo(std::ostream& out) const {
@@ -25439,13 +25840,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other1013) : TException() {
-  message = other1013.message;
-  __isset = other1013.__isset;
+MetaException::MetaException(const MetaException& other1018) : TException() {
+  message = other1018.message;
+  __isset = other1018.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other1014) {
-  message = other1014.message;
-  __isset = other1014.__isset;
+MetaException& MetaException::operator=(const MetaException& other1019) {
+  message = other1019.message;
+  __isset = other1019.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -25536,13 +25937,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other1015) : TException() {
-  message = other1015.message;
-  __isset = other1015.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other1020) : TException() {
+  message = other1020.message;
+  __isset = other1020.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other1016) {
-  message = other1016.message;
-  __isset = other1016.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other1021) {
+  message = other1021.message;
+  __isset = other1021.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -25633,13 +26034,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other1017) : TException() {
-  message = other1017.message;
-  __isset = other1017.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other1022) : TException() {
+  message = other1022.message;
+  __isset = other1022.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other1018) {
-  message = other1018.message;
-  __isset = other1018.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other1023) {
+  message = other1023.message;
+  __isset = other1023.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -25730,13 +26131,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other1019) : TException() {
-  message = other1019.message;
-  __isset = other1019.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other1024) : TException() {
+  message = other1024.message;
+  __isset = other1024.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other1020) {
-  message = other1020.message;
-  __isset = other1020.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other1025) {
+  message = other1025.message;
+  __isset = other1025.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -25827,13 +26228,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other1021) : TException() {
-  message = other1021.message;
-  __isset = other1021.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other1026) : TException() {
+  message = other1026.message;
+  __isset = other1026.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other1022) {
-  message = other1022.message;
-  __isset = other1022.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other1027) {
+  message = other1027.message;
+  __isset = other1027.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -25924,13 +26325,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other1023) : TException() {
-  message = other1023.message;
-  __isset = other1023.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other1028) : TException() {
+  message = other1028.message;
+  __isset = other1028.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other1024) {
-  message = other1024.message;
-  __isset = other1024.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other1029) {
+  message = other1029.message;
+  __isset = other1029.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -26021,13 +26422,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other1025) : TException() {
-  message = other1025.message;
-  __isset = other1025.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other1030) : TException() {
+  message = other1030.message;
+  __isset = other1030.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other1026) {
-  message = other1026.message;
-  __isset = other1026.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other1031) {
+  message = other1031.message;
+  __isset = other1031.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -26118,13 +26519,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other1027) : TException() {
-  message = other1027.message;
-  __isset = other1027.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other1032) : TException() {
+  message = other1032.message;
+  __isset = other1032.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other1028) {
-  message = other1028.message;
-  __isset = other1028.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other1033) {
+  message = other1033.message;
+  __isset = other1033.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -26215,13 +26616,13 @@ void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other1029) : TException() {
-  message = other1029.message;
-  __isset = other1029.__isset;
+IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other1034) : TException() {
+  message = other1034.message;
+  __isset = other1034.__isset;
 }
-IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other1030) {
-  message = other1030.message;
-  __isset = other1030.__isset;
+IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other1035) {
+  message = other1035.message;
+  __isset = other1035.__isset;
   return *this;
 }
 void IndexAlreadyExistsException::printTo(std::ostream& out) const {
@@ -26312,13 +26713,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other1031) : TException() {
-  message = other1031.message;
-  __isset = other1031.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other1036) : TException() {
+  message = other1036.message;
+  __isset = other1036.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other1032) {
-  message = other1032.message;
-  __isset = other1032.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other1037) {
+  message = other1037.message;
+  __isset = other1037.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -26409,13 +26810,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other1033) : TException() {
-  message = other1033.message;
-  __isset = other1033.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other1038) : TException() {
+  message = other1038.message;
+  __isset = other1038.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other1034) {
-  message = other1034.message;
-  __isset = other1034.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other1039) {
+  message = other1039.message;
+  __isset = other1039.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -26506,13 +26907,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other1035) : TException() {
-  message = other1035.message;
-  __isset = other1035.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other1040) : TException() {
+  message = other1040.message;
+  __isset = other1040.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other1036) {
-  message = other1036.message;
-  __isset = other1036.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other1041) {
+  message = other1041.message;
+  __isset = other1041.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -26603,13 +27004,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other1037) : TException() {
-  message = other1037.message;
-  __isset = other1037.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other1042) : TException() {
+  message = other1042.message;
+  __isset = other1042.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other1038) {
-  message = other1038.message;
-  __isset = other1038.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other1043) {
+  message = other1043.message;
+  __isset = other1043.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -26700,13 +27101,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other1039) : TException() {
-  message = other1039.message;
-  __isset = other1039.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other1044) : TException() {
+  message = other1044.message;
+  __isset = other1044.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other1040) {
-  message = other1040.message;
-  __isset = other1040.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other1045) {
+  message = other1045.message;
+  __isset = other1045.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -26797,13 +27198,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other1041) : TException() {
-  message = other1041.message;
-  __isset = other1041.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other1046) : TException() {
+  message = other1046.message;
+  __isset = other1046.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other1042) {
-  message = other1042.message;
-  __isset = other1042.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other1047) {
+  message = other1047.message;
+  __isset = other1047.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -26894,13 +27295,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other1043) : TException() {
-  message = other1043.message;
-  __isset = other1043.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other1048) : TException() {
+  message = other1048.message;
+  __isset = other1048.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other1044) {
-  message = other1044.message;
-  __isset = other1044.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other1049) {
+  message = other1049.message;
+  __isset = other1049.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/900da829/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 3218d9a..4cc6417 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
@@ -466,8 +466,12 @@ class Materialization;
 
 class WMResourcePlan;
 
+class WMNullableResourcePlan;
+
 class WMPool;
 
+class WMNullablePool;
+
 class WMTrigger;
 
 class WMMapping;
@@ -8812,6 +8816,91 @@ inline std::ostream& operator<<(std::ostream& out, const WMResourcePlan& obj)
   return out;
 }
 
+typedef struct _WMNullableResourcePlan__isset {
+  _WMNullableResourcePlan__isset() : status(false), queryParallelism(false), isSetQueryParallelism(false), defaultPoolPath(false), isSetDefaultPoolPath(false) {}
+  bool status :1;
+  bool queryParallelism :1;
+  bool isSetQueryParallelism :1;
+  bool defaultPoolPath :1;
+  bool isSetDefaultPoolPath :1;
+} _WMNullableResourcePlan__isset;
+
+class WMNullableResourcePlan {
+ public:
+
+  WMNullableResourcePlan(const WMNullableResourcePlan&);
+  WMNullableResourcePlan& operator=(const WMNullableResourcePlan&);
+  WMNullableResourcePlan() : name(), status((WMResourcePlanStatus::type)0), queryParallelism(0), isSetQueryParallelism(0), defaultPoolPath(), isSetDefaultPoolPath(0) {
+  }
+
+  virtual ~WMNullableResourcePlan() throw();
+  std::string name;
+  WMResourcePlanStatus::type status;
+  int32_t queryParallelism;
+  bool isSetQueryParallelism;
+  std::string defaultPoolPath;
+  bool isSetDefaultPoolPath;
+
+  _WMNullableResourcePlan__isset __isset;
+
+  void __set_name(const std::string& val);
+
+  void __set_status(const WMResourcePlanStatus::type val);
+
+  void __set_queryParallelism(const int32_t val);
+
+  void __set_isSetQueryParallelism(const bool val);
+
+  void __set_defaultPoolPath(const std::string& val);
+
+  void __set_isSetDefaultPoolPath(const bool val);
+
+  bool operator == (const WMNullableResourcePlan & rhs) const
+  {
+    if (!(name == rhs.name))
+      return false;
+    if (__isset.status != rhs.__isset.status)
+      return false;
+    else if (__isset.status && !(status == rhs.status))
+      return false;
+    if (__isset.queryParallelism != rhs.__isset.queryParallelism)
+      return false;
+    else if (__isset.queryParallelism && !(queryParallelism == rhs.queryParallelism))
+      return false;
+    if (__isset.isSetQueryParallelism != rhs.__isset.isSetQueryParallelism)
+      return false;
+    else if (__isset.isSetQueryParallelism && !(isSetQueryParallelism == rhs.isSetQueryParallelism))
+      return false;
+    if (__isset.defaultPoolPath != rhs.__isset.defaultPoolPath)
+      return false;
+    else if (__isset.defaultPoolPath && !(defaultPoolPath == rhs.defaultPoolPath))
+      return false;
+    if (__isset.isSetDefaultPoolPath != rhs.__isset.isSetDefaultPoolPath)
+      return false;
+    else if (__isset.isSetDefaultPoolPath && !(isSetDefaultPoolPath == rhs.isSetDefaultPoolPath))
+      return false;
+    return true;
+  }
+  bool operator != (const WMNullableResourcePlan &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const WMNullableResourcePlan & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(WMNullableResourcePlan &a, WMNullableResourcePlan &b);
+
+inline std::ostream& operator<<(std::ostream& out, const WMNullableResourcePlan& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _WMPool__isset {
   _WMPool__isset() : allocFraction(false), queryParallelism(false), schedulingPolicy(false) {}
   bool allocFraction :1;
@@ -8886,6 +8975,88 @@ inline std::ostream& operator<<(std::ostream& out, const WMPool& obj)
   return out;
 }
 
+typedef struct _WMNullablePool__isset {
+  _WMNullablePool__isset() : allocFraction(false), queryParallelism(false), schedulingPolicy(false), isSetSchedulingPolicy(false) {}
+  bool allocFraction :1;
+  bool queryParallelism :1;
+  bool schedulingPolicy :1;
+  bool isSetSchedulingPolicy :1;
+} _WMNullablePool__isset;
+
+class WMNullablePool {
+ public:
+
+  WMNullablePool(const WMNullablePool&);
+  WMNullablePool& operator=(const WMNullablePool&);
+  WMNullablePool() : resourcePlanName(), poolPath(), allocFraction(0), queryParallelism(0), schedulingPolicy(), isSetSchedulingPolicy(0) {
+  }
+
+  virtual ~WMNullablePool() throw();
+  std::string resourcePlanName;
+  std::string poolPath;
+  double allocFraction;
+  int32_t queryParallelism;
+  std::string schedulingPolicy;
+  bool isSetSchedulingPolicy;
+
+  _WMNullablePool__isset __isset;
+
+  void __set_resourcePlanName(const std::string& val);
+
+  void __set_poolPath(const std::string& val);
+
+  void __set_allocFraction(const double val);
+
+  void __set_queryParallelism(const int32_t val);
+
+  void __set_schedulingPolicy(const std::string& val);
+
+  void __set_isSetSchedulingPolicy(const bool val);
+
+  bool operator == (const WMNullablePool & rhs) const
+  {
+    if (!(resourcePlanName == rhs.resourcePlanName))
+      return false;
+    if (!(poolPath == rhs.poolPath))
+      return false;
+    if (__isset.allocFraction != rhs.__isset.allocFraction)
+      return false;
+    else if (__isset.allocFraction && !(allocFraction == rhs.allocFraction))
+      return false;
+    if (__isset.queryParallelism != rhs.__isset.queryParallelism)
+      return false;
+    else if (__isset.queryParallelism && !(queryParallelism == rhs.queryParallelism))
+      return false;
+    if (__isset.schedulingPolicy != rhs.__isset.schedulingPolicy)
+      return false;
+    else if (__isset.schedulingPolicy && !(schedulingPolicy == rhs.schedulingPolicy))
+      return false;
+    if (__isset.isSetSchedulingPolicy != rhs.__isset.isSetSchedulingPolicy)
+      return false;
+    else if (__isset.isSetSchedulingPolicy && !(isSetSchedulingPolicy == rhs.isSetSchedulingPolicy))
+      return false;
+    return true;
+  }
+  bool operator != (const WMNullablePool &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const WMNullablePool & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(WMNullablePool &a, WMNullablePool &b);
+
+inline std::ostream& operator<<(std::ostream& out, const WMNullablePool& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _WMTrigger__isset {
   _WMTrigger__isset() : triggerExpression(false), actionExpression(false), isInUnmanaged(false) {}
   bool triggerExpression :1;
@@ -9522,7 +9693,7 @@ class WMAlterResourcePlanRequest {
 
   virtual ~WMAlterResourcePlanRequest() throw();
   std::string resourcePlanName;
-  WMResourcePlan resourcePlan;
+  WMNullableResourcePlan resourcePlan;
   bool isEnableAndActivate;
   bool isForceDeactivate;
   bool isReplace;
@@ -9531,7 +9702,7 @@ class WMAlterResourcePlanRequest {
 
   void __set_resourcePlanName(const std::string& val);
 
-  void __set_resourcePlan(const WMResourcePlan& val);
+  void __set_resourcePlan(const WMNullableResourcePlan& val);
 
   void __set_isEnableAndActivate(const bool val);
 
@@ -10261,12 +10432,12 @@ class WMAlterPoolRequest {
   }
 
   virtual ~WMAlterPoolRequest() throw();
-  WMPool pool;
+  WMNullablePool pool;
   std::string poolPath;
 
   _WMAlterPoolRequest__isset __isset;
 
-  void __set_pool(const WMPool& val);
+  void __set_pool(const WMNullablePool& val);
 
   void __set_poolPath(const std::string& val);