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 2016/10/04 18:46:42 UTC

[01/15] hive git commit: HIVE-14768: Add a new UDTF Replicate_Rows (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/hive-14535 ad3df23b9 -> 0ce24b93e


HIVE-14768: Add a new UDTF Replicate_Rows (Pengcheng Xiong, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hive-14535
Commit: e19f0e35e09ca283e5de46ae7e2db1e11396335e
Parents: 0a4b3d8
Author: Pengcheng Xiong <px...@apache.org>
Authored: Mon Oct 3 22:07:24 2016 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Mon Oct 3 22:07:24 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   1 +
 .../udf/generic/GenericUDTFReplicateRows.java   |  88 +++++++++++++++
 .../clientpositive/udtf_replicate_rows.q        |  23 ++++
 .../results/clientpositive/show_functions.q.out |   1 +
 .../clientpositive/udtf_replicate_rows.q.out    | 107 +++++++++++++++++++
 5 files changed, 220 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e19f0e35/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index 6870dfa..6b29be1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -468,6 +468,7 @@ public final class FunctionRegistry {
 
     // Generic UDTF's
     system.registerGenericUDTF("explode", GenericUDTFExplode.class);
+    system.registerGenericUDTF("replicate_rows", GenericUDTFReplicateRows.class);
     system.registerGenericUDTF("inline", GenericUDTFInline.class);
     system.registerGenericUDTF("json_tuple", GenericUDTFJSONTuple.class);
     system.registerGenericUDTF("parse_url_tuple", GenericUDTFParseUrlTuple.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/e19f0e35/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFReplicateRows.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFReplicateRows.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFReplicateRows.java
new file mode 100644
index 0000000..164445d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFReplicateRows.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ReturnObjectInspectorResolver;
+import org.apache.hadoop.hive.serde2.lazy.LazyLong;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * Takes a row of data and repeats n times.
+ */
+@Description(name = "replicate_rows", value = "_FUNC_(n, cols...) - turns 1 row into n rows")
+public class GenericUDTFReplicateRows extends GenericUDTF {
+  @Override
+  public void close() throws HiveException {
+  }
+
+  private transient List<ObjectInspector> argOIs = new ArrayList<ObjectInspector>();
+
+  @Override
+  public StructObjectInspector initialize(ObjectInspector[] args) throws UDFArgumentException {
+    if (args.length < 2) {
+      throw new UDFArgumentException("UDTFReplicateRows() expects at least two arguments.");
+    }
+    if (!(args[0] instanceof LongObjectInspector)) {
+      throw new UDFArgumentException(
+          "The first argument to UDTFReplicateRows() must be a long (got "
+              + args[0].getTypeName() + " instead).");
+    }
+
+    ArrayList<String> fieldNames = new ArrayList<String>();
+    ArrayList<ObjectInspector> fieldOIs = new ArrayList<ObjectInspector>();
+    for (int index = 0; index < args.length; ++index) {
+      fieldNames.add("col" + index);
+      fieldOIs.add(args[index]);
+    }
+    argOIs = fieldOIs;
+    return ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames, fieldOIs);
+  }
+
+  @Override
+  public void process(Object[] args) throws HiveException, UDFArgumentException {
+
+    long numRows = ((LongObjectInspector) argOIs.get(0)).get(args[0]);
+
+    for (long n = 0; n < numRows; n++) {
+      forward(args);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "UDTFReplicateRows";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/e19f0e35/ql/src/test/queries/clientpositive/udtf_replicate_rows.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udtf_replicate_rows.q b/ql/src/test/queries/clientpositive/udtf_replicate_rows.q
new file mode 100644
index 0000000..a074a78
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/udtf_replicate_rows.q
@@ -0,0 +1,23 @@
+set hive.mapred.mode=nonstrict;
+set hive.cbo.enable=false;
+
+DESCRIBE FUNCTION replicate_rows;
+DESCRIBE FUNCTION EXTENDED replicate_rows;
+
+create table t (x bigint, y string, z int);
+
+insert into table t values (3,'2',0),(2,'3',1),(0,'2',2),(-1,'k',3);
+
+SELECT replicate_rows(x,y) FROM t;
+
+SELECT replicate_rows(x,y,y) FROM t;
+
+SELECT replicate_rows(x,y,y,y,z) FROM t;
+
+select y,x from (SELECT replicate_rows(x,y) as (x,y) FROM t)subq;
+
+select z,y,x from(SELECT replicate_rows(x,y,y) as (z,y,x) FROM t)subq;
+
+SELECT replicate_rows(x,concat(y,'...'),y) FROM t;
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/e19f0e35/ql/src/test/results/clientpositive/show_functions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/show_functions.q.out b/ql/src/test/results/clientpositive/show_functions.q.out
index 4a40094..7b746ff 100644
--- a/ql/src/test/results/clientpositive/show_functions.q.out
+++ b/ql/src/test/results/clientpositive/show_functions.q.out
@@ -186,6 +186,7 @@ regexp_extract
 regexp_replace
 repeat
 replace
+replicate_rows
 reverse
 rlike
 round

http://git-wip-us.apache.org/repos/asf/hive/blob/e19f0e35/ql/src/test/results/clientpositive/udtf_replicate_rows.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udtf_replicate_rows.q.out b/ql/src/test/results/clientpositive/udtf_replicate_rows.q.out
new file mode 100644
index 0000000..f76a584
--- /dev/null
+++ b/ql/src/test/results/clientpositive/udtf_replicate_rows.q.out
@@ -0,0 +1,107 @@
+PREHOOK: query: DESCRIBE FUNCTION replicate_rows
+PREHOOK: type: DESCFUNCTION
+POSTHOOK: query: DESCRIBE FUNCTION replicate_rows
+POSTHOOK: type: DESCFUNCTION
+replicate_rows(n, cols...) - turns 1 row into n rows
+PREHOOK: query: DESCRIBE FUNCTION EXTENDED replicate_rows
+PREHOOK: type: DESCFUNCTION
+POSTHOOK: query: DESCRIBE FUNCTION EXTENDED replicate_rows
+POSTHOOK: type: DESCFUNCTION
+replicate_rows(n, cols...) - turns 1 row into n rows
+PREHOOK: query: create table t (x bigint, y string, z int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t
+POSTHOOK: query: create table t (x bigint, y string, z int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t
+PREHOOK: query: insert into table t values (3,'2',0),(2,'3',1),(0,'2',2),(-1,'k',3)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@t
+POSTHOOK: query: insert into table t values (3,'2',0),(2,'3',1),(0,'2',2),(-1,'k',3)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@t
+POSTHOOK: Lineage: t.x EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: t.y SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: t.z EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+PREHOOK: query: SELECT replicate_rows(x,y) FROM t
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT replicate_rows(x,y) FROM t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t
+#### A masked pattern was here ####
+3	2
+3	2
+3	2
+2	3
+2	3
+PREHOOK: query: SELECT replicate_rows(x,y,y) FROM t
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT replicate_rows(x,y,y) FROM t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t
+#### A masked pattern was here ####
+3	2	2
+3	2	2
+3	2	2
+2	3	3
+2	3	3
+PREHOOK: query: SELECT replicate_rows(x,y,y,y,z) FROM t
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT replicate_rows(x,y,y,y,z) FROM t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t
+#### A masked pattern was here ####
+3	2	2	2	0
+3	2	2	2	0
+3	2	2	2	0
+2	3	3	3	1
+2	3	3	3	1
+PREHOOK: query: select y,x from (SELECT replicate_rows(x,y) as (x,y) FROM t)subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t
+#### A masked pattern was here ####
+POSTHOOK: query: select y,x from (SELECT replicate_rows(x,y) as (x,y) FROM t)subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t
+#### A masked pattern was here ####
+2	3
+2	3
+2	3
+3	2
+3	2
+PREHOOK: query: select z,y,x from(SELECT replicate_rows(x,y,y) as (z,y,x) FROM t)subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t
+#### A masked pattern was here ####
+POSTHOOK: query: select z,y,x from(SELECT replicate_rows(x,y,y) as (z,y,x) FROM t)subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t
+#### A masked pattern was here ####
+3	2	2
+3	2	2
+3	2	2
+2	3	3
+2	3	3
+PREHOOK: query: SELECT replicate_rows(x,concat(y,'...'),y) FROM t
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT replicate_rows(x,concat(y,'...'),y) FROM t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t
+#### A masked pattern was here ####
+3	2...	2
+3	2...	2
+3	2...	2
+2	3...	3
+2	3...	3


[10/15] hive git commit: HIVE-14671 : merge master into hive-14535 (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 22c0cc6,10778f2..6f157d8
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@@ -14858,6 -14695,183 +14927,183 @@@ class get_tables_result
    def __ne__(self, other):
      return not (self == other)
  
+ class get_tables_by_type_args:
+   """
+   Attributes:
+    - db_name
+    - pattern
+    - tableType
+   """
+ 
+   thrift_spec = (
+     None, # 0
+     (1, TType.STRING, 'db_name', None, None, ), # 1
+     (2, TType.STRING, 'pattern', None, None, ), # 2
+     (3, TType.STRING, 'tableType', None, None, ), # 3
+   )
+ 
+   def __init__(self, db_name=None, pattern=None, tableType=None,):
+     self.db_name = db_name
+     self.pattern = pattern
+     self.tableType = tableType
+ 
+   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.db_name = iprot.readString()
+         else:
+           iprot.skip(ftype)
+       elif fid == 2:
+         if ftype == TType.STRING:
+           self.pattern = iprot.readString()
+         else:
+           iprot.skip(ftype)
+       elif fid == 3:
+         if ftype == TType.STRING:
+           self.tableType = iprot.readString()
+         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('get_tables_by_type_args')
+     if self.db_name is not None:
+       oprot.writeFieldBegin('db_name', TType.STRING, 1)
+       oprot.writeString(self.db_name)
+       oprot.writeFieldEnd()
+     if self.pattern is not None:
+       oprot.writeFieldBegin('pattern', TType.STRING, 2)
+       oprot.writeString(self.pattern)
+       oprot.writeFieldEnd()
+     if self.tableType is not None:
+       oprot.writeFieldBegin('tableType', TType.STRING, 3)
+       oprot.writeString(self.tableType)
+       oprot.writeFieldEnd()
+     oprot.writeFieldStop()
+     oprot.writeStructEnd()
+ 
+   def validate(self):
+     return
+ 
+ 
+   def __hash__(self):
+     value = 17
+     value = (value * 31) ^ hash(self.db_name)
+     value = (value * 31) ^ hash(self.pattern)
+     value = (value * 31) ^ hash(self.tableType)
+     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 get_tables_by_type_result:
+   """
+   Attributes:
+    - success
+    - o1
+   """
+ 
+   thrift_spec = (
+     (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0
+     (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
+   )
+ 
+   def __init__(self, success=None, o1=None,):
+     self.success = success
+     self.o1 = o1
+ 
+   def read(self, iprot):
+     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+       fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+       return
+     iprot.readStructBegin()
+     while True:
+       (fname, ftype, fid) = iprot.readFieldBegin()
+       if ftype == TType.STOP:
+         break
+       if fid == 0:
+         if ftype == TType.LIST:
+           self.success = []
 -          (_etype641, _size638) = iprot.readListBegin()
 -          for _i642 in xrange(_size638):
 -            _elem643 = iprot.readString()
 -            self.success.append(_elem643)
++          (_etype648, _size645) = iprot.readListBegin()
++          for _i649 in xrange(_size645):
++            _elem650 = iprot.readString()
++            self.success.append(_elem650)
+           iprot.readListEnd()
+         else:
+           iprot.skip(ftype)
+       elif fid == 1:
+         if ftype == TType.STRUCT:
+           self.o1 = MetaException()
+           self.o1.read(iprot)
+         else:
+           iprot.skip(ftype)
+       else:
+         iprot.skip(ftype)
+       iprot.readFieldEnd()
+     iprot.readStructEnd()
+ 
+   def write(self, oprot):
+     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+       return
+     oprot.writeStructBegin('get_tables_by_type_result')
+     if self.success is not None:
+       oprot.writeFieldBegin('success', TType.LIST, 0)
+       oprot.writeListBegin(TType.STRING, len(self.success))
 -      for iter644 in self.success:
 -        oprot.writeString(iter644)
++      for iter651 in self.success:
++        oprot.writeString(iter651)
+       oprot.writeListEnd()
+       oprot.writeFieldEnd()
+     if self.o1 is not None:
+       oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+       self.o1.write(oprot)
+       oprot.writeFieldEnd()
+     oprot.writeFieldStop()
+     oprot.writeStructEnd()
+ 
+   def validate(self):
+     return
+ 
+ 
+   def __hash__(self):
+     value = 17
+     value = (value * 31) ^ hash(self.success)
+     value = (value * 31) ^ hash(self.o1)
+     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 get_table_meta_args:
    """
    Attributes:
@@@ -14900,10 -14914,10 +15146,10 @@@
        elif fid == 3:
          if ftype == TType.LIST:
            self.tbl_types = []
--          (_etype648, _size645) = iprot.readListBegin()
--          for _i649 in xrange(_size645):
--            _elem650 = iprot.readString()
--            self.tbl_types.append(_elem650)
++          (_etype655, _size652) = iprot.readListBegin()
++          for _i656 in xrange(_size652):
++            _elem657 = iprot.readString()
++            self.tbl_types.append(_elem657)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -14928,8 -14942,8 +15174,8 @@@
      if self.tbl_types is not None:
        oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.tbl_types))
--      for iter651 in self.tbl_types:
--        oprot.writeString(iter651)
++      for iter658 in self.tbl_types:
++        oprot.writeString(iter658)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -14985,11 -14999,11 +15231,11 @@@ class get_table_meta_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype655, _size652) = iprot.readListBegin()
--          for _i656 in xrange(_size652):
--            _elem657 = TableMeta()
--            _elem657.read(iprot)
--            self.success.append(_elem657)
++          (_etype662, _size659) = iprot.readListBegin()
++          for _i663 in xrange(_size659):
++            _elem664 = TableMeta()
++            _elem664.read(iprot)
++            self.success.append(_elem664)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -15012,8 -15026,8 +15258,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
--      for iter658 in self.success:
--        iter658.write(oprot)
++      for iter665 in self.success:
++        iter665.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -15137,10 -15151,10 +15383,10 @@@ class get_all_tables_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype662, _size659) = iprot.readListBegin()
--          for _i663 in xrange(_size659):
--            _elem664 = iprot.readString()
--            self.success.append(_elem664)
++          (_etype669, _size666) = iprot.readListBegin()
++          for _i670 in xrange(_size666):
++            _elem671 = iprot.readString()
++            self.success.append(_elem671)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -15163,8 -15177,8 +15409,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
--      for iter665 in self.success:
--        oprot.writeString(iter665)
++      for iter672 in self.success:
++        oprot.writeString(iter672)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -15400,10 -15414,10 +15646,10 @@@ class get_table_objects_by_name_args
        elif fid == 2:
          if ftype == TType.LIST:
            self.tbl_names = []
--          (_etype669, _size666) = iprot.readListBegin()
--          for _i670 in xrange(_size666):
--            _elem671 = iprot.readString()
--            self.tbl_names.append(_elem671)
++          (_etype676, _size673) = iprot.readListBegin()
++          for _i677 in xrange(_size673):
++            _elem678 = iprot.readString()
++            self.tbl_names.append(_elem678)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -15424,8 -15438,8 +15670,8 @@@
      if self.tbl_names is not None:
        oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
        oprot.writeListBegin(TType.STRING, len(self.tbl_names))
--      for iter672 in self.tbl_names:
--        oprot.writeString(iter672)
++      for iter679 in self.tbl_names:
++        oprot.writeString(iter679)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -15486,11 -15500,11 +15732,11 @@@ class get_table_objects_by_name_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype676, _size673) = iprot.readListBegin()
--          for _i677 in xrange(_size673):
--            _elem678 = Table()
--            _elem678.read(iprot)
--            self.success.append(_elem678)
++          (_etype683, _size680) = iprot.readListBegin()
++          for _i684 in xrange(_size680):
++            _elem685 = Table()
++            _elem685.read(iprot)
++            self.success.append(_elem685)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -15525,8 -15539,8 +15771,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
--      for iter679 in self.success:
--        iter679.write(oprot)
++      for iter686 in self.success:
++        iter686.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -15692,10 -15706,10 +15938,10 @@@ class get_table_names_by_filter_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype683, _size680) = iprot.readListBegin()
--          for _i684 in xrange(_size680):
--            _elem685 = iprot.readString()
--            self.success.append(_elem685)
++          (_etype690, _size687) = iprot.readListBegin()
++          for _i691 in xrange(_size687):
++            _elem692 = iprot.readString()
++            self.success.append(_elem692)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -15730,8 -15744,8 +15976,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
--      for iter686 in self.success:
--        oprot.writeString(iter686)
++      for iter693 in self.success:
++        oprot.writeString(iter693)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -16701,11 -16715,11 +16947,11 @@@ class add_partitions_args
        if fid == 1:
          if ftype == TType.LIST:
            self.new_parts = []
--          (_etype690, _size687) = iprot.readListBegin()
--          for _i691 in xrange(_size687):
--            _elem692 = Partition()
--            _elem692.read(iprot)
--            self.new_parts.append(_elem692)
++          (_etype697, _size694) = iprot.readListBegin()
++          for _i698 in xrange(_size694):
++            _elem699 = Partition()
++            _elem699.read(iprot)
++            self.new_parts.append(_elem699)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -16722,8 -16736,8 +16968,8 @@@
      if self.new_parts is not None:
        oprot.writeFieldBegin('new_parts', TType.LIST, 1)
        oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
--      for iter693 in self.new_parts:
--        iter693.write(oprot)
++      for iter700 in self.new_parts:
++        iter700.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -16881,11 -16895,11 +17127,11 @@@ class add_partitions_pspec_args
        if fid == 1:
          if ftype == TType.LIST:
            self.new_parts = []
--          (_etype697, _size694) = iprot.readListBegin()
--          for _i698 in xrange(_size694):
--            _elem699 = PartitionSpec()
--            _elem699.read(iprot)
--            self.new_parts.append(_elem699)
++          (_etype704, _size701) = iprot.readListBegin()
++          for _i705 in xrange(_size701):
++            _elem706 = PartitionSpec()
++            _elem706.read(iprot)
++            self.new_parts.append(_elem706)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -16902,8 -16916,8 +17148,8 @@@
      if self.new_parts is not None:
        oprot.writeFieldBegin('new_parts', TType.LIST, 1)
        oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
--      for iter700 in self.new_parts:
--        iter700.write(oprot)
++      for iter707 in self.new_parts:
++        iter707.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -17077,10 -17091,10 +17323,10 @@@ class append_partition_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
--          (_etype704, _size701) = iprot.readListBegin()
--          for _i705 in xrange(_size701):
--            _elem706 = iprot.readString()
--            self.part_vals.append(_elem706)
++          (_etype711, _size708) = iprot.readListBegin()
++          for _i712 in xrange(_size708):
++            _elem713 = iprot.readString()
++            self.part_vals.append(_elem713)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -17105,8 -17119,8 +17351,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
--      for iter707 in self.part_vals:
--        oprot.writeString(iter707)
++      for iter714 in self.part_vals:
++        oprot.writeString(iter714)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -17459,10 -17473,10 +17705,10 @@@ class append_partition_with_environment
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
--          (_etype711, _size708) = iprot.readListBegin()
--          for _i712 in xrange(_size708):
--            _elem713 = iprot.readString()
--            self.part_vals.append(_elem713)
++          (_etype718, _size715) = iprot.readListBegin()
++          for _i719 in xrange(_size715):
++            _elem720 = iprot.readString()
++            self.part_vals.append(_elem720)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -17493,8 -17507,8 +17739,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
--      for iter714 in self.part_vals:
--        oprot.writeString(iter714)
++      for iter721 in self.part_vals:
++        oprot.writeString(iter721)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.environment_context is not None:
@@@ -18089,10 -18103,10 +18335,10 @@@ class drop_partition_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
--          (_etype718, _size715) = iprot.readListBegin()
--          for _i719 in xrange(_size715):
--            _elem720 = iprot.readString()
--            self.part_vals.append(_elem720)
++          (_etype725, _size722) = iprot.readListBegin()
++          for _i726 in xrange(_size722):
++            _elem727 = iprot.readString()
++            self.part_vals.append(_elem727)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -18122,8 -18136,8 +18368,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
--      for iter721 in self.part_vals:
--        oprot.writeString(iter721)
++      for iter728 in self.part_vals:
++        oprot.writeString(iter728)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.deleteData is not None:
@@@ -18296,10 -18310,10 +18542,10 @@@ class drop_partition_with_environment_c
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
--          (_etype725, _size722) = iprot.readListBegin()
--          for _i726 in xrange(_size722):
--            _elem727 = iprot.readString()
--            self.part_vals.append(_elem727)
++          (_etype732, _size729) = iprot.readListBegin()
++          for _i733 in xrange(_size729):
++            _elem734 = iprot.readString()
++            self.part_vals.append(_elem734)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -18335,8 -18349,8 +18581,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
--      for iter728 in self.part_vals:
--        oprot.writeString(iter728)
++      for iter735 in self.part_vals:
++        oprot.writeString(iter735)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.deleteData is not None:
@@@ -19073,10 -19087,10 +19319,10 @@@ class get_partition_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
--          (_etype732, _size729) = iprot.readListBegin()
--          for _i733 in xrange(_size729):
--            _elem734 = iprot.readString()
--            self.part_vals.append(_elem734)
++          (_etype739, _size736) = iprot.readListBegin()
++          for _i740 in xrange(_size736):
++            _elem741 = iprot.readString()
++            self.part_vals.append(_elem741)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -19101,8 -19115,8 +19347,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
--      for iter735 in self.part_vals:
--        oprot.writeString(iter735)
++      for iter742 in self.part_vals:
++        oprot.writeString(iter742)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -19261,11 -19275,11 +19507,11 @@@ class exchange_partition_args
        if fid == 1:
          if ftype == TType.MAP:
            self.partitionSpecs = {}
--          (_ktype737, _vtype738, _size736 ) = iprot.readMapBegin()
--          for _i740 in xrange(_size736):
--            _key741 = iprot.readString()
--            _val742 = iprot.readString()
--            self.partitionSpecs[_key741] = _val742
++          (_ktype744, _vtype745, _size743 ) = iprot.readMapBegin()
++          for _i747 in xrange(_size743):
++            _key748 = iprot.readString()
++            _val749 = iprot.readString()
++            self.partitionSpecs[_key748] = _val749
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -19302,9 -19316,9 +19548,9 @@@
      if self.partitionSpecs is not None:
        oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
        oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
--      for kiter743,viter744 in self.partitionSpecs.items():
--        oprot.writeString(kiter743)
--        oprot.writeString(viter744)
++      for kiter750,viter751 in self.partitionSpecs.items():
++        oprot.writeString(kiter750)
++        oprot.writeString(viter751)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      if self.source_db is not None:
@@@ -19509,11 -19523,11 +19755,11 @@@ class exchange_partitions_args
        if fid == 1:
          if ftype == TType.MAP:
            self.partitionSpecs = {}
--          (_ktype746, _vtype747, _size745 ) = iprot.readMapBegin()
--          for _i749 in xrange(_size745):
--            _key750 = iprot.readString()
--            _val751 = iprot.readString()
--            self.partitionSpecs[_key750] = _val751
++          (_ktype753, _vtype754, _size752 ) = iprot.readMapBegin()
++          for _i756 in xrange(_size752):
++            _key757 = iprot.readString()
++            _val758 = iprot.readString()
++            self.partitionSpecs[_key757] = _val758
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -19550,9 -19564,9 +19796,9 @@@
      if self.partitionSpecs is not None:
        oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
        oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
--      for kiter752,viter753 in self.partitionSpecs.items():
--        oprot.writeString(kiter752)
--        oprot.writeString(viter753)
++      for kiter759,viter760 in self.partitionSpecs.items():
++        oprot.writeString(kiter759)
++        oprot.writeString(viter760)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      if self.source_db is not None:
@@@ -19635,11 -19649,11 +19881,11 @@@ class exchange_partitions_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype757, _size754) = iprot.readListBegin()
--          for _i758 in xrange(_size754):
--            _elem759 = Partition()
--            _elem759.read(iprot)
--            self.success.append(_elem759)
++          (_etype764, _size761) = iprot.readListBegin()
++          for _i765 in xrange(_size761):
++            _elem766 = Partition()
++            _elem766.read(iprot)
++            self.success.append(_elem766)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -19680,8 -19694,8 +19926,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
--      for iter760 in self.success:
--        iter760.write(oprot)
++      for iter767 in self.success:
++        iter767.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -19775,10 -19789,10 +20021,10 @@@ class get_partition_with_auth_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
--          (_etype764, _size761) = iprot.readListBegin()
--          for _i765 in xrange(_size761):
--            _elem766 = iprot.readString()
--            self.part_vals.append(_elem766)
++          (_etype771, _size768) = iprot.readListBegin()
++          for _i772 in xrange(_size768):
++            _elem773 = iprot.readString()
++            self.part_vals.append(_elem773)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -19790,10 -19804,10 +20036,10 @@@
        elif fid == 5:
          if ftype == TType.LIST:
            self.group_names = []
--          (_etype770, _size767) = iprot.readListBegin()
--          for _i771 in xrange(_size767):
--            _elem772 = iprot.readString()
--            self.group_names.append(_elem772)
++          (_etype777, _size774) = iprot.readListBegin()
++          for _i778 in xrange(_size774):
++            _elem779 = iprot.readString()
++            self.group_names.append(_elem779)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -19818,8 -19832,8 +20064,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
--      for iter773 in self.part_vals:
--        oprot.writeString(iter773)
++      for iter780 in self.part_vals:
++        oprot.writeString(iter780)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.user_name is not None:
@@@ -19829,8 -19843,8 +20075,8 @@@
      if self.group_names is not None:
        oprot.writeFieldBegin('group_names', TType.LIST, 5)
        oprot.writeListBegin(TType.STRING, len(self.group_names))
--      for iter774 in self.group_names:
--        oprot.writeString(iter774)
++      for iter781 in self.group_names:
++        oprot.writeString(iter781)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -20259,11 -20273,11 +20505,11 @@@ class get_partitions_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype778, _size775) = iprot.readListBegin()
--          for _i779 in xrange(_size775):
--            _elem780 = Partition()
--            _elem780.read(iprot)
--            self.success.append(_elem780)
++          (_etype785, _size782) = iprot.readListBegin()
++          for _i786 in xrange(_size782):
++            _elem787 = Partition()
++            _elem787.read(iprot)
++            self.success.append(_elem787)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -20292,8 -20306,8 +20538,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
--      for iter781 in self.success:
--        iter781.write(oprot)
++      for iter788 in self.success:
++        iter788.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -20387,10 -20401,10 +20633,10 @@@ class get_partitions_with_auth_args
        elif fid == 5:
          if ftype == TType.LIST:
            self.group_names = []
--          (_etype785, _size782) = iprot.readListBegin()
--          for _i786 in xrange(_size782):
--            _elem787 = iprot.readString()
--            self.group_names.append(_elem787)
++          (_etype792, _size789) = iprot.readListBegin()
++          for _i793 in xrange(_size789):
++            _elem794 = iprot.readString()
++            self.group_names.append(_elem794)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -20423,8 -20437,8 +20669,8 @@@
      if self.group_names is not None:
        oprot.writeFieldBegin('group_names', TType.LIST, 5)
        oprot.writeListBegin(TType.STRING, len(self.group_names))
--      for iter788 in self.group_names:
--        oprot.writeString(iter788)
++      for iter795 in self.group_names:
++        oprot.writeString(iter795)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -20485,11 -20499,11 +20731,11 @@@ class get_partitions_with_auth_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype792, _size789) = iprot.readListBegin()
--          for _i793 in xrange(_size789):
--            _elem794 = Partition()
--            _elem794.read(iprot)
--            self.success.append(_elem794)
++          (_etype799, _size796) = iprot.readListBegin()
++          for _i800 in xrange(_size796):
++            _elem801 = Partition()
++            _elem801.read(iprot)
++            self.success.append(_elem801)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -20518,8 -20532,8 +20764,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
--      for iter795 in self.success:
--        iter795.write(oprot)
++      for iter802 in self.success:
++        iter802.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -20677,11 -20691,11 +20923,11 @@@ class get_partitions_pspec_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype799, _size796) = iprot.readListBegin()
--          for _i800 in xrange(_size796):
--            _elem801 = PartitionSpec()
--            _elem801.read(iprot)
--            self.success.append(_elem801)
++          (_etype806, _size803) = iprot.readListBegin()
++          for _i807 in xrange(_size803):
++            _elem808 = PartitionSpec()
++            _elem808.read(iprot)
++            self.success.append(_elem808)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -20710,8 -20724,8 +20956,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
--      for iter802 in self.success:
--        iter802.write(oprot)
++      for iter809 in self.success:
++        iter809.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -20866,10 -20880,10 +21112,10 @@@ class get_partition_names_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype806, _size803) = iprot.readListBegin()
--          for _i807 in xrange(_size803):
--            _elem808 = iprot.readString()
--            self.success.append(_elem808)
++          (_etype813, _size810) = iprot.readListBegin()
++          for _i814 in xrange(_size810):
++            _elem815 = iprot.readString()
++            self.success.append(_elem815)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -20892,8 -20906,8 +21138,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
--      for iter809 in self.success:
--        oprot.writeString(iter809)
++      for iter816 in self.success:
++        oprot.writeString(iter816)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o2 is not None:
@@@ -20969,10 -20983,10 +21215,10 @@@ class get_partitions_ps_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
--          (_etype813, _size810) = iprot.readListBegin()
--          for _i814 in xrange(_size810):
--            _elem815 = iprot.readString()
--            self.part_vals.append(_elem815)
++          (_etype820, _size817) = iprot.readListBegin()
++          for _i821 in xrange(_size817):
++            _elem822 = iprot.readString()
++            self.part_vals.append(_elem822)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -21002,8 -21016,8 +21248,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
--      for iter816 in self.part_vals:
--        oprot.writeString(iter816)
++      for iter823 in self.part_vals:
++        oprot.writeString(iter823)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.max_parts is not None:
@@@ -21067,11 -21081,11 +21313,11 @@@ class get_partitions_ps_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype820, _size817) = iprot.readListBegin()
--          for _i821 in xrange(_size817):
--            _elem822 = Partition()
--            _elem822.read(iprot)
--            self.success.append(_elem822)
++          (_etype827, _size824) = iprot.readListBegin()
++          for _i828 in xrange(_size824):
++            _elem829 = Partition()
++            _elem829.read(iprot)
++            self.success.append(_elem829)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -21100,8 -21114,8 +21346,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
--      for iter823 in self.success:
--        iter823.write(oprot)
++      for iter830 in self.success:
++        iter830.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -21188,10 -21202,10 +21434,10 @@@ class get_partitions_ps_with_auth_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
--          (_etype827, _size824) = iprot.readListBegin()
--          for _i828 in xrange(_size824):
--            _elem829 = iprot.readString()
--            self.part_vals.append(_elem829)
++          (_etype834, _size831) = iprot.readListBegin()
++          for _i835 in xrange(_size831):
++            _elem836 = iprot.readString()
++            self.part_vals.append(_elem836)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -21208,10 -21222,10 +21454,10 @@@
        elif fid == 6:
          if ftype == TType.LIST:
            self.group_names = []
--          (_etype833, _size830) = iprot.readListBegin()
--          for _i834 in xrange(_size830):
--            _elem835 = iprot.readString()
--            self.group_names.append(_elem835)
++          (_etype840, _size837) = iprot.readListBegin()
++          for _i841 in xrange(_size837):
++            _elem842 = iprot.readString()
++            self.group_names.append(_elem842)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -21236,8 -21250,8 +21482,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
--      for iter836 in self.part_vals:
--        oprot.writeString(iter836)
++      for iter843 in self.part_vals:
++        oprot.writeString(iter843)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.max_parts is not None:
@@@ -21251,8 -21265,8 +21497,8 @@@
      if self.group_names is not None:
        oprot.writeFieldBegin('group_names', TType.LIST, 6)
        oprot.writeListBegin(TType.STRING, len(self.group_names))
--      for iter837 in self.group_names:
--        oprot.writeString(iter837)
++      for iter844 in self.group_names:
++        oprot.writeString(iter844)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -21314,11 -21328,11 +21560,11 @@@ class get_partitions_ps_with_auth_resul
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype841, _size838) = iprot.readListBegin()
--          for _i842 in xrange(_size838):
--            _elem843 = Partition()
--            _elem843.read(iprot)
--            self.success.append(_elem843)
++          (_etype848, _size845) = iprot.readListBegin()
++          for _i849 in xrange(_size845):
++            _elem850 = Partition()
++            _elem850.read(iprot)
++            self.success.append(_elem850)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -21347,8 -21361,8 +21593,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
--      for iter844 in self.success:
--        iter844.write(oprot)
++      for iter851 in self.success:
++        iter851.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -21429,10 -21443,10 +21675,10 @@@ class get_partition_names_ps_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
--          (_etype848, _size845) = iprot.readListBegin()
--          for _i849 in xrange(_size845):
--            _elem850 = iprot.readString()
--            self.part_vals.append(_elem850)
++          (_etype855, _size852) = iprot.readListBegin()
++          for _i856 in xrange(_size852):
++            _elem857 = iprot.readString()
++            self.part_vals.append(_elem857)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -21462,8 -21476,8 +21708,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
--      for iter851 in self.part_vals:
--        oprot.writeString(iter851)
++      for iter858 in self.part_vals:
++        oprot.writeString(iter858)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.max_parts is not None:
@@@ -21527,10 -21541,10 +21773,10 @@@ class get_partition_names_ps_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype855, _size852) = iprot.readListBegin()
--          for _i856 in xrange(_size852):
--            _elem857 = iprot.readString()
--            self.success.append(_elem857)
++          (_etype862, _size859) = iprot.readListBegin()
++          for _i863 in xrange(_size859):
++            _elem864 = iprot.readString()
++            self.success.append(_elem864)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -21559,8 -21573,8 +21805,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
--      for iter858 in self.success:
--        oprot.writeString(iter858)
++      for iter865 in self.success:
++        oprot.writeString(iter865)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -21731,11 -21745,11 +21977,11 @@@ class get_partitions_by_filter_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype862, _size859) = iprot.readListBegin()
--          for _i863 in xrange(_size859):
--            _elem864 = Partition()
--            _elem864.read(iprot)
--            self.success.append(_elem864)
++          (_etype869, _size866) = iprot.readListBegin()
++          for _i870 in xrange(_size866):
++            _elem871 = Partition()
++            _elem871.read(iprot)
++            self.success.append(_elem871)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -21764,8 -21778,8 +22010,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
--      for iter865 in self.success:
--        iter865.write(oprot)
++      for iter872 in self.success:
++        iter872.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -21936,11 -21950,11 +22182,11 @@@ class get_part_specs_by_filter_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype869, _size866) = iprot.readListBegin()
--          for _i870 in xrange(_size866):
--            _elem871 = PartitionSpec()
--            _elem871.read(iprot)
--            self.success.append(_elem871)
++          (_etype876, _size873) = iprot.readListBegin()
++          for _i877 in xrange(_size873):
++            _elem878 = PartitionSpec()
++            _elem878.read(iprot)
++            self.success.append(_elem878)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -21969,8 -21983,8 +22215,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
--      for iter872 in self.success:
--        iter872.write(oprot)
++      for iter879 in self.success:
++        iter879.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -22390,10 -22404,10 +22636,10 @@@ class get_partitions_by_names_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.names = []
--          (_etype876, _size873) = iprot.readListBegin()
--          for _i877 in xrange(_size873):
--            _elem878 = iprot.readString()
--            self.names.append(_elem878)
++          (_etype883, _size880) = iprot.readListBegin()
++          for _i884 in xrange(_size880):
++            _elem885 = iprot.readString()
++            self.names.append(_elem885)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -22418,8 -22432,8 +22664,8 @@@
      if self.names is not None:
        oprot.writeFieldBegin('names', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.names))
--      for iter879 in self.names:
--        oprot.writeString(iter879)
++      for iter886 in self.names:
++        oprot.writeString(iter886)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -22478,11 -22492,11 +22724,11 @@@ class get_partitions_by_names_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype883, _size880) = iprot.readListBegin()
--          for _i884 in xrange(_size880):
--            _elem885 = Partition()
--            _elem885.read(iprot)
--            self.success.append(_elem885)
++          (_etype890, _size887) = iprot.readListBegin()
++          for _i891 in xrange(_size887):
++            _elem892 = Partition()
++            _elem892.read(iprot)
++            self.success.append(_elem892)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -22511,8 -22525,8 +22757,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
--      for iter886 in self.success:
--        iter886.write(oprot)
++      for iter893 in self.success:
++        iter893.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -22762,11 -22776,11 +23008,11 @@@ class alter_partitions_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.new_parts = []
--          (_etype890, _size887) = iprot.readListBegin()
--          for _i891 in xrange(_size887):
--            _elem892 = Partition()
--            _elem892.read(iprot)
--            self.new_parts.append(_elem892)
++          (_etype897, _size894) = iprot.readListBegin()
++          for _i898 in xrange(_size894):
++            _elem899 = Partition()
++            _elem899.read(iprot)
++            self.new_parts.append(_elem899)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -22791,8 -22805,8 +23037,8 @@@
      if self.new_parts is not None:
        oprot.writeFieldBegin('new_parts', TType.LIST, 3)
        oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
--      for iter893 in self.new_parts:
--        iter893.write(oprot)
++      for iter900 in self.new_parts:
++        iter900.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -22945,11 -22959,11 +23191,11 @@@ class alter_partitions_with_environment
        elif fid == 3:
          if ftype == TType.LIST:
            self.new_parts = []
--          (_etype897, _size894) = iprot.readListBegin()
--          for _i898 in xrange(_size894):
--            _elem899 = Partition()
--            _elem899.read(iprot)
--            self.new_parts.append(_elem899)
++          (_etype904, _size901) = iprot.readListBegin()
++          for _i905 in xrange(_size901):
++            _elem906 = Partition()
++            _elem906.read(iprot)
++            self.new_parts.append(_elem906)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -22980,8 -22994,8 +23226,8 @@@
      if self.new_parts is not None:
        oprot.writeFieldBegin('new_parts', TType.LIST, 3)
        oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
--      for iter900 in self.new_parts:
--        iter900.write(oprot)
++      for iter907 in self.new_parts:
++        iter907.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.environment_context is not None:
@@@ -23325,10 -23339,10 +23571,10 @@@ class rename_partition_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
--          (_etype904, _size901) = iprot.readListBegin()
--          for _i905 in xrange(_size901):
--            _elem906 = iprot.readString()
--            self.part_vals.append(_elem906)
++          (_etype911, _size908) = iprot.readListBegin()
++          for _i912 in xrange(_size908):
++            _elem913 = iprot.readString()
++            self.part_vals.append(_elem913)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -23359,8 -23373,8 +23605,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
--      for iter907 in self.part_vals:
--        oprot.writeString(iter907)
++      for iter914 in self.part_vals:
++        oprot.writeString(iter914)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.new_part is not None:
@@@ -23502,10 -23516,10 +23748,10 @@@ class partition_name_has_valid_characte
        if fid == 1:
          if ftype == TType.LIST:
            self.part_vals = []
--          (_etype911, _size908) = iprot.readListBegin()
--          for _i912 in xrange(_size908):
--            _elem913 = iprot.readString()
--            self.part_vals.append(_elem913)
++          (_etype918, _size915) = iprot.readListBegin()
++          for _i919 in xrange(_size915):
++            _elem920 = iprot.readString()
++            self.part_vals.append(_elem920)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -23527,8 -23541,8 +23773,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 1)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
--      for iter914 in self.part_vals:
--        oprot.writeString(iter914)
++      for iter921 in self.part_vals:
++        oprot.writeString(iter921)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.throw_exception is not None:
@@@ -23886,10 -23900,10 +24132,10 @@@ class partition_name_to_vals_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype918, _size915) = iprot.readListBegin()
--          for _i919 in xrange(_size915):
--            _elem920 = iprot.readString()
--            self.success.append(_elem920)
++          (_etype925, _size922) = iprot.readListBegin()
++          for _i926 in xrange(_size922):
++            _elem927 = iprot.readString()
++            self.success.append(_elem927)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -23912,8 -23926,8 +24158,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
--      for iter921 in self.success:
--        oprot.writeString(iter921)
++      for iter928 in self.success:
++        oprot.writeString(iter928)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -24037,11 -24051,11 +24283,11 @@@ class partition_name_to_spec_result
        if fid == 0:
          if ftype == TType.MAP:
            self.success = {}
--          (_ktype923, _vtype924, _size922 ) = iprot.readMapBegin()
--          for _i926 in xrange(_size922):
--            _key927 = iprot.readString()
--            _val928 = iprot.readString()
--            self.success[_key927] = _val928
++          (_ktype930, _vtype931, _size929 ) = iprot.readMapBegin()
++          for _i933 in xrange(_size929):
++            _key934 = iprot.readString()
++            _val935 = iprot.readString()
++            self.success[_key934] = _val935
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -24064,9 -24078,9 +24310,9 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.MAP, 0)
        oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success))
--      for kiter929,viter930 in self.success.items():
--        oprot.writeString(kiter929)
--        oprot.writeString(viter930)
++      for kiter936,viter937 in self.success.items():
++        oprot.writeString(kiter936)
++        oprot.writeString(viter937)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -24142,11 -24156,11 +24388,11 @@@ class markPartitionForEvent_args
        elif fid == 3:
          if ftype == TType.MAP:
            self.part_vals = {}
--          (_ktype932, _vtype933, _size931 ) = iprot.readMapBegin()
--          for _i935 in xrange(_size931):
--            _key936 = iprot.readString()
--            _val937 = iprot.readString()
--            self.part_vals[_key936] = _val937
++          (_ktype939, _vtype940, _size938 ) = iprot.readMapBegin()
++          for _i942 in xrange(_size938):
++            _key943 = iprot.readString()
++            _val944 = iprot.readString()
++            self.part_vals[_key943] = _val944
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -24176,9 -24190,9 +24422,9 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.MAP, 3)
        oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
--      for kiter938,viter939 in self.part_vals.items():
--        oprot.writeString(kiter938)
--        oprot.writeString(viter939)
++      for kiter945,viter946 in self.part_vals.items():
++        oprot.writeString(kiter945)
++        oprot.writeString(viter946)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      if self.eventType is not None:
@@@ -24392,11 -24406,11 +24638,11 @@@ class isPartitionMarkedForEvent_args
        elif fid == 3:
          if ftype == TType.MAP:
            self.part_vals = {}
--          (_ktype941, _vtype942, _size940 ) = iprot.readMapBegin()
--          for _i944 in xrange(_size940):
--            _key945 = iprot.readString()
--            _val946 = iprot.readString()
--            self.part_vals[_key945] = _val946
++          (_ktype948, _vtype949, _size947 ) = iprot.readMapBegin()
++          for _i951 in xrange(_size947):
++            _key952 = iprot.readString()
++            _val953 = iprot.readString()
++            self.part_vals[_key952] = _val953
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -24426,9 -24440,9 +24672,9 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.MAP, 3)
        oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
--      for kiter947,viter948 in self.part_vals.items():
--        oprot.writeString(kiter947)
--        oprot.writeString(viter948)
++      for kiter954,viter955 in self.part_vals.items():
++        oprot.writeString(kiter954)
++        oprot.writeString(viter955)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      if self.eventType is not None:
@@@ -25483,11 -25497,11 +25729,11 @@@ class get_indexes_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype952, _size949) = iprot.readListBegin()
--          for _i953 in xrange(_size949):
--            _elem954 = Index()
--            _elem954.read(iprot)
--            self.success.append(_elem954)
++          (_etype959, _size956) = iprot.readListBegin()
++          for _i960 in xrange(_size956):
++            _elem961 = Index()
++            _elem961.read(iprot)
++            self.success.append(_elem961)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -25516,8 -25530,8 +25762,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
--      for iter955 in self.success:
--        iter955.write(oprot)
++      for iter962 in self.success:
++        iter962.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -25672,10 -25686,10 +25918,10 @@@ class get_index_names_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype959, _size956) = iprot.readListBegin()
--          for _i960 in xrange(_size956):
--            _elem961 = iprot.readString()
--            self.success.append(_elem961)
++          (_etype966, _size963) = iprot.readListBegin()
++          for _i967 in xrange(_size963):
++            _elem968 = iprot.readString()
++            self.success.append(_elem968)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -25698,8 -25712,8 +25944,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
--      for iter962 in self.success:
--        oprot.writeString(iter962)
++      for iter969 in self.success:
++        oprot.writeString(iter969)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o2 is not None:
@@@ -28565,10 -28579,10 +28811,10 @@@ class get_functions_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype966, _size963) = iprot.readListBegin()
--          for _i967 in xrange(_size963):
--            _elem968 = iprot.readString()
--            self.success.append(_elem968)
++          (_etype973, _size970) = iprot.readListBegin()
++          for _i974 in xrange(_size970):
++            _elem975 = iprot.readString()
++            self.success.append(_elem975)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -28591,8 -28605,8 +28837,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
--      for iter969 in self.success:
--        oprot.writeString(iter969)
++      for iter976 in self.success:
++        oprot.writeString(iter976)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -29280,10 -29294,10 +29526,10 @@@ class get_role_names_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype973, _size970) = iprot.readListBegin()
--          for _i974 in xrange(_size970):
--            _elem975 = iprot.readString()
--            self.success.append(_elem975)
++          (_etype980, _size977) = iprot.readListBegin()
++          for _i981 in xrange(_size977):
++            _elem982 = iprot.readString()
++            self.success.append(_elem982)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -29306,8 -29320,8 +29552,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
--      for iter976 in self.success:
--        oprot.writeString(iter976)
++      for iter983 in self.success:
++        oprot.writeString(iter983)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -29821,11 -29835,11 +30067,11 @@@ class list_roles_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype980, _size977) = iprot.readListBegin()
--          for _i981 in xrange(_size977):
--            _elem982 = Role()
--            _elem982.read(iprot)
--            self.success.append(_elem982)
++          (_etype987, _size984) = iprot.readListBegin()
++          for _i988 in xrange(_size984):
++            _elem989 = Role()
++            _elem989.read(iprot)
++            self.success.append(_elem989)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -29848,8 -29862,8 +30094,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
--      for iter983 in self.success:
--        iter983.write(oprot)
++      for iter990 in self.success:
++        iter990.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -30358,10 -30372,10 +30604,10 @@@ class get_privilege_set_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.group_names = []
--          (_etype987, _size984) = iprot.readListBegin()
--          for _i988 in xrange(_size984):
--            _elem989 = iprot.readString()
--            self.group_names.append(_elem989)
++          (_etype994, _size991) = iprot.readListBegin()
++          for _i995 in xrange(_size991):
++            _elem996 = iprot.readString()
++            self.group_names.append(_elem996)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -30386,8 -30400,8 +30632,8 @@@
      if self.group_names is not None:
        oprot.writeFieldBegin('group_names', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.group_names))
--      for iter990 in self.group_names:
--        oprot.writeString(iter990)
++      for iter997 in self.group_names:
++        oprot.writeString(iter997)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -30614,11 -30628,11 +30860,11 @@@ class list_privileges_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype994, _size991) = iprot.readListBegin()
--          for _i995 in xrange(_size991):
--            _elem996 = HiveObjectPrivilege()
--            _elem996.read(iprot)
--            self.success.append(_elem996)
++          (_etype1001, _size998) = iprot.readListBegin()
++          for _i1002 in xrange(_size998):
++            _elem1003 = HiveObjectPrivilege()
++            _elem1003.read(iprot)
++            self.success.append(_elem1003)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -30641,8 -30655,8 +30887,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
--      for iter997 in self.success:
--        iter997.write(oprot)
++      for iter1004 in self.success:
++        iter1004.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -31140,10 -31154,10 +31386,10 @@@ class set_ugi_args
        elif fid == 2:
          if ftype == TType.LIST:
            self.group_names = []
--          (_etype1001, _size998) = iprot.readListBegin()
--          for _i1002 in xrange(_size998):
--            _elem1003 = iprot.readString()
--            self.group_names.append(_elem1003)
++          (_etype1008, _size1005) = iprot.readListBegin()
++          for _i1009 in xrange(_size1005):
++            _elem1010 = iprot.readString()
++            self.group_names.append(_elem1010)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -31164,8 -31178,8 +31410,8 @@@
      if self.group_names is not None:
        oprot.writeFieldBegin('group_names', TType.LIST, 2)
        oprot.writeListBegin(TType.STRING, len(self.group_names))
--      for iter1004 in self.group_names:
--        oprot.writeString(iter1004)
++      for iter1011 in self.group_names:
++        oprot.writeString(iter1011)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -31220,10 -31234,10 +31466,10 @@@ class set_ugi_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype1008, _size1005) = iprot.readListBegin()
--          for _i1009 in xrange(_size1005):
--            _elem1010 = iprot.readString()
--            self.success.append(_elem1010)
++          (_etype1015, _size1012) = iprot.readListBegin()
++          for _i1016 in xrange(_size1012):
++            _elem1017 = iprot.readString()
++            self.success.append(_elem1017)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -31246,8 -31260,8 +31492,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
--      for iter1011 in self.success:
--        oprot.writeString(iter1011)
++      for iter1018 in self.success:
++        oprot.writeString(iter1018)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -32179,10 -32193,10 +32425,10 @@@ class get_all_token_identifiers_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype1015, _size1012) = iprot.readListBegin()
--          for _i1016 in xrange(_size1012):
--            _elem1017 = iprot.readString()
--            self.success.append(_elem1017)
++          (_etype1022, _size1019) = iprot.readListBegin()
++          for _i1023 in xrange(_size1019):
++            _elem1024 = iprot.readString()
++            self.success.append(_elem1024)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -32199,8 -32213,8 +32445,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
--      for iter1018 in self.success:
--        oprot.writeString(iter1018)
++      for iter1025 in self.success:
++        oprot.writeString(iter1025)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -32727,10 -32741,10 +32973,10 @@@ class get_master_keys_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
--          (_etype1022, _size1019) = iprot.readListBegin()
--          for _i1023 in xrange(_size1019):
--            _elem1024 = iprot.readString()
--            self.success.append(_elem1024)
++          (_etype1029, _size1026) = iprot.readListBegin()
++          for _i1030 in xrange(_size1026):
++            _elem1031 = iprot.readString()
++            self.success.append(_elem1031)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -32747,8 -32761,8 +32993,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
--      for iter1025 in self.success:
--        oprot.writeString(iter1025)
++      for iter1032 in self.success:
++        oprot.writeString(iter1032)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --cc metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 0e7bdf2,530d2f4..9f16eb2
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@@ -47,7 -47,9 +47,8 @@@ import org.apache.hadoop.hive.common.me
  import org.apache.hadoop.hive.common.metrics.common.MetricsVariable;
  import org.apache.hadoop.hive.conf.HiveConf;
  import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+ import org.apache.hadoop.hive.metastore.TableType;
  import org.apache.hadoop.hive.metastore.api.*;
 -import org.apache.hadoop.hive.metastore.api.Table;
  import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
  import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
  import org.apache.hadoop.hive.metastore.events.AlterIndexEvent;

http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --cc metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 0325854,c32486f..73e14c4
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@@ -27,7 -27,105 +27,8 @@@ import org.apache.hadoop.hive.common.cl
  import org.apache.hadoop.hive.conf.HiveConf;
  import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
  import org.apache.hadoop.hive.conf.HiveConfUtil;
 +import org.apache.hadoop.hive.metastore.api.*;
+ import org.apache.hadoop.hive.metastore.TableType;
 -import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
 -import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest;
 -import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
 -import org.apache.hadoop.hive.metastore.api.AddForeignKeyRequest;
 -import org.apache.hadoop.hive.metastore.api.AddPartitionsRequest;
 -import org.apache.hadoop.hive.metastore.api.AddPartitionsResult;
 -import org.apache.hadoop.hive.metastore.api.AddPrimaryKeyRequest;
 -import org.apache.hadoop.hive.metastore.api.AggrStats;
 -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 -import org.apache.hadoop.hive.metastore.api.CacheFileMetadataRequest;
 -import org.apache.hadoop.hive.metastore.api.CacheFileMetadataResult;
 -import org.apache.hadoop.hive.metastore.api.CheckLockRequest;
 -import org.apache.hadoop.hive.metastore.api.ClearFileMetadataRequest;
 -import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 -import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 -import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
 -import org.apache.hadoop.hive.metastore.api.CompactionRequest;
 -import org.apache.hadoop.hive.metastore.api.CompactionType;
 -import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
 -import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 -import org.apache.hadoop.hive.metastore.api.DataOperationType;
 -import org.apache.hadoop.hive.metastore.api.Database;
 -import org.apache.hadoop.hive.metastore.api.DropConstraintRequest;
 -import org.apache.hadoop.hive.metastore.api.DropPartitionsExpr;
 -import org.apache.hadoop.hive.metastore.api.DropPartitionsRequest;
 -import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 -import org.apache.hadoop.hive.metastore.api.FieldSchema;
 -import org.apache.hadoop.hive.metastore.api.FireEventRequest;
 -import org.apache.hadoop.hive.metastore.api.FireEventResponse;
 -import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
 -import org.apache.hadoop.hive.metastore.api.Function;
 -import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
 -import org.apache.hadoop.hive.metastore.api.GetFileMetadataByExprRequest;
 -import org.apache.hadoop.hive.metastore.api.GetFileMetadataByExprResult;
 -import org.apache.hadoop.hive.metastore.api.GetFileMetadataRequest;
 -import org.apache.hadoop.hive.metastore.api.GetFileMetadataResult;
 -import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
 -import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest;
 -import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleResponse;
 -import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest;
 -import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalResponse;
 -import org.apache.hadoop.hive.metastore.api.GrantRevokePrivilegeRequest;
 -import org.apache.hadoop.hive.metastore.api.GrantRevokePrivilegeResponse;
 -import org.apache.hadoop.hive.metastore.api.GrantRevokeRoleRequest;
 -import org.apache.hadoop.hive.metastore.api.GrantRevokeRoleResponse;
 -import org.apache.hadoop.hive.metastore.api.GrantRevokeType;
 -import org.apache.hadoop.hive.metastore.api.HeartbeatRequest;
 -import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeRequest;
 -import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
 -import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 -import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
 -import org.apache.hadoop.hive.metastore.api.Index;
 -import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 -import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 -import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 -import org.apache.hadoop.hive.metastore.api.InvalidPartitionException;
 -import org.apache.hadoop.hive.metastore.api.LockRequest;
 -import org.apache.hadoop.hive.metastore.api.LockResponse;
 -import org.apache.hadoop.hive.metastore.api.MetaException;
 -import org.apache.hadoop.hive.metastore.api.MetadataPpdResult;
 -import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
 -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 -import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
 -import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 -import org.apache.hadoop.hive.metastore.api.NotificationEventRequest;
 -import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
 -import org.apache.hadoop.hive.metastore.api.OpenTxnRequest;
 -import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
 -import org.apache.hadoop.hive.metastore.api.Partition;
 -import org.apache.hadoop.hive.metastore.api.PartitionEventType;
 -import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
 -import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
 -import org.apache.hadoop.hive.metastore.api.PartitionsStatsRequest;
 -import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
 -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.PutFileMetadataRequest;
 -import org.apache.hadoop.hive.metastore.api.RequestPartsSpec;
 -import org.apache.hadoop.hive.metastore.api.Role;
 -import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 -import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 -import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
 -import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
 -import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 -import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
 -import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
 -import org.apache.hadoop.hive.metastore.api.Table;
 -import org.apache.hadoop.hive.metastore.api.TableMeta;
 -import org.apache.hadoop.hive.metastore.api.TableStatsRequest;
 -import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore;
 -import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 -import org.apache.hadoop.hive.metastore.api.TxnOpenException;
 -import org.apache.hadoop.hive.metastore.api.Type;
 -import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 -import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 -import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 -import org.apache.hadoop.hive.metastore.api.UnlockRequest;
  import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
  import org.apache.hadoop.hive.metastore.txn.TxnUtils;
  import org.apache.hadoop.hive.shims.ShimLoader;

http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --cc metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index ddc5a62,f9fad4c..e5b469b
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@@ -33,8 -33,8 +33,9 @@@ import org.apache.hadoop.hive.metastore
  import org.apache.hadoop.hive.metastore.PartFilterExprUtil;
  import org.apache.hadoop.hive.metastore.PartitionExpressionProxy;
  import org.apache.hadoop.hive.metastore.RawStore;
+ import org.apache.hadoop.hive.metastore.TableType;
  import org.apache.hadoop.hive.metastore.Warehouse;
 +import org.apache.hadoop.hive.metastore.RawStore.CanNotRetry;
  import org.apache.hadoop.hive.metastore.api.AggrStats;
  import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
  import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;

http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------


[13/15] hive git commit: HIVE-14671 : merge master into hive-14535 (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 0b6fe91,b4a05b2..27985f6
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@@ -6186,6 -6186,277 +6186,277 @@@ uint32_t ThriftHiveMetastore_get_tables
  }
  
  
+ ThriftHiveMetastore_get_tables_by_type_args::~ThriftHiveMetastore_get_tables_by_type_args() throw() {
+ }
+ 
+ 
+ uint32_t ThriftHiveMetastore_get_tables_by_type_args::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;
+ 
+ 
+   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->db_name);
+           this->__isset.db_name = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
+       case 2:
+         if (ftype == ::apache::thrift::protocol::T_STRING) {
+           xfer += iprot->readString(this->pattern);
+           this->__isset.pattern = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
+       case 3:
+         if (ftype == ::apache::thrift::protocol::T_STRING) {
+           xfer += iprot->readString(this->tableType);
+           this->__isset.tableType = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
+       default:
+         xfer += iprot->skip(ftype);
+         break;
+     }
+     xfer += iprot->readFieldEnd();
+   }
+ 
+   xfer += iprot->readStructEnd();
+ 
+   return xfer;
+ }
+ 
+ uint32_t ThriftHiveMetastore_get_tables_by_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+   uint32_t xfer = 0;
+   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+   xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_args");
+ 
+   xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
+   xfer += oprot->writeString(this->db_name);
+   xfer += oprot->writeFieldEnd();
+ 
+   xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2);
+   xfer += oprot->writeString(this->pattern);
+   xfer += oprot->writeFieldEnd();
+ 
+   xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3);
+   xfer += oprot->writeString(this->tableType);
+   xfer += oprot->writeFieldEnd();
+ 
+   xfer += oprot->writeFieldStop();
+   xfer += oprot->writeStructEnd();
+   return xfer;
+ }
+ 
+ 
+ ThriftHiveMetastore_get_tables_by_type_pargs::~ThriftHiveMetastore_get_tables_by_type_pargs() throw() {
+ }
+ 
+ 
+ uint32_t ThriftHiveMetastore_get_tables_by_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+   uint32_t xfer = 0;
+   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+   xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_pargs");
+ 
+   xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
+   xfer += oprot->writeString((*(this->db_name)));
+   xfer += oprot->writeFieldEnd();
+ 
+   xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2);
+   xfer += oprot->writeString((*(this->pattern)));
+   xfer += oprot->writeFieldEnd();
+ 
+   xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3);
+   xfer += oprot->writeString((*(this->tableType)));
+   xfer += oprot->writeFieldEnd();
+ 
+   xfer += oprot->writeFieldStop();
+   xfer += oprot->writeStructEnd();
+   return xfer;
+ }
+ 
+ 
+ ThriftHiveMetastore_get_tables_by_type_result::~ThriftHiveMetastore_get_tables_by_type_result() throw() {
+ }
+ 
+ 
+ uint32_t ThriftHiveMetastore_get_tables_by_type_result::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;
+ 
+ 
+   while (true)
+   {
+     xfer += iprot->readFieldBegin(fname, ftype, fid);
+     if (ftype == ::apache::thrift::protocol::T_STOP) {
+       break;
+     }
+     switch (fid)
+     {
+       case 0:
+         if (ftype == ::apache::thrift::protocol::T_LIST) {
+           {
+             this->success.clear();
 -            uint32_t _size887;
 -            ::apache::thrift::protocol::TType _etype890;
 -            xfer += iprot->readListBegin(_etype890, _size887);
 -            this->success.resize(_size887);
 -            uint32_t _i891;
 -            for (_i891 = 0; _i891 < _size887; ++_i891)
++            uint32_t _size909;
++            ::apache::thrift::protocol::TType _etype912;
++            xfer += iprot->readListBegin(_etype912, _size909);
++            this->success.resize(_size909);
++            uint32_t _i913;
++            for (_i913 = 0; _i913 < _size909; ++_i913)
+             {
 -              xfer += iprot->readString(this->success[_i891]);
++              xfer += iprot->readString(this->success[_i913]);
+             }
+             xfer += iprot->readListEnd();
+           }
+           this->__isset.success = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
+       case 1:
+         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+           xfer += this->o1.read(iprot);
+           this->__isset.o1 = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
+       default:
+         xfer += iprot->skip(ftype);
+         break;
+     }
+     xfer += iprot->readFieldEnd();
+   }
+ 
+   xfer += iprot->readStructEnd();
+ 
+   return xfer;
+ }
+ 
+ uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+ 
+   uint32_t xfer = 0;
+ 
+   xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_result");
+ 
+   if (this->__isset.success) {
+     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 _iter892;
 -      for (_iter892 = this->success.begin(); _iter892 != this->success.end(); ++_iter892)
++      std::vector<std::string> ::const_iterator _iter914;
++      for (_iter914 = this->success.begin(); _iter914 != this->success.end(); ++_iter914)
+       {
 -        xfer += oprot->writeString((*_iter892));
++        xfer += oprot->writeString((*_iter914));
+       }
+       xfer += oprot->writeListEnd();
+     }
+     xfer += oprot->writeFieldEnd();
+   } else if (this->__isset.o1) {
+     xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
+     xfer += this->o1.write(oprot);
+     xfer += oprot->writeFieldEnd();
+   }
+   xfer += oprot->writeFieldStop();
+   xfer += oprot->writeStructEnd();
+   return xfer;
+ }
+ 
+ 
+ ThriftHiveMetastore_get_tables_by_type_presult::~ThriftHiveMetastore_get_tables_by_type_presult() throw() {
+ }
+ 
+ 
+ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::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;
+ 
+ 
+   while (true)
+   {
+     xfer += iprot->readFieldBegin(fname, ftype, fid);
+     if (ftype == ::apache::thrift::protocol::T_STOP) {
+       break;
+     }
+     switch (fid)
+     {
+       case 0:
+         if (ftype == ::apache::thrift::protocol::T_LIST) {
+           {
+             (*(this->success)).clear();
 -            uint32_t _size893;
 -            ::apache::thrift::protocol::TType _etype896;
 -            xfer += iprot->readListBegin(_etype896, _size893);
 -            (*(this->success)).resize(_size893);
 -            uint32_t _i897;
 -            for (_i897 = 0; _i897 < _size893; ++_i897)
++            uint32_t _size915;
++            ::apache::thrift::protocol::TType _etype918;
++            xfer += iprot->readListBegin(_etype918, _size915);
++            (*(this->success)).resize(_size915);
++            uint32_t _i919;
++            for (_i919 = 0; _i919 < _size915; ++_i919)
+             {
 -              xfer += iprot->readString((*(this->success))[_i897]);
++              xfer += iprot->readString((*(this->success))[_i919]);
+             }
+             xfer += iprot->readListEnd();
+           }
+           this->__isset.success = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
+       case 1:
+         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+           xfer += this->o1.read(iprot);
+           this->__isset.o1 = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
+       default:
+         xfer += iprot->skip(ftype);
+         break;
+     }
+     xfer += iprot->readFieldEnd();
+   }
+ 
+   xfer += iprot->readStructEnd();
+ 
+   return xfer;
+ }
+ 
+ 
  ThriftHiveMetastore_get_table_meta_args::~ThriftHiveMetastore_get_table_meta_args() throw() {
  }
  
@@@ -6231,14 -6502,14 +6502,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->tbl_types.clear();
-             uint32_t _size909;
-             ::apache::thrift::protocol::TType _etype912;
-             xfer += iprot->readListBegin(_etype912, _size909);
-             this->tbl_types.resize(_size909);
-             uint32_t _i913;
-             for (_i913 = 0; _i913 < _size909; ++_i913)
 -            uint32_t _size898;
 -            ::apache::thrift::protocol::TType _etype901;
 -            xfer += iprot->readListBegin(_etype901, _size898);
 -            this->tbl_types.resize(_size898);
 -            uint32_t _i902;
 -            for (_i902 = 0; _i902 < _size898; ++_i902)
++            uint32_t _size920;
++            ::apache::thrift::protocol::TType _etype923;
++            xfer += iprot->readListBegin(_etype923, _size920);
++            this->tbl_types.resize(_size920);
++            uint32_t _i924;
++            for (_i924 = 0; _i924 < _size920; ++_i924)
              {
-               xfer += iprot->readString(this->tbl_types[_i913]);
 -              xfer += iprot->readString(this->tbl_types[_i902]);
++              xfer += iprot->readString(this->tbl_types[_i924]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -6275,10 -6546,10 +6546,10 @@@ uint32_t ThriftHiveMetastore_get_table_
    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 _iter914;
-     for (_iter914 = this->tbl_types.begin(); _iter914 != this->tbl_types.end(); ++_iter914)
 -    std::vector<std::string> ::const_iterator _iter903;
 -    for (_iter903 = this->tbl_types.begin(); _iter903 != this->tbl_types.end(); ++_iter903)
++    std::vector<std::string> ::const_iterator _iter925;
++    for (_iter925 = this->tbl_types.begin(); _iter925 != this->tbl_types.end(); ++_iter925)
      {
-       xfer += oprot->writeString((*_iter914));
 -      xfer += oprot->writeString((*_iter903));
++      xfer += oprot->writeString((*_iter925));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -6310,10 -6581,10 +6581,10 @@@ uint32_t ThriftHiveMetastore_get_table_
    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 _iter915;
-     for (_iter915 = (*(this->tbl_types)).begin(); _iter915 != (*(this->tbl_types)).end(); ++_iter915)
 -    std::vector<std::string> ::const_iterator _iter904;
 -    for (_iter904 = (*(this->tbl_types)).begin(); _iter904 != (*(this->tbl_types)).end(); ++_iter904)
++    std::vector<std::string> ::const_iterator _iter926;
++    for (_iter926 = (*(this->tbl_types)).begin(); _iter926 != (*(this->tbl_types)).end(); ++_iter926)
      {
-       xfer += oprot->writeString((*_iter915));
 -      xfer += oprot->writeString((*_iter904));
++      xfer += oprot->writeString((*_iter926));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -6354,14 -6625,14 +6625,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size916;
-             ::apache::thrift::protocol::TType _etype919;
-             xfer += iprot->readListBegin(_etype919, _size916);
-             this->success.resize(_size916);
-             uint32_t _i920;
-             for (_i920 = 0; _i920 < _size916; ++_i920)
 -            uint32_t _size905;
 -            ::apache::thrift::protocol::TType _etype908;
 -            xfer += iprot->readListBegin(_etype908, _size905);
 -            this->success.resize(_size905);
 -            uint32_t _i909;
 -            for (_i909 = 0; _i909 < _size905; ++_i909)
++            uint32_t _size927;
++            ::apache::thrift::protocol::TType _etype930;
++            xfer += iprot->readListBegin(_etype930, _size927);
++            this->success.resize(_size927);
++            uint32_t _i931;
++            for (_i931 = 0; _i931 < _size927; ++_i931)
              {
-               xfer += this->success[_i920].read(iprot);
 -              xfer += this->success[_i909].read(iprot);
++              xfer += this->success[_i931].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -6400,10 -6671,10 +6671,10 @@@ uint32_t ThriftHiveMetastore_get_table_
      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 _iter921;
-       for (_iter921 = this->success.begin(); _iter921 != this->success.end(); ++_iter921)
 -      std::vector<TableMeta> ::const_iterator _iter910;
 -      for (_iter910 = this->success.begin(); _iter910 != this->success.end(); ++_iter910)
++      std::vector<TableMeta> ::const_iterator _iter932;
++      for (_iter932 = this->success.begin(); _iter932 != this->success.end(); ++_iter932)
        {
-         xfer += (*_iter921).write(oprot);
 -        xfer += (*_iter910).write(oprot);
++        xfer += (*_iter932).write(oprot);
        }
        xfer += oprot->writeListEnd();
      }
@@@ -6448,14 -6719,14 +6719,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size922;
-             ::apache::thrift::protocol::TType _etype925;
-             xfer += iprot->readListBegin(_etype925, _size922);
-             (*(this->success)).resize(_size922);
-             uint32_t _i926;
-             for (_i926 = 0; _i926 < _size922; ++_i926)
 -            uint32_t _size911;
 -            ::apache::thrift::protocol::TType _etype914;
 -            xfer += iprot->readListBegin(_etype914, _size911);
 -            (*(this->success)).resize(_size911);
 -            uint32_t _i915;
 -            for (_i915 = 0; _i915 < _size911; ++_i915)
++            uint32_t _size933;
++            ::apache::thrift::protocol::TType _etype936;
++            xfer += iprot->readListBegin(_etype936, _size933);
++            (*(this->success)).resize(_size933);
++            uint32_t _i937;
++            for (_i937 = 0; _i937 < _size933; ++_i937)
              {
-               xfer += (*(this->success))[_i926].read(iprot);
 -              xfer += (*(this->success))[_i915].read(iprot);
++              xfer += (*(this->success))[_i937].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -6593,14 -6864,14 +6864,14 @@@ uint32_t ThriftHiveMetastore_get_all_ta
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size927;
-             ::apache::thrift::protocol::TType _etype930;
-             xfer += iprot->readListBegin(_etype930, _size927);
-             this->success.resize(_size927);
-             uint32_t _i931;
-             for (_i931 = 0; _i931 < _size927; ++_i931)
 -            uint32_t _size916;
 -            ::apache::thrift::protocol::TType _etype919;
 -            xfer += iprot->readListBegin(_etype919, _size916);
 -            this->success.resize(_size916);
 -            uint32_t _i920;
 -            for (_i920 = 0; _i920 < _size916; ++_i920)
++            uint32_t _size938;
++            ::apache::thrift::protocol::TType _etype941;
++            xfer += iprot->readListBegin(_etype941, _size938);
++            this->success.resize(_size938);
++            uint32_t _i942;
++            for (_i942 = 0; _i942 < _size938; ++_i942)
              {
-               xfer += iprot->readString(this->success[_i931]);
 -              xfer += iprot->readString(this->success[_i920]);
++              xfer += iprot->readString(this->success[_i942]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -6639,10 -6910,10 +6910,10 @@@ uint32_t ThriftHiveMetastore_get_all_ta
      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 _iter932;
-       for (_iter932 = this->success.begin(); _iter932 != this->success.end(); ++_iter932)
 -      std::vector<std::string> ::const_iterator _iter921;
 -      for (_iter921 = this->success.begin(); _iter921 != this->success.end(); ++_iter921)
++      std::vector<std::string> ::const_iterator _iter943;
++      for (_iter943 = this->success.begin(); _iter943 != this->success.end(); ++_iter943)
        {
-         xfer += oprot->writeString((*_iter932));
 -        xfer += oprot->writeString((*_iter921));
++        xfer += oprot->writeString((*_iter943));
        }
        xfer += oprot->writeListEnd();
      }
@@@ -6687,14 -6958,14 +6958,14 @@@ uint32_t ThriftHiveMetastore_get_all_ta
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size933;
-             ::apache::thrift::protocol::TType _etype936;
-             xfer += iprot->readListBegin(_etype936, _size933);
-             (*(this->success)).resize(_size933);
-             uint32_t _i937;
-             for (_i937 = 0; _i937 < _size933; ++_i937)
 -            uint32_t _size922;
 -            ::apache::thrift::protocol::TType _etype925;
 -            xfer += iprot->readListBegin(_etype925, _size922);
 -            (*(this->success)).resize(_size922);
 -            uint32_t _i926;
 -            for (_i926 = 0; _i926 < _size922; ++_i926)
++            uint32_t _size944;
++            ::apache::thrift::protocol::TType _etype947;
++            xfer += iprot->readListBegin(_etype947, _size944);
++            (*(this->success)).resize(_size944);
++            uint32_t _i948;
++            for (_i948 = 0; _i948 < _size944; ++_i948)
              {
-               xfer += iprot->readString((*(this->success))[_i937]);
 -              xfer += iprot->readString((*(this->success))[_i926]);
++              xfer += iprot->readString((*(this->success))[_i948]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -7004,14 -7275,14 +7275,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->tbl_names.clear();
-             uint32_t _size938;
-             ::apache::thrift::protocol::TType _etype941;
-             xfer += iprot->readListBegin(_etype941, _size938);
-             this->tbl_names.resize(_size938);
-             uint32_t _i942;
-             for (_i942 = 0; _i942 < _size938; ++_i942)
 -            uint32_t _size927;
 -            ::apache::thrift::protocol::TType _etype930;
 -            xfer += iprot->readListBegin(_etype930, _size927);
 -            this->tbl_names.resize(_size927);
 -            uint32_t _i931;
 -            for (_i931 = 0; _i931 < _size927; ++_i931)
++            uint32_t _size949;
++            ::apache::thrift::protocol::TType _etype952;
++            xfer += iprot->readListBegin(_etype952, _size949);
++            this->tbl_names.resize(_size949);
++            uint32_t _i953;
++            for (_i953 = 0; _i953 < _size949; ++_i953)
              {
-               xfer += iprot->readString(this->tbl_names[_i942]);
 -              xfer += iprot->readString(this->tbl_names[_i931]);
++              xfer += iprot->readString(this->tbl_names[_i953]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -7044,10 -7315,10 +7315,10 @@@ uint32_t ThriftHiveMetastore_get_table_
    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 _iter943;
-     for (_iter943 = this->tbl_names.begin(); _iter943 != this->tbl_names.end(); ++_iter943)
 -    std::vector<std::string> ::const_iterator _iter932;
 -    for (_iter932 = this->tbl_names.begin(); _iter932 != this->tbl_names.end(); ++_iter932)
++    std::vector<std::string> ::const_iterator _iter954;
++    for (_iter954 = this->tbl_names.begin(); _iter954 != this->tbl_names.end(); ++_iter954)
      {
-       xfer += oprot->writeString((*_iter943));
 -      xfer += oprot->writeString((*_iter932));
++      xfer += oprot->writeString((*_iter954));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -7075,10 -7346,10 +7346,10 @@@ uint32_t ThriftHiveMetastore_get_table_
    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 _iter944;
-     for (_iter944 = (*(this->tbl_names)).begin(); _iter944 != (*(this->tbl_names)).end(); ++_iter944)
 -    std::vector<std::string> ::const_iterator _iter933;
 -    for (_iter933 = (*(this->tbl_names)).begin(); _iter933 != (*(this->tbl_names)).end(); ++_iter933)
++    std::vector<std::string> ::const_iterator _iter955;
++    for (_iter955 = (*(this->tbl_names)).begin(); _iter955 != (*(this->tbl_names)).end(); ++_iter955)
      {
-       xfer += oprot->writeString((*_iter944));
 -      xfer += oprot->writeString((*_iter933));
++      xfer += oprot->writeString((*_iter955));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -7119,14 -7390,14 +7390,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size945;
-             ::apache::thrift::protocol::TType _etype948;
-             xfer += iprot->readListBegin(_etype948, _size945);
-             this->success.resize(_size945);
-             uint32_t _i949;
-             for (_i949 = 0; _i949 < _size945; ++_i949)
 -            uint32_t _size934;
 -            ::apache::thrift::protocol::TType _etype937;
 -            xfer += iprot->readListBegin(_etype937, _size934);
 -            this->success.resize(_size934);
 -            uint32_t _i938;
 -            for (_i938 = 0; _i938 < _size934; ++_i938)
++            uint32_t _size956;
++            ::apache::thrift::protocol::TType _etype959;
++            xfer += iprot->readListBegin(_etype959, _size956);
++            this->success.resize(_size956);
++            uint32_t _i960;
++            for (_i960 = 0; _i960 < _size956; ++_i960)
              {
-               xfer += this->success[_i949].read(iprot);
 -              xfer += this->success[_i938].read(iprot);
++              xfer += this->success[_i960].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -7181,10 -7452,10 +7452,10 @@@ uint32_t ThriftHiveMetastore_get_table_
      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 _iter950;
-       for (_iter950 = this->success.begin(); _iter950 != this->success.end(); ++_iter950)
 -      std::vector<Table> ::const_iterator _iter939;
 -      for (_iter939 = this->success.begin(); _iter939 != this->success.end(); ++_iter939)
++      std::vector<Table> ::const_iterator _iter961;
++      for (_iter961 = this->success.begin(); _iter961 != this->success.end(); ++_iter961)
        {
-         xfer += (*_iter950).write(oprot);
 -        xfer += (*_iter939).write(oprot);
++        xfer += (*_iter961).write(oprot);
        }
        xfer += oprot->writeListEnd();
      }
@@@ -7237,14 -7508,14 +7508,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size951;
-             ::apache::thrift::protocol::TType _etype954;
-             xfer += iprot->readListBegin(_etype954, _size951);
-             (*(this->success)).resize(_size951);
-             uint32_t _i955;
-             for (_i955 = 0; _i955 < _size951; ++_i955)
 -            uint32_t _size940;
 -            ::apache::thrift::protocol::TType _etype943;
 -            xfer += iprot->readListBegin(_etype943, _size940);
 -            (*(this->success)).resize(_size940);
 -            uint32_t _i944;
 -            for (_i944 = 0; _i944 < _size940; ++_i944)
++            uint32_t _size962;
++            ::apache::thrift::protocol::TType _etype965;
++            xfer += iprot->readListBegin(_etype965, _size962);
++            (*(this->success)).resize(_size962);
++            uint32_t _i966;
++            for (_i966 = 0; _i966 < _size962; ++_i966)
              {
-               xfer += (*(this->success))[_i955].read(iprot);
 -              xfer += (*(this->success))[_i944].read(iprot);
++              xfer += (*(this->success))[_i966].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -7430,14 -7701,14 +7701,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size956;
-             ::apache::thrift::protocol::TType _etype959;
-             xfer += iprot->readListBegin(_etype959, _size956);
-             this->success.resize(_size956);
-             uint32_t _i960;
-             for (_i960 = 0; _i960 < _size956; ++_i960)
 -            uint32_t _size945;
 -            ::apache::thrift::protocol::TType _etype948;
 -            xfer += iprot->readListBegin(_etype948, _size945);
 -            this->success.resize(_size945);
 -            uint32_t _i949;
 -            for (_i949 = 0; _i949 < _size945; ++_i949)
++            uint32_t _size967;
++            ::apache::thrift::protocol::TType _etype970;
++            xfer += iprot->readListBegin(_etype970, _size967);
++            this->success.resize(_size967);
++            uint32_t _i971;
++            for (_i971 = 0; _i971 < _size967; ++_i971)
              {
-               xfer += iprot->readString(this->success[_i960]);
 -              xfer += iprot->readString(this->success[_i949]);
++              xfer += iprot->readString(this->success[_i971]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -7492,10 -7763,10 +7763,10 @@@ uint32_t ThriftHiveMetastore_get_table_
      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 _iter961;
-       for (_iter961 = this->success.begin(); _iter961 != this->success.end(); ++_iter961)
 -      std::vector<std::string> ::const_iterator _iter950;
 -      for (_iter950 = this->success.begin(); _iter950 != this->success.end(); ++_iter950)
++      std::vector<std::string> ::const_iterator _iter972;
++      for (_iter972 = this->success.begin(); _iter972 != this->success.end(); ++_iter972)
        {
-         xfer += oprot->writeString((*_iter961));
 -        xfer += oprot->writeString((*_iter950));
++        xfer += oprot->writeString((*_iter972));
        }
        xfer += oprot->writeListEnd();
      }
@@@ -7548,14 -7819,14 +7819,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size962;
-             ::apache::thrift::protocol::TType _etype965;
-             xfer += iprot->readListBegin(_etype965, _size962);
-             (*(this->success)).resize(_size962);
-             uint32_t _i966;
-             for (_i966 = 0; _i966 < _size962; ++_i966)
 -            uint32_t _size951;
 -            ::apache::thrift::protocol::TType _etype954;
 -            xfer += iprot->readListBegin(_etype954, _size951);
 -            (*(this->success)).resize(_size951);
 -            uint32_t _i955;
 -            for (_i955 = 0; _i955 < _size951; ++_i955)
++            uint32_t _size973;
++            ::apache::thrift::protocol::TType _etype976;
++            xfer += iprot->readListBegin(_etype976, _size973);
++            (*(this->success)).resize(_size973);
++            uint32_t _i977;
++            for (_i977 = 0; _i977 < _size973; ++_i977)
              {
-               xfer += iprot->readString((*(this->success))[_i966]);
 -              xfer += iprot->readString((*(this->success))[_i955]);
++              xfer += iprot->readString((*(this->success))[_i977]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -8889,14 -9160,14 +9160,14 @@@ uint32_t ThriftHiveMetastore_add_partit
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->new_parts.clear();
-             uint32_t _size967;
-             ::apache::thrift::protocol::TType _etype970;
-             xfer += iprot->readListBegin(_etype970, _size967);
-             this->new_parts.resize(_size967);
-             uint32_t _i971;
-             for (_i971 = 0; _i971 < _size967; ++_i971)
 -            uint32_t _size956;
 -            ::apache::thrift::protocol::TType _etype959;
 -            xfer += iprot->readListBegin(_etype959, _size956);
 -            this->new_parts.resize(_size956);
 -            uint32_t _i960;
 -            for (_i960 = 0; _i960 < _size956; ++_i960)
++            uint32_t _size978;
++            ::apache::thrift::protocol::TType _etype981;
++            xfer += iprot->readListBegin(_etype981, _size978);
++            this->new_parts.resize(_size978);
++            uint32_t _i982;
++            for (_i982 = 0; _i982 < _size978; ++_i982)
              {
-               xfer += this->new_parts[_i971].read(iprot);
 -              xfer += this->new_parts[_i960].read(iprot);
++              xfer += this->new_parts[_i982].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -8925,10 -9196,10 +9196,10 @@@ uint32_t ThriftHiveMetastore_add_partit
    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 _iter972;
-     for (_iter972 = this->new_parts.begin(); _iter972 != this->new_parts.end(); ++_iter972)
 -    std::vector<Partition> ::const_iterator _iter961;
 -    for (_iter961 = this->new_parts.begin(); _iter961 != this->new_parts.end(); ++_iter961)
++    std::vector<Partition> ::const_iterator _iter983;
++    for (_iter983 = this->new_parts.begin(); _iter983 != this->new_parts.end(); ++_iter983)
      {
-       xfer += (*_iter972).write(oprot);
 -      xfer += (*_iter961).write(oprot);
++      xfer += (*_iter983).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -8952,10 -9223,10 +9223,10 @@@ uint32_t ThriftHiveMetastore_add_partit
    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 _iter973;
-     for (_iter973 = (*(this->new_parts)).begin(); _iter973 != (*(this->new_parts)).end(); ++_iter973)
 -    std::vector<Partition> ::const_iterator _iter962;
 -    for (_iter962 = (*(this->new_parts)).begin(); _iter962 != (*(this->new_parts)).end(); ++_iter962)
++    std::vector<Partition> ::const_iterator _iter984;
++    for (_iter984 = (*(this->new_parts)).begin(); _iter984 != (*(this->new_parts)).end(); ++_iter984)
      {
-       xfer += (*_iter973).write(oprot);
 -      xfer += (*_iter962).write(oprot);
++      xfer += (*_iter984).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -9164,14 -9435,14 +9435,14 @@@ uint32_t ThriftHiveMetastore_add_partit
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->new_parts.clear();
-             uint32_t _size974;
-             ::apache::thrift::protocol::TType _etype977;
-             xfer += iprot->readListBegin(_etype977, _size974);
-             this->new_parts.resize(_size974);
-             uint32_t _i978;
-             for (_i978 = 0; _i978 < _size974; ++_i978)
 -            uint32_t _size963;
 -            ::apache::thrift::protocol::TType _etype966;
 -            xfer += iprot->readListBegin(_etype966, _size963);
 -            this->new_parts.resize(_size963);
 -            uint32_t _i967;
 -            for (_i967 = 0; _i967 < _size963; ++_i967)
++            uint32_t _size985;
++            ::apache::thrift::protocol::TType _etype988;
++            xfer += iprot->readListBegin(_etype988, _size985);
++            this->new_parts.resize(_size985);
++            uint32_t _i989;
++            for (_i989 = 0; _i989 < _size985; ++_i989)
              {
-               xfer += this->new_parts[_i978].read(iprot);
 -              xfer += this->new_parts[_i967].read(iprot);
++              xfer += this->new_parts[_i989].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -9200,10 -9471,10 +9471,10 @@@ uint32_t ThriftHiveMetastore_add_partit
    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 _iter979;
-     for (_iter979 = this->new_parts.begin(); _iter979 != this->new_parts.end(); ++_iter979)
 -    std::vector<PartitionSpec> ::const_iterator _iter968;
 -    for (_iter968 = this->new_parts.begin(); _iter968 != this->new_parts.end(); ++_iter968)
++    std::vector<PartitionSpec> ::const_iterator _iter990;
++    for (_iter990 = this->new_parts.begin(); _iter990 != this->new_parts.end(); ++_iter990)
      {
-       xfer += (*_iter979).write(oprot);
 -      xfer += (*_iter968).write(oprot);
++      xfer += (*_iter990).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -9227,10 -9498,10 +9498,10 @@@ uint32_t ThriftHiveMetastore_add_partit
    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 _iter980;
-     for (_iter980 = (*(this->new_parts)).begin(); _iter980 != (*(this->new_parts)).end(); ++_iter980)
 -    std::vector<PartitionSpec> ::const_iterator _iter969;
 -    for (_iter969 = (*(this->new_parts)).begin(); _iter969 != (*(this->new_parts)).end(); ++_iter969)
++    std::vector<PartitionSpec> ::const_iterator _iter991;
++    for (_iter991 = (*(this->new_parts)).begin(); _iter991 != (*(this->new_parts)).end(); ++_iter991)
      {
-       xfer += (*_iter980).write(oprot);
 -      xfer += (*_iter969).write(oprot);
++      xfer += (*_iter991).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -9455,14 -9726,14 +9726,14 @@@ uint32_t ThriftHiveMetastore_append_par
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->part_vals.clear();
-             uint32_t _size981;
-             ::apache::thrift::protocol::TType _etype984;
-             xfer += iprot->readListBegin(_etype984, _size981);
-             this->part_vals.resize(_size981);
-             uint32_t _i985;
-             for (_i985 = 0; _i985 < _size981; ++_i985)
 -            uint32_t _size970;
 -            ::apache::thrift::protocol::TType _etype973;
 -            xfer += iprot->readListBegin(_etype973, _size970);
 -            this->part_vals.resize(_size970);
 -            uint32_t _i974;
 -            for (_i974 = 0; _i974 < _size970; ++_i974)
++            uint32_t _size992;
++            ::apache::thrift::protocol::TType _etype995;
++            xfer += iprot->readListBegin(_etype995, _size992);
++            this->part_vals.resize(_size992);
++            uint32_t _i996;
++            for (_i996 = 0; _i996 < _size992; ++_i996)
              {
-               xfer += iprot->readString(this->part_vals[_i985]);
 -              xfer += iprot->readString(this->part_vals[_i974]);
++              xfer += iprot->readString(this->part_vals[_i996]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -9499,10 -9770,10 +9770,10 @@@ uint32_t ThriftHiveMetastore_append_par
    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 _iter986;
-     for (_iter986 = this->part_vals.begin(); _iter986 != this->part_vals.end(); ++_iter986)
 -    std::vector<std::string> ::const_iterator _iter975;
 -    for (_iter975 = this->part_vals.begin(); _iter975 != this->part_vals.end(); ++_iter975)
++    std::vector<std::string> ::const_iterator _iter997;
++    for (_iter997 = this->part_vals.begin(); _iter997 != this->part_vals.end(); ++_iter997)
      {
-       xfer += oprot->writeString((*_iter986));
 -      xfer += oprot->writeString((*_iter975));
++      xfer += oprot->writeString((*_iter997));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -9534,10 -9805,10 +9805,10 @@@ uint32_t ThriftHiveMetastore_append_par
    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 _iter987;
-     for (_iter987 = (*(this->part_vals)).begin(); _iter987 != (*(this->part_vals)).end(); ++_iter987)
 -    std::vector<std::string> ::const_iterator _iter976;
 -    for (_iter976 = (*(this->part_vals)).begin(); _iter976 != (*(this->part_vals)).end(); ++_iter976)
++    std::vector<std::string> ::const_iterator _iter998;
++    for (_iter998 = (*(this->part_vals)).begin(); _iter998 != (*(this->part_vals)).end(); ++_iter998)
      {
-       xfer += oprot->writeString((*_iter987));
 -      xfer += oprot->writeString((*_iter976));
++      xfer += oprot->writeString((*_iter998));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -10009,14 -10280,14 +10280,14 @@@ uint32_t ThriftHiveMetastore_append_par
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->part_vals.clear();
-             uint32_t _size988;
-             ::apache::thrift::protocol::TType _etype991;
-             xfer += iprot->readListBegin(_etype991, _size988);
-             this->part_vals.resize(_size988);
-             uint32_t _i992;
-             for (_i992 = 0; _i992 < _size988; ++_i992)
 -            uint32_t _size977;
 -            ::apache::thrift::protocol::TType _etype980;
 -            xfer += iprot->readListBegin(_etype980, _size977);
 -            this->part_vals.resize(_size977);
 -            uint32_t _i981;
 -            for (_i981 = 0; _i981 < _size977; ++_i981)
++            uint32_t _size999;
++            ::apache::thrift::protocol::TType _etype1002;
++            xfer += iprot->readListBegin(_etype1002, _size999);
++            this->part_vals.resize(_size999);
++            uint32_t _i1003;
++            for (_i1003 = 0; _i1003 < _size999; ++_i1003)
              {
-               xfer += iprot->readString(this->part_vals[_i992]);
 -              xfer += iprot->readString(this->part_vals[_i981]);
++              xfer += iprot->readString(this->part_vals[_i1003]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -10061,10 -10332,10 +10332,10 @@@ uint32_t ThriftHiveMetastore_append_par
    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 _iter993;
-     for (_iter993 = this->part_vals.begin(); _iter993 != this->part_vals.end(); ++_iter993)
 -    std::vector<std::string> ::const_iterator _iter982;
 -    for (_iter982 = this->part_vals.begin(); _iter982 != this->part_vals.end(); ++_iter982)
++    std::vector<std::string> ::const_iterator _iter1004;
++    for (_iter1004 = this->part_vals.begin(); _iter1004 != this->part_vals.end(); ++_iter1004)
      {
-       xfer += oprot->writeString((*_iter993));
 -      xfer += oprot->writeString((*_iter982));
++      xfer += oprot->writeString((*_iter1004));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -10100,10 -10371,10 +10371,10 @@@ uint32_t ThriftHiveMetastore_append_par
    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 _iter994;
-     for (_iter994 = (*(this->part_vals)).begin(); _iter994 != (*(this->part_vals)).end(); ++_iter994)
 -    std::vector<std::string> ::const_iterator _iter983;
 -    for (_iter983 = (*(this->part_vals)).begin(); _iter983 != (*(this->part_vals)).end(); ++_iter983)
++    std::vector<std::string> ::const_iterator _iter1005;
++    for (_iter1005 = (*(this->part_vals)).begin(); _iter1005 != (*(this->part_vals)).end(); ++_iter1005)
      {
-       xfer += oprot->writeString((*_iter994));
 -      xfer += oprot->writeString((*_iter983));
++      xfer += oprot->writeString((*_iter1005));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -10906,14 -11177,14 +11177,14 @@@ uint32_t ThriftHiveMetastore_drop_parti
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->part_vals.clear();
-             uint32_t _size995;
-             ::apache::thrift::protocol::TType _etype998;
-             xfer += iprot->readListBegin(_etype998, _size995);
-             this->part_vals.resize(_size995);
-             uint32_t _i999;
-             for (_i999 = 0; _i999 < _size995; ++_i999)
 -            uint32_t _size984;
 -            ::apache::thrift::protocol::TType _etype987;
 -            xfer += iprot->readListBegin(_etype987, _size984);
 -            this->part_vals.resize(_size984);
 -            uint32_t _i988;
 -            for (_i988 = 0; _i988 < _size984; ++_i988)
++            uint32_t _size1006;
++            ::apache::thrift::protocol::TType _etype1009;
++            xfer += iprot->readListBegin(_etype1009, _size1006);
++            this->part_vals.resize(_size1006);
++            uint32_t _i1010;
++            for (_i1010 = 0; _i1010 < _size1006; ++_i1010)
              {
-               xfer += iprot->readString(this->part_vals[_i999]);
 -              xfer += iprot->readString(this->part_vals[_i988]);
++              xfer += iprot->readString(this->part_vals[_i1010]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -10958,10 -11229,10 +11229,10 @@@ uint32_t ThriftHiveMetastore_drop_parti
    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 _iter1000;
-     for (_iter1000 = this->part_vals.begin(); _iter1000 != this->part_vals.end(); ++_iter1000)
 -    std::vector<std::string> ::const_iterator _iter989;
 -    for (_iter989 = this->part_vals.begin(); _iter989 != this->part_vals.end(); ++_iter989)
++    std::vector<std::string> ::const_iterator _iter1011;
++    for (_iter1011 = this->part_vals.begin(); _iter1011 != this->part_vals.end(); ++_iter1011)
      {
-       xfer += oprot->writeString((*_iter1000));
 -      xfer += oprot->writeString((*_iter989));
++      xfer += oprot->writeString((*_iter1011));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -10997,10 -11268,10 +11268,10 @@@ uint32_t ThriftHiveMetastore_drop_parti
    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 _iter1001;
-     for (_iter1001 = (*(this->part_vals)).begin(); _iter1001 != (*(this->part_vals)).end(); ++_iter1001)
 -    std::vector<std::string> ::const_iterator _iter990;
 -    for (_iter990 = (*(this->part_vals)).begin(); _iter990 != (*(this->part_vals)).end(); ++_iter990)
++    std::vector<std::string> ::const_iterator _iter1012;
++    for (_iter1012 = (*(this->part_vals)).begin(); _iter1012 != (*(this->part_vals)).end(); ++_iter1012)
      {
-       xfer += oprot->writeString((*_iter1001));
 -      xfer += oprot->writeString((*_iter990));
++      xfer += oprot->writeString((*_iter1012));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -11209,14 -11480,14 +11480,14 @@@ uint32_t ThriftHiveMetastore_drop_parti
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->part_vals.clear();
-             uint32_t _size1002;
-             ::apache::thrift::protocol::TType _etype1005;
-             xfer += iprot->readListBegin(_etype1005, _size1002);
-             this->part_vals.resize(_size1002);
-             uint32_t _i1006;
-             for (_i1006 = 0; _i1006 < _size1002; ++_i1006)
 -            uint32_t _size991;
 -            ::apache::thrift::protocol::TType _etype994;
 -            xfer += iprot->readListBegin(_etype994, _size991);
 -            this->part_vals.resize(_size991);
 -            uint32_t _i995;
 -            for (_i995 = 0; _i995 < _size991; ++_i995)
++            uint32_t _size1013;
++            ::apache::thrift::protocol::TType _etype1016;
++            xfer += iprot->readListBegin(_etype1016, _size1013);
++            this->part_vals.resize(_size1013);
++            uint32_t _i1017;
++            for (_i1017 = 0; _i1017 < _size1013; ++_i1017)
              {
-               xfer += iprot->readString(this->part_vals[_i1006]);
 -              xfer += iprot->readString(this->part_vals[_i995]);
++              xfer += iprot->readString(this->part_vals[_i1017]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -11269,10 -11540,10 +11540,10 @@@ uint32_t ThriftHiveMetastore_drop_parti
    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 _iter1007;
-     for (_iter1007 = this->part_vals.begin(); _iter1007 != this->part_vals.end(); ++_iter1007)
 -    std::vector<std::string> ::const_iterator _iter996;
 -    for (_iter996 = this->part_vals.begin(); _iter996 != this->part_vals.end(); ++_iter996)
++    std::vector<std::string> ::const_iterator _iter1018;
++    for (_iter1018 = this->part_vals.begin(); _iter1018 != this->part_vals.end(); ++_iter1018)
      {
-       xfer += oprot->writeString((*_iter1007));
 -      xfer += oprot->writeString((*_iter996));
++      xfer += oprot->writeString((*_iter1018));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -11312,10 -11583,10 +11583,10 @@@ uint32_t ThriftHiveMetastore_drop_parti
    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 _iter1008;
-     for (_iter1008 = (*(this->part_vals)).begin(); _iter1008 != (*(this->part_vals)).end(); ++_iter1008)
 -    std::vector<std::string> ::const_iterator _iter997;
 -    for (_iter997 = (*(this->part_vals)).begin(); _iter997 != (*(this->part_vals)).end(); ++_iter997)
++    std::vector<std::string> ::const_iterator _iter1019;
++    for (_iter1019 = (*(this->part_vals)).begin(); _iter1019 != (*(this->part_vals)).end(); ++_iter1019)
      {
-       xfer += oprot->writeString((*_iter1008));
 -      xfer += oprot->writeString((*_iter997));
++      xfer += oprot->writeString((*_iter1019));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -12321,14 -12592,14 +12592,14 @@@ uint32_t ThriftHiveMetastore_get_partit
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->part_vals.clear();
-             uint32_t _size1009;
-             ::apache::thrift::protocol::TType _etype1012;
-             xfer += iprot->readListBegin(_etype1012, _size1009);
-             this->part_vals.resize(_size1009);
-             uint32_t _i1013;
-             for (_i1013 = 0; _i1013 < _size1009; ++_i1013)
 -            uint32_t _size998;
 -            ::apache::thrift::protocol::TType _etype1001;
 -            xfer += iprot->readListBegin(_etype1001, _size998);
 -            this->part_vals.resize(_size998);
 -            uint32_t _i1002;
 -            for (_i1002 = 0; _i1002 < _size998; ++_i1002)
++            uint32_t _size1020;
++            ::apache::thrift::protocol::TType _etype1023;
++            xfer += iprot->readListBegin(_etype1023, _size1020);
++            this->part_vals.resize(_size1020);
++            uint32_t _i1024;
++            for (_i1024 = 0; _i1024 < _size1020; ++_i1024)
              {
-               xfer += iprot->readString(this->part_vals[_i1013]);
 -              xfer += iprot->readString(this->part_vals[_i1002]);
++              xfer += iprot->readString(this->part_vals[_i1024]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -12365,10 -12636,10 +12636,10 @@@ uint32_t ThriftHiveMetastore_get_partit
    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 _iter1014;
-     for (_iter1014 = this->part_vals.begin(); _iter1014 != this->part_vals.end(); ++_iter1014)
 -    std::vector<std::string> ::const_iterator _iter1003;
 -    for (_iter1003 = this->part_vals.begin(); _iter1003 != this->part_vals.end(); ++_iter1003)
++    std::vector<std::string> ::const_iterator _iter1025;
++    for (_iter1025 = this->part_vals.begin(); _iter1025 != this->part_vals.end(); ++_iter1025)
      {
-       xfer += oprot->writeString((*_iter1014));
 -      xfer += oprot->writeString((*_iter1003));
++      xfer += oprot->writeString((*_iter1025));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -12400,10 -12671,10 +12671,10 @@@ uint32_t ThriftHiveMetastore_get_partit
    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 _iter1015;
-     for (_iter1015 = (*(this->part_vals)).begin(); _iter1015 != (*(this->part_vals)).end(); ++_iter1015)
 -    std::vector<std::string> ::const_iterator _iter1004;
 -    for (_iter1004 = (*(this->part_vals)).begin(); _iter1004 != (*(this->part_vals)).end(); ++_iter1004)
++    std::vector<std::string> ::const_iterator _iter1026;
++    for (_iter1026 = (*(this->part_vals)).begin(); _iter1026 != (*(this->part_vals)).end(); ++_iter1026)
      {
-       xfer += oprot->writeString((*_iter1015));
 -      xfer += oprot->writeString((*_iter1004));
++      xfer += oprot->writeString((*_iter1026));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -12592,17 -12863,17 +12863,17 @@@ uint32_t ThriftHiveMetastore_exchange_p
          if (ftype == ::apache::thrift::protocol::T_MAP) {
            {
              this->partitionSpecs.clear();
-             uint32_t _size1016;
-             ::apache::thrift::protocol::TType _ktype1017;
-             ::apache::thrift::protocol::TType _vtype1018;
-             xfer += iprot->readMapBegin(_ktype1017, _vtype1018, _size1016);
-             uint32_t _i1020;
-             for (_i1020 = 0; _i1020 < _size1016; ++_i1020)
 -            uint32_t _size1005;
 -            ::apache::thrift::protocol::TType _ktype1006;
 -            ::apache::thrift::protocol::TType _vtype1007;
 -            xfer += iprot->readMapBegin(_ktype1006, _vtype1007, _size1005);
 -            uint32_t _i1009;
 -            for (_i1009 = 0; _i1009 < _size1005; ++_i1009)
++            uint32_t _size1027;
++            ::apache::thrift::protocol::TType _ktype1028;
++            ::apache::thrift::protocol::TType _vtype1029;
++            xfer += iprot->readMapBegin(_ktype1028, _vtype1029, _size1027);
++            uint32_t _i1031;
++            for (_i1031 = 0; _i1031 < _size1027; ++_i1031)
              {
-               std::string _key1021;
-               xfer += iprot->readString(_key1021);
-               std::string& _val1022 = this->partitionSpecs[_key1021];
-               xfer += iprot->readString(_val1022);
 -              std::string _key1010;
 -              xfer += iprot->readString(_key1010);
 -              std::string& _val1011 = this->partitionSpecs[_key1010];
 -              xfer += iprot->readString(_val1011);
++              std::string _key1032;
++              xfer += iprot->readString(_key1032);
++              std::string& _val1033 = this->partitionSpecs[_key1032];
++              xfer += iprot->readString(_val1033);
              }
              xfer += iprot->readMapEnd();
            }
@@@ -12663,11 -12934,11 +12934,11 @@@ uint32_t ThriftHiveMetastore_exchange_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 _iter1023;
-     for (_iter1023 = this->partitionSpecs.begin(); _iter1023 != this->partitionSpecs.end(); ++_iter1023)
 -    std::map<std::string, std::string> ::const_iterator _iter1012;
 -    for (_iter1012 = this->partitionSpecs.begin(); _iter1012 != this->partitionSpecs.end(); ++_iter1012)
++    std::map<std::string, std::string> ::const_iterator _iter1034;
++    for (_iter1034 = this->partitionSpecs.begin(); _iter1034 != this->partitionSpecs.end(); ++_iter1034)
      {
-       xfer += oprot->writeString(_iter1023->first);
-       xfer += oprot->writeString(_iter1023->second);
 -      xfer += oprot->writeString(_iter1012->first);
 -      xfer += oprot->writeString(_iter1012->second);
++      xfer += oprot->writeString(_iter1034->first);
++      xfer += oprot->writeString(_iter1034->second);
      }
      xfer += oprot->writeMapEnd();
    }
@@@ -12707,11 -12978,11 +12978,11 @@@ uint32_t ThriftHiveMetastore_exchange_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 _iter1024;
-     for (_iter1024 = (*(this->partitionSpecs)).begin(); _iter1024 != (*(this->partitionSpecs)).end(); ++_iter1024)
 -    std::map<std::string, std::string> ::const_iterator _iter1013;
 -    for (_iter1013 = (*(this->partitionSpecs)).begin(); _iter1013 != (*(this->partitionSpecs)).end(); ++_iter1013)
++    std::map<std::string, std::string> ::const_iterator _iter1035;
++    for (_iter1035 = (*(this->partitionSpecs)).begin(); _iter1035 != (*(this->partitionSpecs)).end(); ++_iter1035)
      {
-       xfer += oprot->writeString(_iter1024->first);
-       xfer += oprot->writeString(_iter1024->second);
 -      xfer += oprot->writeString(_iter1013->first);
 -      xfer += oprot->writeString(_iter1013->second);
++      xfer += oprot->writeString(_iter1035->first);
++      xfer += oprot->writeString(_iter1035->second);
      }
      xfer += oprot->writeMapEnd();
    }
@@@ -12956,17 -13227,17 +13227,17 @@@ uint32_t ThriftHiveMetastore_exchange_p
          if (ftype == ::apache::thrift::protocol::T_MAP) {
            {
              this->partitionSpecs.clear();
-             uint32_t _size1025;
-             ::apache::thrift::protocol::TType _ktype1026;
-             ::apache::thrift::protocol::TType _vtype1027;
-             xfer += iprot->readMapBegin(_ktype1026, _vtype1027, _size1025);
-             uint32_t _i1029;
-             for (_i1029 = 0; _i1029 < _size1025; ++_i1029)
 -            uint32_t _size1014;
 -            ::apache::thrift::protocol::TType _ktype1015;
 -            ::apache::thrift::protocol::TType _vtype1016;
 -            xfer += iprot->readMapBegin(_ktype1015, _vtype1016, _size1014);
 -            uint32_t _i1018;
 -            for (_i1018 = 0; _i1018 < _size1014; ++_i1018)
++            uint32_t _size1036;
++            ::apache::thrift::protocol::TType _ktype1037;
++            ::apache::thrift::protocol::TType _vtype1038;
++            xfer += iprot->readMapBegin(_ktype1037, _vtype1038, _size1036);
++            uint32_t _i1040;
++            for (_i1040 = 0; _i1040 < _size1036; ++_i1040)
              {
-               std::string _key1030;
-               xfer += iprot->readString(_key1030);
-               std::string& _val1031 = this->partitionSpecs[_key1030];
-               xfer += iprot->readString(_val1031);
 -              std::string _key1019;
 -              xfer += iprot->readString(_key1019);
 -              std::string& _val1020 = this->partitionSpecs[_key1019];
 -              xfer += iprot->readString(_val1020);
++              std::string _key1041;
++              xfer += iprot->readString(_key1041);
++              std::string& _val1042 = this->partitionSpecs[_key1041];
++              xfer += iprot->readString(_val1042);
              }
              xfer += iprot->readMapEnd();
            }
@@@ -13027,11 -13298,11 +13298,11 @@@ uint32_t ThriftHiveMetastore_exchange_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 _iter1032;
-     for (_iter1032 = this->partitionSpecs.begin(); _iter1032 != this->partitionSpecs.end(); ++_iter1032)
 -    std::map<std::string, std::string> ::const_iterator _iter1021;
 -    for (_iter1021 = this->partitionSpecs.begin(); _iter1021 != this->partitionSpecs.end(); ++_iter1021)
++    std::map<std::string, std::string> ::const_iterator _iter1043;
++    for (_iter1043 = this->partitionSpecs.begin(); _iter1043 != this->partitionSpecs.end(); ++_iter1043)
      {
-       xfer += oprot->writeString(_iter1032->first);
-       xfer += oprot->writeString(_iter1032->second);
 -      xfer += oprot->writeString(_iter1021->first);
 -      xfer += oprot->writeString(_iter1021->second);
++      xfer += oprot->writeString(_iter1043->first);
++      xfer += oprot->writeString(_iter1043->second);
      }
      xfer += oprot->writeMapEnd();
    }
@@@ -13071,11 -13342,11 +13342,11 @@@ uint32_t ThriftHiveMetastore_exchange_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 _iter1033;
-     for (_iter1033 = (*(this->partitionSpecs)).begin(); _iter1033 != (*(this->partitionSpecs)).end(); ++_iter1033)
 -    std::map<std::string, std::string> ::const_iterator _iter1022;
 -    for (_iter1022 = (*(this->partitionSpecs)).begin(); _iter1022 != (*(this->partitionSpecs)).end(); ++_iter1022)
++    std::map<std::string, std::string> ::const_iterator _iter1044;
++    for (_iter1044 = (*(this->partitionSpecs)).begin(); _iter1044 != (*(this->partitionSpecs)).end(); ++_iter1044)
      {
-       xfer += oprot->writeString(_iter1033->first);
-       xfer += oprot->writeString(_iter1033->second);
 -      xfer += oprot->writeString(_iter1022->first);
 -      xfer += oprot->writeString(_iter1022->second);
++      xfer += oprot->writeString(_iter1044->first);
++      xfer += oprot->writeString(_iter1044->second);
      }
      xfer += oprot->writeMapEnd();
    }
@@@ -13132,14 -13403,14 +13403,14 @@@ uint32_t ThriftHiveMetastore_exchange_p
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1034;
-             ::apache::thrift::protocol::TType _etype1037;
-             xfer += iprot->readListBegin(_etype1037, _size1034);
-             this->success.resize(_size1034);
-             uint32_t _i1038;
-             for (_i1038 = 0; _i1038 < _size1034; ++_i1038)
 -            uint32_t _size1023;
 -            ::apache::thrift::protocol::TType _etype1026;
 -            xfer += iprot->readListBegin(_etype1026, _size1023);
 -            this->success.resize(_size1023);
 -            uint32_t _i1027;
 -            for (_i1027 = 0; _i1027 < _size1023; ++_i1027)
++            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)
              {
-               xfer += this->success[_i1038].read(iprot);
 -              xfer += this->success[_i1027].read(iprot);
++              xfer += this->success[_i1049].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -13202,10 -13473,10 +13473,10 @@@ uint32_t ThriftHiveMetastore_exchange_p
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-       std::vector<Partition> ::const_iterator _iter1039;
-       for (_iter1039 = this->success.begin(); _iter1039 != this->success.end(); ++_iter1039)
 -      std::vector<Partition> ::const_iterator _iter1028;
 -      for (_iter1028 = this->success.begin(); _iter1028 != this->success.end(); ++_iter1028)
++      std::vector<Partition> ::const_iterator _iter1050;
++      for (_iter1050 = this->success.begin(); _iter1050 != this->success.end(); ++_iter1050)
        {
-         xfer += (*_iter1039).write(oprot);
 -        xfer += (*_iter1028).write(oprot);
++        xfer += (*_iter1050).write(oprot);
        }
        xfer += oprot->writeListEnd();
      }
@@@ -13262,14 -13533,14 +13533,14 @@@ uint32_t ThriftHiveMetastore_exchange_p
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1040;
-             ::apache::thrift::protocol::TType _etype1043;
-             xfer += iprot->readListBegin(_etype1043, _size1040);
-             (*(this->success)).resize(_size1040);
-             uint32_t _i1044;
-             for (_i1044 = 0; _i1044 < _size1040; ++_i1044)
 -            uint32_t _size1029;
 -            ::apache::thrift::protocol::TType _etype1032;
 -            xfer += iprot->readListBegin(_etype1032, _size1029);
 -            (*(this->success)).resize(_size1029);
 -            uint32_t _i1033;
 -            for (_i1033 = 0; _i1033 < _size1029; ++_i1033)
++            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)
              {
-               xfer += (*(this->success))[_i1044].read(iprot);
 -              xfer += (*(this->success))[_i1033].read(iprot);
++              xfer += (*(this->success))[_i1055].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -13368,14 -13639,14 +13639,14 @@@ uint32_t ThriftHiveMetastore_get_partit
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->part_vals.clear();
-             uint32_t _size1045;
-             ::apache::thrift::protocol::TType _etype1048;
-             xfer += iprot->readListBegin(_etype1048, _size1045);
-             this->part_vals.resize(_size1045);
-             uint32_t _i1049;
-             for (_i1049 = 0; _i1049 < _size1045; ++_i1049)
 -            uint32_t _size1034;
 -            ::apache::thrift::protocol::TType _etype1037;
 -            xfer += iprot->readListBegin(_etype1037, _size1034);
 -            this->part_vals.resize(_size1034);
 -            uint32_t _i1038;
 -            for (_i1038 = 0; _i1038 < _size1034; ++_i1038)
++            uint32_t _size1056;
++            ::apache::thrift::protocol::TType _etype1059;
++            xfer += iprot->readListBegin(_etype1059, _size1056);
++            this->part_vals.resize(_size1056);
++            uint32_t _i1060;
++            for (_i1060 = 0; _i1060 < _size1056; ++_i1060)
              {
-               xfer += iprot->readString(this->part_vals[_i1049]);
 -              xfer += iprot->readString(this->part_vals[_i1038]);
++              xfer += iprot->readString(this->part_vals[_i1060]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -13396,14 -13667,14 +13667,14 @@@
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->group_names.clear();
-             uint32_t _size1050;
-             ::apache::thrift::protocol::TType _etype1053;
-             xfer += iprot->readListBegin(_etype1053, _size1050);
-             this->group_names.resize(_size1050);
-             uint32_t _i1054;
-             for (_i1054 = 0; _i1054 < _size1050; ++_i1054)
 -            uint32_t _size1039;
 -            ::apache::thrift::protocol::TType _etype1042;
 -            xfer += iprot->readListBegin(_etype1042, _size1039);
 -            this->group_names.resize(_size1039);
 -            uint32_t _i1043;
 -            for (_i1043 = 0; _i1043 < _size1039; ++_i1043)
++            uint32_t _size1061;
++            ::apache::thrift::protocol::TType _etype1064;
++            xfer += iprot->readListBegin(_etype1064, _size1061);
++            this->group_names.resize(_size1061);
++            uint32_t _i1065;
++            for (_i1065 = 0; _i1065 < _size1061; ++_i1065)
              {
-               xfer += iprot->readString(this->group_names[_i1054]);
 -              xfer += iprot->readString(this->group_names[_i1043]);
++              xfer += iprot->readString(this->group_names[_i1065]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -13440,10 -13711,10 +13711,10 @@@ uint32_t ThriftHiveMetastore_get_partit
    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 _iter1055;
-     for (_iter1055 = this->part_vals.begin(); _iter1055 != this->part_vals.end(); ++_iter1055)
 -    std::vector<std::string> ::const_iterator _iter1044;
 -    for (_iter1044 = this->part_vals.begin(); _iter1044 != this->part_vals.end(); ++_iter1044)
++    std::vector<std::string> ::const_iterator _iter1066;
++    for (_iter1066 = this->part_vals.begin(); _iter1066 != this->part_vals.end(); ++_iter1066)
      {
-       xfer += oprot->writeString((*_iter1055));
 -      xfer += oprot->writeString((*_iter1044));
++      xfer += oprot->writeString((*_iter1066));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -13456,10 -13727,10 +13727,10 @@@
    xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-     std::vector<std::string> ::const_iterator _iter1056;
-     for (_iter1056 = this->group_names.begin(); _iter1056 != this->group_names.end(); ++_iter1056)
 -    std::vector<std::string> ::const_iterator _iter1045;
 -    for (_iter1045 = this->group_names.begin(); _iter1045 != this->group_names.end(); ++_iter1045)
++    std::vector<std::string> ::const_iterator _iter1067;
++    for (_iter1067 = this->group_names.begin(); _iter1067 != this->group_names.end(); ++_iter1067)
      {
-       xfer += oprot->writeString((*_iter1056));
 -      xfer += oprot->writeString((*_iter1045));
++      xfer += oprot->writeString((*_iter1067));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -13491,10 -13762,10 +13762,10 @@@ uint32_t ThriftHiveMetastore_get_partit
    xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-     std::vector<std::string> ::const_iterator _iter1057;
-     for (_iter1057 = (*(this->part_vals)).begin(); _iter1057 != (*(this->part_vals)).end(); ++_iter1057)
 -    std::vector<std::string> ::const_iterator _iter1046;
 -    for (_iter1046 = (*(this->part_vals)).begin(); _iter1046 != (*(this->part_vals)).end(); ++_iter1046)
++    std::vector<std::string> ::const_iterator _iter1068;
++    for (_iter1068 = (*(this->part_vals)).begin(); _iter1068 != (*(this->part_vals)).end(); ++_iter1068)
      {
-       xfer += oprot->writeString((*_iter1057));
 -      xfer += oprot->writeString((*_iter1046));
++      xfer += oprot->writeString((*_iter1068));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -13507,10 -13778,10 +13778,10 @@@
    xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-     std::vector<std::string> ::const_iterator _iter1058;
-     for (_iter1058 = (*(this->group_names)).begin(); _iter1058 != (*(this->group_names)).end(); ++_iter1058)
 -    std::vector<std::string> ::const_iterator _iter1047;
 -    for (_iter1047 = (*(this->group_names)).begin(); _iter1047 != (*(this->group_names)).end(); ++_iter1047)
++    std::vector<std::string> ::const_iterator _iter1069;
++    for (_iter1069 = (*(this->group_names)).begin(); _iter1069 != (*(this->group_names)).end(); ++_iter1069)
      {
-       xfer += oprot->writeString((*_iter1058));
 -      xfer += oprot->writeString((*_iter1047));
++      xfer += oprot->writeString((*_iter1069));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -14069,14 -14340,14 +14340,14 @@@ uint32_t ThriftHiveMetastore_get_partit
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1059;
-             ::apache::thrift::protocol::TType _etype1062;
-             xfer += iprot->readListBegin(_etype1062, _size1059);
-             this->success.resize(_size1059);
-             uint32_t _i1063;
-             for (_i1063 = 0; _i1063 < _size1059; ++_i1063)
 -            uint32_t _size1048;
 -            ::apache::thrift::protocol::TType _etype1051;
 -            xfer += iprot->readListBegin(_etype1051, _size1048);
 -            this->success.resize(_size1048);
 -            uint32_t _i1052;
 -            for (_i1052 = 0; _i1052 < _size1048; ++_i1052)
++            uint32_t _size1070;
++            ::apache::thrift::protocol::TType _etype1073;
++            xfer += iprot->readListBegin(_etype1073, _size1070);
++            this->success.resize(_size1070);
++            uint32_t _i1074;
++            for (_i1074 = 0; _i1074 < _size1070; ++_i1074)
              {
-               xfer += this->success[_i1063].read(iprot);
 -              xfer += this->success[_i1052].read(iprot);
++              xfer += this->success[_i1074].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -14123,10 -14394,10 +14394,10 @@@ uint32_t ThriftHiveMetastore_get_partit
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-       std::vector<Partition> ::const_iterator _iter1064;
-       for (_iter1064 = this->success.begin(); _iter1064 != this->success.end(); ++_iter1064)
 -      std::vector<Partition> ::const_iterator _iter1053;
 -      for (_iter1053 = this->success.begin(); _iter1053 != this->success.end(); ++_iter1053)
++      std::vector<Partition> ::const_iterator _iter1075;
++      for (_iter1075 = this->success.begin(); _iter1075 != this->success.end(); ++_iter1075)
        {
-         xfer += (*_iter1064).write(oprot);
 -        xfer += (*_iter1053).write(oprot);
++        xfer += (*_iter1075).write(oprot);
        }
        xfer += oprot->writeListEnd();
      }
@@@ -14175,14 -14446,14 +14446,14 @@@ uint32_t ThriftHiveMetastore_get_partit
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1065;
-             ::apache::thrift::protocol::TType _etype1068;
-             xfer += iprot->readListBegin(_etype1068, _size1065);
-             (*(this->success)).resize(_size1065);
-             uint32_t _i1069;
-             for (_i1069 = 0; _i1069 < _size1065; ++_i1069)
 -            uint32_t _size1054;
 -            ::apache::thrift::protocol::TType _etype1057;
 -            xfer += iprot->readListBegin(_etype1057, _size1054);
 -            (*(this->success)).resize(_size1054);
 -            uint32_t _i1058;
 -            for (_i1058 = 0; _i1058 < _size1054; ++_i1058)
++            uint32_t _size1076;
++            ::apache::thrift::protocol::TType _etype1079;
++            xfer += iprot->readListBegin(_etype1079, _size1076);
++            (*(this->success)).resize(_size1076);
++            uint32_t _i1080;
++            for (_i1080 = 0; _i1080 < _size1076; ++_i1080)
              {
-               xfer += (*(this->success))[_i1069].read(iprot);
 -              xfer += (*(this->success))[_i1058].read(iprot);
++              xfer += (*(this->success))[_i1080].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -14281,14 -14552,14 +14552,14 @@@ uint32_t ThriftHiveMetastore_get_partit
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->group_names.clear();
-             uint32_t _size1070;
-             ::apache::thrift::protocol::TType _etype1073;
-             xfer += iprot->readListBegin(_etype1073, _size1070);
-             this->group_names.resize(_size1070);
-             uint32_t _i1074;
-             for (_i1074 = 0; _i1074 < _size1070; ++_i1074)
 -            uint32_t _size1059;
 -            ::apache::thrift::protocol::TType _etype1062;
 -            xfer += iprot->readListBegin(_etype1062, _size1059);
 -            this->group_names.resize(_size1059);
 -            uint32_t _i1063;
 -            for (_i1063 = 0; _i1063 < _size1059; ++_i1063)
++            uint32_t _size1081;
++            ::apache::thrift::protocol::TType _etype1084;
++            xfer += iprot->readListBegin(_etype1084, _size1081);
++            this->group_names.resize(_size1081);
++            uint32_t _i1085;
++            for (_i1085 = 0; _i1085 < _size1081; ++_i1085)
              {
-               xfer += iprot->readString(this->group_names[_i1074]);
 -              xfer += iprot->readString(this->group_names[_i1063]);
++              xfer += iprot->readString(this->group_names[_i1085]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -14333,10 -14604,10 +14604,10 @@@ uint32_t ThriftHiveMetastore_get_partit
    xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-     std::vector<std::string> ::const_iterator _iter1075;
-     for (_iter1075 = this->group_names.begin(); _iter1075 != this->group_names.end(); ++_iter1075)
 -    std::vector<std::string> ::const_iterator _iter1064;
 -    for (_iter1064 = this->group_names.begin(); _iter1064 != this->group_names.end(); ++_iter1064)
++    std::vector<std::string> ::const_iterator _iter1086;
++    for (_iter1086 = this->group_names.begin(); _iter1086 != this->group_names.end(); ++_iter1086)
      {
-       xfer += oprot->writeString((*_iter1075));
 -      xfer += oprot->writeString((*_iter1064));
++      xfer += oprot->writeString((*_iter1086));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -14376,10 -14647,10 +14647,10 @@@ uint32_t ThriftHiveMetastore_get_partit
    xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-     std::vector<std::string> ::const_iterator _iter1076;
-     for (_iter1076 = (*(this->group_names)).begin(); _iter1076 != (*(this->group_names)).end(); ++_iter1076)
 -    std::vector<std::string> ::const_iterator _iter1065;
 -    for (_iter1065 = (*(this->group_names)).begin(); _iter1065 != (*(this->group_names)).end(); ++_iter1065)
++    std::vector<std::string> ::const_iterator _iter1087;
++    for (_iter1087 = (*(this->group_names)).begin(); _iter1087 != (*(this->group_names)).end(); ++_iter1087)
      {
-       xfer += oprot->writeString((*_iter1076));
 -      xfer += oprot->writeString((*_iter1065));
++      xfer += oprot->writeString((*_iter1087));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -14420,14 -14691,14 +14691,14 @@@ uint32_t ThriftHiveMetastore_get_partit
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1077;
-             ::apache::thrift::protocol::TType _etype1080;
-             xfer += iprot->readListBegin(_etype1080, _size1077);
-             this->success.resize(_size1077);
-             uint32_t _i1081;
-             for (_i1081 = 0; _i1081 < _size1077; ++_i1081)
 -            uint32_t _size1066;
 -            ::apache::thrift::protocol::TType _etype1069;
 -            xfer += iprot->readListBegin(_etype1069, _size1066);
 -            this->success.resize(_size1066);
 -            uint32_t _i1070;
 -            for (_i1070 = 0; _i1070 < _size1066; ++_i1070)
++            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)
              {
-               xfer += this->success[_i1081].read(iprot);
 -              xfer += this->success[_i1070].read(iprot);
++              xfer += this->success[_i1092].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -14474,10 -14745,10 +14745,10 @@@ uint32_t ThriftHiveMetastore_get_partit
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-       std::vector<Partition> ::const_iterator _iter1082;
-       for (_iter1082 = this->success.begin(); _iter1082 != this->success.end(); ++_iter1082)
 -      std::vector<Partition> ::const_iterator _iter1071;
 -      for (_iter1071 = this->success.begin(); _iter1071 != this->success.end(); ++_iter1071)
++      std::vector<Partition> ::const_iterator _iter1093;
++      for (_iter1093 = this->success.begin(); _iter1093 != this->success.end(); ++_iter1093)
        {
-         xfer += (*_iter1082).write(oprot);
 -        xfer += (*_iter1071).write(oprot);
++        xfer += (*_iter1093).write(oprot);
        }
        xfer += oprot->writeListEnd();
      }
@@@ -14526,14 -14797,14 +14797,14 @@@ uint32_t ThriftHiveMetastore_get_partit
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1083;
-             ::apache::thrift::protocol::TType _etype1086;
-             xfer += iprot->readListBegin(_etype1086, _size1083);
-             (*(this->success)).resize(_size1083);
-             uint32_t _i1087;
-             for (_i1087 = 0; _i1087 < _size1083; ++_i1087)
 -            uint32_t _size1072;
 -            ::apache::thrift::protocol::TType _etype1075;
 -            xfer += iprot->readListBegin(_etype1075, _size1072);
 -            (*(this->success)).resize(_size1072);
 -            uint32_t _i1076;
 -            for (_i1076 = 0; _i1076 < _size1072; ++_i1076)
++            uint32_t _size1094;
++            ::apache::thrift::protocol::TType _etype1097;
++            xfer += iprot->readListBegin(_etype1097, _size1094);
++            (*(this->success)).resize(_size1094);
++            uint32_t _i1098;
++            for (_i1098 = 0; _i1098 < _size1094; ++_i1098)
              {
-               xfer += (*(this->success))[_i1087].read(iprot);
 -              xfer += (*(this->success))[_i1076].read(iprot);
++              xfer += (*(this->success))[_i1098].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -14711,14 -14982,14 +14982,14 @@@ uint32_t ThriftHiveMetastore_get_partit
          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 _size1077;
 -            ::apache::thrift::protocol::TType _etype1080;
 -            xfer += iprot->readListBegin(_etype1080, _size1077);
 -            this->success.resize(_size1077);
 -            uint32_t _i1081;
 -            for (_i1081 = 0; _i1081 < _size1077; ++_i1081)
++            uint32_t _size1099;
++            ::apache::thrift::protocol::TType _etype1102;
++            xfer += iprot->readListBegin(_etype1102, _size1099);
++            this->success.resize(_size1099);
++            uint32_t _i1103;
++            for (_i1103 = 0; _i1103 < _size1099; ++_i1103)
              {
-               xfer += this->success[_i1092].read(iprot);
 -              xfer += this->success[_i1081].read(iprot);
++              xfer += this->success[_i1103].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -14765,10 -15036,10 +15036,10 @@@ uint32_t ThriftHiveMetastore_get_partit
      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<PartitionSpec> ::const_iterator _iter1093;
-       for (_iter1093 = this->success.begin(); _iter1093 != this->success.end(); ++_iter1093)
 -      std::vector<PartitionSpec> ::const_iterator _iter1082;
 -      for (_iter1082 = this->success.begin(); _iter1082 != this->success.end(); ++_iter1082)
++      std::vector<PartitionSpec> ::const_iterator _iter1104;
++      for (_iter1104 = this->success.begin(); _iter1104 != this->success.end(); ++_iter1104)
        {
-         xfer += (*_iter1093).write(oprot);
 -        xfer += (*_iter1082).write(oprot);
++        xfer += (*_iter1104).write(oprot);
        }
        xfer += oprot->writeListEnd();
      }
@@@ -14817,14 -15088,14 +15088,14 @@@ uint32_t ThriftHiveMetastore_get_partit
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1094;
-             ::apache::thrift::protocol::TType _etype1097;
-             xfer += iprot->readListBegin(_etype1097, _size1094);
-             (*(this->success)).resize(_size1094);
-             uint32_t _i1098;
-             for (_i1098 = 0; _i1098 < _size1094; ++_i1098)
 -            uint32_t _size1083;
 -            ::apache::thrift::protocol::TType _etype1086;
 -            xfer += iprot->readListBegin(_etype1086, _size1083);
 -            (*(this->success)).resize(_size1083);
 -            uint32_t _i1087;
 -            for (_i1087 = 0; _i1087 < _size1083; ++_i1087)
++            uint32_t _size1105;
++            ::apache::thrift::protocol::TType _etype1108;
++            xfer += iprot->readListBegin(_etype1108, _size1105);
++            (*(this->success)).resize(_size1105);
++            uint32_t _i1109;
++            for (_i1109 = 0; _i1109 < _size1105; ++_i1109)
              {
-               xfer += (*(this->success))[_i1098].read(iprot);
 -              xfer += (*(this->success))[_i1087].read(iprot);
++              xfer += (*(this->success))[_i1109].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -15002,14 -15273,14 +15273,14 @@@ uint32_t ThriftHiveMetastore_get_partit
          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 _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 _size1110;
++            ::apache::thrift::protocol::TType _etype1113;
++            xfer += iprot->readListBegin(_etype1113, _size1110);
++            this->success.resize(_size1110);
++            uint32_t _i1114;
++            for (_i1114 = 0; _i1114 < _size1110; ++_i1114)
              {
-               xfer += iprot->readString(this->success[_i1103]);
 -              xfer += iprot->readString(this->success[_i1092]);
++              xfer += iprot->readString(this->success[_i1114]);
              }
              x

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------


[14/15] hive git commit: HIVE-14671 : merge master into hive-14535 (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-14671 : merge master into hive-14535 (Sergey Shelukhin)

Conflicts:
	metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
	metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java


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

Branch: refs/heads/hive-14535
Commit: 754443e6ed654da6479575346aae22c5bd1ae7cd
Parents: ad3df23 efe9c84
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Oct 4 11:43:56 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Oct 4 11:43:56 2016 -0700

----------------------------------------------------------------------
 metastore/if/hive_metastore.thrift              |    1 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2163 +++++++-----
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  148 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../hive/metastore/api/ThriftHiveMetastore.java | 3184 ++++++++++++------
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1341 +++++---
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       |  938 ++++--
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   65 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   23 +
 .../hive/metastore/HiveMetaStoreClient.java     |   12 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |   15 +
 .../hadoop/hive/metastore/ObjectStore.java      |    8 +
 .../apache/hadoop/hive/metastore/RawStore.java  |    4 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |    6 +
 .../DummyRawStoreControlledCommit.java          |    6 +
 .../DummyRawStoreForJdoConnection.java          |    6 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   45 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |    1 +
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   46 +-
 .../reloperators/HiveTableFunctionScan.java     |   77 +
 .../calcite/translator/ASTConverter.java        |   83 +-
 .../translator/SqlFunctionConverter.java        |    9 +
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |  211 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   43 +
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |    1 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |    2 +
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |    2 +
 .../org/apache/hadoop/hive/ql/plan/DDLWork.java |    2 +-
 .../hadoop/hive/ql/plan/HiveOperation.java      |    1 +
 .../hadoop/hive/ql/plan/ShowTablesDesc.java     |   29 +
 .../authorization/plugin/HiveOperationType.java |    1 +
 .../plugin/sqlstd/Operation2Privilege.java      |    2 +
 .../udf/generic/GenericUDTFReplicateRows.java   |   88 +
 ql/src/test/queries/clientpositive/show_views.q |   56 +
 .../clientpositive/udtf_replicate_rows.q        |   23 +
 .../clientpositive/allcolref_in_udf.q.out       |   80 +-
 .../clientpositive/lateral_view_noalias.q.out   |   12 +-
 .../test/results/clientpositive/ppd_udtf.q.out  |    4 +-
 .../results/clientpositive/show_functions.q.out |    1 +
 .../results/clientpositive/show_views.q.out     |  320 ++
 .../results/clientpositive/udf_inline.q.out     |   12 +-
 .../results/clientpositive/udtf_explode.q.out   |   76 +-
 .../clientpositive/udtf_json_tuple.q.out        |   12 +-
 .../clientpositive/udtf_parse_url_tuple.q.out   |   12 +-
 .../clientpositive/udtf_replicate_rows.q.out    |  107 +
 46 files changed, 6472 insertions(+), 2818 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------


[06/15] hive git commit: HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index 6498eb1..525408b 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -47,6 +47,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) = 0;
   virtual void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) = 0;
   virtual void get_tables(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern) = 0;
+  virtual void get_tables_by_type(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType) = 0;
   virtual void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types) = 0;
   virtual void get_all_tables(std::vector<std::string> & _return, const std::string& db_name) = 0;
   virtual void get_table(Table& _return, const std::string& dbname, const std::string& tbl_name) = 0;
@@ -277,6 +278,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void get_tables(std::vector<std::string> & /* _return */, const std::string& /* db_name */, const std::string& /* pattern */) {
     return;
   }
+  void get_tables_by_type(std::vector<std::string> & /* _return */, const std::string& /* db_name */, const std::string& /* pattern */, const std::string& /* tableType */) {
+    return;
+  }
   void get_table_meta(std::vector<TableMeta> & /* _return */, const std::string& /* db_patterns */, const std::string& /* tbl_patterns */, const std::vector<std::string> & /* tbl_types */) {
     return;
   }
@@ -3758,6 +3762,132 @@ class ThriftHiveMetastore_get_tables_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_get_tables_by_type_args__isset {
+  _ThriftHiveMetastore_get_tables_by_type_args__isset() : db_name(false), pattern(false), tableType(false) {}
+  bool db_name :1;
+  bool pattern :1;
+  bool tableType :1;
+} _ThriftHiveMetastore_get_tables_by_type_args__isset;
+
+class ThriftHiveMetastore_get_tables_by_type_args {
+ public:
+
+  ThriftHiveMetastore_get_tables_by_type_args(const ThriftHiveMetastore_get_tables_by_type_args&);
+  ThriftHiveMetastore_get_tables_by_type_args& operator=(const ThriftHiveMetastore_get_tables_by_type_args&);
+  ThriftHiveMetastore_get_tables_by_type_args() : db_name(), pattern(), tableType() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_tables_by_type_args() throw();
+  std::string db_name;
+  std::string pattern;
+  std::string tableType;
+
+  _ThriftHiveMetastore_get_tables_by_type_args__isset __isset;
+
+  void __set_db_name(const std::string& val);
+
+  void __set_pattern(const std::string& val);
+
+  void __set_tableType(const std::string& val);
+
+  bool operator == (const ThriftHiveMetastore_get_tables_by_type_args & rhs) const
+  {
+    if (!(db_name == rhs.db_name))
+      return false;
+    if (!(pattern == rhs.pattern))
+      return false;
+    if (!(tableType == rhs.tableType))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_tables_by_type_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_tables_by_type_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_tables_by_type_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_tables_by_type_pargs() throw();
+  const std::string* db_name;
+  const std::string* pattern;
+  const std::string* tableType;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_tables_by_type_result__isset {
+  _ThriftHiveMetastore_get_tables_by_type_result__isset() : success(false), o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_get_tables_by_type_result__isset;
+
+class ThriftHiveMetastore_get_tables_by_type_result {
+ public:
+
+  ThriftHiveMetastore_get_tables_by_type_result(const ThriftHiveMetastore_get_tables_by_type_result&);
+  ThriftHiveMetastore_get_tables_by_type_result& operator=(const ThriftHiveMetastore_get_tables_by_type_result&);
+  ThriftHiveMetastore_get_tables_by_type_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_tables_by_type_result() throw();
+  std::vector<std::string>  success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_get_tables_by_type_result__isset __isset;
+
+  void __set_success(const std::vector<std::string> & val);
+
+  void __set_o1(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_tables_by_type_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_tables_by_type_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_tables_by_type_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_tables_by_type_presult__isset {
+  _ThriftHiveMetastore_get_tables_by_type_presult__isset() : success(false), o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_get_tables_by_type_presult__isset;
+
+class ThriftHiveMetastore_get_tables_by_type_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_tables_by_type_presult() throw();
+  std::vector<std::string> * success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_get_tables_by_type_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_get_table_meta_args__isset {
   _ThriftHiveMetastore_get_table_meta_args__isset() : db_patterns(false), tbl_patterns(false), tbl_types(false) {}
   bool db_patterns :1;
@@ -19217,6 +19347,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void get_tables(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern);
   void send_get_tables(const std::string& db_name, const std::string& pattern);
   void recv_get_tables(std::vector<std::string> & _return);
+  void get_tables_by_type(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType);
+  void send_get_tables_by_type(const std::string& db_name, const std::string& pattern, const std::string& tableType);
+  void recv_get_tables_by_type(std::vector<std::string> & _return);
   void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
   void send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
   void recv_get_table_meta(std::vector<TableMeta> & _return);
@@ -19624,6 +19757,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_drop_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_drop_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_get_tables_by_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_table_meta(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_all_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -19777,6 +19911,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["drop_table"] = &ThriftHiveMetastoreProcessor::process_drop_table;
     processMap_["drop_table_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_drop_table_with_environment_context;
     processMap_["get_tables"] = &ThriftHiveMetastoreProcessor::process_get_tables;
+    processMap_["get_tables_by_type"] = &ThriftHiveMetastoreProcessor::process_get_tables_by_type;
     processMap_["get_table_meta"] = &ThriftHiveMetastoreProcessor::process_get_table_meta;
     processMap_["get_all_tables"] = &ThriftHiveMetastoreProcessor::process_get_all_tables;
     processMap_["get_table"] = &ThriftHiveMetastoreProcessor::process_get_table;
@@ -20170,6 +20305,16 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     return;
   }
 
+  void get_tables_by_type(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_tables_by_type(_return, db_name, pattern, tableType);
+    }
+    ifaces_[i]->get_tables_by_type(_return, db_name, pattern, tableType);
+    return;
+  }
+
   void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -21452,6 +21597,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void get_tables(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern);
   int32_t send_get_tables(const std::string& db_name, const std::string& pattern);
   void recv_get_tables(std::vector<std::string> & _return, const int32_t seqid);
+  void get_tables_by_type(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType);
+  int32_t send_get_tables_by_type(const std::string& db_name, const std::string& pattern, const std::string& tableType);
+  void recv_get_tables_by_type(std::vector<std::string> & _return, const int32_t seqid);
   void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
   int32_t send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
   void recv_get_table_meta(std::vector<TableMeta> & _return, const int32_t seqid);

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index 3330b2f..d66fdbe 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -147,6 +147,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("get_tables\n");
   }
 
+  void get_tables_by_type(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType) {
+    // Your implementation goes here
+    printf("get_tables_by_type\n");
+  }
+
   void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types) {
     // Your implementation goes here
     printf("get_table_meta\n");


[11/15] hive git commit: HIVE-14671 : merge master into hive-14535 (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 17a8fa3,24b3ba1..93e0bc8
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@@ -15537,6 -15377,253 +15601,253 @@@ class ThriftHiveMetastore_get_tables_re
  
  }
  
+ class ThriftHiveMetastore_get_tables_by_type_args {
+   static $_TSPEC;
+ 
+   /**
+    * @var string
+    */
+   public $db_name = null;
+   /**
+    * @var string
+    */
+   public $pattern = null;
+   /**
+    * @var string
+    */
+   public $tableType = null;
+ 
+   public function __construct($vals=null) {
+     if (!isset(self::$_TSPEC)) {
+       self::$_TSPEC = array(
+         1 => array(
+           'var' => 'db_name',
+           'type' => TType::STRING,
+           ),
+         2 => array(
+           'var' => 'pattern',
+           'type' => TType::STRING,
+           ),
+         3 => array(
+           'var' => 'tableType',
+           'type' => TType::STRING,
+           ),
+         );
+     }
+     if (is_array($vals)) {
+       if (isset($vals['db_name'])) {
+         $this->db_name = $vals['db_name'];
+       }
+       if (isset($vals['pattern'])) {
+         $this->pattern = $vals['pattern'];
+       }
+       if (isset($vals['tableType'])) {
+         $this->tableType = $vals['tableType'];
+       }
+     }
+   }
+ 
+   public function getName() {
+     return 'ThriftHiveMetastore_get_tables_by_type_args';
+   }
+ 
+   public function read($input)
+   {
+     $xfer = 0;
+     $fname = null;
+     $ftype = 0;
+     $fid = 0;
+     $xfer += $input->readStructBegin($fname);
+     while (true)
+     {
+       $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+       if ($ftype == TType::STOP) {
+         break;
+       }
+       switch ($fid)
+       {
+         case 1:
+           if ($ftype == TType::STRING) {
+             $xfer += $input->readString($this->db_name);
+           } else {
+             $xfer += $input->skip($ftype);
+           }
+           break;
+         case 2:
+           if ($ftype == TType::STRING) {
+             $xfer += $input->readString($this->pattern);
+           } else {
+             $xfer += $input->skip($ftype);
+           }
+           break;
+         case 3:
+           if ($ftype == TType::STRING) {
+             $xfer += $input->readString($this->tableType);
+           } else {
+             $xfer += $input->skip($ftype);
+           }
+           break;
+         default:
+           $xfer += $input->skip($ftype);
+           break;
+       }
+       $xfer += $input->readFieldEnd();
+     }
+     $xfer += $input->readStructEnd();
+     return $xfer;
+   }
+ 
+   public function write($output) {
+     $xfer = 0;
+     $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_tables_by_type_args');
+     if ($this->db_name !== null) {
+       $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1);
+       $xfer += $output->writeString($this->db_name);
+       $xfer += $output->writeFieldEnd();
+     }
+     if ($this->pattern !== null) {
+       $xfer += $output->writeFieldBegin('pattern', TType::STRING, 2);
+       $xfer += $output->writeString($this->pattern);
+       $xfer += $output->writeFieldEnd();
+     }
+     if ($this->tableType !== null) {
+       $xfer += $output->writeFieldBegin('tableType', TType::STRING, 3);
+       $xfer += $output->writeString($this->tableType);
+       $xfer += $output->writeFieldEnd();
+     }
+     $xfer += $output->writeFieldStop();
+     $xfer += $output->writeStructEnd();
+     return $xfer;
+   }
+ 
+ }
+ 
+ class ThriftHiveMetastore_get_tables_by_type_result {
+   static $_TSPEC;
+ 
+   /**
+    * @var string[]
+    */
+   public $success = null;
+   /**
+    * @var \metastore\MetaException
+    */
+   public $o1 = null;
+ 
+   public function __construct($vals=null) {
+     if (!isset(self::$_TSPEC)) {
+       self::$_TSPEC = array(
+         0 => array(
+           'var' => 'success',
+           'type' => TType::LST,
+           'etype' => TType::STRING,
+           'elem' => array(
+             'type' => TType::STRING,
+             ),
+           ),
+         1 => array(
+           'var' => 'o1',
+           'type' => TType::STRUCT,
+           'class' => '\metastore\MetaException',
+           ),
+         );
+     }
+     if (is_array($vals)) {
+       if (isset($vals['success'])) {
+         $this->success = $vals['success'];
+       }
+       if (isset($vals['o1'])) {
+         $this->o1 = $vals['o1'];
+       }
+     }
+   }
+ 
+   public function getName() {
+     return 'ThriftHiveMetastore_get_tables_by_type_result';
+   }
+ 
+   public function read($input)
+   {
+     $xfer = 0;
+     $fname = null;
+     $ftype = 0;
+     $fid = 0;
+     $xfer += $input->readStructBegin($fname);
+     while (true)
+     {
+       $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+       if ($ftype == TType::STOP) {
+         break;
+       }
+       switch ($fid)
+       {
+         case 0:
+           if ($ftype == TType::LST) {
+             $this->success = array();
 -            $_size641 = 0;
 -            $_etype644 = 0;
 -            $xfer += $input->readListBegin($_etype644, $_size641);
 -            for ($_i645 = 0; $_i645 < $_size641; ++$_i645)
++            $_size648 = 0;
++            $_etype651 = 0;
++            $xfer += $input->readListBegin($_etype651, $_size648);
++            for ($_i652 = 0; $_i652 < $_size648; ++$_i652)
+             {
 -              $elem646 = null;
 -              $xfer += $input->readString($elem646);
 -              $this->success []= $elem646;
++              $elem653 = null;
++              $xfer += $input->readString($elem653);
++              $this->success []= $elem653;
+             }
+             $xfer += $input->readListEnd();
+           } else {
+             $xfer += $input->skip($ftype);
+           }
+           break;
+         case 1:
+           if ($ftype == TType::STRUCT) {
+             $this->o1 = new \metastore\MetaException();
+             $xfer += $this->o1->read($input);
+           } else {
+             $xfer += $input->skip($ftype);
+           }
+           break;
+         default:
+           $xfer += $input->skip($ftype);
+           break;
+       }
+       $xfer += $input->readFieldEnd();
+     }
+     $xfer += $input->readStructEnd();
+     return $xfer;
+   }
+ 
+   public function write($output) {
+     $xfer = 0;
+     $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_tables_by_type_result');
+     if ($this->success !== null) {
+       if (!is_array($this->success)) {
+         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+       }
+       $xfer += $output->writeFieldBegin('success', TType::LST, 0);
+       {
+         $output->writeListBegin(TType::STRING, count($this->success));
+         {
 -          foreach ($this->success as $iter647)
++          foreach ($this->success as $iter654)
+           {
 -            $xfer += $output->writeString($iter647);
++            $xfer += $output->writeString($iter654);
+           }
+         }
+         $output->writeListEnd();
+       }
+       $xfer += $output->writeFieldEnd();
+     }
+     if ($this->o1 !== null) {
+       $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+       $xfer += $this->o1->write($output);
+       $xfer += $output->writeFieldEnd();
+     }
+     $xfer += $output->writeFieldStop();
+     $xfer += $output->writeStructEnd();
+     return $xfer;
+   }
+ 
+ }
+ 
  class ThriftHiveMetastore_get_table_meta_args {
    static $_TSPEC;
  
@@@ -15623,14 -15710,14 +15934,14 @@@
          case 3:
            if ($ftype == TType::LST) {
              $this->tbl_types = array();
--            $_size648 = 0;
--            $_etype651 = 0;
--            $xfer += $input->readListBegin($_etype651, $_size648);
--            for ($_i652 = 0; $_i652 < $_size648; ++$_i652)
++            $_size655 = 0;
++            $_etype658 = 0;
++            $xfer += $input->readListBegin($_etype658, $_size655);
++            for ($_i659 = 0; $_i659 < $_size655; ++$_i659)
              {
--              $elem653 = null;
--              $xfer += $input->readString($elem653);
--              $this->tbl_types []= $elem653;
++              $elem660 = null;
++              $xfer += $input->readString($elem660);
++              $this->tbl_types []= $elem660;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -15668,9 -15755,9 +15979,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->tbl_types));
          {
--          foreach ($this->tbl_types as $iter654)
++          foreach ($this->tbl_types as $iter661)
            {
--            $xfer += $output->writeString($iter654);
++            $xfer += $output->writeString($iter661);
            }
          }
          $output->writeListEnd();
@@@ -15747,15 -15834,15 +16058,15 @@@ class ThriftHiveMetastore_get_table_met
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size655 = 0;
--            $_etype658 = 0;
--            $xfer += $input->readListBegin($_etype658, $_size655);
--            for ($_i659 = 0; $_i659 < $_size655; ++$_i659)
++            $_size662 = 0;
++            $_etype665 = 0;
++            $xfer += $input->readListBegin($_etype665, $_size662);
++            for ($_i666 = 0; $_i666 < $_size662; ++$_i666)
              {
--              $elem660 = null;
--              $elem660 = new \metastore\TableMeta();
--              $xfer += $elem660->read($input);
--              $this->success []= $elem660;
++              $elem667 = null;
++              $elem667 = new \metastore\TableMeta();
++              $xfer += $elem667->read($input);
++              $this->success []= $elem667;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -15791,9 -15878,9 +16102,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
--          foreach ($this->success as $iter661)
++          foreach ($this->success as $iter668)
            {
--            $xfer += $iter661->write($output);
++            $xfer += $iter668->write($output);
            }
          }
          $output->writeListEnd();
@@@ -15949,14 -16036,14 +16260,14 @@@ class ThriftHiveMetastore_get_all_table
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size662 = 0;
--            $_etype665 = 0;
--            $xfer += $input->readListBegin($_etype665, $_size662);
--            for ($_i666 = 0; $_i666 < $_size662; ++$_i666)
++            $_size669 = 0;
++            $_etype672 = 0;
++            $xfer += $input->readListBegin($_etype672, $_size669);
++            for ($_i673 = 0; $_i673 < $_size669; ++$_i673)
              {
--              $elem667 = null;
--              $xfer += $input->readString($elem667);
--              $this->success []= $elem667;
++              $elem674 = null;
++              $xfer += $input->readString($elem674);
++              $this->success []= $elem674;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -15992,9 -16079,9 +16303,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
--          foreach ($this->success as $iter668)
++          foreach ($this->success as $iter675)
            {
--            $xfer += $output->writeString($iter668);
++            $xfer += $output->writeString($iter675);
            }
          }
          $output->writeListEnd();
@@@ -16309,14 -16396,14 +16620,14 @@@ class ThriftHiveMetastore_get_table_obj
          case 2:
            if ($ftype == TType::LST) {
              $this->tbl_names = array();
--            $_size669 = 0;
--            $_etype672 = 0;
--            $xfer += $input->readListBegin($_etype672, $_size669);
--            for ($_i673 = 0; $_i673 < $_size669; ++$_i673)
++            $_size676 = 0;
++            $_etype679 = 0;
++            $xfer += $input->readListBegin($_etype679, $_size676);
++            for ($_i680 = 0; $_i680 < $_size676; ++$_i680)
              {
--              $elem674 = null;
--              $xfer += $input->readString($elem674);
--              $this->tbl_names []= $elem674;
++              $elem681 = null;
++              $xfer += $input->readString($elem681);
++              $this->tbl_names []= $elem681;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -16349,9 -16436,9 +16660,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->tbl_names));
          {
--          foreach ($this->tbl_names as $iter675)
++          foreach ($this->tbl_names as $iter682)
            {
--            $xfer += $output->writeString($iter675);
++            $xfer += $output->writeString($iter682);
            }
          }
          $output->writeListEnd();
@@@ -16452,15 -16539,15 +16763,15 @@@ class ThriftHiveMetastore_get_table_obj
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size676 = 0;
--            $_etype679 = 0;
--            $xfer += $input->readListBegin($_etype679, $_size676);
--            for ($_i680 = 0; $_i680 < $_size676; ++$_i680)
++            $_size683 = 0;
++            $_etype686 = 0;
++            $xfer += $input->readListBegin($_etype686, $_size683);
++            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
              {
--              $elem681 = null;
--              $elem681 = new \metastore\Table();
--              $xfer += $elem681->read($input);
--              $this->success []= $elem681;
++              $elem688 = null;
++              $elem688 = new \metastore\Table();
++              $xfer += $elem688->read($input);
++              $this->success []= $elem688;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -16512,9 -16599,9 +16823,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
--          foreach ($this->success as $iter682)
++          foreach ($this->success as $iter689)
            {
--            $xfer += $iter682->write($output);
++            $xfer += $iter689->write($output);
            }
          }
          $output->writeListEnd();
@@@ -16750,14 -16837,14 +17061,14 @@@ class ThriftHiveMetastore_get_table_nam
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size683 = 0;
--            $_etype686 = 0;
--            $xfer += $input->readListBegin($_etype686, $_size683);
--            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
++            $_size690 = 0;
++            $_etype693 = 0;
++            $xfer += $input->readListBegin($_etype693, $_size690);
++            for ($_i694 = 0; $_i694 < $_size690; ++$_i694)
              {
--              $elem688 = null;
--              $xfer += $input->readString($elem688);
--              $this->success []= $elem688;
++              $elem695 = null;
++              $xfer += $input->readString($elem695);
++              $this->success []= $elem695;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -16809,9 -16896,9 +17120,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
--          foreach ($this->success as $iter689)
++          foreach ($this->success as $iter696)
            {
--            $xfer += $output->writeString($iter689);
++            $xfer += $output->writeString($iter696);
            }
          }
          $output->writeListEnd();
@@@ -18124,15 -18211,15 +18435,15 @@@ class ThriftHiveMetastore_add_partition
          case 1:
            if ($ftype == TType::LST) {
              $this->new_parts = array();
--            $_size690 = 0;
--            $_etype693 = 0;
--            $xfer += $input->readListBegin($_etype693, $_size690);
--            for ($_i694 = 0; $_i694 < $_size690; ++$_i694)
++            $_size697 = 0;
++            $_etype700 = 0;
++            $xfer += $input->readListBegin($_etype700, $_size697);
++            for ($_i701 = 0; $_i701 < $_size697; ++$_i701)
              {
--              $elem695 = null;
--              $elem695 = new \metastore\Partition();
--              $xfer += $elem695->read($input);
--              $this->new_parts []= $elem695;
++              $elem702 = null;
++              $elem702 = new \metastore\Partition();
++              $xfer += $elem702->read($input);
++              $this->new_parts []= $elem702;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -18160,9 -18247,9 +18471,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->new_parts));
          {
--          foreach ($this->new_parts as $iter696)
++          foreach ($this->new_parts as $iter703)
            {
--            $xfer += $iter696->write($output);
++            $xfer += $iter703->write($output);
            }
          }
          $output->writeListEnd();
@@@ -18377,15 -18464,15 +18688,15 @@@ class ThriftHiveMetastore_add_partition
          case 1:
            if ($ftype == TType::LST) {
              $this->new_parts = array();
--            $_size697 = 0;
--            $_etype700 = 0;
--            $xfer += $input->readListBegin($_etype700, $_size697);
--            for ($_i701 = 0; $_i701 < $_size697; ++$_i701)
++            $_size704 = 0;
++            $_etype707 = 0;
++            $xfer += $input->readListBegin($_etype707, $_size704);
++            for ($_i708 = 0; $_i708 < $_size704; ++$_i708)
              {
--              $elem702 = null;
--              $elem702 = new \metastore\PartitionSpec();
--              $xfer += $elem702->read($input);
--              $this->new_parts []= $elem702;
++              $elem709 = null;
++              $elem709 = new \metastore\PartitionSpec();
++              $xfer += $elem709->read($input);
++              $this->new_parts []= $elem709;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -18413,9 -18500,9 +18724,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->new_parts));
          {
--          foreach ($this->new_parts as $iter703)
++          foreach ($this->new_parts as $iter710)
            {
--            $xfer += $iter703->write($output);
++            $xfer += $iter710->write($output);
            }
          }
          $output->writeListEnd();
@@@ -18665,14 -18752,14 +18976,14 @@@ class ThriftHiveMetastore_append_partit
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
--            $_size704 = 0;
--            $_etype707 = 0;
--            $xfer += $input->readListBegin($_etype707, $_size704);
--            for ($_i708 = 0; $_i708 < $_size704; ++$_i708)
++            $_size711 = 0;
++            $_etype714 = 0;
++            $xfer += $input->readListBegin($_etype714, $_size711);
++            for ($_i715 = 0; $_i715 < $_size711; ++$_i715)
              {
--              $elem709 = null;
--              $xfer += $input->readString($elem709);
--              $this->part_vals []= $elem709;
++              $elem716 = null;
++              $xfer += $input->readString($elem716);
++              $this->part_vals []= $elem716;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -18710,9 -18797,9 +19021,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
--          foreach ($this->part_vals as $iter710)
++          foreach ($this->part_vals as $iter717)
            {
--            $xfer += $output->writeString($iter710);
++            $xfer += $output->writeString($iter717);
            }
          }
          $output->writeListEnd();
@@@ -19214,14 -19301,14 +19525,14 @@@ class ThriftHiveMetastore_append_partit
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
--            $_size711 = 0;
--            $_etype714 = 0;
--            $xfer += $input->readListBegin($_etype714, $_size711);
--            for ($_i715 = 0; $_i715 < $_size711; ++$_i715)
++            $_size718 = 0;
++            $_etype721 = 0;
++            $xfer += $input->readListBegin($_etype721, $_size718);
++            for ($_i722 = 0; $_i722 < $_size718; ++$_i722)
              {
--              $elem716 = null;
--              $xfer += $input->readString($elem716);
--              $this->part_vals []= $elem716;
++              $elem723 = null;
++              $xfer += $input->readString($elem723);
++              $this->part_vals []= $elem723;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -19267,9 -19354,9 +19578,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
--          foreach ($this->part_vals as $iter717)
++          foreach ($this->part_vals as $iter724)
            {
--            $xfer += $output->writeString($iter717);
++            $xfer += $output->writeString($iter724);
            }
          }
          $output->writeListEnd();
@@@ -20123,14 -20210,14 +20434,14 @@@ class ThriftHiveMetastore_drop_partitio
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
--            $_size718 = 0;
--            $_etype721 = 0;
--            $xfer += $input->readListBegin($_etype721, $_size718);
--            for ($_i722 = 0; $_i722 < $_size718; ++$_i722)
++            $_size725 = 0;
++            $_etype728 = 0;
++            $xfer += $input->readListBegin($_etype728, $_size725);
++            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
              {
--              $elem723 = null;
--              $xfer += $input->readString($elem723);
--              $this->part_vals []= $elem723;
++              $elem730 = null;
++              $xfer += $input->readString($elem730);
++              $this->part_vals []= $elem730;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -20175,9 -20262,9 +20486,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
--          foreach ($this->part_vals as $iter724)
++          foreach ($this->part_vals as $iter731)
            {
--            $xfer += $output->writeString($iter724);
++            $xfer += $output->writeString($iter731);
            }
          }
          $output->writeListEnd();
@@@ -20430,14 -20517,14 +20741,14 @@@ class ThriftHiveMetastore_drop_partitio
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
--            $_size725 = 0;
--            $_etype728 = 0;
--            $xfer += $input->readListBegin($_etype728, $_size725);
--            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
++            $_size732 = 0;
++            $_etype735 = 0;
++            $xfer += $input->readListBegin($_etype735, $_size732);
++            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
              {
--              $elem730 = null;
--              $xfer += $input->readString($elem730);
--              $this->part_vals []= $elem730;
++              $elem737 = null;
++              $xfer += $input->readString($elem737);
++              $this->part_vals []= $elem737;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -20490,9 -20577,9 +20801,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
--          foreach ($this->part_vals as $iter731)
++          foreach ($this->part_vals as $iter738)
            {
--            $xfer += $output->writeString($iter731);
++            $xfer += $output->writeString($iter738);
            }
          }
          $output->writeListEnd();
@@@ -21506,14 -21593,14 +21817,14 @@@ class ThriftHiveMetastore_get_partition
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
--            $_size732 = 0;
--            $_etype735 = 0;
--            $xfer += $input->readListBegin($_etype735, $_size732);
--            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
++            $_size739 = 0;
++            $_etype742 = 0;
++            $xfer += $input->readListBegin($_etype742, $_size739);
++            for ($_i743 = 0; $_i743 < $_size739; ++$_i743)
              {
--              $elem737 = null;
--              $xfer += $input->readString($elem737);
--              $this->part_vals []= $elem737;
++              $elem744 = null;
++              $xfer += $input->readString($elem744);
++              $this->part_vals []= $elem744;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -21551,9 -21638,9 +21862,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
--          foreach ($this->part_vals as $iter738)
++          foreach ($this->part_vals as $iter745)
            {
--            $xfer += $output->writeString($iter738);
++            $xfer += $output->writeString($iter745);
            }
          }
          $output->writeListEnd();
@@@ -21795,17 -21882,17 +22106,17 @@@ class ThriftHiveMetastore_exchange_part
          case 1:
            if ($ftype == TType::MAP) {
              $this->partitionSpecs = array();
--            $_size739 = 0;
--            $_ktype740 = 0;
--            $_vtype741 = 0;
--            $xfer += $input->readMapBegin($_ktype740, $_vtype741, $_size739);
--            for ($_i743 = 0; $_i743 < $_size739; ++$_i743)
++            $_size746 = 0;
++            $_ktype747 = 0;
++            $_vtype748 = 0;
++            $xfer += $input->readMapBegin($_ktype747, $_vtype748, $_size746);
++            for ($_i750 = 0; $_i750 < $_size746; ++$_i750)
              {
--              $key744 = '';
--              $val745 = '';
--              $xfer += $input->readString($key744);
--              $xfer += $input->readString($val745);
--              $this->partitionSpecs[$key744] = $val745;
++              $key751 = '';
++              $val752 = '';
++              $xfer += $input->readString($key751);
++              $xfer += $input->readString($val752);
++              $this->partitionSpecs[$key751] = $val752;
              }
              $xfer += $input->readMapEnd();
            } else {
@@@ -21861,10 -21948,10 +22172,10 @@@
        {
          $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
          {
--          foreach ($this->partitionSpecs as $kiter746 => $viter747)
++          foreach ($this->partitionSpecs as $kiter753 => $viter754)
            {
--            $xfer += $output->writeString($kiter746);
--            $xfer += $output->writeString($viter747);
++            $xfer += $output->writeString($kiter753);
++            $xfer += $output->writeString($viter754);
            }
          }
          $output->writeMapEnd();
@@@ -22176,17 -22263,17 +22487,17 @@@ class ThriftHiveMetastore_exchange_part
          case 1:
            if ($ftype == TType::MAP) {
              $this->partitionSpecs = array();
--            $_size748 = 0;
--            $_ktype749 = 0;
--            $_vtype750 = 0;
--            $xfer += $input->readMapBegin($_ktype749, $_vtype750, $_size748);
--            for ($_i752 = 0; $_i752 < $_size748; ++$_i752)
++            $_size755 = 0;
++            $_ktype756 = 0;
++            $_vtype757 = 0;
++            $xfer += $input->readMapBegin($_ktype756, $_vtype757, $_size755);
++            for ($_i759 = 0; $_i759 < $_size755; ++$_i759)
              {
--              $key753 = '';
--              $val754 = '';
--              $xfer += $input->readString($key753);
--              $xfer += $input->readString($val754);
--              $this->partitionSpecs[$key753] = $val754;
++              $key760 = '';
++              $val761 = '';
++              $xfer += $input->readString($key760);
++              $xfer += $input->readString($val761);
++              $this->partitionSpecs[$key760] = $val761;
              }
              $xfer += $input->readMapEnd();
            } else {
@@@ -22242,10 -22329,10 +22553,10 @@@
        {
          $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
          {
--          foreach ($this->partitionSpecs as $kiter755 => $viter756)
++          foreach ($this->partitionSpecs as $kiter762 => $viter763)
            {
--            $xfer += $output->writeString($kiter755);
--            $xfer += $output->writeString($viter756);
++            $xfer += $output->writeString($kiter762);
++            $xfer += $output->writeString($viter763);
            }
          }
          $output->writeMapEnd();
@@@ -22378,15 -22465,15 +22689,15 @@@ class ThriftHiveMetastore_exchange_part
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size757 = 0;
--            $_etype760 = 0;
--            $xfer += $input->readListBegin($_etype760, $_size757);
--            for ($_i761 = 0; $_i761 < $_size757; ++$_i761)
++            $_size764 = 0;
++            $_etype767 = 0;
++            $xfer += $input->readListBegin($_etype767, $_size764);
++            for ($_i768 = 0; $_i768 < $_size764; ++$_i768)
              {
--              $elem762 = null;
--              $elem762 = new \metastore\Partition();
--              $xfer += $elem762->read($input);
--              $this->success []= $elem762;
++              $elem769 = null;
++              $elem769 = new \metastore\Partition();
++              $xfer += $elem769->read($input);
++              $this->success []= $elem769;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -22446,9 -22533,9 +22757,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
--          foreach ($this->success as $iter763)
++          foreach ($this->success as $iter770)
            {
--            $xfer += $iter763->write($output);
++            $xfer += $iter770->write($output);
            }
          }
          $output->writeListEnd();
@@@ -22594,14 -22681,14 +22905,14 @@@ class ThriftHiveMetastore_get_partition
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
--            $_size764 = 0;
--            $_etype767 = 0;
--            $xfer += $input->readListBegin($_etype767, $_size764);
--            for ($_i768 = 0; $_i768 < $_size764; ++$_i768)
++            $_size771 = 0;
++            $_etype774 = 0;
++            $xfer += $input->readListBegin($_etype774, $_size771);
++            for ($_i775 = 0; $_i775 < $_size771; ++$_i775)
              {
--              $elem769 = null;
--              $xfer += $input->readString($elem769);
--              $this->part_vals []= $elem769;
++              $elem776 = null;
++              $xfer += $input->readString($elem776);
++              $this->part_vals []= $elem776;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -22618,14 -22705,14 +22929,14 @@@
          case 5:
            if ($ftype == TType::LST) {
              $this->group_names = array();
--            $_size770 = 0;
--            $_etype773 = 0;
--            $xfer += $input->readListBegin($_etype773, $_size770);
--            for ($_i774 = 0; $_i774 < $_size770; ++$_i774)
++            $_size777 = 0;
++            $_etype780 = 0;
++            $xfer += $input->readListBegin($_etype780, $_size777);
++            for ($_i781 = 0; $_i781 < $_size777; ++$_i781)
              {
--              $elem775 = null;
--              $xfer += $input->readString($elem775);
--              $this->group_names []= $elem775;
++              $elem782 = null;
++              $xfer += $input->readString($elem782);
++              $this->group_names []= $elem782;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -22663,9 -22750,9 +22974,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
--          foreach ($this->part_vals as $iter776)
++          foreach ($this->part_vals as $iter783)
            {
--            $xfer += $output->writeString($iter776);
++            $xfer += $output->writeString($iter783);
            }
          }
          $output->writeListEnd();
@@@ -22685,9 -22772,9 +22996,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->group_names));
          {
--          foreach ($this->group_names as $iter777)
++          foreach ($this->group_names as $iter784)
            {
--            $xfer += $output->writeString($iter777);
++            $xfer += $output->writeString($iter784);
            }
          }
          $output->writeListEnd();
@@@ -23278,15 -23365,15 +23589,15 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size778 = 0;
--            $_etype781 = 0;
--            $xfer += $input->readListBegin($_etype781, $_size778);
--            for ($_i782 = 0; $_i782 < $_size778; ++$_i782)
++            $_size785 = 0;
++            $_etype788 = 0;
++            $xfer += $input->readListBegin($_etype788, $_size785);
++            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
              {
--              $elem783 = null;
--              $elem783 = new \metastore\Partition();
--              $xfer += $elem783->read($input);
--              $this->success []= $elem783;
++              $elem790 = null;
++              $elem790 = new \metastore\Partition();
++              $xfer += $elem790->read($input);
++              $this->success []= $elem790;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -23330,9 -23417,9 +23641,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
--          foreach ($this->success as $iter784)
++          foreach ($this->success as $iter791)
            {
--            $xfer += $iter784->write($output);
++            $xfer += $iter791->write($output);
            }
          }
          $output->writeListEnd();
@@@ -23478,14 -23565,14 +23789,14 @@@ class ThriftHiveMetastore_get_partition
          case 5:
            if ($ftype == TType::LST) {
              $this->group_names = array();
--            $_size785 = 0;
--            $_etype788 = 0;
--            $xfer += $input->readListBegin($_etype788, $_size785);
--            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
++            $_size792 = 0;
++            $_etype795 = 0;
++            $xfer += $input->readListBegin($_etype795, $_size792);
++            for ($_i796 = 0; $_i796 < $_size792; ++$_i796)
              {
--              $elem790 = null;
--              $xfer += $input->readString($elem790);
--              $this->group_names []= $elem790;
++              $elem797 = null;
++              $xfer += $input->readString($elem797);
++              $this->group_names []= $elem797;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -23533,9 -23620,9 +23844,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->group_names));
          {
--          foreach ($this->group_names as $iter791)
++          foreach ($this->group_names as $iter798)
            {
--            $xfer += $output->writeString($iter791);
++            $xfer += $output->writeString($iter798);
            }
          }
          $output->writeListEnd();
@@@ -23624,15 -23711,15 +23935,15 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size792 = 0;
--            $_etype795 = 0;
--            $xfer += $input->readListBegin($_etype795, $_size792);
--            for ($_i796 = 0; $_i796 < $_size792; ++$_i796)
++            $_size799 = 0;
++            $_etype802 = 0;
++            $xfer += $input->readListBegin($_etype802, $_size799);
++            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
              {
--              $elem797 = null;
--              $elem797 = new \metastore\Partition();
--              $xfer += $elem797->read($input);
--              $this->success []= $elem797;
++              $elem804 = null;
++              $elem804 = new \metastore\Partition();
++              $xfer += $elem804->read($input);
++              $this->success []= $elem804;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -23676,9 -23763,9 +23987,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
--          foreach ($this->success as $iter798)
++          foreach ($this->success as $iter805)
            {
--            $xfer += $iter798->write($output);
++            $xfer += $iter805->write($output);
            }
          }
          $output->writeListEnd();
@@@ -23898,15 -23985,15 +24209,15 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size799 = 0;
--            $_etype802 = 0;
--            $xfer += $input->readListBegin($_etype802, $_size799);
--            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
++            $_size806 = 0;
++            $_etype809 = 0;
++            $xfer += $input->readListBegin($_etype809, $_size806);
++            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
              {
--              $elem804 = null;
--              $elem804 = new \metastore\PartitionSpec();
--              $xfer += $elem804->read($input);
--              $this->success []= $elem804;
++              $elem811 = null;
++              $elem811 = new \metastore\PartitionSpec();
++              $xfer += $elem811->read($input);
++              $this->success []= $elem811;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -23950,9 -24037,9 +24261,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
--          foreach ($this->success as $iter805)
++          foreach ($this->success as $iter812)
            {
--            $xfer += $iter805->write($output);
++            $xfer += $iter812->write($output);
            }
          }
          $output->writeListEnd();
@@@ -24159,14 -24246,14 +24470,14 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size806 = 0;
--            $_etype809 = 0;
--            $xfer += $input->readListBegin($_etype809, $_size806);
--            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
++            $_size813 = 0;
++            $_etype816 = 0;
++            $xfer += $input->readListBegin($_etype816, $_size813);
++            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
              {
--              $elem811 = null;
--              $xfer += $input->readString($elem811);
--              $this->success []= $elem811;
++              $elem818 = null;
++              $xfer += $input->readString($elem818);
++              $this->success []= $elem818;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -24202,9 -24289,9 +24513,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
--          foreach ($this->success as $iter812)
++          foreach ($this->success as $iter819)
            {
--            $xfer += $output->writeString($iter812);
++            $xfer += $output->writeString($iter819);
            }
          }
          $output->writeListEnd();
@@@ -24320,14 -24407,14 +24631,14 @@@ class ThriftHiveMetastore_get_partition
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
--            $_size813 = 0;
--            $_etype816 = 0;
--            $xfer += $input->readListBegin($_etype816, $_size813);
--            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
++            $_size820 = 0;
++            $_etype823 = 0;
++            $xfer += $input->readListBegin($_etype823, $_size820);
++            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
              {
--              $elem818 = null;
--              $xfer += $input->readString($elem818);
--              $this->part_vals []= $elem818;
++              $elem825 = null;
++              $xfer += $input->readString($elem825);
++              $this->part_vals []= $elem825;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -24372,9 -24459,9 +24683,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
--          foreach ($this->part_vals as $iter819)
++          foreach ($this->part_vals as $iter826)
            {
--            $xfer += $output->writeString($iter819);
++            $xfer += $output->writeString($iter826);
            }
          }
          $output->writeListEnd();
@@@ -24468,15 -24555,15 +24779,15 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size820 = 0;
--            $_etype823 = 0;
--            $xfer += $input->readListBegin($_etype823, $_size820);
--            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
++            $_size827 = 0;
++            $_etype830 = 0;
++            $xfer += $input->readListBegin($_etype830, $_size827);
++            for ($_i831 = 0; $_i831 < $_size827; ++$_i831)
              {
--              $elem825 = null;
--              $elem825 = new \metastore\Partition();
--              $xfer += $elem825->read($input);
--              $this->success []= $elem825;
++              $elem832 = null;
++              $elem832 = new \metastore\Partition();
++              $xfer += $elem832->read($input);
++              $this->success []= $elem832;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -24520,9 -24607,9 +24831,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
--          foreach ($this->success as $iter826)
++          foreach ($this->success as $iter833)
            {
--            $xfer += $iter826->write($output);
++            $xfer += $iter833->write($output);
            }
          }
          $output->writeListEnd();
@@@ -24669,14 -24756,14 +24980,14 @@@ class ThriftHiveMetastore_get_partition
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
--            $_size827 = 0;
--            $_etype830 = 0;
--            $xfer += $input->readListBegin($_etype830, $_size827);
--            for ($_i831 = 0; $_i831 < $_size827; ++$_i831)
++            $_size834 = 0;
++            $_etype837 = 0;
++            $xfer += $input->readListBegin($_etype837, $_size834);
++            for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
              {
--              $elem832 = null;
--              $xfer += $input->readString($elem832);
--              $this->part_vals []= $elem832;
++              $elem839 = null;
++              $xfer += $input->readString($elem839);
++              $this->part_vals []= $elem839;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -24700,14 -24787,14 +25011,14 @@@
          case 6:
            if ($ftype == TType::LST) {
              $this->group_names = array();
--            $_size833 = 0;
--            $_etype836 = 0;
--            $xfer += $input->readListBegin($_etype836, $_size833);
--            for ($_i837 = 0; $_i837 < $_size833; ++$_i837)
++            $_size840 = 0;
++            $_etype843 = 0;
++            $xfer += $input->readListBegin($_etype843, $_size840);
++            for ($_i844 = 0; $_i844 < $_size840; ++$_i844)
              {
--              $elem838 = null;
--              $xfer += $input->readString($elem838);
--              $this->group_names []= $elem838;
++              $elem845 = null;
++              $xfer += $input->readString($elem845);
++              $this->group_names []= $elem845;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -24745,9 -24832,9 +25056,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
--          foreach ($this->part_vals as $iter839)
++          foreach ($this->part_vals as $iter846)
            {
--            $xfer += $output->writeString($iter839);
++            $xfer += $output->writeString($iter846);
            }
          }
          $output->writeListEnd();
@@@ -24772,9 -24859,9 +25083,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->group_names));
          {
--          foreach ($this->group_names as $iter840)
++          foreach ($this->group_names as $iter847)
            {
--            $xfer += $output->writeString($iter840);
++            $xfer += $output->writeString($iter847);
            }
          }
          $output->writeListEnd();
@@@ -24863,15 -24950,15 +25174,15 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size841 = 0;
--            $_etype844 = 0;
--            $xfer += $input->readListBegin($_etype844, $_size841);
--            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
++            $_size848 = 0;
++            $_etype851 = 0;
++            $xfer += $input->readListBegin($_etype851, $_size848);
++            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
              {
--              $elem846 = null;
--              $elem846 = new \metastore\Partition();
--              $xfer += $elem846->read($input);
--              $this->success []= $elem846;
++              $elem853 = null;
++              $elem853 = new \metastore\Partition();
++              $xfer += $elem853->read($input);
++              $this->success []= $elem853;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -24915,9 -25002,9 +25226,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
--          foreach ($this->success as $iter847)
++          foreach ($this->success as $iter854)
            {
--            $xfer += $iter847->write($output);
++            $xfer += $iter854->write($output);
            }
          }
          $output->writeListEnd();
@@@ -25038,14 -25125,14 +25349,14 @@@ class ThriftHiveMetastore_get_partition
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
--            $_size848 = 0;
--            $_etype851 = 0;
--            $xfer += $input->readListBegin($_etype851, $_size848);
--            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
++            $_size855 = 0;
++            $_etype858 = 0;
++            $xfer += $input->readListBegin($_etype858, $_size855);
++            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
              {
--              $elem853 = null;
--              $xfer += $input->readString($elem853);
--              $this->part_vals []= $elem853;
++              $elem860 = null;
++              $xfer += $input->readString($elem860);
++              $this->part_vals []= $elem860;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -25090,9 -25177,9 +25401,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
--          foreach ($this->part_vals as $iter854)
++          foreach ($this->part_vals as $iter861)
            {
--            $xfer += $output->writeString($iter854);
++            $xfer += $output->writeString($iter861);
            }
          }
          $output->writeListEnd();
@@@ -25185,14 -25272,14 +25496,14 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size855 = 0;
--            $_etype858 = 0;
--            $xfer += $input->readListBegin($_etype858, $_size855);
--            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
++            $_size862 = 0;
++            $_etype865 = 0;
++            $xfer += $input->readListBegin($_etype865, $_size862);
++            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
              {
--              $elem860 = null;
--              $xfer += $input->readString($elem860);
--              $this->success []= $elem860;
++              $elem867 = null;
++              $xfer += $input->readString($elem867);
++              $this->success []= $elem867;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -25236,9 -25323,9 +25547,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
--          foreach ($this->success as $iter861)
++          foreach ($this->success as $iter868)
            {
--            $xfer += $output->writeString($iter861);
++            $xfer += $output->writeString($iter868);
            }
          }
          $output->writeListEnd();
@@@ -25481,15 -25568,15 +25792,15 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size862 = 0;
--            $_etype865 = 0;
--            $xfer += $input->readListBegin($_etype865, $_size862);
--            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
++            $_size869 = 0;
++            $_etype872 = 0;
++            $xfer += $input->readListBegin($_etype872, $_size869);
++            for ($_i873 = 0; $_i873 < $_size869; ++$_i873)
              {
--              $elem867 = null;
--              $elem867 = new \metastore\Partition();
--              $xfer += $elem867->read($input);
--              $this->success []= $elem867;
++              $elem874 = null;
++              $elem874 = new \metastore\Partition();
++              $xfer += $elem874->read($input);
++              $this->success []= $elem874;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -25533,9 -25620,9 +25844,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
--          foreach ($this->success as $iter868)
++          foreach ($this->success as $iter875)
            {
--            $xfer += $iter868->write($output);
++            $xfer += $iter875->write($output);
            }
          }
          $output->writeListEnd();
@@@ -25778,15 -25865,15 +26089,15 @@@ class ThriftHiveMetastore_get_part_spec
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size869 = 0;
--            $_etype872 = 0;
--            $xfer += $input->readListBegin($_etype872, $_size869);
--            for ($_i873 = 0; $_i873 < $_size869; ++$_i873)
++            $_size876 = 0;
++            $_etype879 = 0;
++            $xfer += $input->readListBegin($_etype879, $_size876);
++            for ($_i880 = 0; $_i880 < $_size876; ++$_i880)
              {
--              $elem874 = null;
--              $elem874 = new \metastore\PartitionSpec();
--              $xfer += $elem874->read($input);
--              $this->success []= $elem874;
++              $elem881 = null;
++              $elem881 = new \metastore\PartitionSpec();
++              $xfer += $elem881->read($input);
++              $this->success []= $elem881;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -25830,9 -25917,9 +26141,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
--          foreach ($this->success as $iter875)
++          foreach ($this->success as $iter882)
            {
--            $xfer += $iter875->write($output);
++            $xfer += $iter882->write($output);
            }
          }
          $output->writeListEnd();
@@@ -26398,14 -26485,14 +26709,14 @@@ class ThriftHiveMetastore_get_partition
          case 3:
            if ($ftype == TType::LST) {
              $this->names = array();
--            $_size876 = 0;
--            $_etype879 = 0;
--            $xfer += $input->readListBegin($_etype879, $_size876);
--            for ($_i880 = 0; $_i880 < $_size876; ++$_i880)
++            $_size883 = 0;
++            $_etype886 = 0;
++            $xfer += $input->readListBegin($_etype886, $_size883);
++            for ($_i887 = 0; $_i887 < $_size883; ++$_i887)
              {
--              $elem881 = null;
--              $xfer += $input->readString($elem881);
--              $this->names []= $elem881;
++              $elem888 = null;
++              $xfer += $input->readString($elem888);
++              $this->names []= $elem888;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -26443,9 -26530,9 +26754,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->names));
          {
--          foreach ($this->names as $iter882)
++          foreach ($this->names as $iter889)
            {
--            $xfer += $output->writeString($iter882);
++            $xfer += $output->writeString($iter889);
            }
          }
          $output->writeListEnd();
@@@ -26534,15 -26621,15 +26845,15 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size883 = 0;
--            $_etype886 = 0;
--            $xfer += $input->readListBegin($_etype886, $_size883);
--            for ($_i887 = 0; $_i887 < $_size883; ++$_i887)
++            $_size890 = 0;
++            $_etype893 = 0;
++            $xfer += $input->readListBegin($_etype893, $_size890);
++            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
              {
--              $elem888 = null;
--              $elem888 = new \metastore\Partition();
--              $xfer += $elem888->read($input);
--              $this->success []= $elem888;
++              $elem895 = null;
++              $elem895 = new \metastore\Partition();
++              $xfer += $elem895->read($input);
++              $this->success []= $elem895;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -26586,9 -26673,9 +26897,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
--          foreach ($this->success as $iter889)
++          foreach ($this->success as $iter896)
            {
--            $xfer += $iter889->write($output);
++            $xfer += $iter896->write($output);
            }
          }
          $output->writeListEnd();
@@@ -26927,15 -27014,15 +27238,15 @@@ class ThriftHiveMetastore_alter_partiti
          case 3:
            if ($ftype == TType::LST) {
              $this->new_parts = array();
--            $_size890 = 0;
--            $_etype893 = 0;
--            $xfer += $input->readListBegin($_etype893, $_size890);
--            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
++            $_size897 = 0;
++            $_etype900 = 0;
++            $xfer += $input->readListBegin($_etype900, $_size897);
++            for ($_i901 = 0; $_i901 < $_size897; ++$_i901)
              {
--              $elem895 = null;
--              $elem895 = new \metastore\Partition();
--              $xfer += $elem895->read($input);
--              $this->new_parts []= $elem895;
++              $elem902 = null;
++              $elem902 = new \metastore\Partition();
++              $xfer += $elem902->read($input);
++              $this->new_parts []= $elem902;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -26973,9 -27060,9 +27284,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->new_parts));
          {
--          foreach ($this->new_parts as $iter896)
++          foreach ($this->new_parts as $iter903)
            {
--            $xfer += $iter896->write($output);
++            $xfer += $iter903->write($output);
            }
          }
          $output->writeListEnd();
@@@ -27190,15 -27277,15 +27501,15 @@@ class ThriftHiveMetastore_alter_partiti
          case 3:
            if ($ftype == TType::LST) {
              $this->new_parts = array();
--            $_size897 = 0;
--            $_etype900 = 0;
--            $xfer += $input->readListBegin($_etype900, $_size897);
--            for ($_i901 = 0; $_i901 < $_size897; ++$_i901)
++            $_size904 = 0;
++            $_etype907 = 0;
++            $xfer += $input->readListBegin($_etype907, $_size904);
++            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
              {
--              $elem902 = null;
--              $elem902 = new \metastore\Partition();
--              $xfer += $elem902->read($input);
--              $this->new_parts []= $elem902;
++              $elem909 = null;
++              $elem909 = new \metastore\Partition();
++              $xfer += $elem909->read($input);
++              $this->new_parts []= $elem909;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -27244,9 -27331,9 +27555,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->new_parts));
          {
--          foreach ($this->new_parts as $iter903)
++          foreach ($this->new_parts as $iter910)
            {
--            $xfer += $iter903->write($output);
++            $xfer += $iter910->write($output);
            }
          }
          $output->writeListEnd();
@@@ -27724,14 -27811,14 +28035,14 @@@ class ThriftHiveMetastore_rename_partit
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
--            $_size904 = 0;
--            $_etype907 = 0;
--            $xfer += $input->readListBegin($_etype907, $_size904);
--            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
++            $_size911 = 0;
++            $_etype914 = 0;
++            $xfer += $input->readListBegin($_etype914, $_size911);
++            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
              {
--              $elem909 = null;
--              $xfer += $input->readString($elem909);
--              $this->part_vals []= $elem909;
++              $elem916 = null;
++              $xfer += $input->readString($elem916);
++              $this->part_vals []= $elem916;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -27777,9 -27864,9 +28088,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
--          foreach ($this->part_vals as $iter910)
++          foreach ($this->part_vals as $iter917)
            {
--            $xfer += $output->writeString($iter910);
++            $xfer += $output->writeString($iter917);
            }
          }
          $output->writeListEnd();
@@@ -27964,14 -28051,14 +28275,14 @@@ class ThriftHiveMetastore_partition_nam
          case 1:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
--            $_size911 = 0;
--            $_etype914 = 0;
--            $xfer += $input->readListBegin($_etype914, $_size911);
--            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
++            $_size918 = 0;
++            $_etype921 = 0;
++            $xfer += $input->readListBegin($_etype921, $_size918);
++            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
              {
--              $elem916 = null;
--              $xfer += $input->readString($elem916);
--              $this->part_vals []= $elem916;
++              $elem923 = null;
++              $xfer += $input->readString($elem923);
++              $this->part_vals []= $elem923;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -28006,9 -28093,9 +28317,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
--          foreach ($this->part_vals as $iter917)
++          foreach ($this->part_vals as $iter924)
            {
--            $xfer += $output->writeString($iter917);
++            $xfer += $output->writeString($iter924);
            }
          }
          $output->writeListEnd();
@@@ -28462,14 -28549,14 +28773,14 @@@ class ThriftHiveMetastore_partition_nam
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size918 = 0;
--            $_etype921 = 0;
--            $xfer += $input->readListBegin($_etype921, $_size918);
--            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
++            $_size925 = 0;
++            $_etype928 = 0;
++            $xfer += $input->readListBegin($_etype928, $_size925);
++            for ($_i929 = 0; $_i929 < $_size925; ++$_i929)
              {
--              $elem923 = null;
--              $xfer += $input->readString($elem923);
--              $this->success []= $elem923;
++              $elem930 = null;
++              $xfer += $input->readString($elem930);
++              $this->success []= $elem930;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -28505,9 -28592,9 +28816,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
--          foreach ($this->success as $iter924)
++          foreach ($this->success as $iter931)
            {
--            $xfer += $output->writeString($iter924);
++            $xfer += $output->writeString($iter931);
            }
          }
          $output->writeListEnd();
@@@ -28667,17 -28754,17 +28978,17 @@@ class ThriftHiveMetastore_partition_nam
          case 0:
            if ($ftype == TType::MAP) {
              $this->success = array();
--            $_size925 = 0;
--            $_ktype926 = 0;
--            $_vtype927 = 0;
--            $xfer += $input->readMapBegin($_ktype926, $_vtype927, $_size925);
--            for ($_i929 = 0; $_i929 < $_size925; ++$_i929)
++            $_size932 = 0;
++            $_ktype933 = 0;
++            $_vtype934 = 0;
++            $xfer += $input->readMapBegin($_ktype933, $_vtype934, $_size932);
++            for ($_i936 = 0; $_i936 < $_size932; ++$_i936)
              {
--              $key930 = '';
--              $val931 = '';
--              $xfer += $input->readString($key930);
--              $xfer += $input->readString($val931);
--              $this->success[$key930] = $val931;
++              $key937 = '';
++              $val938 = '';
++              $xfer += $input->readString($key937);
++              $xfer += $input->readString($val938);
++              $this->success[$key937] = $val938;
              }
              $xfer += $input->readMapEnd();
            } else {
@@@ -28713,10 -28800,10 +29024,10 @@@
        {
          $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
          {
--          foreach ($this->success as $kiter932 => $viter933)
++          foreach ($this->success as $kiter939 => $viter940)
            {
--            $xfer += $output->writeString($kiter932);
--            $xfer += $output->writeString($viter933);
++            $xfer += $output->writeString($kiter939);
++            $xfer += $output->writeString($viter940);
            }
          }
          $output->writeMapEnd();
@@@ -28836,17 -28923,17 +29147,17 @@@ class ThriftHiveMetastore_markPartition
          case 3:
            if ($ftype == TType::MAP) {
              $this->part_vals = array();
--            $_size934 = 0;
--            $_ktype935 = 0;
--            $_vtype936 = 0;
--            $xfer += $input->readMapBegin($_ktype935, $_vtype936, $_size934);
--            for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
++            $_size941 = 0;
++            $_ktype942 = 0;
++            $_vtype943 = 0;
++            $xfer += $input->readMapBegin($_ktype942, $_vtype943, $_size941);
++            for ($_i945 = 0; $_i945 < $_size941; ++$_i945)
              {
--              $key939 = '';
--              $val940 = '';
--              $xfer += $input->readString($key939);
--              $xfer += $input->readString($val940);
--              $this->part_vals[$key939] = $val940;
++              $key946 = '';
++              $val947 = '';
++              $xfer += $input->readString($key946);
++              $xfer += $input->readString($val947);
++              $this->part_vals[$key946] = $val947;
              }
              $xfer += $input->readMapEnd();
            } else {
@@@ -28891,10 -28978,10 +29202,10 @@@
        {
          $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
          {
--          foreach ($this->part_vals as $kiter941 => $viter942)
++          foreach ($this->part_vals as $kiter948 => $viter949)
            {
--            $xfer += $output->writeString($kiter941);
--            $xfer += $output->writeString($viter942);
++            $xfer += $output->writeString($kiter948);
++            $xfer += $output->writeString($viter949);
            }
          }
          $output->writeMapEnd();
@@@ -29216,17 -29303,17 +29527,17 @@@ class ThriftHiveMetastore_isPartitionMa
          case 3:
            if ($ftype == TType::MAP) {
              $this->part_vals = array();
--            $_size943 = 0;
--            $_ktype944 = 0;
--            $_vtype945 = 0;
--            $xfer += $input->readMapBegin($_ktype944, $_vtype945, $_size943);
--            for ($_i947 = 0; $_i947 < $_size943; ++$_i947)
++            $_size950 = 0;
++            $_ktype951 = 0;
++            $_vtype952 = 0;
++            $xfer += $input->readMapBegin($_ktype951, $_vtype952, $_size950);
++            for ($_i954 = 0; $_i954 < $_size950; ++$_i954)
              {
--              $key948 = '';
--              $val949 = '';
--              $xfer += $input->readString($key948);
--              $xfer += $input->readString($val949);
--              $this->part_vals[$key948] = $val949;
++              $key955 = '';
++              $val956 = '';
++              $xfer += $input->readString($key955);
++              $xfer += $input->readString($val956);
++              $this->part_vals[$key955] = $val956;
              }
              $xfer += $input->readMapEnd();
            } else {
@@@ -29271,10 -29358,10 +29582,10 @@@
        {
          $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
          {
--          foreach ($this->part_vals as $kiter950 => $viter951)
++          foreach ($this->part_vals as $kiter957 => $viter958)
            {
--            $xfer += $output->writeString($kiter950);
--            $xfer += $output->writeString($viter951);
++            $xfer += $output->writeString($kiter957);
++            $xfer += $output->writeString($viter958);
            }
          }
          $output->writeMapEnd();
@@@ -30748,15 -30835,15 +31059,15 @@@ class ThriftHiveMetastore_get_indexes_r
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size952 = 0;
--            $_etype955 = 0;
--            $xfer += $input->readListBegin($_etype955, $_size952);
--            for ($_i956 = 0; $_i956 < $_size952; ++$_i956)
++            $_size959 = 0;
++            $_etype962 = 0;
++            $xfer += $input->readListBegin($_etype962, $_size959);
++            for ($_i963 = 0; $_i963 < $_size959; ++$_i963)
              {
--              $elem957 = null;
--              $elem957 = new \metastore\Index();
--              $xfer += $elem957->read($input);
--              $this->success []= $elem957;
++              $elem964 = null;
++              $elem964 = new \metastore\Index();
++              $xfer += $elem964->read($input);
++              $this->success []= $elem964;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -30800,9 -30887,9 +31111,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
--          foreach ($this->success as $iter958)
++          foreach ($this->success as $iter965)
            {
--            $xfer += $iter958->write($output);
++            $xfer += $iter965->write($output);
            }
          }
          $output->writeListEnd();
@@@ -31009,14 -31096,14 +31320,14 @@@ class ThriftHiveMetastore_get_index_nam
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size959 = 0;
--            $_etype962 = 0;
--            $xfer += $input->readListBegin($_etype962, $_size959);
--            for ($_i963 = 0; $_i963 < $_size959; ++$_i963)
++            $_size966 = 0;
++            $_etype969 = 0;
++            $xfer += $input->readListBegin($_etype969, $_size966);
++            for ($_i970 = 0; $_i970 < $_size966; ++$_i970)
              {
--              $elem964 = null;
--              $xfer += $input->readString($elem964);
--              $this->success []= $elem964;
++              $elem971 = null;
++              $xfer += $input->readString($elem971);
++              $this->success []= $elem971;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -31052,9 -31139,9 +31363,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
--          foreach ($this->success as $iter965)
++          foreach ($this->success as $iter972)
            {
--            $xfer += $output->writeString($iter965);
++            $xfer += $output->writeString($iter972);
            }
          }
          $output->writeListEnd();
@@@ -34948,14 -35035,14 +35259,14 @@@ class ThriftHiveMetastore_get_functions
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size966 = 0;
--            $_etype969 = 0;
--            $xfer += $input->readListBegin($_etype969, $_size966);
--            for ($_i970 = 0; $_i970 < $_size966; ++$_i970)
++            $_size973 = 0;
++            $_etype976 = 0;
++            $xfer += $input->readListBegin($_etype976, $_size973);
++            for ($_i977 = 0; $_i977 < $_size973; ++$_i977)
              {
--              $elem971 = null;
--              $xfer += $input->readString($elem971);
--              $this->success []= $elem971;
++              $elem978 = null;
++              $xfer += $input->readString($elem978);
++              $this->success []= $elem978;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -34991,9 -35078,9 +35302,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
--          foreach ($this->success as $iter972)
++          foreach ($this->success as $iter979)
            {
--            $xfer += $output->writeString($iter972);
++            $xfer += $output->writeString($iter979);
            }
          }
          $output->writeListEnd();
@@@ -35862,14 -35949,14 +36173,14 @@@ class ThriftHiveMetastore_get_role_name
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size973 = 0;
--            $_etype976 = 0;
--            $xfer += $input->readListBegin($_etype976, $_size973);
--            for ($_i977 = 0; $_i977 < $_size973; ++$_i977)
++            $_size980 = 0;
++            $_etype983 = 0;
++            $xfer += $input->readListBegin($_etype983, $_size980);
++            for ($_i984 = 0; $_i984 < $_size980; ++$_i984)
              {
--              $elem978 = null;
--              $xfer += $input->readString($elem978);
--              $this->success []= $elem978;
++              $elem985 = null;
++              $xfer += $input->readString($elem985);
++              $this->success []= $elem985;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -35905,9 -35992,9 +36216,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
--          foreach ($this->success as $iter979)
++          foreach ($this->success as $iter986)
            {
--            $xfer += $output->writeString($iter979);
++            $xfer += $output->writeString($iter986);
            }
          }
          $output->writeListEnd();
@@@ -36598,15 -36685,15 +36909,15 @@@ class ThriftHiveMetastore_list_roles_re
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size980 = 0;
--            $_etype983 = 0;
--            $xfer += $input->readListBegin($_etype983, $_size980);
--            for ($_i984 = 0; $_i984 < $_size980; ++$_i984)
++            $_size987 = 0;
++            $_etype990 = 0;
++            $xfer += $input->readListBegin($_etype990, $_size987);
++            for ($_i991 = 0; $_i991 < $_size987; ++$_i991)
              {
--              $elem985 = null;
--              $elem985 = new \metastore\Role();
--              $xfer += $elem985->read($input);
--              $this->success []= $elem985;
++              $elem992 = null;
++              $elem992 = new \metastore\Role();
++              $xfer += $elem992->read($input);
++              $this->success []= $elem992;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -36642,9 -36729,9 +36953,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
--          foreach ($this->success as $iter986)
++          foreach ($this->success as $iter993)
            {
--            $xfer += $iter986->write($output);
++            $xfer += $iter993->write($output);
            }
          }
          $output->writeListEnd();
@@@ -37306,14 -37393,14 +37617,14 @@@ class ThriftHiveMetastore_get_privilege
          case 3:
            if ($ftype == TType::LST) {
              $this->group_names = array();
--            $_size987 = 0;
--            $_etype990 = 0;
--            $xfer += $input->readListBegin($_etype990, $_size987);
--            for ($_i991 = 0; $_i991 < $_size987; ++$_i991)
++            $_size994 = 0;
++            $_etype997 = 0;
++            $xfer += $input->readListBegin($_etype997, $_size994);
++            for ($_i998 = 0; $_i998 < $_size994; ++$_i998)
              {
--              $elem992 = null;
--              $xfer += $input->readString($elem992);
--              $this->group_names []= $elem992;
++              $elem999 = null;
++              $xfer += $input->readString($elem999);
++              $this->group_names []= $elem999;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -37354,9 -37441,9 +37665,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->group_names));
          {
--          foreach ($this->group_names as $iter993)
++          foreach ($this->group_names as $iter1000)
            {
--            $xfer += $output->writeString($iter993);
++            $xfer += $output->writeString($iter1000);
            }
          }
          $output->writeListEnd();
@@@ -37664,15 -37751,15 +37975,15 @@@ class ThriftHiveMetastore_list_privileg
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size994 = 0;
--            $_etype997 = 0;
--            $xfer += $input->readListBegin($_etype997, $_size994);
--            for ($_i998 = 0; $_i998 < $_size994; ++$_i998)
++            $_size1001 = 0;
++            $_etype1004 = 0;
++            $xfer += $input->readListBegin($_etype1004, $_size1001);
++            for ($_i1005 = 0; $_i1005 < $_size1001; ++$_i1005)
              {
--              $elem999 = null;
--              $elem999 = new \metastore\HiveObjectPrivilege();
--              $xfer += $elem999->read($input);
--              $this->success []= $elem999;
++              $elem1006 = null;
++              $elem1006 = new \metastore\HiveObjectPrivilege();
++              $xfer += $elem1006->read($input);
++              $this->success []= $elem1006;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -37708,9 -37795,9 +38019,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
--          foreach ($this->success as $iter1000)
++          foreach ($this->success as $iter1007)
            {
--            $xfer += $iter1000->write($output);
++            $xfer += $iter1007->write($output);
            }
          }
          $output->writeListEnd();
@@@ -38342,14 -38429,14 +38653,14 @@@ class ThriftHiveMetastore_set_ugi_args 
          case 2:
            if ($ftype == TType::LST) {
              $this->group_names = array();
--            $_size1001 = 0;
--            $_etype1004 = 0;
--            $xfer += $input->readListBegin($_etype1004, $_size1001);
--            for ($_i1005 = 0; $_i1005 < $_size1001; ++$_i1005)
++            $_size1008 = 0;
++            $_etype1011 = 0;
++            $xfer += $input->readListBegin($_etype1011, $_size1008);
++            for ($_i1012 = 0; $_i1012 < $_size1008; ++$_i1012)
              {
--              $elem1006 = null;
--              $xfer += $input->readString($elem1006);
--              $this->group_names []= $elem1006;
++              $elem1013 = null;
++              $xfer += $input->readString($elem1013);
++              $this->group_names []= $elem1013;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -38382,9 -38469,9 +38693,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->group_names));
          {
--          foreach ($this->group_names as $iter1007)
++          foreach ($this->group_names as $iter1014)
            {
--            $xfer += $output->writeString($iter1007);
++            $xfer += $output->writeString($iter1014);
            }
          }
          $output->writeListEnd();
@@@ -38460,14 -38547,14 +38771,14 @@@ class ThriftHiveMetastore_set_ugi_resul
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size1008 = 0;
--            $_etype1011 = 0;
--            $xfer += $input->readListBegin($_etype1011, $_size1008);
--            for ($_i1012 = 0; $_i1012 < $_size1008; ++$_i1012)
++            $_size1015 = 0;
++            $_etype1018 = 0;
++            $xfer += $input->readListBegin($_etype1018, $_size1015);
++            for ($_i1019 = 0; $_i1019 < $_size1015; ++$_i1019)
              {
--              $elem1013 = null;
--              $xfer += $input->readString($elem1013);
--              $this->success []= $elem1013;
++              $elem1020 = null;
++              $xfer += $input->readString($elem1020);
++              $this->success []= $elem1020;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -38503,9 -38590,9 +38814,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
--          foreach ($this->success as $iter1014)
++          foreach ($this->success as $iter1021)
            {
--            $xfer += $output->writeString($iter1014);
++            $xfer += $output->writeString($iter1021);
            }
          }
          $output->writeListEnd();
@@@ -39622,14 -39709,14 +39933,14 @@@ class ThriftHiveMetastore_get_all_token
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size1015 = 0;
--            $_etype1018 = 0;
--            $xfer += $input->readListBegin($_etype1018, $_size1015);
--            for ($_i1019 = 0; $_i1019 < $_size1015; ++$_i1019)
++            $_size1022 = 0;
++            $_etype1025 = 0;
++            $xfer += $input->readListBegin($_etype1025, $_size1022);
++            for ($_i1026 = 0; $_i1026 < $_size1022; ++$_i1026)
              {
--              $elem1020 = null;
--              $xfer += $input->readString($elem1020);
--              $this->success []= $elem1020;
++              $elem1027 = null;
++              $xfer += $input->readString($elem1027);
++              $this->success []= $elem1027;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -39657,9 -39744,9 +39968,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
--          foreach ($this->success as $iter1021)
++          foreach ($this->success as $iter1028)
            {
--            $xfer += $output->writeString($iter1021);
++            $xfer += $output->writeString($iter1028);
            }
          }
          $output->writeListEnd();
@@@ -40298,14 -40385,14 +40609,14 @@@ class ThriftHiveMetastore_get_master_ke
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
--            $_size1022 = 0;
--            $_etype1025 = 0;
--            $xfer += $input->readListBegin($_etype1025, $_size1022);
--            for ($_i1026 = 0; $_i1026 < $_size1022; ++$_i1026)
++            $_size1029 = 0;
++            $_etype1032 = 0;
++            $xfer += $input->readListBegin($_etype1032, $_size1029);
++            for ($_i1033 = 0; $_i1033 < $_size1029; ++$_i1033)
              {
--              $elem1027 = null;
--              $xfer += $input->readString($elem1027);
--              $this->success []= $elem1027;
++              $elem1034 = null;
++              $xfer += $input->readString($elem1034);
++              $this->success []= $elem1034;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -40333,9 -40420,9 +40644,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
--          foreach ($this->success as $iter1028)
++          foreach ($this->success as $iter1035)
            {
--            $xfer += $output->writeString($iter1028);
++            $xfer += $output->writeString($iter1035);
            }
          }
          $output->writeListEnd();

http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------


[07/15] hive git commit: HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 5a35a50..b4a05b2 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -6186,6 +6186,277 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
 }
 
 
+ThriftHiveMetastore_get_tables_by_type_args::~ThriftHiveMetastore_get_tables_by_type_args() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_tables_by_type_args::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;
+
+
+  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->db_name);
+          this->__isset.db_name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->pattern);
+          this->__isset.pattern = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->tableType);
+          this->__isset.tableType = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_get_tables_by_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_args");
+
+  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->db_name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->pattern);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->tableType);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_tables_by_type_pargs::~ThriftHiveMetastore_get_tables_by_type_pargs() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_tables_by_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_pargs");
+
+  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->db_name)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString((*(this->pattern)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString((*(this->tableType)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_tables_by_type_result::~ThriftHiveMetastore_get_tables_by_type_result() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_tables_by_type_result::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;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 0:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->success.clear();
+            uint32_t _size887;
+            ::apache::thrift::protocol::TType _etype890;
+            xfer += iprot->readListBegin(_etype890, _size887);
+            this->success.resize(_size887);
+            uint32_t _i891;
+            for (_i891 = 0; _i891 < _size887; ++_i891)
+            {
+              xfer += iprot->readString(this->success[_i891]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_result");
+
+  if (this->__isset.success) {
+    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 _iter892;
+      for (_iter892 = this->success.begin(); _iter892 != this->success.end(); ++_iter892)
+      {
+        xfer += oprot->writeString((*_iter892));
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o1) {
+    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
+    xfer += this->o1.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_tables_by_type_presult::~ThriftHiveMetastore_get_tables_by_type_presult() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_tables_by_type_presult::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;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 0:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            (*(this->success)).clear();
+            uint32_t _size893;
+            ::apache::thrift::protocol::TType _etype896;
+            xfer += iprot->readListBegin(_etype896, _size893);
+            (*(this->success)).resize(_size893);
+            uint32_t _i897;
+            for (_i897 = 0; _i897 < _size893; ++_i897)
+            {
+              xfer += iprot->readString((*(this->success))[_i897]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+
 ThriftHiveMetastore_get_table_meta_args::~ThriftHiveMetastore_get_table_meta_args() throw() {
 }
 
@@ -6231,14 +6502,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_types.clear();
-            uint32_t _size887;
-            ::apache::thrift::protocol::TType _etype890;
-            xfer += iprot->readListBegin(_etype890, _size887);
-            this->tbl_types.resize(_size887);
-            uint32_t _i891;
-            for (_i891 = 0; _i891 < _size887; ++_i891)
+            uint32_t _size898;
+            ::apache::thrift::protocol::TType _etype901;
+            xfer += iprot->readListBegin(_etype901, _size898);
+            this->tbl_types.resize(_size898);
+            uint32_t _i902;
+            for (_i902 = 0; _i902 < _size898; ++_i902)
             {
-              xfer += iprot->readString(this->tbl_types[_i891]);
+              xfer += iprot->readString(this->tbl_types[_i902]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6275,10 +6546,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_types.size()));
-    std::vector<std::string> ::const_iterator _iter892;
-    for (_iter892 = this->tbl_types.begin(); _iter892 != this->tbl_types.end(); ++_iter892)
+    std::vector<std::string> ::const_iterator _iter903;
+    for (_iter903 = this->tbl_types.begin(); _iter903 != this->tbl_types.end(); ++_iter903)
     {
-      xfer += oprot->writeString((*_iter892));
+      xfer += oprot->writeString((*_iter903));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6310,10 +6581,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_types)).size()));
-    std::vector<std::string> ::const_iterator _iter893;
-    for (_iter893 = (*(this->tbl_types)).begin(); _iter893 != (*(this->tbl_types)).end(); ++_iter893)
+    std::vector<std::string> ::const_iterator _iter904;
+    for (_iter904 = (*(this->tbl_types)).begin(); _iter904 != (*(this->tbl_types)).end(); ++_iter904)
     {
-      xfer += oprot->writeString((*_iter893));
+      xfer += oprot->writeString((*_iter904));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6354,14 +6625,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size894;
-            ::apache::thrift::protocol::TType _etype897;
-            xfer += iprot->readListBegin(_etype897, _size894);
-            this->success.resize(_size894);
-            uint32_t _i898;
-            for (_i898 = 0; _i898 < _size894; ++_i898)
+            uint32_t _size905;
+            ::apache::thrift::protocol::TType _etype908;
+            xfer += iprot->readListBegin(_etype908, _size905);
+            this->success.resize(_size905);
+            uint32_t _i909;
+            for (_i909 = 0; _i909 < _size905; ++_i909)
             {
-              xfer += this->success[_i898].read(iprot);
+              xfer += this->success[_i909].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6400,10 +6671,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<TableMeta> ::const_iterator _iter899;
-      for (_iter899 = this->success.begin(); _iter899 != this->success.end(); ++_iter899)
+      std::vector<TableMeta> ::const_iterator _iter910;
+      for (_iter910 = this->success.begin(); _iter910 != this->success.end(); ++_iter910)
       {
-        xfer += (*_iter899).write(oprot);
+        xfer += (*_iter910).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6448,14 +6719,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size900;
-            ::apache::thrift::protocol::TType _etype903;
-            xfer += iprot->readListBegin(_etype903, _size900);
-            (*(this->success)).resize(_size900);
-            uint32_t _i904;
-            for (_i904 = 0; _i904 < _size900; ++_i904)
+            uint32_t _size911;
+            ::apache::thrift::protocol::TType _etype914;
+            xfer += iprot->readListBegin(_etype914, _size911);
+            (*(this->success)).resize(_size911);
+            uint32_t _i915;
+            for (_i915 = 0; _i915 < _size911; ++_i915)
             {
-              xfer += (*(this->success))[_i904].read(iprot);
+              xfer += (*(this->success))[_i915].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6593,14 +6864,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size905;
-            ::apache::thrift::protocol::TType _etype908;
-            xfer += iprot->readListBegin(_etype908, _size905);
-            this->success.resize(_size905);
-            uint32_t _i909;
-            for (_i909 = 0; _i909 < _size905; ++_i909)
+            uint32_t _size916;
+            ::apache::thrift::protocol::TType _etype919;
+            xfer += iprot->readListBegin(_etype919, _size916);
+            this->success.resize(_size916);
+            uint32_t _i920;
+            for (_i920 = 0; _i920 < _size916; ++_i920)
             {
-              xfer += iprot->readString(this->success[_i909]);
+              xfer += iprot->readString(this->success[_i920]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6639,10 +6910,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter910;
-      for (_iter910 = this->success.begin(); _iter910 != this->success.end(); ++_iter910)
+      std::vector<std::string> ::const_iterator _iter921;
+      for (_iter921 = this->success.begin(); _iter921 != this->success.end(); ++_iter921)
       {
-        xfer += oprot->writeString((*_iter910));
+        xfer += oprot->writeString((*_iter921));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6687,14 +6958,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size911;
-            ::apache::thrift::protocol::TType _etype914;
-            xfer += iprot->readListBegin(_etype914, _size911);
-            (*(this->success)).resize(_size911);
-            uint32_t _i915;
-            for (_i915 = 0; _i915 < _size911; ++_i915)
+            uint32_t _size922;
+            ::apache::thrift::protocol::TType _etype925;
+            xfer += iprot->readListBegin(_etype925, _size922);
+            (*(this->success)).resize(_size922);
+            uint32_t _i926;
+            for (_i926 = 0; _i926 < _size922; ++_i926)
             {
-              xfer += iprot->readString((*(this->success))[_i915]);
+              xfer += iprot->readString((*(this->success))[_i926]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7004,14 +7275,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size916;
-            ::apache::thrift::protocol::TType _etype919;
-            xfer += iprot->readListBegin(_etype919, _size916);
-            this->tbl_names.resize(_size916);
-            uint32_t _i920;
-            for (_i920 = 0; _i920 < _size916; ++_i920)
+            uint32_t _size927;
+            ::apache::thrift::protocol::TType _etype930;
+            xfer += iprot->readListBegin(_etype930, _size927);
+            this->tbl_names.resize(_size927);
+            uint32_t _i931;
+            for (_i931 = 0; _i931 < _size927; ++_i931)
             {
-              xfer += iprot->readString(this->tbl_names[_i920]);
+              xfer += iprot->readString(this->tbl_names[_i931]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7044,10 +7315,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter921;
-    for (_iter921 = this->tbl_names.begin(); _iter921 != this->tbl_names.end(); ++_iter921)
+    std::vector<std::string> ::const_iterator _iter932;
+    for (_iter932 = this->tbl_names.begin(); _iter932 != this->tbl_names.end(); ++_iter932)
     {
-      xfer += oprot->writeString((*_iter921));
+      xfer += oprot->writeString((*_iter932));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7075,10 +7346,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter922;
-    for (_iter922 = (*(this->tbl_names)).begin(); _iter922 != (*(this->tbl_names)).end(); ++_iter922)
+    std::vector<std::string> ::const_iterator _iter933;
+    for (_iter933 = (*(this->tbl_names)).begin(); _iter933 != (*(this->tbl_names)).end(); ++_iter933)
     {
-      xfer += oprot->writeString((*_iter922));
+      xfer += oprot->writeString((*_iter933));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7119,14 +7390,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size923;
-            ::apache::thrift::protocol::TType _etype926;
-            xfer += iprot->readListBegin(_etype926, _size923);
-            this->success.resize(_size923);
-            uint32_t _i927;
-            for (_i927 = 0; _i927 < _size923; ++_i927)
+            uint32_t _size934;
+            ::apache::thrift::protocol::TType _etype937;
+            xfer += iprot->readListBegin(_etype937, _size934);
+            this->success.resize(_size934);
+            uint32_t _i938;
+            for (_i938 = 0; _i938 < _size934; ++_i938)
             {
-              xfer += this->success[_i927].read(iprot);
+              xfer += this->success[_i938].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7181,10 +7452,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 _iter928;
-      for (_iter928 = this->success.begin(); _iter928 != this->success.end(); ++_iter928)
+      std::vector<Table> ::const_iterator _iter939;
+      for (_iter939 = this->success.begin(); _iter939 != this->success.end(); ++_iter939)
       {
-        xfer += (*_iter928).write(oprot);
+        xfer += (*_iter939).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -7237,14 +7508,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 _size929;
-            ::apache::thrift::protocol::TType _etype932;
-            xfer += iprot->readListBegin(_etype932, _size929);
-            (*(this->success)).resize(_size929);
-            uint32_t _i933;
-            for (_i933 = 0; _i933 < _size929; ++_i933)
+            uint32_t _size940;
+            ::apache::thrift::protocol::TType _etype943;
+            xfer += iprot->readListBegin(_etype943, _size940);
+            (*(this->success)).resize(_size940);
+            uint32_t _i944;
+            for (_i944 = 0; _i944 < _size940; ++_i944)
             {
-              xfer += (*(this->success))[_i933].read(iprot);
+              xfer += (*(this->success))[_i944].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7430,14 +7701,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 _size934;
-            ::apache::thrift::protocol::TType _etype937;
-            xfer += iprot->readListBegin(_etype937, _size934);
-            this->success.resize(_size934);
-            uint32_t _i938;
-            for (_i938 = 0; _i938 < _size934; ++_i938)
+            uint32_t _size945;
+            ::apache::thrift::protocol::TType _etype948;
+            xfer += iprot->readListBegin(_etype948, _size945);
+            this->success.resize(_size945);
+            uint32_t _i949;
+            for (_i949 = 0; _i949 < _size945; ++_i949)
             {
-              xfer += iprot->readString(this->success[_i938]);
+              xfer += iprot->readString(this->success[_i949]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7492,10 +7763,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 _iter939;
-      for (_iter939 = this->success.begin(); _iter939 != this->success.end(); ++_iter939)
+      std::vector<std::string> ::const_iterator _iter950;
+      for (_iter950 = this->success.begin(); _iter950 != this->success.end(); ++_iter950)
       {
-        xfer += oprot->writeString((*_iter939));
+        xfer += oprot->writeString((*_iter950));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7548,14 +7819,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 _size940;
-            ::apache::thrift::protocol::TType _etype943;
-            xfer += iprot->readListBegin(_etype943, _size940);
-            (*(this->success)).resize(_size940);
-            uint32_t _i944;
-            for (_i944 = 0; _i944 < _size940; ++_i944)
+            uint32_t _size951;
+            ::apache::thrift::protocol::TType _etype954;
+            xfer += iprot->readListBegin(_etype954, _size951);
+            (*(this->success)).resize(_size951);
+            uint32_t _i955;
+            for (_i955 = 0; _i955 < _size951; ++_i955)
             {
-              xfer += iprot->readString((*(this->success))[_i944]);
+              xfer += iprot->readString((*(this->success))[_i955]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8889,14 +9160,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size945;
-            ::apache::thrift::protocol::TType _etype948;
-            xfer += iprot->readListBegin(_etype948, _size945);
-            this->new_parts.resize(_size945);
-            uint32_t _i949;
-            for (_i949 = 0; _i949 < _size945; ++_i949)
+            uint32_t _size956;
+            ::apache::thrift::protocol::TType _etype959;
+            xfer += iprot->readListBegin(_etype959, _size956);
+            this->new_parts.resize(_size956);
+            uint32_t _i960;
+            for (_i960 = 0; _i960 < _size956; ++_i960)
             {
-              xfer += this->new_parts[_i949].read(iprot);
+              xfer += this->new_parts[_i960].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8925,10 +9196,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 _iter950;
-    for (_iter950 = this->new_parts.begin(); _iter950 != this->new_parts.end(); ++_iter950)
+    std::vector<Partition> ::const_iterator _iter961;
+    for (_iter961 = this->new_parts.begin(); _iter961 != this->new_parts.end(); ++_iter961)
     {
-      xfer += (*_iter950).write(oprot);
+      xfer += (*_iter961).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8952,10 +9223,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 _iter951;
-    for (_iter951 = (*(this->new_parts)).begin(); _iter951 != (*(this->new_parts)).end(); ++_iter951)
+    std::vector<Partition> ::const_iterator _iter962;
+    for (_iter962 = (*(this->new_parts)).begin(); _iter962 != (*(this->new_parts)).end(); ++_iter962)
     {
-      xfer += (*_iter951).write(oprot);
+      xfer += (*_iter962).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9164,14 +9435,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 _size952;
-            ::apache::thrift::protocol::TType _etype955;
-            xfer += iprot->readListBegin(_etype955, _size952);
-            this->new_parts.resize(_size952);
-            uint32_t _i956;
-            for (_i956 = 0; _i956 < _size952; ++_i956)
+            uint32_t _size963;
+            ::apache::thrift::protocol::TType _etype966;
+            xfer += iprot->readListBegin(_etype966, _size963);
+            this->new_parts.resize(_size963);
+            uint32_t _i967;
+            for (_i967 = 0; _i967 < _size963; ++_i967)
             {
-              xfer += this->new_parts[_i956].read(iprot);
+              xfer += this->new_parts[_i967].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9200,10 +9471,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 _iter957;
-    for (_iter957 = this->new_parts.begin(); _iter957 != this->new_parts.end(); ++_iter957)
+    std::vector<PartitionSpec> ::const_iterator _iter968;
+    for (_iter968 = this->new_parts.begin(); _iter968 != this->new_parts.end(); ++_iter968)
     {
-      xfer += (*_iter957).write(oprot);
+      xfer += (*_iter968).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9227,10 +9498,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 _iter958;
-    for (_iter958 = (*(this->new_parts)).begin(); _iter958 != (*(this->new_parts)).end(); ++_iter958)
+    std::vector<PartitionSpec> ::const_iterator _iter969;
+    for (_iter969 = (*(this->new_parts)).begin(); _iter969 != (*(this->new_parts)).end(); ++_iter969)
     {
-      xfer += (*_iter958).write(oprot);
+      xfer += (*_iter969).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9455,14 +9726,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size959;
-            ::apache::thrift::protocol::TType _etype962;
-            xfer += iprot->readListBegin(_etype962, _size959);
-            this->part_vals.resize(_size959);
-            uint32_t _i963;
-            for (_i963 = 0; _i963 < _size959; ++_i963)
+            uint32_t _size970;
+            ::apache::thrift::protocol::TType _etype973;
+            xfer += iprot->readListBegin(_etype973, _size970);
+            this->part_vals.resize(_size970);
+            uint32_t _i974;
+            for (_i974 = 0; _i974 < _size970; ++_i974)
             {
-              xfer += iprot->readString(this->part_vals[_i963]);
+              xfer += iprot->readString(this->part_vals[_i974]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9499,10 +9770,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 _iter964;
-    for (_iter964 = this->part_vals.begin(); _iter964 != this->part_vals.end(); ++_iter964)
+    std::vector<std::string> ::const_iterator _iter975;
+    for (_iter975 = this->part_vals.begin(); _iter975 != this->part_vals.end(); ++_iter975)
     {
-      xfer += oprot->writeString((*_iter964));
+      xfer += oprot->writeString((*_iter975));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9534,10 +9805,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 _iter965;
-    for (_iter965 = (*(this->part_vals)).begin(); _iter965 != (*(this->part_vals)).end(); ++_iter965)
+    std::vector<std::string> ::const_iterator _iter976;
+    for (_iter976 = (*(this->part_vals)).begin(); _iter976 != (*(this->part_vals)).end(); ++_iter976)
     {
-      xfer += oprot->writeString((*_iter965));
+      xfer += oprot->writeString((*_iter976));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10009,14 +10280,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size966;
-            ::apache::thrift::protocol::TType _etype969;
-            xfer += iprot->readListBegin(_etype969, _size966);
-            this->part_vals.resize(_size966);
-            uint32_t _i970;
-            for (_i970 = 0; _i970 < _size966; ++_i970)
+            uint32_t _size977;
+            ::apache::thrift::protocol::TType _etype980;
+            xfer += iprot->readListBegin(_etype980, _size977);
+            this->part_vals.resize(_size977);
+            uint32_t _i981;
+            for (_i981 = 0; _i981 < _size977; ++_i981)
             {
-              xfer += iprot->readString(this->part_vals[_i970]);
+              xfer += iprot->readString(this->part_vals[_i981]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10061,10 +10332,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 _iter971;
-    for (_iter971 = this->part_vals.begin(); _iter971 != this->part_vals.end(); ++_iter971)
+    std::vector<std::string> ::const_iterator _iter982;
+    for (_iter982 = this->part_vals.begin(); _iter982 != this->part_vals.end(); ++_iter982)
     {
-      xfer += oprot->writeString((*_iter971));
+      xfer += oprot->writeString((*_iter982));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10100,10 +10371,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 _iter972;
-    for (_iter972 = (*(this->part_vals)).begin(); _iter972 != (*(this->part_vals)).end(); ++_iter972)
+    std::vector<std::string> ::const_iterator _iter983;
+    for (_iter983 = (*(this->part_vals)).begin(); _iter983 != (*(this->part_vals)).end(); ++_iter983)
     {
-      xfer += oprot->writeString((*_iter972));
+      xfer += oprot->writeString((*_iter983));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10906,14 +11177,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size973;
-            ::apache::thrift::protocol::TType _etype976;
-            xfer += iprot->readListBegin(_etype976, _size973);
-            this->part_vals.resize(_size973);
-            uint32_t _i977;
-            for (_i977 = 0; _i977 < _size973; ++_i977)
+            uint32_t _size984;
+            ::apache::thrift::protocol::TType _etype987;
+            xfer += iprot->readListBegin(_etype987, _size984);
+            this->part_vals.resize(_size984);
+            uint32_t _i988;
+            for (_i988 = 0; _i988 < _size984; ++_i988)
             {
-              xfer += iprot->readString(this->part_vals[_i977]);
+              xfer += iprot->readString(this->part_vals[_i988]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10958,10 +11229,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 _iter978;
-    for (_iter978 = this->part_vals.begin(); _iter978 != this->part_vals.end(); ++_iter978)
+    std::vector<std::string> ::const_iterator _iter989;
+    for (_iter989 = this->part_vals.begin(); _iter989 != this->part_vals.end(); ++_iter989)
     {
-      xfer += oprot->writeString((*_iter978));
+      xfer += oprot->writeString((*_iter989));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10997,10 +11268,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 _iter979;
-    for (_iter979 = (*(this->part_vals)).begin(); _iter979 != (*(this->part_vals)).end(); ++_iter979)
+    std::vector<std::string> ::const_iterator _iter990;
+    for (_iter990 = (*(this->part_vals)).begin(); _iter990 != (*(this->part_vals)).end(); ++_iter990)
     {
-      xfer += oprot->writeString((*_iter979));
+      xfer += oprot->writeString((*_iter990));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11209,14 +11480,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size980;
-            ::apache::thrift::protocol::TType _etype983;
-            xfer += iprot->readListBegin(_etype983, _size980);
-            this->part_vals.resize(_size980);
-            uint32_t _i984;
-            for (_i984 = 0; _i984 < _size980; ++_i984)
+            uint32_t _size991;
+            ::apache::thrift::protocol::TType _etype994;
+            xfer += iprot->readListBegin(_etype994, _size991);
+            this->part_vals.resize(_size991);
+            uint32_t _i995;
+            for (_i995 = 0; _i995 < _size991; ++_i995)
             {
-              xfer += iprot->readString(this->part_vals[_i984]);
+              xfer += iprot->readString(this->part_vals[_i995]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11269,10 +11540,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 _iter985;
-    for (_iter985 = this->part_vals.begin(); _iter985 != this->part_vals.end(); ++_iter985)
+    std::vector<std::string> ::const_iterator _iter996;
+    for (_iter996 = this->part_vals.begin(); _iter996 != this->part_vals.end(); ++_iter996)
     {
-      xfer += oprot->writeString((*_iter985));
+      xfer += oprot->writeString((*_iter996));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11312,10 +11583,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 _iter986;
-    for (_iter986 = (*(this->part_vals)).begin(); _iter986 != (*(this->part_vals)).end(); ++_iter986)
+    std::vector<std::string> ::const_iterator _iter997;
+    for (_iter997 = (*(this->part_vals)).begin(); _iter997 != (*(this->part_vals)).end(); ++_iter997)
     {
-      xfer += oprot->writeString((*_iter986));
+      xfer += oprot->writeString((*_iter997));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12321,14 +12592,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size987;
-            ::apache::thrift::protocol::TType _etype990;
-            xfer += iprot->readListBegin(_etype990, _size987);
-            this->part_vals.resize(_size987);
-            uint32_t _i991;
-            for (_i991 = 0; _i991 < _size987; ++_i991)
+            uint32_t _size998;
+            ::apache::thrift::protocol::TType _etype1001;
+            xfer += iprot->readListBegin(_etype1001, _size998);
+            this->part_vals.resize(_size998);
+            uint32_t _i1002;
+            for (_i1002 = 0; _i1002 < _size998; ++_i1002)
             {
-              xfer += iprot->readString(this->part_vals[_i991]);
+              xfer += iprot->readString(this->part_vals[_i1002]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12365,10 +12636,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 _iter992;
-    for (_iter992 = this->part_vals.begin(); _iter992 != this->part_vals.end(); ++_iter992)
+    std::vector<std::string> ::const_iterator _iter1003;
+    for (_iter1003 = this->part_vals.begin(); _iter1003 != this->part_vals.end(); ++_iter1003)
     {
-      xfer += oprot->writeString((*_iter992));
+      xfer += oprot->writeString((*_iter1003));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12400,10 +12671,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 _iter993;
-    for (_iter993 = (*(this->part_vals)).begin(); _iter993 != (*(this->part_vals)).end(); ++_iter993)
+    std::vector<std::string> ::const_iterator _iter1004;
+    for (_iter1004 = (*(this->part_vals)).begin(); _iter1004 != (*(this->part_vals)).end(); ++_iter1004)
     {
-      xfer += oprot->writeString((*_iter993));
+      xfer += oprot->writeString((*_iter1004));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12592,17 +12863,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size994;
-            ::apache::thrift::protocol::TType _ktype995;
-            ::apache::thrift::protocol::TType _vtype996;
-            xfer += iprot->readMapBegin(_ktype995, _vtype996, _size994);
-            uint32_t _i998;
-            for (_i998 = 0; _i998 < _size994; ++_i998)
+            uint32_t _size1005;
+            ::apache::thrift::protocol::TType _ktype1006;
+            ::apache::thrift::protocol::TType _vtype1007;
+            xfer += iprot->readMapBegin(_ktype1006, _vtype1007, _size1005);
+            uint32_t _i1009;
+            for (_i1009 = 0; _i1009 < _size1005; ++_i1009)
             {
-              std::string _key999;
-              xfer += iprot->readString(_key999);
-              std::string& _val1000 = this->partitionSpecs[_key999];
-              xfer += iprot->readString(_val1000);
+              std::string _key1010;
+              xfer += iprot->readString(_key1010);
+              std::string& _val1011 = this->partitionSpecs[_key1010];
+              xfer += iprot->readString(_val1011);
             }
             xfer += iprot->readMapEnd();
           }
@@ -12663,11 +12934,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 _iter1001;
-    for (_iter1001 = this->partitionSpecs.begin(); _iter1001 != this->partitionSpecs.end(); ++_iter1001)
+    std::map<std::string, std::string> ::const_iterator _iter1012;
+    for (_iter1012 = this->partitionSpecs.begin(); _iter1012 != this->partitionSpecs.end(); ++_iter1012)
     {
-      xfer += oprot->writeString(_iter1001->first);
-      xfer += oprot->writeString(_iter1001->second);
+      xfer += oprot->writeString(_iter1012->first);
+      xfer += oprot->writeString(_iter1012->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12707,11 +12978,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 _iter1002;
-    for (_iter1002 = (*(this->partitionSpecs)).begin(); _iter1002 != (*(this->partitionSpecs)).end(); ++_iter1002)
+    std::map<std::string, std::string> ::const_iterator _iter1013;
+    for (_iter1013 = (*(this->partitionSpecs)).begin(); _iter1013 != (*(this->partitionSpecs)).end(); ++_iter1013)
     {
-      xfer += oprot->writeString(_iter1002->first);
-      xfer += oprot->writeString(_iter1002->second);
+      xfer += oprot->writeString(_iter1013->first);
+      xfer += oprot->writeString(_iter1013->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12956,17 +13227,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size1003;
-            ::apache::thrift::protocol::TType _ktype1004;
-            ::apache::thrift::protocol::TType _vtype1005;
-            xfer += iprot->readMapBegin(_ktype1004, _vtype1005, _size1003);
-            uint32_t _i1007;
-            for (_i1007 = 0; _i1007 < _size1003; ++_i1007)
+            uint32_t _size1014;
+            ::apache::thrift::protocol::TType _ktype1015;
+            ::apache::thrift::protocol::TType _vtype1016;
+            xfer += iprot->readMapBegin(_ktype1015, _vtype1016, _size1014);
+            uint32_t _i1018;
+            for (_i1018 = 0; _i1018 < _size1014; ++_i1018)
             {
-              std::string _key1008;
-              xfer += iprot->readString(_key1008);
-              std::string& _val1009 = this->partitionSpecs[_key1008];
-              xfer += iprot->readString(_val1009);
+              std::string _key1019;
+              xfer += iprot->readString(_key1019);
+              std::string& _val1020 = this->partitionSpecs[_key1019];
+              xfer += iprot->readString(_val1020);
             }
             xfer += iprot->readMapEnd();
           }
@@ -13027,11 +13298,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 _iter1010;
-    for (_iter1010 = this->partitionSpecs.begin(); _iter1010 != this->partitionSpecs.end(); ++_iter1010)
+    std::map<std::string, std::string> ::const_iterator _iter1021;
+    for (_iter1021 = this->partitionSpecs.begin(); _iter1021 != this->partitionSpecs.end(); ++_iter1021)
     {
-      xfer += oprot->writeString(_iter1010->first);
-      xfer += oprot->writeString(_iter1010->second);
+      xfer += oprot->writeString(_iter1021->first);
+      xfer += oprot->writeString(_iter1021->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13071,11 +13342,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_pargs::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter1011;
-    for (_iter1011 = (*(this->partitionSpecs)).begin(); _iter1011 != (*(this->partitionSpecs)).end(); ++_iter1011)
+    std::map<std::string, std::string> ::const_iterator _iter1022;
+    for (_iter1022 = (*(this->partitionSpecs)).begin(); _iter1022 != (*(this->partitionSpecs)).end(); ++_iter1022)
     {
-      xfer += oprot->writeString(_iter1011->first);
-      xfer += oprot->writeString(_iter1011->second);
+      xfer += oprot->writeString(_iter1022->first);
+      xfer += oprot->writeString(_iter1022->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13132,14 +13403,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1012;
-            ::apache::thrift::protocol::TType _etype1015;
-            xfer += iprot->readListBegin(_etype1015, _size1012);
-            this->success.resize(_size1012);
-            uint32_t _i1016;
-            for (_i1016 = 0; _i1016 < _size1012; ++_i1016)
+            uint32_t _size1023;
+            ::apache::thrift::protocol::TType _etype1026;
+            xfer += iprot->readListBegin(_etype1026, _size1023);
+            this->success.resize(_size1023);
+            uint32_t _i1027;
+            for (_i1027 = 0; _i1027 < _size1023; ++_i1027)
             {
-              xfer += this->success[_i1016].read(iprot);
+              xfer += this->success[_i1027].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13202,10 +13473,10 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter1017;
-      for (_iter1017 = this->success.begin(); _iter1017 != this->success.end(); ++_iter1017)
+      std::vector<Partition> ::const_iterator _iter1028;
+      for (_iter1028 = this->success.begin(); _iter1028 != this->success.end(); ++_iter1028)
       {
-        xfer += (*_iter1017).write(oprot);
+        xfer += (*_iter1028).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13262,14 +13533,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1018;
-            ::apache::thrift::protocol::TType _etype1021;
-            xfer += iprot->readListBegin(_etype1021, _size1018);
-            (*(this->success)).resize(_size1018);
-            uint32_t _i1022;
-            for (_i1022 = 0; _i1022 < _size1018; ++_i1022)
+            uint32_t _size1029;
+            ::apache::thrift::protocol::TType _etype1032;
+            xfer += iprot->readListBegin(_etype1032, _size1029);
+            (*(this->success)).resize(_size1029);
+            uint32_t _i1033;
+            for (_i1033 = 0; _i1033 < _size1029; ++_i1033)
             {
-              xfer += (*(this->success))[_i1022].read(iprot);
+              xfer += (*(this->success))[_i1033].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13368,14 +13639,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1023;
-            ::apache::thrift::protocol::TType _etype1026;
-            xfer += iprot->readListBegin(_etype1026, _size1023);
-            this->part_vals.resize(_size1023);
-            uint32_t _i1027;
-            for (_i1027 = 0; _i1027 < _size1023; ++_i1027)
+            uint32_t _size1034;
+            ::apache::thrift::protocol::TType _etype1037;
+            xfer += iprot->readListBegin(_etype1037, _size1034);
+            this->part_vals.resize(_size1034);
+            uint32_t _i1038;
+            for (_i1038 = 0; _i1038 < _size1034; ++_i1038)
             {
-              xfer += iprot->readString(this->part_vals[_i1027]);
+              xfer += iprot->readString(this->part_vals[_i1038]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13396,14 +13667,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size1028;
-            ::apache::thrift::protocol::TType _etype1031;
-            xfer += iprot->readListBegin(_etype1031, _size1028);
-            this->group_names.resize(_size1028);
-            uint32_t _i1032;
-            for (_i1032 = 0; _i1032 < _size1028; ++_i1032)
+            uint32_t _size1039;
+            ::apache::thrift::protocol::TType _etype1042;
+            xfer += iprot->readListBegin(_etype1042, _size1039);
+            this->group_names.resize(_size1039);
+            uint32_t _i1043;
+            for (_i1043 = 0; _i1043 < _size1039; ++_i1043)
             {
-              xfer += iprot->readString(this->group_names[_i1032]);
+              xfer += iprot->readString(this->group_names[_i1043]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13440,10 +13711,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter1033;
-    for (_iter1033 = this->part_vals.begin(); _iter1033 != this->part_vals.end(); ++_iter1033)
+    std::vector<std::string> ::const_iterator _iter1044;
+    for (_iter1044 = this->part_vals.begin(); _iter1044 != this->part_vals.end(); ++_iter1044)
     {
-      xfer += oprot->writeString((*_iter1033));
+      xfer += oprot->writeString((*_iter1044));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13456,10 +13727,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter1034;
-    for (_iter1034 = this->group_names.begin(); _iter1034 != this->group_names.end(); ++_iter1034)
+    std::vector<std::string> ::const_iterator _iter1045;
+    for (_iter1045 = this->group_names.begin(); _iter1045 != this->group_names.end(); ++_iter1045)
     {
-      xfer += oprot->writeString((*_iter1034));
+      xfer += oprot->writeString((*_iter1045));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13491,10 +13762,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter1035;
-    for (_iter1035 = (*(this->part_vals)).begin(); _iter1035 != (*(this->part_vals)).end(); ++_iter1035)
+    std::vector<std::string> ::const_iterator _iter1046;
+    for (_iter1046 = (*(this->part_vals)).begin(); _iter1046 != (*(this->part_vals)).end(); ++_iter1046)
     {
-      xfer += oprot->writeString((*_iter1035));
+      xfer += oprot->writeString((*_iter1046));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13507,10 +13778,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1036;
-    for (_iter1036 = (*(this->group_names)).begin(); _iter1036 != (*(this->group_names)).end(); ++_iter1036)
+    std::vector<std::string> ::const_iterator _iter1047;
+    for (_iter1047 = (*(this->group_names)).begin(); _iter1047 != (*(this->group_names)).end(); ++_iter1047)
     {
-      xfer += oprot->writeString((*_iter1036));
+      xfer += oprot->writeString((*_iter1047));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14069,14 +14340,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1037;
-            ::apache::thrift::protocol::TType _etype1040;
-            xfer += iprot->readListBegin(_etype1040, _size1037);
-            this->success.resize(_size1037);
-            uint32_t _i1041;
-            for (_i1041 = 0; _i1041 < _size1037; ++_i1041)
+            uint32_t _size1048;
+            ::apache::thrift::protocol::TType _etype1051;
+            xfer += iprot->readListBegin(_etype1051, _size1048);
+            this->success.resize(_size1048);
+            uint32_t _i1052;
+            for (_i1052 = 0; _i1052 < _size1048; ++_i1052)
             {
-              xfer += this->success[_i1041].read(iprot);
+              xfer += this->success[_i1052].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14123,10 +14394,10 @@ uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter1042;
-      for (_iter1042 = this->success.begin(); _iter1042 != this->success.end(); ++_iter1042)
+      std::vector<Partition> ::const_iterator _iter1053;
+      for (_iter1053 = this->success.begin(); _iter1053 != this->success.end(); ++_iter1053)
       {
-        xfer += (*_iter1042).write(oprot);
+        xfer += (*_iter1053).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14175,14 +14446,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1043;
-            ::apache::thrift::protocol::TType _etype1046;
-            xfer += iprot->readListBegin(_etype1046, _size1043);
-            (*(this->success)).resize(_size1043);
-            uint32_t _i1047;
-            for (_i1047 = 0; _i1047 < _size1043; ++_i1047)
+            uint32_t _size1054;
+            ::apache::thrift::protocol::TType _etype1057;
+            xfer += iprot->readListBegin(_etype1057, _size1054);
+            (*(this->success)).resize(_size1054);
+            uint32_t _i1058;
+            for (_i1058 = 0; _i1058 < _size1054; ++_i1058)
             {
-              xfer += (*(this->success))[_i1047].read(iprot);
+              xfer += (*(this->success))[_i1058].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14281,14 +14552,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size1048;
-            ::apache::thrift::protocol::TType _etype1051;
-            xfer += iprot->readListBegin(_etype1051, _size1048);
-            this->group_names.resize(_size1048);
-            uint32_t _i1052;
-            for (_i1052 = 0; _i1052 < _size1048; ++_i1052)
+            uint32_t _size1059;
+            ::apache::thrift::protocol::TType _etype1062;
+            xfer += iprot->readListBegin(_etype1062, _size1059);
+            this->group_names.resize(_size1059);
+            uint32_t _i1063;
+            for (_i1063 = 0; _i1063 < _size1059; ++_i1063)
             {
-              xfer += iprot->readString(this->group_names[_i1052]);
+              xfer += iprot->readString(this->group_names[_i1063]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14333,10 +14604,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter1053;
-    for (_iter1053 = this->group_names.begin(); _iter1053 != this->group_names.end(); ++_iter1053)
+    std::vector<std::string> ::const_iterator _iter1064;
+    for (_iter1064 = this->group_names.begin(); _iter1064 != this->group_names.end(); ++_iter1064)
     {
-      xfer += oprot->writeString((*_iter1053));
+      xfer += oprot->writeString((*_iter1064));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14376,10 +14647,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1054;
-    for (_iter1054 = (*(this->group_names)).begin(); _iter1054 != (*(this->group_names)).end(); ++_iter1054)
+    std::vector<std::string> ::const_iterator _iter1065;
+    for (_iter1065 = (*(this->group_names)).begin(); _iter1065 != (*(this->group_names)).end(); ++_iter1065)
     {
-      xfer += oprot->writeString((*_iter1054));
+      xfer += oprot->writeString((*_iter1065));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14420,14 +14691,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1055;
-            ::apache::thrift::protocol::TType _etype1058;
-            xfer += iprot->readListBegin(_etype1058, _size1055);
-            this->success.resize(_size1055);
-            uint32_t _i1059;
-            for (_i1059 = 0; _i1059 < _size1055; ++_i1059)
+            uint32_t _size1066;
+            ::apache::thrift::protocol::TType _etype1069;
+            xfer += iprot->readListBegin(_etype1069, _size1066);
+            this->success.resize(_size1066);
+            uint32_t _i1070;
+            for (_i1070 = 0; _i1070 < _size1066; ++_i1070)
             {
-              xfer += this->success[_i1059].read(iprot);
+              xfer += this->success[_i1070].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14474,10 +14745,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter1060;
-      for (_iter1060 = this->success.begin(); _iter1060 != this->success.end(); ++_iter1060)
+      std::vector<Partition> ::const_iterator _iter1071;
+      for (_iter1071 = this->success.begin(); _iter1071 != this->success.end(); ++_iter1071)
       {
-        xfer += (*_iter1060).write(oprot);
+        xfer += (*_iter1071).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14526,14 +14797,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1061;
-            ::apache::thrift::protocol::TType _etype1064;
-            xfer += iprot->readListBegin(_etype1064, _size1061);
-            (*(this->success)).resize(_size1061);
-            uint32_t _i1065;
-            for (_i1065 = 0; _i1065 < _size1061; ++_i1065)
+            uint32_t _size1072;
+            ::apache::thrift::protocol::TType _etype1075;
+            xfer += iprot->readListBegin(_etype1075, _size1072);
+            (*(this->success)).resize(_size1072);
+            uint32_t _i1076;
+            for (_i1076 = 0; _i1076 < _size1072; ++_i1076)
             {
-              xfer += (*(this->success))[_i1065].read(iprot);
+              xfer += (*(this->success))[_i1076].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14711,14 +14982,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1066;
-            ::apache::thrift::protocol::TType _etype1069;
-            xfer += iprot->readListBegin(_etype1069, _size1066);
-            this->success.resize(_size1066);
-            uint32_t _i1070;
-            for (_i1070 = 0; _i1070 < _size1066; ++_i1070)
+            uint32_t _size1077;
+            ::apache::thrift::protocol::TType _etype1080;
+            xfer += iprot->readListBegin(_etype1080, _size1077);
+            this->success.resize(_size1077);
+            uint32_t _i1081;
+            for (_i1081 = 0; _i1081 < _size1077; ++_i1081)
             {
-              xfer += this->success[_i1070].read(iprot);
+              xfer += this->success[_i1081].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14765,10 +15036,10 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::write(::apache::thrift
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<PartitionSpec> ::const_iterator _iter1071;
-      for (_iter1071 = this->success.begin(); _iter1071 != this->success.end(); ++_iter1071)
+      std::vector<PartitionSpec> ::const_iterator _iter1082;
+      for (_iter1082 = this->success.begin(); _iter1082 != this->success.end(); ++_iter1082)
       {
-        xfer += (*_iter1071).write(oprot);
+        xfer += (*_iter1082).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14817,14 +15088,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1072;
-            ::apache::thrift::protocol::TType _etype1075;
-            xfer += iprot->readListBegin(_etype1075, _size1072);
-            (*(this->success)).resize(_size1072);
-            uint32_t _i1076;
-            for (_i1076 = 0; _i1076 < _size1072; ++_i1076)
+            uint32_t _size1083;
+            ::apache::thrift::protocol::TType _etype1086;
+            xfer += iprot->readListBegin(_etype1086, _size1083);
+            (*(this->success)).resize(_size1083);
+            uint32_t _i1087;
+            for (_i1087 = 0; _i1087 < _size1083; ++_i1087)
             {
-              xfer += (*(this->success))[_i1076].read(iprot);
+              xfer += (*(this->success))[_i1087].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15002,14 +15273,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1077;
-            ::apache::thrift::protocol::TType _etype1080;
-            xfer += iprot->readListBegin(_etype1080, _size1077);
-            this->success.resize(_size1077);
-            uint32_t _i1081;
-            for (_i1081 = 0; _i1081 < _size1077; ++_i1081)
+            uint32_t _size1088;
+            ::apache::thrift::protocol::TType _etype1091;
+            xfer += iprot->readListBegin(_etype1091, _size1088);
+            this->success.resize(_size1088);
+            uint32_t _i1092;
+            for (_i1092 = 0; _i1092 < _size1088; ++_i1092)
             {
-              xfer += iprot->readString(this->success[_i1081]);
+              xfer += iprot->readString(this->success[_i1092]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15048,10 +15319,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_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 _iter1082;
-      for (_iter1082 = this->success.begin(); _iter1082 != this->success.end(); ++_iter1082)
+      std::vector<std::string> ::const_iterator _iter1093;
+      for (_iter1093 = this->success.begin(); _iter1093 != this->success.end(); ++_iter1093)
       {
-        xfer += oprot->writeString((*_iter1082));
+        xfer += oprot->writeString((*_iter1093));
       }
       xfer += oprot->writeListEnd();
     }
@@ -15096,14 +15367,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1083;
-            ::apache::thrift::protocol::TType _etype1086;
-            xfer += iprot->readListBegin(_etype1086, _size1083);
-            (*(this->success)).resize(_size1083);
-            uint32_t _i1087;
-            for (_i1087 = 0; _i1087 < _size1083; ++_i1087)
+            uint32_t _size1094;
+            ::apache::thrift::protocol::TType _etype1097;
+            xfer += iprot->readListBegin(_etype1097, _size1094);
+            (*(this->success)).resize(_size1094);
+            uint32_t _i1098;
+            for (_i1098 = 0; _i1098 < _size1094; ++_i1098)
             {
-              xfer += iprot->readString((*(this->success))[_i1087]);
+              xfer += iprot->readString((*(this->success))[_i1098]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15178,14 +15449,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1088;
-            ::apache::thrift::protocol::TType _etype1091;
-            xfer += iprot->readListBegin(_etype1091, _size1088);
-            this->part_vals.resize(_size1088);
-            uint32_t _i1092;
-            for (_i1092 = 0; _i1092 < _size1088; ++_i1092)
+            uint32_t _size1099;
+            ::apache::thrift::protocol::TType _etype1102;
+            xfer += iprot->readListBegin(_etype1102, _size1099);
+            this->part_vals.resize(_size1099);
+            uint32_t _i1103;
+            for (_i1103 = 0; _i1103 < _size1099; ++_i1103)
             {
-              xfer += iprot->readString(this->part_vals[_i1092]);
+              xfer += iprot->readString(this->part_vals[_i1103]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15230,10 +15501,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::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 _iter1093;
-    for (_iter1093 = this->part_vals.begin(); _iter1093 != this->part_vals.end(); ++_iter1093)
+    std::vector<std::string> ::const_iterator _iter1104;
+    for (_iter1104 = this->part_vals.begin(); _iter1104 != this->part_vals.end(); ++_iter1104)
     {
-      xfer += oprot->writeString((*_iter1093));
+      xfer += oprot->writeString((*_iter1104));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15269,10 +15540,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_pargs::write(::apache::thrift::pr
   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 _iter1094;
-    for (_iter1094 = (*(this->part_vals)).begin(); _iter1094 != (*(this->part_vals)).end(); ++_iter1094)
+    std::vector<std::string> ::const_iterator _iter1105;
+    for (_iter1105 = (*(this->part_vals)).begin(); _iter1105 != (*(this->part_vals)).end(); ++_iter1105)
     {
-      xfer += oprot->writeString((*_iter1094));
+      xfer += oprot->writeString((*_iter1105));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15317,14 +15588,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1095;
-            ::apache::thrift::protocol::TType _etype1098;
-            xfer += iprot->readListBegin(_etype1098, _size1095);
-            this->success.resize(_size1095);
-            uint32_t _i1099;
-            for (_i1099 = 0; _i1099 < _size1095; ++_i1099)
+            uint32_t _size1106;
+            ::apache::thrift::protocol::TType _etype1109;
+            xfer += iprot->readListBegin(_etype1109, _size1106);
+            this->success.resize(_size1106);
+            uint32_t _i1110;
+            for (_i1110 = 0; _i1110 < _size1106; ++_i1110)
             {
-              xfer += this->success[_i1099].read(iprot);
+              xfer += this->success[_i1110].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15371,10 +15642,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter1100;
-      for (_iter1100 = this->success.begin(); _iter1100 != this->success.end(); ++_iter1100)
+      std::vector<Partition> ::const_iterator _iter1111;
+      for (_iter1111 = this->success.begin(); _iter1111 != this->success.end(); ++_iter1111)
       {
-        xfer += (*_iter1100).write(oprot);
+        xfer += (*_iter1111).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -15423,14 +15694,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1101;
-            ::apache::thrift::protocol::TType _etype1104;
-            xfer += iprot->readListBegin(_etype1104, _size1101);
-            (*(this->success)).resize(_size1101);
-            uint32_t _i1105;
-            for (_i1105 = 0; _i1105 < _size1101; ++_i1105)
+            uint32_t _size1112;
+            ::apache::thrift::protocol::TType _etype1115;
+            xfer += iprot->readListBegin(_etype1115, _size1112);
+            (*(this->success)).resize(_size1112);
+            uint32_t _i1116;
+            for (_i1116 = 0; _i1116 < _size1112; ++_i1116)
             {
-              xfer += (*(this->success))[_i1105].read(iprot);
+              xfer += (*(this->success))[_i1116].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15513,14 +15784,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1106;
-            ::apache::thrift::protocol::TType _etype1109;
-            xfer += iprot->readListBegin(_etype1109, _size1106);
-            this->part_vals.resize(_size1106);
-            uint32_t _i1110;
-            for (_i1110 = 0; _i1110 < _size1106; ++_i1110)
+            uint32_t _size1117;
+            ::apache::thrift::protocol::TType _etype1120;
+            xfer += iprot->readListBegin(_etype1120, _size1117);
+            this->part_vals.resize(_size1117);
+            uint32_t _i1121;
+            for (_i1121 = 0; _i1121 < _size1117; ++_i1121)
             {
-              xfer += iprot->readString(this->part_vals[_i1110]);
+              xfer += iprot->readString(this->part_vals[_i1121]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15549,14 +15820,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size1111;
-            ::apache::thrift::protocol::TType _etype1114;
-            xfer += iprot->readListBegin(_etype1114, _size1111);
-            this->group_names.resize(_size1111);
-            uint32_t _i1115;
-            for (_i1115 = 0; _i1115 < _size1111; ++_i1115)
+            uint32_t _size1122;
+            ::apache::thrift::protocol::TType _etype1125;
+            xfer += iprot->readListBegin(_etype1125, _size1122);
+            this->group_names.resize(_size1122);
+            uint32_t _i1126;
+            for (_i1126 = 0; _i1126 < _size1122; ++_i1126)
             {
-              xfer += iprot->readString(this->group_names[_i1115]);
+              xfer += iprot->readString(this->group_names[_i1126]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15593,10 +15864,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t
   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 _iter1116;
-    for (_iter1116 = this->part_vals.begin(); _iter1116 != this->part_vals.end(); ++_iter1116)
+    std::vector<std::string> ::const_iterator _iter1127;
+    for (_iter1127 = this->part_vals.begin(); _iter1127 != this->part_vals.end(); ++_iter1127)
     {
-      xfer += oprot->writeString((*_iter1116));
+      xfer += oprot->writeString((*_iter1127));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15613,10 +15884,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter1117;
-    for (_iter1117 = this->group_names.begin(); _iter1117 != this->group_names.end(); ++_iter1117)
+    std::vector<std::string> ::const_iterator _iter1128;
+    for (_iter1128 = this->group_names.begin(); _iter1128 != this->group_names.end(); ++_iter1128)
     {
-      xfer += oprot->writeString((*_iter1117));
+      xfer += oprot->writeString((*_iter1128));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15648,10 +15919,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache::
   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 _iter1118;
-    for (_iter1118 = (*(this->part_vals)).begin(); _iter1118 != (*(this->part_vals)).end(); ++_iter1118)
+    std::vector<std::string> ::const_iterator _iter1129;
+    for (_iter1129 = (*(this->part_vals)).begin(); _iter1129 != (*(this->part_vals)).end(); ++_iter1129)
     {
-      xfer += oprot->writeString((*_iter1118));
+      xfer += oprot->writeString((*_iter1129));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15668,10 +15939,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache::
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1119;
-    for (_iter1119 = (*(this->group_names)).begin(); _iter1119 != (*(this->group_names)).end(); ++_iter1119)
+    std::vector<std::string> ::const_iterator _iter1130;
+    for (_iter1130 = (*(this->group_names)).begin(); _iter1130 != (*(this->group_names)).end(); ++_iter1130)
     {
-      xfer += oprot->writeString((*_iter1119));
+      xfer += oprot->writeString((*_iter1130));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15712,14 +15983,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1120;
-            ::apache::thrift::protocol::TType _etype1123;
-            xfer += iprot->readListBegin(_etype1123, _size1120);
-            this->success.resize(_size1120);
-            uint32_t _i1124;
-            for (_i1124 = 0; _i1124 < _size1120; ++_i1124)
+            uint32_t _size1131;
+            ::apache::thrift::protocol::TType _etype1134;
+            xfer += iprot->readListBegin(_etype1134, _size1131);
+            this->success.resize(_size1131);
+            uint32_t _i1135;
+            for (_i1135 = 0; _i1135 < _size1131; ++_i1135)
             {
-              xfer += this->success[_i1124].read(iprot);
+              xfer += this->success[_i1135].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15766,10 +16037,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::write(::apache:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter1125;
-      for (_iter1125 = this->success.begin(); _iter1125 != this->success.end(); ++_iter1125)
+      std::vector<Partition> ::const_iterator _iter1136;
+      for (_iter1136 = this->success.begin(); _iter1136 != this->success.end(); ++_iter1136)
       {
-        xfer += (*_iter1125).write(oprot);
+        xfer += (*_iter1136).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -15818,14 +16089,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_presult::read(::apache:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1126;
-            ::apache::thrift::protocol::TType _etype1129;
-            xfer += iprot->readListBegin(_etype1129, _size1126);
-            (*(this->success)).resize(_size1126);
-            uint32_t _i1130;
-            for (_i1130 = 0; _i1130 < _size1126; ++_i1130)
+            uint32_t _size1137;
+            ::apache::thrift::protocol::TType _etype1140;
+            xfer += iprot->readListBegin(_etype1140, _size1137);
+            (*(this->success)).resize(_size1137);
+            uint32_t _i1141;
+            for (_i1141 = 0; _i1141 < _size1137; ++_i1141)
             {
-              xfer += (*(this->success))[_i1130].read(iprot);
+              xfer += (*(this->success))[_i1141].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15908,14 +16179,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_args::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1131;
-            ::apache::thrift::protocol::TType _etype1134;
-            xfer += iprot->readListBegin(_etype1134, _size1131);
-            this->part_vals.resize(_size1131);
-            uint32_t _i1135;
-            for (_i1135 = 0; _i1135 < _size1131; ++_i1135)
+            uint32_t _size1142;
+            ::apache::thrift::protocol::TType _etype1145;
+            xfer += iprot->readListBegin(_etype1145, _size1142);
+            this->part_vals.resize(_size1142);
+            uint32_t _i1146;
+            for (_i1146 = 0; _i1146 < _size1142; ++_i1146)
             {
-              xfer += iprot->readString(this->part_vals[_i1135]);
+              xfer += iprot->readString(this->part_vals[_i1146]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15960,10 +16231,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_args::write(::apache::thrift
   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 _iter1136;
-    for (_iter1136 = this->part_vals.begin(); _iter1136 != this->part_vals.end(); ++_iter1136)
+    std::vector<std::string> ::const_iterator _iter1147;
+    for (_iter1147 = this->part_vals.begin(); _iter1147 != this->part_vals.end(); ++_iter1147)
     {
-      xfer += oprot->writeString((*_iter1136));
+      xfer += oprot->writeString((*_iter1147));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15999,10 +16270,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_pargs::write(::apache::thrif
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter1137;
-    for (_iter1137 = (*(this->part_vals)).begin(); _iter1137 != (*(this->part_vals)).end(); ++_iter1137)
+    std::vector<std::string> ::const_iterator _iter1148;
+    for (_iter1148 = (*(this->part_vals)).begin(); _iter1148 != (*(this->part_vals)).end(); ++_iter1148)
     {
-      xfer += oprot->writeString((*_iter1137));
+      xfer += oprot->writeString((*_iter1148));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16047,14 +16318,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1138;
-            ::apache::thrift::protocol::TType _etype1141;
-            xfer += iprot->readListBegin(_etype1141, _size1138);
-            this->success.resize(_size1138);
-            uint32_t _i1142;
-            for (_i1142 = 0; _i1142 < _size1138; ++_i1142)
+            uint32_t _size1149;
+            ::apache::thrift::protocol::TType _etype1152;
+            xfer += iprot->readListBegin(_etype1152, _size1149);
+            this->success.resize(_size1149);
+            uint32_t _i1153;
+            for (_i1153 = 0; _i1153 < _size1149; ++_i1153)
             {
-              xfer += iprot->readString(this->success[_i1142]);
+              xfer += iprot->readString(this->success[_i1153]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16101,10 +16372,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::write(::apache::thri
     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 _iter1143;
-      for (_iter1143 = this->success.begin(); _iter1143 != this->success.end(); ++_iter1143)
+      std::vector<std::string> ::const_iterator _ite

<TRUNCATED>

[15/15] hive git commit: HIVE-14638 : handle unions (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-14638 : handle unions (Sergey Shelukhin)


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

Branch: refs/heads/hive-14535
Commit: 0ce24b93e1ba92930c316dee0eb1262a27a101c8
Parents: 754443e
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Oct 4 11:46:19 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Oct 4 11:46:19 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/common/HiveStatsUtils.java      |  37 +-
 .../hadoop/hive/common/ValidWriteIds.java       |   5 -
 .../hadoop/hive/ql/exec/FileSinkOperator.java   | 137 ++++---
 .../hadoop/hive/ql/io/HiveInputFormat.java      |  46 ++-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   3 +
 ql/src/test/queries/clientpositive/mm_all.q     |  67 +++-
 ql/src/test/queries/clientpositive/mm_current.q |  31 +-
 .../results/clientpositive/llap/mm_all.q.out    | 377 ++++++++++++++++---
 .../clientpositive/llap/mm_current.q.out        | 186 +++++----
 9 files changed, 690 insertions(+), 199 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0ce24b93/common/src/java/org/apache/hadoop/hive/common/HiveStatsUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/HiveStatsUtils.java b/common/src/java/org/apache/hadoop/hive/common/HiveStatsUtils.java
index 111d99c..745a868 100644
--- a/common/src/java/org/apache/hadoop/hive/common/HiveStatsUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/HiveStatsUtils.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.common;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
@@ -30,6 +31,8 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Lists;
+
 /**
  * HiveStatsUtils.
  * A collection of utilities used for hive statistics.
@@ -53,11 +56,17 @@ public class HiveStatsUtils {
    */
   public static FileStatus[] getFileStatusRecurse(Path path, int level,  FileSystem fs)
       throws IOException {
-    return getFileStatusRecurse(path, level, fs, FileUtils.HIDDEN_FILES_PATH_FILTER);
+    return getFileStatusRecurse(path, level, fs, FileUtils.HIDDEN_FILES_PATH_FILTER, false);
   }
 
   public static FileStatus[] getFileStatusRecurse(
       Path path, int level, FileSystem fs, PathFilter filter) throws IOException {
+    return getFileStatusRecurse(path, level, fs, filter, false);
+  }
+
+  public static FileStatus[] getFileStatusRecurse(
+      Path path, int level, FileSystem fs, PathFilter filter, boolean allLevelsBelow)
+          throws IOException {
 
     // if level is <0, the return all files/directories under the specified path
     if (level < 0) {
@@ -81,7 +90,31 @@ public class HiveStatsUtils {
       sb.append(Path.SEPARATOR).append("*");
     }
     Path pathPattern = new Path(path, sb.toString());
-    return fs.globStatus(pathPattern, filter);
+    if (!allLevelsBelow) {
+      return fs.globStatus(pathPattern, filter);
+    }
+    LinkedList<FileStatus> queue = new LinkedList<>();
+    List<FileStatus> results = new ArrayList<FileStatus>();
+    for (FileStatus status : fs.globStatus(pathPattern)) {
+      if (filter.accept(status.getPath())) {
+        results.add(status);
+      }
+      if (status.isDirectory()) {
+        queue.add(status);
+      }
+    }
+    while (!queue.isEmpty()) {
+      FileStatus status = queue.poll();
+      for (FileStatus child : fs.listStatus(status.getPath())) {
+        if (filter.accept(child.getPath())) {
+          results.add(child);
+        }
+        if (child.isDirectory()) {
+          queue.add(child);
+        }
+      }
+    }
+    return results.toArray(new FileStatus[results.size()]);
   }
 
   public static int getNumBitVectorsForNDVEstimation(Configuration conf) throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/0ce24b93/common/src/java/org/apache/hadoop/hive/common/ValidWriteIds.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/ValidWriteIds.java b/common/src/java/org/apache/hadoop/hive/common/ValidWriteIds.java
index b939b43..160f4c0 100644
--- a/common/src/java/org/apache/hadoop/hive/common/ValidWriteIds.java
+++ b/common/src/java/org/apache/hadoop/hive/common/ValidWriteIds.java
@@ -116,11 +116,6 @@ public class ValidWriteIds {
     return ids != null && (areIdsValid == ids.contains(writeId));
   }
 
-  public boolean isValidInput(Path file) {
-    Long writeId = extractWriteId(file);
-    return (writeId != null) && isValid(writeId);
-  }
-
   public static String getMmFilePrefix(long mmWriteId) {
     return MM_PREFIX + "_" + mmWriteId;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0ce24b93/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
index f11a7c3..00115fe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
@@ -20,15 +20,12 @@ package org.apache.hadoop.hive.ql.exec;
 
 import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_TEMPORARY_TABLE_STORAGE;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.Serializable;
 import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -40,6 +37,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.HiveStatsUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
@@ -96,6 +94,7 @@ import com.google.common.collect.Lists;
 /**
  * File Sink operator implementation.
  **/
+@SuppressWarnings("deprecation")
 public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     Serializable {
 
@@ -386,12 +385,16 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     // 'Parent'
     if ((!conf.isLinkedFileSink()) || (dpCtx == null)) {
       specPath = conf.getDirName();
+      Utilities.LOG14535.info("Setting up FSOP " + System.identityHashCode(this) + " ("
+          + conf.isLinkedFileSink() + ") with " + taskId + " and " + specPath);
       childSpecPathDynLinkedPartitions = null;
       return;
     }
 
     specPath = conf.getParentDir();
     childSpecPathDynLinkedPartitions = conf.getDirName().getName();
+    Utilities.LOG14535.info("Setting up FSOP " + System.identityHashCode(this) + " ("
+        + conf.isLinkedFileSink() + ") with " + taskId + " and " + specPath);
   }
 
   /** Kryo ctor. */
@@ -1126,7 +1129,8 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
         }
       }
       if (!commitPaths.isEmpty()) {
-        Path manifestPath = new Path(specPath, "_tmp." + ValidWriteIds.getMmFilePrefix(
+        Path manifestPath = getManifestDir(specPath, childSpecPathDynLinkedPartitions);
+        manifestPath = new Path(manifestPath, "_tmp." + ValidWriteIds.getMmFilePrefix(
             conf.getMmWriteId()) + "_" + taskId + MANIFEST_EXTENSION);
         Utilities.LOG14535.info("Writing manifest to " + manifestPath + " with " + commitPaths);
         try {
@@ -1161,6 +1165,10 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     super.closeOp(abort);
   }
 
+  private static Path getManifestDir(Path specPath, String unionSuffix) {
+    return (unionSuffix == null) ? specPath : new Path(specPath, unionSuffix);
+  }
+
   /**
    * @return the name of the operator
    */
@@ -1179,15 +1187,17 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     try {
       if ((conf != null) && isNativeTable) {
         Path specPath = conf.getDirName();
+        String unionSuffix = null;
         DynamicPartitionCtx dpCtx = conf.getDynPartCtx();
         if (conf.isLinkedFileSink() && (dpCtx != null)) {
           specPath = conf.getParentDir();
           Utilities.LOG14535.info("Setting specPath to " + specPath + " for dynparts");
+          unionSuffix = conf.getDirName().getName();
         }
         if (!conf.isMmTable()) {
           Utilities.mvFileToFinalPath(specPath, hconf, success, LOG, dpCtx, conf, reporter); // TODO# other callers
         } else {
-          handleMmTable(specPath, hconf, success, dpCtx, conf, reporter);
+          handleMmTable(specPath, unionSuffix, hconf, success, dpCtx, conf, reporter);
         }
       }
     } catch (IOException e) {
@@ -1196,33 +1206,64 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     super.jobCloseOp(hconf, success);
   }
 
-  private void handleMmTable(Path specPath, Configuration hconf, boolean success,
-      DynamicPartitionCtx dpCtx, FileSinkDesc conf, Reporter reporter)
+  private static FileStatus[] getMmDirectoryCandidates(FileSystem fs, Path path,
+      int dpLevels, String unionSuffix, PathFilter filter) throws IOException {
+    StringBuilder sb = new StringBuilder(path.toUri().getPath());
+    for (int i = 0; i < dpLevels; i++) {
+      sb.append(Path.SEPARATOR).append("*");
+    }
+    if (unionSuffix != null) {
+      sb.append(Path.SEPARATOR).append(unionSuffix);
+    }
+    sb.append(Path.SEPARATOR).append("*"); // TODO: we could add exact mm prefix here
+    Path pathPattern = new Path(path, sb.toString());
+    return fs.globStatus(pathPattern, filter);
+  }
+
+  private void handleMmTable(Path specPath, String unionSuffix, Configuration hconf,
+      boolean success, DynamicPartitionCtx dpCtx, FileSinkDesc conf, Reporter reporter)
           throws IOException, HiveException {
     FileSystem fs = specPath.getFileSystem(hconf);
     // Manifests would be at the root level, but the results at target level.
     // TODO# special case - doesn't take bucketing into account
-    int targetLevel = (dpCtx == null) ? 1 : (dpCtx.getNumDPCols() + 1);
-    int manifestLevel = 1;
+    Path manifestDir = getManifestDir(specPath, unionSuffix);
+
     ValidWriteIds.IdPathFilter filter = new ValidWriteIds.IdPathFilter(conf.getMmWriteId(), true);
     if (!success) {
-      deleteMatchingFiles(specPath, fs, targetLevel, filter);
-      deleteMatchingFiles(specPath, fs, manifestLevel, filter);
+      tryDeleteAllMmFiles(fs, specPath, manifestDir, dpCtx, unionSuffix, filter);
       return;
     }
-    FileStatus[] files = HiveStatsUtils.getFileStatusRecurse(specPath, manifestLevel, fs, filter);
-    List<Path> manifests = new ArrayList<>(files.length);
+    FileStatus[] files = HiveStatsUtils.getFileStatusRecurse(manifestDir, 1, fs, filter);
+    Utilities.LOG14535.info("Looking for manifests in: " + manifestDir);
+    List<Path> manifests = new ArrayList<>();
     if (files != null) {
       for (FileStatus status : files) {
-        if (status.getPath().getName().endsWith(MANIFEST_EXTENSION)) {
-          manifests.add(status.getPath());
-        } else if (!status.isDirectory()) {
-          Path path = status.getPath();
-          Utilities.LOG14535.warn("Unknown file found - neither a manifest nor directory: " + path);
-          tryDelete(fs, path);
+        Path path = status.getPath();
+        if (path.getName().endsWith(MANIFEST_EXTENSION)) {
+          manifests.add(path);
         }
       }
     }
+
+    Utilities.LOG14535.info("Looking for files in: " + specPath);
+    files = getMmDirectoryCandidates(fs, specPath,
+        dpCtx == null ? 0 : dpCtx.getNumDPCols(), unionSuffix, filter);
+    ArrayList<FileStatus> results = new ArrayList<>();
+    if (files != null) {
+      for (FileStatus status : files) {
+        Path path = status.getPath();
+        Utilities.LOG14535.info("Looking at path: " + path + " from " + System.identityHashCode(this));
+        if (!status.isDirectory()) {
+          if (!path.getName().endsWith(MANIFEST_EXTENSION)) {
+            Utilities.LOG14535.warn("Unknown file found, deleting: " + path);
+            tryDelete(fs, path);
+          }
+        } else {
+          results.add(status);
+        }
+      }
+    }
+
     HashSet<String> committed = new HashSet<>();
     for (Path mfp : manifests) {
       try (FSDataInputStream mdis = fs.open(mfp)) {
@@ -1236,22 +1277,14 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
       }
     }
 
-    files = HiveStatsUtils.getFileStatusRecurse(specPath, targetLevel, fs, filter);
-    LinkedList<FileStatus> results = new LinkedList<>();
-    for (FileStatus status : files) {
-      if (!status.isDirectory()) {
-        Path path = status.getPath();
-        Utilities.LOG14535.warn("Unknown file found - neither a manifest nor directory: " + path);
-        tryDelete(fs, path);
-      } else {
-        for (FileStatus child : fs.listStatus(status.getPath())) {
-          Path path = child.getPath();
-          if (committed.remove(path.toString())) continue; // A good file.
-          Utilities.LOG14535.info("Deleting " + path + " that was not committed");
-          // We should actually succeed here - if we fail, don't commit the query.
-          if (!fs.delete(path, true)) {
-            throw new HiveException("Failed to delete an uncommitted path " + path);
-          }
+    for (FileStatus status : results) {
+      for (FileStatus child : fs.listStatus(status.getPath())) {
+        Path childPath = child.getPath();
+        if (committed.remove(childPath.toString())) continue; // A good file.
+        Utilities.LOG14535.info("Deleting " + childPath + " that was not committed");
+        // We should actually succeed here - if we fail, don't commit the query.
+        if (!fs.delete(childPath, true)) {
+          throw new HiveException("Failed to delete an uncommitted path " + childPath);
         }
       }
     }
@@ -1263,11 +1296,19 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
       Utilities.LOG14535.info("Deleting manifest " + mfp);
       tryDelete(fs, mfp);
     }
+    // Delete the manifest directory if we only created it for manifests; otherwise the
+    // dynamic partition loader will find it and try to load it as a partition... what a mess.
+    if (manifestDir != specPath) {
+      FileStatus[] remainingFiles = fs.listStatus(manifestDir);
+      if (remainingFiles == null || remainingFiles.length == 0) {
+        Utilities.LOG14535.info("Deleting directory " + manifestDir);
+        tryDelete(fs, manifestDir);
+      }
+    }
 
     if (results.isEmpty()) return;
     FileStatus[] finalResults = results.toArray(new FileStatus[results.size()]);
 
-    // TODO# dp may break - removeTempOrDuplicateFiles assumes dirs in results. Why? We recurse...
     List<Path> emptyBuckets = Utilities.removeTempOrDuplicateFiles(
         fs, finalResults, dpCtx, conf, hconf);
     // create empty buckets if necessary
@@ -1276,15 +1317,27 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     }
   }
 
-  private void deleteMatchingFiles(Path specPath, FileSystem fs,
-      int targetLevel, ValidWriteIds.IdPathFilter filter) throws IOException {
-    for (FileStatus status : HiveStatsUtils.getFileStatusRecurse(specPath, targetLevel, fs,
-        filter)) {
-      Utilities.LOG14535.info("Deleting " + status.getPath() + " on failure");
-      tryDelete(fs, status.getPath());
+  private void tryDeleteAllMmFiles(FileSystem fs, Path specPath, Path manifestDir,
+      DynamicPartitionCtx dpCtx, String unionSuffix,
+      ValidWriteIds.IdPathFilter filter) throws IOException {
+    FileStatus[] files = getMmDirectoryCandidates(fs, specPath,
+        dpCtx == null ? 0 : dpCtx.getNumDPCols(), unionSuffix, filter);
+    if (files != null) {
+      for (FileStatus status : files) {
+        Utilities.LOG14535.info("Deleting " + status.getPath() + " on failure");
+        tryDelete(fs, status.getPath());
+      }
+    }
+    files = HiveStatsUtils.getFileStatusRecurse(manifestDir, 1, fs, filter);
+    if (files != null) {
+      for (FileStatus status : files) {
+        Utilities.LOG14535.info("Deleting " + status.getPath() + " on failure");
+        tryDelete(fs, status.getPath());
+      }
     }
   }
 
+
   private void tryDelete(FileSystem fs, Path path) {
     try {
       fs.delete(path, true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0ce24b93/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
index 0510e08..c3e2681 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
@@ -29,6 +29,7 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -42,6 +43,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.ValidWriteIds;
@@ -352,7 +354,9 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
       TableDesc table, Map<String, ValidWriteIds> writeIdMap, List<InputSplit> result)
           throws IOException {
     ValidWriteIds writeIds = extractWriteIds(writeIdMap, conf, table.getTableName());
-    Utilities.LOG14535.info("Observing " + table.getTableName() + ": " + writeIds);
+    if (writeIds != null) {
+      Utilities.LOG14535.info("Observing " + table.getTableName() + ": " + writeIds);
+    }
 
     Utilities.copyTablePropertiesToConf(table, conf);
 
@@ -394,22 +398,40 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
 
   private void processForWriteIds(Path dir, JobConf conf,
       ValidWriteIds writeIds, List<Path> finalPaths) throws IOException {
-    FileStatus[] files = dir.getFileSystem(conf).listStatus(dir); // TODO: batch?
+    FileSystem fs = dir.getFileSystem(conf);
+    FileStatus[] files = fs.listStatus(dir); // TODO: batch?
+    LinkedList<Path> subdirs = new LinkedList<>();
     for (FileStatus file : files) {
-      Path subdir = file.getPath();
-      if (!file.isDirectory()) {
-        Utilities.LOG14535.warn("Found a file not in subdirectory " + subdir);
-        continue;
-      }
-      if (!writeIds.isValidInput(subdir)) {
-        Utilities.LOG14535.warn("Ignoring an uncommitted directory " + subdir);
-        continue;
+      handleNonMmDirChild(file, writeIds, subdirs, finalPaths);
+    }
+    while (!subdirs.isEmpty()) {
+      Path subdir = subdirs.poll();
+      for (FileStatus file : fs.listStatus(subdir)) {
+        handleNonMmDirChild(file, writeIds, subdirs, finalPaths);
       }
-      Utilities.LOG14535.info("Adding input " + subdir);
-      finalPaths.add(subdir);
     }
   }
 
+  private void handleNonMmDirChild(FileStatus file, ValidWriteIds writeIds,
+      LinkedList<Path> subdirs, List<Path> finalPaths) {
+    Path path = file.getPath();
+    if (!file.isDirectory()) {
+      Utilities.LOG14535.warn("Ignoring a file not in MM directory " + path);
+      return;
+    }
+    Long writeId = ValidWriteIds.extractWriteId(path);
+    if (writeId == null) {
+      subdirs.add(path);
+      return;
+    }
+    if (!writeIds.isValid(writeId)) {
+      Utilities.LOG14535.warn("Ignoring an uncommitted directory " + path);
+      return;
+    }
+    Utilities.LOG14535.info("Adding input " + path);
+    finalPaths.add(path);
+  }
+
   Path[] getInputPaths(JobConf job) throws IOException {
     Path[] dirs;
     if (HiveConf.getVar(job, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("spark")) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0ce24b93/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 6cd0500..73a3b19 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
@@ -1597,6 +1597,7 @@ public class Hive {
               getConf(), new ValidWriteIds.IdPathFilter(mmWriteId, false));
         }
       } else {
+        Utilities.LOG14535.info("moving " + loadPath + " to " + newPartPath);
         if (replace || (oldPart == null && !isAcid)) {
           replaceFiles(tbl.getPath(), loadPath, newPartPath, oldPartPath, getConf(),
               isSrcLocal);
@@ -2016,6 +2017,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
       newFiles = Collections.synchronizedList(new ArrayList<Path>());
     }
     if (mmWriteId == null) {
+      Utilities.LOG14535.info("moving " + loadPath + " to " + tbl.getPath());
       if (replace) {
         Path tableDest = tbl.getPath();
         replaceFiles(tableDest, loadPath, tableDest, tableDest, sessionConf, isSrcLocal);
@@ -2029,6 +2031,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
         }
       }
     } else {
+      Utilities.LOG14535.info("not moving " + loadPath + " to " + tbl.getPath());
       if (replace) {
         Path tableDest = tbl.getPath();
         deleteOldPathForReplace(tableDest, tableDest, sessionConf,

http://git-wip-us.apache.org/repos/asf/hive/blob/0ce24b93/ql/src/test/queries/clientpositive/mm_all.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/mm_all.q b/ql/src/test/queries/clientpositive/mm_all.q
index 59171af..cc44c19 100644
--- a/ql/src/test/queries/clientpositive/mm_all.q
+++ b/ql/src/test/queries/clientpositive/mm_all.q
@@ -1,6 +1,10 @@
 set hive.mapred.mode=nonstrict;
 set hive.explain.user=false;
 set hive.fetch.task.conversion=none;
+set tez.grouping.min-size=1;
+set tez.grouping.max-size=2;
+set hive.exec.dynamic.partition.mode=nonstrict;
+
 
 -- Force multiple writers when reading
 drop table intermediate;
@@ -22,6 +26,8 @@ create table simple_mm(key int) stored as orc tblproperties ('hivecommit'='true'
 insert into table simple_mm select key from intermediate;
 insert overwrite table simple_mm select key from intermediate;
 select * from simple_mm;
+insert into table simple_mm select key from intermediate;
+select * from simple_mm;
 drop table simple_mm;
 
 
@@ -50,6 +56,65 @@ drop table dp_no_mm;
 drop table dp_mm;
 
 
+-- union
+
+create table union_mm(id int)  tblproperties ('hivecommit'='true'); 
+insert into table union_mm 
+select temps.p from (
+select key as p from intermediate 
+union all 
+select key + 1 as p from intermediate ) temps;
+
+select * from union_mm order by id;
+
+insert into table union_mm 
+select p from
+(
+select key + 1 as p from intermediate
+union all
+select key from intermediate
+) tab group by p
+union all
+select key + 2 as p from intermediate;
+
+select * from union_mm order by id;
+
+insert into table union_mm
+SELECT p FROM
+(
+  SELECT key + 1 as p FROM intermediate
+  UNION ALL
+  SELECT key as p FROM ( 
+    SELECT distinct key FROM (
+      SELECT key FROM (
+        SELECT key + 2 as key FROM intermediate
+        UNION ALL
+        SELECT key FROM intermediate
+      )t1 
+    group by key)t2
+  )t3
+)t4
+group by p;
+
+
+select * from union_mm order by id;
+drop table union_mm;
+
+
+create table partunion_mm(id int) partitioned by (key int) tblproperties ('hivecommit'='true'); 
+insert into table partunion_mm partition(key)
+select temps.* from (
+select key as p, key from intermediate 
+union all 
+select key + 1 as p, key + 1 from intermediate ) temps;
+
+select * from partunion_mm;
+drop table partunion_mm;
+
+-- TODO# from here, fix it
+
+
+
 
 -- future
 
@@ -110,7 +175,7 @@ drop table dp_mm;
 --INSERT OVERWRITE TABLE skew_mm
 --SELECT a.key as k1, a.val as k2, b.key as k3, b.val as k4 FROM T1 a JOIN T2 b ON a.key = b.key;
 --
----- TODO load, acid, etc
+---- TODO load, multi-insert etc
 --
 --
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0ce24b93/ql/src/test/queries/clientpositive/mm_current.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/mm_current.q b/ql/src/test/queries/clientpositive/mm_current.q
index b551176..e1fb3d9 100644
--- a/ql/src/test/queries/clientpositive/mm_current.q
+++ b/ql/src/test/queries/clientpositive/mm_current.q
@@ -12,27 +12,28 @@ insert into table intermediate partition(p='455') select key from src limit 2;
 insert into table intermediate partition(p='456') select key from src limit 2;
 
 
+create table partunion_no_mm(id int) partitioned by (key int); 
+insert into table partunion_no_mm partition(key)
+select temps.* from (
+select key as p, key from intermediate 
+union all 
+select key + 1 as p, key + 1 from intermediate ) temps;
 
-drop table dp_no_mm;
-drop table dp_mm;
+select * from partunion_no_mm;
+drop table partunion_no_mm;
 
-set hive.merge.mapredfiles=false;
-set hive.merge.sparkfiles=false;
-set hive.merge.tezfiles=false;
 
-create table dp_no_mm (key int) partitioned by (key1 string, key2 int) stored as orc;
-create table dp_mm (key int) partitioned by (key1 string, key2 int) stored as orc
-  tblproperties ('hivecommit'='true');
+create table partunion_mm(id int) partitioned by (key int) tblproperties ('hivecommit'='true'); 
+insert into table partunion_mm partition(key)
+select temps.* from (
+select key as p, key from intermediate 
+union all 
+select key + 1 as p, key + 1 from intermediate ) temps;
 
-insert into table dp_no_mm partition (key1='123', key2) select key, key from intermediate;
+select * from partunion_mm;
+drop table partunion_mm;
 
-insert into table dp_mm partition (key1='123', key2) select key, key from intermediate;
 
-select * from dp_no_mm;
-select * from dp_mm;
-
-drop table dp_no_mm;
-drop table dp_mm;
 
 drop table intermediate;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0ce24b93/ql/src/test/results/clientpositive/llap/mm_all.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/mm_all.q.out b/ql/src/test/results/clientpositive/llap/mm_all.q.out
index b0c9c0a..0a8bb40 100644
--- a/ql/src/test/results/clientpositive/llap/mm_all.q.out
+++ b/ql/src/test/results/clientpositive/llap/mm_all.q.out
@@ -147,18 +147,18 @@ POSTHOOK: Input: default@part_mm
 POSTHOOK: Input: default@part_mm@key_mm=455
 POSTHOOK: Input: default@part_mm@key_mm=456
 #### A masked pattern was here ####
-238	455
-86	455
-238	455
-86	455
-238	455
-86	455
-238	455
-86	455
-238	456
-86	456
-238	456
-86	456
+0	455
+455	455
+0	455
+455	455
+0	455
+455	455
+0	455
+455	455
+0	456
+455	456
+0	456
+455	456
 PREHOOK: query: drop table part_mm
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@part_mm
@@ -213,10 +213,39 @@ POSTHOOK: query: select * from simple_mm
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@simple_mm
 #### A masked pattern was here ####
-238
-86
-238
-86
+0
+455
+0
+455
+PREHOOK: query: insert into table simple_mm select key from intermediate
+PREHOOK: type: QUERY
+PREHOOK: Input: default@intermediate
+PREHOOK: Input: default@intermediate@p=455
+PREHOOK: Input: default@intermediate@p=456
+PREHOOK: Output: default@simple_mm
+POSTHOOK: query: insert into table simple_mm select key from intermediate
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@intermediate
+POSTHOOK: Input: default@intermediate@p=455
+POSTHOOK: Input: default@intermediate@p=456
+POSTHOOK: Output: default@simple_mm
+POSTHOOK: Lineage: simple_mm.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select * from simple_mm
+PREHOOK: type: QUERY
+PREHOOK: Input: default@simple_mm
+#### A masked pattern was here ####
+POSTHOOK: query: select * from simple_mm
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@simple_mm
+#### A masked pattern was here ####
+0
+455
+0
+455
+0
+455
+0
+455
 PREHOOK: query: drop table simple_mm
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@simple_mm
@@ -264,10 +293,10 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@intermediate
 POSTHOOK: Input: default@intermediate@p=455
 POSTHOOK: Input: default@intermediate@p=456
-POSTHOOK: Output: default@dp_no_mm@key1=123/key2=238
-POSTHOOK: Output: default@dp_no_mm@key1=123/key2=86
-POSTHOOK: Lineage: dp_no_mm PARTITION(key1=123,key2=238).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: dp_no_mm PARTITION(key1=123,key2=86).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Output: default@dp_no_mm@key1=123/key2=0
+POSTHOOK: Output: default@dp_no_mm@key1=123/key2=455
+POSTHOOK: Lineage: dp_no_mm PARTITION(key1=123,key2=0).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: dp_no_mm PARTITION(key1=123,key2=455).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 PREHOOK: query: insert into table dp_mm partition (key1='123', key2) select key, key from intermediate
 PREHOOK: type: QUERY
 PREHOOK: Input: default@intermediate
@@ -279,42 +308,42 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@intermediate
 POSTHOOK: Input: default@intermediate@p=455
 POSTHOOK: Input: default@intermediate@p=456
-POSTHOOK: Output: default@dp_mm@key1=123/key2=238
-POSTHOOK: Output: default@dp_mm@key1=123/key2=86
-POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=238).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=86).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Output: default@dp_mm@key1=123/key2=0
+POSTHOOK: Output: default@dp_mm@key1=123/key2=455
+POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=0).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=455).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 PREHOOK: query: select * from dp_no_mm
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dp_no_mm
-PREHOOK: Input: default@dp_no_mm@key1=123/key2=238
-PREHOOK: Input: default@dp_no_mm@key1=123/key2=86
+PREHOOK: Input: default@dp_no_mm@key1=123/key2=0
+PREHOOK: Input: default@dp_no_mm@key1=123/key2=455
 #### A masked pattern was here ####
 POSTHOOK: query: select * from dp_no_mm
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@dp_no_mm
-POSTHOOK: Input: default@dp_no_mm@key1=123/key2=238
-POSTHOOK: Input: default@dp_no_mm@key1=123/key2=86
+POSTHOOK: Input: default@dp_no_mm@key1=123/key2=0
+POSTHOOK: Input: default@dp_no_mm@key1=123/key2=455
 #### A masked pattern was here ####
-238	123	238
-238	123	238
-86	123	86
-86	123	86
+455	123	455
+455	123	455
+0	123	0
+0	123	0
 PREHOOK: query: select * from dp_mm
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dp_mm
-PREHOOK: Input: default@dp_mm@key1=123/key2=238
-PREHOOK: Input: default@dp_mm@key1=123/key2=86
+PREHOOK: Input: default@dp_mm@key1=123/key2=0
+PREHOOK: Input: default@dp_mm@key1=123/key2=455
 #### A masked pattern was here ####
 POSTHOOK: query: select * from dp_mm
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@dp_mm
-POSTHOOK: Input: default@dp_mm@key1=123/key2=238
-POSTHOOK: Input: default@dp_mm@key1=123/key2=86
+POSTHOOK: Input: default@dp_mm@key1=123/key2=0
+POSTHOOK: Input: default@dp_mm@key1=123/key2=455
 #### A masked pattern was here ####
-238	123	238
-238	123	238
-86	123	86
-86	123	86
+455	123	455
+455	123	455
+0	123	0
+0	123	0
 PREHOOK: query: drop table dp_no_mm
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@dp_no_mm
@@ -331,7 +360,262 @@ POSTHOOK: query: drop table dp_mm
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@dp_mm
 POSTHOOK: Output: default@dp_mm
-PREHOOK: query: -- future
+PREHOOK: query: -- union
+
+create table union_mm(id int)  tblproperties ('hivecommit'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@union_mm
+POSTHOOK: query: -- union
+
+create table union_mm(id int)  tblproperties ('hivecommit'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@union_mm
+PREHOOK: query: insert into table union_mm 
+select temps.p from (
+select key as p from intermediate 
+union all 
+select key + 1 as p from intermediate ) temps
+PREHOOK: type: QUERY
+PREHOOK: Input: default@intermediate
+PREHOOK: Input: default@intermediate@p=455
+PREHOOK: Input: default@intermediate@p=456
+PREHOOK: Output: default@union_mm
+POSTHOOK: query: insert into table union_mm 
+select temps.p from (
+select key as p from intermediate 
+union all 
+select key + 1 as p from intermediate ) temps
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@intermediate
+POSTHOOK: Input: default@intermediate@p=455
+POSTHOOK: Input: default@intermediate@p=456
+POSTHOOK: Output: default@union_mm
+POSTHOOK: Lineage: union_mm.id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select * from union_mm order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@union_mm
+#### A masked pattern was here ####
+POSTHOOK: query: select * from union_mm order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@union_mm
+#### A masked pattern was here ####
+0
+0
+1
+1
+455
+455
+456
+456
+PREHOOK: query: insert into table union_mm 
+select p from
+(
+select key + 1 as p from intermediate
+union all
+select key from intermediate
+) tab group by p
+union all
+select key + 2 as p from intermediate
+PREHOOK: type: QUERY
+PREHOOK: Input: default@intermediate
+PREHOOK: Input: default@intermediate@p=455
+PREHOOK: Input: default@intermediate@p=456
+PREHOOK: Output: default@union_mm
+POSTHOOK: query: insert into table union_mm 
+select p from
+(
+select key + 1 as p from intermediate
+union all
+select key from intermediate
+) tab group by p
+union all
+select key + 2 as p from intermediate
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@intermediate
+POSTHOOK: Input: default@intermediate@p=455
+POSTHOOK: Input: default@intermediate@p=456
+POSTHOOK: Output: default@union_mm
+POSTHOOK: Lineage: union_mm.id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select * from union_mm order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@union_mm
+#### A masked pattern was here ####
+POSTHOOK: query: select * from union_mm order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@union_mm
+#### A masked pattern was here ####
+0
+0
+0
+1
+1
+1
+2
+2
+455
+455
+455
+456
+456
+456
+457
+457
+PREHOOK: query: insert into table union_mm
+SELECT p FROM
+(
+  SELECT key + 1 as p FROM intermediate
+  UNION ALL
+  SELECT key as p FROM ( 
+    SELECT distinct key FROM (
+      SELECT key FROM (
+        SELECT key + 2 as key FROM intermediate
+        UNION ALL
+        SELECT key FROM intermediate
+      )t1 
+    group by key)t2
+  )t3
+)t4
+group by p
+PREHOOK: type: QUERY
+PREHOOK: Input: default@intermediate
+PREHOOK: Input: default@intermediate@p=455
+PREHOOK: Input: default@intermediate@p=456
+PREHOOK: Output: default@union_mm
+POSTHOOK: query: insert into table union_mm
+SELECT p FROM
+(
+  SELECT key + 1 as p FROM intermediate
+  UNION ALL
+  SELECT key as p FROM ( 
+    SELECT distinct key FROM (
+      SELECT key FROM (
+        SELECT key + 2 as key FROM intermediate
+        UNION ALL
+        SELECT key FROM intermediate
+      )t1 
+    group by key)t2
+  )t3
+)t4
+group by p
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@intermediate
+POSTHOOK: Input: default@intermediate@p=455
+POSTHOOK: Input: default@intermediate@p=456
+POSTHOOK: Output: default@union_mm
+POSTHOOK: Lineage: union_mm.id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select * from union_mm order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@union_mm
+#### A masked pattern was here ####
+POSTHOOK: query: select * from union_mm order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@union_mm
+#### A masked pattern was here ####
+0
+0
+0
+0
+1
+1
+1
+1
+2
+2
+2
+455
+455
+455
+455
+456
+456
+456
+456
+457
+457
+457
+PREHOOK: query: drop table union_mm
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@union_mm
+PREHOOK: Output: default@union_mm
+POSTHOOK: query: drop table union_mm
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@union_mm
+POSTHOOK: Output: default@union_mm
+PREHOOK: query: create table partunion_mm(id int) partitioned by (key int) tblproperties ('hivecommit'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@partunion_mm
+POSTHOOK: query: create table partunion_mm(id int) partitioned by (key int) tblproperties ('hivecommit'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@partunion_mm
+PREHOOK: query: insert into table partunion_mm partition(key)
+select temps.* from (
+select key as p, key from intermediate 
+union all 
+select key + 1 as p, key + 1 from intermediate ) temps
+PREHOOK: type: QUERY
+PREHOOK: Input: default@intermediate
+PREHOOK: Input: default@intermediate@p=455
+PREHOOK: Input: default@intermediate@p=456
+PREHOOK: Output: default@partunion_mm
+POSTHOOK: query: insert into table partunion_mm partition(key)
+select temps.* from (
+select key as p, key from intermediate 
+union all 
+select key + 1 as p, key + 1 from intermediate ) temps
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@intermediate
+POSTHOOK: Input: default@intermediate@p=455
+POSTHOOK: Input: default@intermediate@p=456
+POSTHOOK: Output: default@partunion_mm@key=0
+POSTHOOK: Output: default@partunion_mm@key=1
+POSTHOOK: Output: default@partunion_mm@key=455
+POSTHOOK: Output: default@partunion_mm@key=456
+POSTHOOK: Lineage: partunion_mm PARTITION(key=0).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: partunion_mm PARTITION(key=1).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: partunion_mm PARTITION(key=455).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: partunion_mm PARTITION(key=456).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select * from partunion_mm
+PREHOOK: type: QUERY
+PREHOOK: Input: default@partunion_mm
+PREHOOK: Input: default@partunion_mm@key=0
+PREHOOK: Input: default@partunion_mm@key=1
+PREHOOK: Input: default@partunion_mm@key=455
+PREHOOK: Input: default@partunion_mm@key=456
+#### A masked pattern was here ####
+POSTHOOK: query: select * from partunion_mm
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@partunion_mm
+POSTHOOK: Input: default@partunion_mm@key=0
+POSTHOOK: Input: default@partunion_mm@key=1
+POSTHOOK: Input: default@partunion_mm@key=455
+POSTHOOK: Input: default@partunion_mm@key=456
+#### A masked pattern was here ####
+0	0
+0	0
+1	1
+1	1
+455	455
+455	455
+456	456
+456	456
+PREHOOK: query: drop table partunion_mm
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@partunion_mm
+PREHOOK: Output: default@partunion_mm
+POSTHOOK: query: drop table partunion_mm
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@partunion_mm
+POSTHOOK: Output: default@partunion_mm
+PREHOOK: query: -- TODO# from here, fix it
+
+
+
+
+-- future
 
 
 
@@ -390,7 +674,7 @@ PREHOOK: query: -- future
 --INSERT OVERWRITE TABLE skew_mm
 --SELECT a.key as k1, a.val as k2, b.key as k3, b.val as k4 FROM T1 a JOIN T2 b ON a.key = b.key;
 --
----- TODO load, acid, etc
+---- TODO load, multi-insert etc
 --
 --
 
@@ -398,7 +682,12 @@ drop table intermediate
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@intermediate
 PREHOOK: Output: default@intermediate
-POSTHOOK: query: -- future
+POSTHOOK: query: -- TODO# from here, fix it
+
+
+
+
+-- future
 
 
 
@@ -457,7 +746,7 @@ POSTHOOK: query: -- future
 --INSERT OVERWRITE TABLE skew_mm
 --SELECT a.key as k1, a.val as k2, b.key as k3, b.val as k4 FROM T1 a JOIN T2 b ON a.key = b.key;
 --
----- TODO load, acid, etc
+---- TODO load, multi-insert etc
 --
 --
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0ce24b93/ql/src/test/results/clientpositive/llap/mm_current.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/mm_current.q.out b/ql/src/test/results/clientpositive/llap/mm_current.q.out
index fe1caee..87214ba 100644
--- a/ql/src/test/results/clientpositive/llap/mm_current.q.out
+++ b/ql/src/test/results/clientpositive/llap/mm_current.q.out
@@ -28,110 +28,140 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@intermediate@p=456
 POSTHOOK: Lineage: intermediate PARTITION(p=456).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-PREHOOK: query: drop table dp_no_mm
-PREHOOK: type: DROPTABLE
-POSTHOOK: query: drop table dp_no_mm
-POSTHOOK: type: DROPTABLE
-PREHOOK: query: drop table dp_mm
-PREHOOK: type: DROPTABLE
-POSTHOOK: query: drop table dp_mm
-POSTHOOK: type: DROPTABLE
-PREHOOK: query: create table dp_no_mm (key int) partitioned by (key1 string, key2 int) stored as orc
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dp_no_mm
-POSTHOOK: query: create table dp_no_mm (key int) partitioned by (key1 string, key2 int) stored as orc
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dp_no_mm
-PREHOOK: query: create table dp_mm (key int) partitioned by (key1 string, key2 int) stored as orc
-  tblproperties ('hivecommit'='true')
+PREHOOK: query: create table partunion_no_mm(id int) partitioned by (key int)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
-PREHOOK: Output: default@dp_mm
-POSTHOOK: query: create table dp_mm (key int) partitioned by (key1 string, key2 int) stored as orc
-  tblproperties ('hivecommit'='true')
+PREHOOK: Output: default@partunion_no_mm
+POSTHOOK: query: create table partunion_no_mm(id int) partitioned by (key int)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dp_mm
-PREHOOK: query: insert into table dp_no_mm partition (key1='123', key2) select key, key from intermediate
+POSTHOOK: Output: default@partunion_no_mm
+PREHOOK: query: insert into table partunion_no_mm partition(key)
+select temps.* from (
+select key as p, key from intermediate 
+union all 
+select key + 1 as p, key + 1 from intermediate ) temps
 PREHOOK: type: QUERY
 PREHOOK: Input: default@intermediate
 PREHOOK: Input: default@intermediate@p=455
 PREHOOK: Input: default@intermediate@p=456
-PREHOOK: Output: default@dp_no_mm@key1=123
-POSTHOOK: query: insert into table dp_no_mm partition (key1='123', key2) select key, key from intermediate
+PREHOOK: Output: default@partunion_no_mm
+POSTHOOK: query: insert into table partunion_no_mm partition(key)
+select temps.* from (
+select key as p, key from intermediate 
+union all 
+select key + 1 as p, key + 1 from intermediate ) temps
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@intermediate
 POSTHOOK: Input: default@intermediate@p=455
 POSTHOOK: Input: default@intermediate@p=456
-POSTHOOK: Output: default@dp_no_mm@key1=123/key2=0
-POSTHOOK: Output: default@dp_no_mm@key1=123/key2=455
-POSTHOOK: Lineage: dp_no_mm PARTITION(key1=123,key2=0).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: dp_no_mm PARTITION(key1=123,key2=455).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-PREHOOK: query: insert into table dp_mm partition (key1='123', key2) select key, key from intermediate
+POSTHOOK: Output: default@partunion_no_mm@key=0
+POSTHOOK: Output: default@partunion_no_mm@key=1
+POSTHOOK: Output: default@partunion_no_mm@key=455
+POSTHOOK: Output: default@partunion_no_mm@key=456
+POSTHOOK: Lineage: partunion_no_mm PARTITION(key=0).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: partunion_no_mm PARTITION(key=1).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: partunion_no_mm PARTITION(key=455).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: partunion_no_mm PARTITION(key=456).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select * from partunion_no_mm
+PREHOOK: type: QUERY
+PREHOOK: Input: default@partunion_no_mm
+PREHOOK: Input: default@partunion_no_mm@key=0
+PREHOOK: Input: default@partunion_no_mm@key=1
+PREHOOK: Input: default@partunion_no_mm@key=455
+PREHOOK: Input: default@partunion_no_mm@key=456
+#### A masked pattern was here ####
+POSTHOOK: query: select * from partunion_no_mm
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@partunion_no_mm
+POSTHOOK: Input: default@partunion_no_mm@key=0
+POSTHOOK: Input: default@partunion_no_mm@key=1
+POSTHOOK: Input: default@partunion_no_mm@key=455
+POSTHOOK: Input: default@partunion_no_mm@key=456
+#### A masked pattern was here ####
+0	0
+0	0
+1	1
+1	1
+455	455
+455	455
+456	456
+456	456
+PREHOOK: query: drop table partunion_no_mm
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@partunion_no_mm
+PREHOOK: Output: default@partunion_no_mm
+POSTHOOK: query: drop table partunion_no_mm
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@partunion_no_mm
+POSTHOOK: Output: default@partunion_no_mm
+PREHOOK: query: create table partunion_mm(id int) partitioned by (key int) tblproperties ('hivecommit'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@partunion_mm
+POSTHOOK: query: create table partunion_mm(id int) partitioned by (key int) tblproperties ('hivecommit'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@partunion_mm
+PREHOOK: query: insert into table partunion_mm partition(key)
+select temps.* from (
+select key as p, key from intermediate 
+union all 
+select key + 1 as p, key + 1 from intermediate ) temps
 PREHOOK: type: QUERY
 PREHOOK: Input: default@intermediate
 PREHOOK: Input: default@intermediate@p=455
 PREHOOK: Input: default@intermediate@p=456
-PREHOOK: Output: default@dp_mm@key1=123
-POSTHOOK: query: insert into table dp_mm partition (key1='123', key2) select key, key from intermediate
+PREHOOK: Output: default@partunion_mm
+POSTHOOK: query: insert into table partunion_mm partition(key)
+select temps.* from (
+select key as p, key from intermediate 
+union all 
+select key + 1 as p, key + 1 from intermediate ) temps
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@intermediate
 POSTHOOK: Input: default@intermediate@p=455
 POSTHOOK: Input: default@intermediate@p=456
-POSTHOOK: Output: default@dp_mm@key1=123/key2=0
-POSTHOOK: Output: default@dp_mm@key1=123/key2=455
-POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=0).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=455).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-PREHOOK: query: select * from dp_no_mm
+POSTHOOK: Output: default@partunion_mm@key=0
+POSTHOOK: Output: default@partunion_mm@key=1
+POSTHOOK: Output: default@partunion_mm@key=455
+POSTHOOK: Output: default@partunion_mm@key=456
+POSTHOOK: Lineage: partunion_mm PARTITION(key=0).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: partunion_mm PARTITION(key=1).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: partunion_mm PARTITION(key=455).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: partunion_mm PARTITION(key=456).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select * from partunion_mm
 PREHOOK: type: QUERY
-PREHOOK: Input: default@dp_no_mm
-PREHOOK: Input: default@dp_no_mm@key1=123/key2=0
-PREHOOK: Input: default@dp_no_mm@key1=123/key2=455
+PREHOOK: Input: default@partunion_mm
+PREHOOK: Input: default@partunion_mm@key=0
+PREHOOK: Input: default@partunion_mm@key=1
+PREHOOK: Input: default@partunion_mm@key=455
+PREHOOK: Input: default@partunion_mm@key=456
 #### A masked pattern was here ####
-POSTHOOK: query: select * from dp_no_mm
+POSTHOOK: query: select * from partunion_mm
 POSTHOOK: type: QUERY
-POSTHOOK: Input: default@dp_no_mm
-POSTHOOK: Input: default@dp_no_mm@key1=123/key2=0
-POSTHOOK: Input: default@dp_no_mm@key1=123/key2=455
+POSTHOOK: Input: default@partunion_mm
+POSTHOOK: Input: default@partunion_mm@key=0
+POSTHOOK: Input: default@partunion_mm@key=1
+POSTHOOK: Input: default@partunion_mm@key=455
+POSTHOOK: Input: default@partunion_mm@key=456
 #### A masked pattern was here ####
-455	123	455
-455	123	455
-0	123	0
-0	123	0
-PREHOOK: query: select * from dp_mm
-PREHOOK: type: QUERY
-PREHOOK: Input: default@dp_mm
-PREHOOK: Input: default@dp_mm@key1=123/key2=0
-PREHOOK: Input: default@dp_mm@key1=123/key2=455
-#### A masked pattern was here ####
-POSTHOOK: query: select * from dp_mm
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@dp_mm
-POSTHOOK: Input: default@dp_mm@key1=123/key2=0
-POSTHOOK: Input: default@dp_mm@key1=123/key2=455
-#### A masked pattern was here ####
-455	123	455
-455	123	455
-0	123	0
-0	123	0
-PREHOOK: query: drop table dp_no_mm
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@dp_no_mm
-PREHOOK: Output: default@dp_no_mm
-POSTHOOK: query: drop table dp_no_mm
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@dp_no_mm
-POSTHOOK: Output: default@dp_no_mm
-PREHOOK: query: drop table dp_mm
+0	0
+0	0
+1	1
+1	1
+455	455
+455	455
+456	456
+456	456
+PREHOOK: query: drop table partunion_mm
 PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@dp_mm
-PREHOOK: Output: default@dp_mm
-POSTHOOK: query: drop table dp_mm
+PREHOOK: Input: default@partunion_mm
+PREHOOK: Output: default@partunion_mm
+POSTHOOK: query: drop table partunion_mm
 POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@dp_mm
-POSTHOOK: Output: default@dp_mm
+POSTHOOK: Input: default@partunion_mm
+POSTHOOK: Output: default@partunion_mm
 PREHOOK: query: drop table intermediate
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@intermediate


[03/15] hive git commit: HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 57a748a..10778f2 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -213,6 +213,15 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def get_tables_by_type(self, db_name, pattern, tableType):
+    """
+    Parameters:
+     - db_name
+     - pattern
+     - tableType
+    """
+    pass
+
   def get_table_meta(self, db_patterns, tbl_patterns, tbl_types):
     """
     Parameters:
@@ -2106,6 +2115,43 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o1
     raise TApplicationException(TApplicationException.MISSING_RESULT, "get_tables failed: unknown result")
 
+  def get_tables_by_type(self, db_name, pattern, tableType):
+    """
+    Parameters:
+     - db_name
+     - pattern
+     - tableType
+    """
+    self.send_get_tables_by_type(db_name, pattern, tableType)
+    return self.recv_get_tables_by_type()
+
+  def send_get_tables_by_type(self, db_name, pattern, tableType):
+    self._oprot.writeMessageBegin('get_tables_by_type', TMessageType.CALL, self._seqid)
+    args = get_tables_by_type_args()
+    args.db_name = db_name
+    args.pattern = pattern
+    args.tableType = tableType
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_tables_by_type(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_tables_by_type_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_tables_by_type failed: unknown result")
+
   def get_table_meta(self, db_patterns, tbl_patterns, tbl_types):
     """
     Parameters:
@@ -6625,6 +6671,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["drop_table"] = Processor.process_drop_table
     self._processMap["drop_table_with_environment_context"] = Processor.process_drop_table_with_environment_context
     self._processMap["get_tables"] = Processor.process_get_tables
+    self._processMap["get_tables_by_type"] = Processor.process_get_tables_by_type
     self._processMap["get_table_meta"] = Processor.process_get_table_meta
     self._processMap["get_all_tables"] = Processor.process_get_all_tables
     self._processMap["get_table"] = Processor.process_get_table
@@ -7411,6 +7458,28 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_get_tables_by_type(self, seqid, iprot, oprot):
+    args = get_tables_by_type_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_tables_by_type_result()
+    try:
+      result.success = self._handler.get_tables_by_type(args.db_name, args.pattern, args.tableType)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except MetaException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_tables_by_type", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_get_table_meta(self, seqid, iprot, oprot):
     args = get_table_meta_args()
     args.read(iprot)
@@ -14626,6 +14695,183 @@ class get_tables_result:
   def __ne__(self, other):
     return not (self == other)
 
+class get_tables_by_type_args:
+  """
+  Attributes:
+   - db_name
+   - pattern
+   - tableType
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'db_name', None, None, ), # 1
+    (2, TType.STRING, 'pattern', None, None, ), # 2
+    (3, TType.STRING, 'tableType', None, None, ), # 3
+  )
+
+  def __init__(self, db_name=None, pattern=None, tableType=None,):
+    self.db_name = db_name
+    self.pattern = pattern
+    self.tableType = tableType
+
+  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.db_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.pattern = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.tableType = iprot.readString()
+        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('get_tables_by_type_args')
+    if self.db_name is not None:
+      oprot.writeFieldBegin('db_name', TType.STRING, 1)
+      oprot.writeString(self.db_name)
+      oprot.writeFieldEnd()
+    if self.pattern is not None:
+      oprot.writeFieldBegin('pattern', TType.STRING, 2)
+      oprot.writeString(self.pattern)
+      oprot.writeFieldEnd()
+    if self.tableType is not None:
+      oprot.writeFieldBegin('tableType', TType.STRING, 3)
+      oprot.writeString(self.tableType)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.db_name)
+    value = (value * 31) ^ hash(self.pattern)
+    value = (value * 31) ^ hash(self.tableType)
+    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 get_tables_by_type_result:
+  """
+  Attributes:
+   - success
+   - o1
+  """
+
+  thrift_spec = (
+    (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0
+    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, success=None, o1=None,):
+    self.success = success
+    self.o1 = o1
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.LIST:
+          self.success = []
+          (_etype641, _size638) = iprot.readListBegin()
+          for _i642 in xrange(_size638):
+            _elem643 = iprot.readString()
+            self.success.append(_elem643)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = MetaException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_tables_by_type_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.LIST, 0)
+      oprot.writeListBegin(TType.STRING, len(self.success))
+      for iter644 in self.success:
+        oprot.writeString(iter644)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    value = (value * 31) ^ hash(self.o1)
+    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 get_table_meta_args:
   """
   Attributes:
@@ -14668,10 +14914,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype641, _size638) = iprot.readListBegin()
-          for _i642 in xrange(_size638):
-            _elem643 = iprot.readString()
-            self.tbl_types.append(_elem643)
+          (_etype648, _size645) = iprot.readListBegin()
+          for _i649 in xrange(_size645):
+            _elem650 = iprot.readString()
+            self.tbl_types.append(_elem650)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14696,8 +14942,8 @@ class get_table_meta_args:
     if self.tbl_types is not None:
       oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.tbl_types))
-      for iter644 in self.tbl_types:
-        oprot.writeString(iter644)
+      for iter651 in self.tbl_types:
+        oprot.writeString(iter651)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14753,11 +14999,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype648, _size645) = iprot.readListBegin()
-          for _i649 in xrange(_size645):
-            _elem650 = TableMeta()
-            _elem650.read(iprot)
-            self.success.append(_elem650)
+          (_etype655, _size652) = iprot.readListBegin()
+          for _i656 in xrange(_size652):
+            _elem657 = TableMeta()
+            _elem657.read(iprot)
+            self.success.append(_elem657)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14780,8 +15026,8 @@ class get_table_meta_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter651 in self.success:
-        iter651.write(oprot)
+      for iter658 in self.success:
+        iter658.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14905,10 +15151,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype655, _size652) = iprot.readListBegin()
-          for _i656 in xrange(_size652):
-            _elem657 = iprot.readString()
-            self.success.append(_elem657)
+          (_etype662, _size659) = iprot.readListBegin()
+          for _i663 in xrange(_size659):
+            _elem664 = iprot.readString()
+            self.success.append(_elem664)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14931,8 +15177,8 @@ class get_all_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter658 in self.success:
-        oprot.writeString(iter658)
+      for iter665 in self.success:
+        oprot.writeString(iter665)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15168,10 +15414,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype662, _size659) = iprot.readListBegin()
-          for _i663 in xrange(_size659):
-            _elem664 = iprot.readString()
-            self.tbl_names.append(_elem664)
+          (_etype669, _size666) = iprot.readListBegin()
+          for _i670 in xrange(_size666):
+            _elem671 = iprot.readString()
+            self.tbl_names.append(_elem671)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15192,8 +15438,8 @@ class get_table_objects_by_name_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter665 in self.tbl_names:
-        oprot.writeString(iter665)
+      for iter672 in self.tbl_names:
+        oprot.writeString(iter672)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15254,11 +15500,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype669, _size666) = iprot.readListBegin()
-          for _i670 in xrange(_size666):
-            _elem671 = Table()
-            _elem671.read(iprot)
-            self.success.append(_elem671)
+          (_etype676, _size673) = iprot.readListBegin()
+          for _i677 in xrange(_size673):
+            _elem678 = Table()
+            _elem678.read(iprot)
+            self.success.append(_elem678)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15293,8 +15539,8 @@ class get_table_objects_by_name_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter672 in self.success:
-        iter672.write(oprot)
+      for iter679 in self.success:
+        iter679.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15460,10 +15706,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype676, _size673) = iprot.readListBegin()
-          for _i677 in xrange(_size673):
-            _elem678 = iprot.readString()
-            self.success.append(_elem678)
+          (_etype683, _size680) = iprot.readListBegin()
+          for _i684 in xrange(_size680):
+            _elem685 = iprot.readString()
+            self.success.append(_elem685)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15498,8 +15744,8 @@ class get_table_names_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter679 in self.success:
-        oprot.writeString(iter679)
+      for iter686 in self.success:
+        oprot.writeString(iter686)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16469,11 +16715,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype683, _size680) = iprot.readListBegin()
-          for _i684 in xrange(_size680):
-            _elem685 = Partition()
-            _elem685.read(iprot)
-            self.new_parts.append(_elem685)
+          (_etype690, _size687) = iprot.readListBegin()
+          for _i691 in xrange(_size687):
+            _elem692 = Partition()
+            _elem692.read(iprot)
+            self.new_parts.append(_elem692)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16490,8 +16736,8 @@ class add_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter686 in self.new_parts:
-        iter686.write(oprot)
+      for iter693 in self.new_parts:
+        iter693.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16649,11 +16895,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype690, _size687) = iprot.readListBegin()
-          for _i691 in xrange(_size687):
-            _elem692 = PartitionSpec()
-            _elem692.read(iprot)
-            self.new_parts.append(_elem692)
+          (_etype697, _size694) = iprot.readListBegin()
+          for _i698 in xrange(_size694):
+            _elem699 = PartitionSpec()
+            _elem699.read(iprot)
+            self.new_parts.append(_elem699)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16670,8 +16916,8 @@ class add_partitions_pspec_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter693 in self.new_parts:
-        iter693.write(oprot)
+      for iter700 in self.new_parts:
+        iter700.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16845,10 +17091,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype697, _size694) = iprot.readListBegin()
-          for _i698 in xrange(_size694):
-            _elem699 = iprot.readString()
-            self.part_vals.append(_elem699)
+          (_etype704, _size701) = iprot.readListBegin()
+          for _i705 in xrange(_size701):
+            _elem706 = iprot.readString()
+            self.part_vals.append(_elem706)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16873,8 +17119,8 @@ class append_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter700 in self.part_vals:
-        oprot.writeString(iter700)
+      for iter707 in self.part_vals:
+        oprot.writeString(iter707)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17227,10 +17473,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype704, _size701) = iprot.readListBegin()
-          for _i705 in xrange(_size701):
-            _elem706 = iprot.readString()
-            self.part_vals.append(_elem706)
+          (_etype711, _size708) = iprot.readListBegin()
+          for _i712 in xrange(_size708):
+            _elem713 = iprot.readString()
+            self.part_vals.append(_elem713)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17261,8 +17507,8 @@ class append_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter707 in self.part_vals:
-        oprot.writeString(iter707)
+      for iter714 in self.part_vals:
+        oprot.writeString(iter714)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -17857,10 +18103,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype711, _size708) = iprot.readListBegin()
-          for _i712 in xrange(_size708):
-            _elem713 = iprot.readString()
-            self.part_vals.append(_elem713)
+          (_etype718, _size715) = iprot.readListBegin()
+          for _i719 in xrange(_size715):
+            _elem720 = iprot.readString()
+            self.part_vals.append(_elem720)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17890,8 +18136,8 @@ class drop_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter714 in self.part_vals:
-        oprot.writeString(iter714)
+      for iter721 in self.part_vals:
+        oprot.writeString(iter721)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -18064,10 +18310,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype718, _size715) = iprot.readListBegin()
-          for _i719 in xrange(_size715):
-            _elem720 = iprot.readString()
-            self.part_vals.append(_elem720)
+          (_etype725, _size722) = iprot.readListBegin()
+          for _i726 in xrange(_size722):
+            _elem727 = iprot.readString()
+            self.part_vals.append(_elem727)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18103,8 +18349,8 @@ class drop_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter721 in self.part_vals:
-        oprot.writeString(iter721)
+      for iter728 in self.part_vals:
+        oprot.writeString(iter728)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -18841,10 +19087,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype725, _size722) = iprot.readListBegin()
-          for _i726 in xrange(_size722):
-            _elem727 = iprot.readString()
-            self.part_vals.append(_elem727)
+          (_etype732, _size729) = iprot.readListBegin()
+          for _i733 in xrange(_size729):
+            _elem734 = iprot.readString()
+            self.part_vals.append(_elem734)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18869,8 +19115,8 @@ class get_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter728 in self.part_vals:
-        oprot.writeString(iter728)
+      for iter735 in self.part_vals:
+        oprot.writeString(iter735)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19029,11 +19275,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype730, _vtype731, _size729 ) = iprot.readMapBegin()
-          for _i733 in xrange(_size729):
-            _key734 = iprot.readString()
-            _val735 = iprot.readString()
-            self.partitionSpecs[_key734] = _val735
+          (_ktype737, _vtype738, _size736 ) = iprot.readMapBegin()
+          for _i740 in xrange(_size736):
+            _key741 = iprot.readString()
+            _val742 = iprot.readString()
+            self.partitionSpecs[_key741] = _val742
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -19070,9 +19316,9 @@ class exchange_partition_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter736,viter737 in self.partitionSpecs.items():
-        oprot.writeString(kiter736)
-        oprot.writeString(viter737)
+      for kiter743,viter744 in self.partitionSpecs.items():
+        oprot.writeString(kiter743)
+        oprot.writeString(viter744)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -19277,11 +19523,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype739, _vtype740, _size738 ) = iprot.readMapBegin()
-          for _i742 in xrange(_size738):
-            _key743 = iprot.readString()
-            _val744 = iprot.readString()
-            self.partitionSpecs[_key743] = _val744
+          (_ktype746, _vtype747, _size745 ) = iprot.readMapBegin()
+          for _i749 in xrange(_size745):
+            _key750 = iprot.readString()
+            _val751 = iprot.readString()
+            self.partitionSpecs[_key750] = _val751
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -19318,9 +19564,9 @@ class exchange_partitions_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter745,viter746 in self.partitionSpecs.items():
-        oprot.writeString(kiter745)
-        oprot.writeString(viter746)
+      for kiter752,viter753 in self.partitionSpecs.items():
+        oprot.writeString(kiter752)
+        oprot.writeString(viter753)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -19403,11 +19649,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype750, _size747) = iprot.readListBegin()
-          for _i751 in xrange(_size747):
-            _elem752 = Partition()
-            _elem752.read(iprot)
-            self.success.append(_elem752)
+          (_etype757, _size754) = iprot.readListBegin()
+          for _i758 in xrange(_size754):
+            _elem759 = Partition()
+            _elem759.read(iprot)
+            self.success.append(_elem759)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19448,8 +19694,8 @@ class exchange_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter753 in self.success:
-        iter753.write(oprot)
+      for iter760 in self.success:
+        iter760.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19543,10 +19789,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype757, _size754) = iprot.readListBegin()
-          for _i758 in xrange(_size754):
-            _elem759 = iprot.readString()
-            self.part_vals.append(_elem759)
+          (_etype764, _size761) = iprot.readListBegin()
+          for _i765 in xrange(_size761):
+            _elem766 = iprot.readString()
+            self.part_vals.append(_elem766)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19558,10 +19804,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype763, _size760) = iprot.readListBegin()
-          for _i764 in xrange(_size760):
-            _elem765 = iprot.readString()
-            self.group_names.append(_elem765)
+          (_etype770, _size767) = iprot.readListBegin()
+          for _i771 in xrange(_size767):
+            _elem772 = iprot.readString()
+            self.group_names.append(_elem772)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19586,8 +19832,8 @@ class get_partition_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter766 in self.part_vals:
-        oprot.writeString(iter766)
+      for iter773 in self.part_vals:
+        oprot.writeString(iter773)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -19597,8 +19843,8 @@ class get_partition_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter767 in self.group_names:
-        oprot.writeString(iter767)
+      for iter774 in self.group_names:
+        oprot.writeString(iter774)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20027,11 +20273,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype771, _size768) = iprot.readListBegin()
-          for _i772 in xrange(_size768):
-            _elem773 = Partition()
-            _elem773.read(iprot)
-            self.success.append(_elem773)
+          (_etype778, _size775) = iprot.readListBegin()
+          for _i779 in xrange(_size775):
+            _elem780 = Partition()
+            _elem780.read(iprot)
+            self.success.append(_elem780)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20060,8 +20306,8 @@ class get_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter774 in self.success:
-        iter774.write(oprot)
+      for iter781 in self.success:
+        iter781.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20155,10 +20401,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype778, _size775) = iprot.readListBegin()
-          for _i779 in xrange(_size775):
-            _elem780 = iprot.readString()
-            self.group_names.append(_elem780)
+          (_etype785, _size782) = iprot.readListBegin()
+          for _i786 in xrange(_size782):
+            _elem787 = iprot.readString()
+            self.group_names.append(_elem787)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20191,8 +20437,8 @@ class get_partitions_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter781 in self.group_names:
-        oprot.writeString(iter781)
+      for iter788 in self.group_names:
+        oprot.writeString(iter788)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20253,11 +20499,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype785, _size782) = iprot.readListBegin()
-          for _i786 in xrange(_size782):
-            _elem787 = Partition()
-            _elem787.read(iprot)
-            self.success.append(_elem787)
+          (_etype792, _size789) = iprot.readListBegin()
+          for _i793 in xrange(_size789):
+            _elem794 = Partition()
+            _elem794.read(iprot)
+            self.success.append(_elem794)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20286,8 +20532,8 @@ class get_partitions_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter788 in self.success:
-        iter788.write(oprot)
+      for iter795 in self.success:
+        iter795.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20445,11 +20691,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype792, _size789) = iprot.readListBegin()
-          for _i793 in xrange(_size789):
-            _elem794 = PartitionSpec()
-            _elem794.read(iprot)
-            self.success.append(_elem794)
+          (_etype799, _size796) = iprot.readListBegin()
+          for _i800 in xrange(_size796):
+            _elem801 = PartitionSpec()
+            _elem801.read(iprot)
+            self.success.append(_elem801)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20478,8 +20724,8 @@ class get_partitions_pspec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter795 in self.success:
-        iter795.write(oprot)
+      for iter802 in self.success:
+        iter802.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20634,10 +20880,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype799, _size796) = iprot.readListBegin()
-          for _i800 in xrange(_size796):
-            _elem801 = iprot.readString()
-            self.success.append(_elem801)
+          (_etype806, _size803) = iprot.readListBegin()
+          for _i807 in xrange(_size803):
+            _elem808 = iprot.readString()
+            self.success.append(_elem808)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20660,8 +20906,8 @@ class get_partition_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter802 in self.success:
-        oprot.writeString(iter802)
+      for iter809 in self.success:
+        oprot.writeString(iter809)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -20737,10 +20983,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype806, _size803) = iprot.readListBegin()
-          for _i807 in xrange(_size803):
-            _elem808 = iprot.readString()
-            self.part_vals.append(_elem808)
+          (_etype813, _size810) = iprot.readListBegin()
+          for _i814 in xrange(_size810):
+            _elem815 = iprot.readString()
+            self.part_vals.append(_elem815)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20770,8 +21016,8 @@ class get_partitions_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter809 in self.part_vals:
-        oprot.writeString(iter809)
+      for iter816 in self.part_vals:
+        oprot.writeString(iter816)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -20835,11 +21081,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype813, _size810) = iprot.readListBegin()
-          for _i814 in xrange(_size810):
-            _elem815 = Partition()
-            _elem815.read(iprot)
-            self.success.append(_elem815)
+          (_etype820, _size817) = iprot.readListBegin()
+          for _i821 in xrange(_size817):
+            _elem822 = Partition()
+            _elem822.read(iprot)
+            self.success.append(_elem822)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20868,8 +21114,8 @@ class get_partitions_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter816 in self.success:
-        iter816.write(oprot)
+      for iter823 in self.success:
+        iter823.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20956,10 +21202,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype820, _size817) = iprot.readListBegin()
-          for _i821 in xrange(_size817):
-            _elem822 = iprot.readString()
-            self.part_vals.append(_elem822)
+          (_etype827, _size824) = iprot.readListBegin()
+          for _i828 in xrange(_size824):
+            _elem829 = iprot.readString()
+            self.part_vals.append(_elem829)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20976,10 +21222,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype826, _size823) = iprot.readListBegin()
-          for _i827 in xrange(_size823):
-            _elem828 = iprot.readString()
-            self.group_names.append(_elem828)
+          (_etype833, _size830) = iprot.readListBegin()
+          for _i834 in xrange(_size830):
+            _elem835 = iprot.readString()
+            self.group_names.append(_elem835)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21004,8 +21250,8 @@ class get_partitions_ps_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter829 in self.part_vals:
-        oprot.writeString(iter829)
+      for iter836 in self.part_vals:
+        oprot.writeString(iter836)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -21019,8 +21265,8 @@ class get_partitions_ps_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter830 in self.group_names:
-        oprot.writeString(iter830)
+      for iter837 in self.group_names:
+        oprot.writeString(iter837)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21082,11 +21328,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype834, _size831) = iprot.readListBegin()
-          for _i835 in xrange(_size831):
-            _elem836 = Partition()
-            _elem836.read(iprot)
-            self.success.append(_elem836)
+          (_etype841, _size838) = iprot.readListBegin()
+          for _i842 in xrange(_size838):
+            _elem843 = Partition()
+            _elem843.read(iprot)
+            self.success.append(_elem843)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21115,8 +21361,8 @@ class get_partitions_ps_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter837 in self.success:
-        iter837.write(oprot)
+      for iter844 in self.success:
+        iter844.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21197,10 +21443,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype841, _size838) = iprot.readListBegin()
-          for _i842 in xrange(_size838):
-            _elem843 = iprot.readString()
-            self.part_vals.append(_elem843)
+          (_etype848, _size845) = iprot.readListBegin()
+          for _i849 in xrange(_size845):
+            _elem850 = iprot.readString()
+            self.part_vals.append(_elem850)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21230,8 +21476,8 @@ class get_partition_names_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter844 in self.part_vals:
-        oprot.writeString(iter844)
+      for iter851 in self.part_vals:
+        oprot.writeString(iter851)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -21295,10 +21541,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype848, _size845) = iprot.readListBegin()
-          for _i849 in xrange(_size845):
-            _elem850 = iprot.readString()
-            self.success.append(_elem850)
+          (_etype855, _size852) = iprot.readListBegin()
+          for _i856 in xrange(_size852):
+            _elem857 = iprot.readString()
+            self.success.append(_elem857)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21327,8 +21573,8 @@ class get_partition_names_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter851 in self.success:
-        oprot.writeString(iter851)
+      for iter858 in self.success:
+        oprot.writeString(iter858)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21499,11 +21745,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype855, _size852) = iprot.readListBegin()
-          for _i856 in xrange(_size852):
-            _elem857 = Partition()
-            _elem857.read(iprot)
-            self.success.append(_elem857)
+          (_etype862, _size859) = iprot.readListBegin()
+          for _i863 in xrange(_size859):
+            _elem864 = Partition()
+            _elem864.read(iprot)
+            self.success.append(_elem864)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21532,8 +21778,8 @@ class get_partitions_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter858 in self.success:
-        iter858.write(oprot)
+      for iter865 in self.success:
+        iter865.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21704,11 +21950,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype862, _size859) = iprot.readListBegin()
-          for _i863 in xrange(_size859):
-            _elem864 = PartitionSpec()
-            _elem864.read(iprot)
-            self.success.append(_elem864)
+          (_etype869, _size866) = iprot.readListBegin()
+          for _i870 in xrange(_size866):
+            _elem871 = PartitionSpec()
+            _elem871.read(iprot)
+            self.success.append(_elem871)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21737,8 +21983,8 @@ class get_part_specs_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter865 in self.success:
-        iter865.write(oprot)
+      for iter872 in self.success:
+        iter872.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22158,10 +22404,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype869, _size866) = iprot.readListBegin()
-          for _i870 in xrange(_size866):
-            _elem871 = iprot.readString()
-            self.names.append(_elem871)
+          (_etype876, _size873) = iprot.readListBegin()
+          for _i877 in xrange(_size873):
+            _elem878 = iprot.readString()
+            self.names.append(_elem878)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22186,8 +22432,8 @@ class get_partitions_by_names_args:
     if self.names is not None:
       oprot.writeFieldBegin('names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.names))
-      for iter872 in self.names:
-        oprot.writeString(iter872)
+      for iter879 in self.names:
+        oprot.writeString(iter879)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22246,11 +22492,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype876, _size873) = iprot.readListBegin()
-          for _i877 in xrange(_size873):
-            _elem878 = Partition()
-            _elem878.read(iprot)
-            self.success.append(_elem878)
+          (_etype883, _size880) = iprot.readListBegin()
+          for _i884 in xrange(_size880):
+            _elem885 = Partition()
+            _elem885.read(iprot)
+            self.success.append(_elem885)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22279,8 +22525,8 @@ class get_partitions_by_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter879 in self.success:
-        iter879.write(oprot)
+      for iter886 in self.success:
+        iter886.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22530,11 +22776,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype883, _size880) = iprot.readListBegin()
-          for _i884 in xrange(_size880):
-            _elem885 = Partition()
-            _elem885.read(iprot)
-            self.new_parts.append(_elem885)
+          (_etype890, _size887) = iprot.readListBegin()
+          for _i891 in xrange(_size887):
+            _elem892 = Partition()
+            _elem892.read(iprot)
+            self.new_parts.append(_elem892)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22559,8 +22805,8 @@ class alter_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter886 in self.new_parts:
-        iter886.write(oprot)
+      for iter893 in self.new_parts:
+        iter893.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22713,11 +22959,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype890, _size887) = iprot.readListBegin()
-          for _i891 in xrange(_size887):
-            _elem892 = Partition()
-            _elem892.read(iprot)
-            self.new_parts.append(_elem892)
+          (_etype897, _size894) = iprot.readListBegin()
+          for _i898 in xrange(_size894):
+            _elem899 = Partition()
+            _elem899.read(iprot)
+            self.new_parts.append(_elem899)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22748,8 +22994,8 @@ class alter_partitions_with_environment_context_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter893 in self.new_parts:
-        iter893.write(oprot)
+      for iter900 in self.new_parts:
+        iter900.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -23093,10 +23339,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype897, _size894) = iprot.readListBegin()
-          for _i898 in xrange(_size894):
-            _elem899 = iprot.readString()
-            self.part_vals.append(_elem899)
+          (_etype904, _size901) = iprot.readListBegin()
+          for _i905 in xrange(_size901):
+            _elem906 = iprot.readString()
+            self.part_vals.append(_elem906)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23127,8 +23373,8 @@ class rename_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter900 in self.part_vals:
-        oprot.writeString(iter900)
+      for iter907 in self.part_vals:
+        oprot.writeString(iter907)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -23270,10 +23516,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype904, _size901) = iprot.readListBegin()
-          for _i905 in xrange(_size901):
-            _elem906 = iprot.readString()
-            self.part_vals.append(_elem906)
+          (_etype911, _size908) = iprot.readListBegin()
+          for _i912 in xrange(_size908):
+            _elem913 = iprot.readString()
+            self.part_vals.append(_elem913)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23295,8 +23541,8 @@ class partition_name_has_valid_characters_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter907 in self.part_vals:
-        oprot.writeString(iter907)
+      for iter914 in self.part_vals:
+        oprot.writeString(iter914)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -23654,10 +23900,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype911, _size908) = iprot.readListBegin()
-          for _i912 in xrange(_size908):
-            _elem913 = iprot.readString()
-            self.success.append(_elem913)
+          (_etype918, _size915) = iprot.readListBegin()
+          for _i919 in xrange(_size915):
+            _elem920 = iprot.readString()
+            self.success.append(_elem920)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23680,8 +23926,8 @@ class partition_name_to_vals_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter914 in self.success:
-        oprot.writeString(iter914)
+      for iter921 in self.success:
+        oprot.writeString(iter921)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23805,11 +24051,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype916, _vtype917, _size915 ) = iprot.readMapBegin()
-          for _i919 in xrange(_size915):
-            _key920 = iprot.readString()
-            _val921 = iprot.readString()
-            self.success[_key920] = _val921
+          (_ktype923, _vtype924, _size922 ) = iprot.readMapBegin()
+          for _i926 in xrange(_size922):
+            _key927 = iprot.readString()
+            _val928 = iprot.readString()
+            self.success[_key927] = _val928
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -23832,9 +24078,9 @@ class partition_name_to_spec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success))
-      for kiter922,viter923 in self.success.items():
-        oprot.writeString(kiter922)
-        oprot.writeString(viter923)
+      for kiter929,viter930 in self.success.items():
+        oprot.writeString(kiter929)
+        oprot.writeString(viter930)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23910,11 +24156,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype925, _vtype926, _size924 ) = iprot.readMapBegin()
-          for _i928 in xrange(_size924):
-            _key929 = iprot.readString()
-            _val930 = iprot.readString()
-            self.part_vals[_key929] = _val930
+          (_ktype932, _vtype933, _size931 ) = iprot.readMapBegin()
+          for _i935 in xrange(_size931):
+            _key936 = iprot.readString()
+            _val937 = iprot.readString()
+            self.part_vals[_key936] = _val937
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -23944,9 +24190,9 @@ class markPartitionForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter931,viter932 in self.part_vals.items():
-        oprot.writeString(kiter931)
-        oprot.writeString(viter932)
+      for kiter938,viter939 in self.part_vals.items():
+        oprot.writeString(kiter938)
+        oprot.writeString(viter939)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -24160,11 +24406,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype934, _vtype935, _size933 ) = iprot.readMapBegin()
-          for _i937 in xrange(_size933):
-            _key938 = iprot.readString()
-            _val939 = iprot.readString()
-            self.part_vals[_key938] = _val939
+          (_ktype941, _vtype942, _size940 ) = iprot.readMapBegin()
+          for _i944 in xrange(_size940):
+            _key945 = iprot.readString()
+            _val946 = iprot.readString()
+            self.part_vals[_key945] = _val946
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -24194,9 +24440,9 @@ class isPartitionMarkedForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter940,viter941 in self.part_vals.items():
-        oprot.writeString(kiter940)
-        oprot.writeString(viter941)
+      for kiter947,viter948 in self.part_vals.items():
+        oprot.writeString(kiter947)
+        oprot.writeString(viter948)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -25251,11 +25497,11 @@ class get_indexes_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype945, _size942) = iprot.readListBegin()
-          for _i946 in xrange(_size942):
-            _elem947 = Index()
-            _elem947.read(iprot)
-            self.success.append(_elem947)
+          (_etype952, _size949) = iprot.readListBegin()
+          for _i953 in xrange(_size949):
+            _elem954 = Index()
+            _elem954.read(iprot)
+            self.success.append(_elem954)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25284,8 +25530,8 @@ class get_indexes_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter948 in self.success:
-        iter948.write(oprot)
+      for iter955 in self.success:
+        iter955.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25440,10 +25686,10 @@ class get_index_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype952, _size949) = iprot.readListBegin()
-          for _i953 in xrange(_size949):
-            _elem954 = iprot.readString()
-            self.success.append(_elem954)
+          (_etype959, _size956) = iprot.readListBegin()
+          for _i960 in xrange(_size956):
+            _elem961 = iprot.readString()
+            self.success.append(_elem961)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25466,8 +25712,8 @@ class get_index_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter955 in self.success:
-        oprot.writeString(iter955)
+      for iter962 in self.success:
+        oprot.writeString(iter962)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -28333,10 +28579,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype959, _size956) = iprot.readListBegin()
-          for _i960 in xrange(_size956):
-            _elem961 = iprot.readString()
-            self.success.append(_elem961)
+          (_etype966, _size963) = iprot.readListBegin()
+          for _i967 in xrange(_size963):
+            _elem968 = iprot.readString()
+            self.success.append(_elem968)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28359,8 +28605,8 @@ class get_functions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter962 in self.success:
-        oprot.writeString(iter962)
+      for iter969 in self.success:
+        oprot.writeString(iter969)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29048,10 +29294,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype966, _size963) = iprot.readListBegin()
-          for _i967 in xrange(_size963):
-            _elem968 = iprot.readString()
-            self.success.append(_elem968)
+          (_etype973, _size970) = iprot.readListBegin()
+          for _i974 in xrange(_size970):
+            _elem975 = iprot.readString()
+            self.success.append(_elem975)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29074,8 +29320,8 @@ class get_role_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter969 in self.success:
-        oprot.writeString(iter969)
+      for iter976 in self.success:
+        oprot.writeString(iter976)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29589,11 +29835,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype973, _size970) = iprot.readListBegin()
-          for _i974 in xrange(_size970):
-            _elem975 = Role()
-            _elem975.read(iprot)
-            self.success.append(_elem975)
+          (_etype980, _size977) = iprot.readListBegin()
+          for _i981 in xrange(_size977):
+            _elem982 = Role()
+            _elem982.read(iprot)
+            self.success.append(_elem982)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29616,8 +29862,8 @@ class list_roles_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter976 in self.success:
-        iter976.write(oprot)
+      for iter983 in self.success:
+        iter983.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30126,10 +30372,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype980, _size977) = iprot.readListBegin()
-          for _i981 in xrange(_size977):
-            _elem982 = iprot.readString()
-            self.group_names.append(_elem982)
+          (_etype987, _size984) = iprot.readListBegin()
+          for _i988 in xrange(_size984):
+            _elem989 = iprot.readString()
+            self.group_names.append(_elem989)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30154,8 +30400,8 @@ class get_privilege_set_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter983 in self.group_names:
-        oprot.writeString(iter983)
+      for iter990 in self.group_names:
+        oprot.writeString(iter990)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -30382,11 +30628,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype987, _size984) = iprot.readListBegin()
-          for _i988 in xrange(_size984):
-            _elem989 = HiveObjectPrivilege()
-            _elem989.read(iprot)
-            self.success.append(_elem989)
+          (_etype994, _size991) = iprot.readListBegin()
+          for _i995 in xrange(_size991):
+            _elem996 = HiveObjectPrivilege()
+            _elem996.read(iprot)
+            self.success.append(_elem996)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30409,8 +30655,8 @@ class list_privileges_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter990 in self.success:
-        iter990.write(oprot)
+      for iter997 in self.success:
+        iter997.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30908,10 +31154,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype994, _size991) = iprot.readListBegin()
-          for _i995 in xrange(_size991):
-            _elem996 = iprot.readString()
-            self.group_names.append(_elem996)
+          (_etype1001, _size998) = iprot.readListBegin()
+          for _i1002 in xrange(_size998):
+            _elem1003 = iprot.readString()
+            self.group_names.append(_elem1003)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30932,8 +31178,8 @@ class set_ugi_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter997 in self.group_names:
-        oprot.writeString(iter997)
+      for iter1004 in self.group_names:
+        oprot.writeString(iter1004)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -30988,10 +31234,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1001, _size998) = iprot.readListBegin()
-          for _i1002 in xrange(_size998):
-            _elem1003 = iprot.readString()
-            self.success.append(_elem1003)
+          (_etype1008, _size1005) = iprot.readListBegin()
+          for _i1009 in xrange(_size1005):
+            _elem1010 = iprot.readString()
+            self.success.append(_elem1010)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31014,8 +31260,8 @@ class set_ugi_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1004 in self.success:
-        oprot.writeString(iter1004)
+      for iter1011 in self.success:
+        oprot.writeString(iter1011)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -31947,10 +32193,10 @@ class get_all_token_identifiers_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1008, _size1005) = iprot.readListBegin()
-          for _i1009 in xrange(_size1005):
-            _elem1010 = iprot.readString()
-            self.success.append(_elem1010)
+          (_etype1015, _size1012) = iprot.readListBegin()
+          for _i1016 in xrange(_size1012):
+            _elem1017 = iprot.readString()
+            self.success.append(_elem1017)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31967,8 +32213,8 @@ class get_all_token_identifiers_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1011 in self.success:
-        oprot.writeString(iter1011)
+      for iter1018 in self.success:
+        oprot.writeString(iter1018)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -32495,10 +32741,10 @@ class get_master_keys_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1015, _size1012) = iprot.readListBegin()
-          for _i1016 in xrange(_size1012):
-            _elem1017 = iprot.readString()
-            self.success.append(_elem1017)
+          (_etype1022, _size1019) = iprot.readListBegin()
+          for _i1023 in xrange(_size1019):
+            _elem1024 = iprot.readString()
+            self.success.append(_elem1024)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -32515,8 +32761,8 @@ class get_master_keys_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1018 in self.success:
-        oprot.writeString(iter1018)
+      for iter1025 in self.success:
+        oprot.writeString(iter1025)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index 51f65c6..76a3339 100644
--- a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -432,6 +432,22 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_tables failed: unknown result')
     end
 
+    def get_tables_by_type(db_name, pattern, tableType)
+      send_get_tables_by_type(db_name, pattern, tableType)
+      return recv_get_tables_by_type()
+    end
+
+    def send_get_tables_by_type(db_name, pattern, tableType)
+      send_message('get_tables_by_type', Get_tables_by_type_args, :db_name => db_name, :pattern => pattern, :tableType => tableType)
+    end
+
+    def recv_get_tables_by_type()
+      result = receive_message(Get_tables_by_type_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_tables_by_type failed: unknown result')
+    end
+
     def get_table_meta(db_patterns, tbl_patterns, tbl_types)
       send_get_table_meta(db_patterns, tbl_patterns, tbl_types)
       return recv_get_table_meta()
@@ -2828,6 +2844,17 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'get_tables', seqid)
     end
 
+    def process_get_tables_by_type(seqid, iprot, oprot)
+      args = read_args(iprot, Get_tables_by_type_args)
+      result = Get_tables_by_type_result.new()
+      begin
+        result.success = @handler.get_tables_by_type(args.db_name, args.pattern, args.tableType)
+      rescue ::MetaException => o1
+        result.o1 = o1
+      end
+      write_result(result, oprot, 'get_tables_by_type', seqid)
+    end
+
     def process_get_table_meta(seqid, iprot, oprot)
       args = read_args(iprot, Get_table_meta_args)
       result = Get_table_meta_result.new()
@@ -5261,6 +5288,44 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Get_tables_by_type_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    DB_NAME = 1
+    PATTERN = 2
+    TABLETYPE = 3
+
+    FIELDS = {
+      DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'},
+      PATTERN => {:type => ::Thrift::Types::STRING, :name => 'pattern'},
+      TABLETYPE => {:type => ::Thrift::Types::STRING, :name => 'tableType'}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_tables_by_type_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::LIST, :name => 'success', :element => {:type => ::Thrift::Types::STRING}},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Get_table_meta_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     DB_PATTERNS = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index c4d03eb..530d2f4 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
 import org.apache.hadoop.hive.common.metrics.common.MetricsVariable;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.*;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
@@ -3744,6 +3745,28 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     @Override
+    public List<String> get_tables_by_type(final String dbname, final String pattern, final String tableType)
+        throws MetaException {
+      startFunction("get_tables_by_type", ": db=" + dbname + " pat=" + pattern + ",type=" + tableType);
+
+      List<String> ret = null;
+      Exception ex = null;
+      try {
+        ret = getMS().getTables(dbname, pattern, TableType.valueOf(tableType));
+      } catch (Exception e) {
+        ex = e;
+        if (e instanceof MetaException) {
+          throw (MetaException) e;
+        } else {
+          throw newMetaException(e);
+        }
+      } finally {
+        endFunction("get_tables_by_type", ret != null, ex);
+      }
+      return ret;
+    }
+
+    @Override
     public List<String> get_all_tables(final String dbname) throws MetaException {
       startFunction("get_all_tables", ": db=" + dbname);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 909d8eb..c32486f 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.common.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.conf.HiveConfUtil;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
 import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest;
 import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
@@ -1383,6 +1384,17 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     return null;
   }
 
+  /** {@inheritDoc} */
+  @Override
+  public List<String> getTables(String dbname, String tablePattern, TableType tableType) throws MetaException {
+    try {
+      return filterHook.filterTableNames(dbname, client.get_tables_by_type(dbname, tablePattern, tableType.toString()));
+    } catch (Exception e) {
+      MetaStoreUtils.logAndThrowMetaException(e);
+    }
+    return null;
+  }
+
   @Override
   public List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)
       throws MetaException {

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 8dc4e28..b770559 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience.Public;
 import org.apache.hadoop.hive.common.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.annotation.NoReconnect;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
@@ -172,6 +173,20 @@ public interface IMetaStoreClient {
       throws MetaException, TException, UnknownDBException;
 
   /**
+   * Get the names of all tables in the specified database that satisfy the supplied
+   * table name pattern and table type (MANAGED_TABLE || EXTERNAL_TABLE || VIRTUAL_VIEW)
+   * @param dbName Name of the database to fetch tables in.
+   * @param tablePattern pattern to match for table names.
+   * @param tableType Type of the table in the HMS store. VIRTUAL_VIEW is for views.
+   * @return List of table names.
+   * @throws MetaException
+   * @throws TException
+   * @throws UnknownDBException
+   */
+  List<String> getTables(String dbName, String tablePattern, TableType tableType)
+      throws MetaException, TException, UnknownDBException;
+
+  /**
    * For quick GetTablesOperation
    */
   List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 74bde3d..90ea641 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -1197,6 +1197,11 @@ public class ObjectStore implements RawStore, Configurable {
 
   @Override
   public List<String> getTables(String dbName, String pattern) throws MetaException {
+    return getTables(dbName, pattern, null);
+  }
+
+  @Override
+  public List<String> getTables(String dbName, String pattern, TableType tableType) throws MetaException {
     boolean commited = false;
     Query query = null;
     List<String> tbls = null;
@@ -1219,6 +1224,9 @@ public class ObjectStore implements RawStore, Configurable {
         first = false;
       }
       queryStr = queryStr + ")";
+      if (tableType != null) {
+        queryStr = queryStr + " && tableType.matches(\"" + tableType.toString() + "\")";
+      }
       query = pm.newQuery(queryStr);
       query.declareParameters("java.lang.String dbName");
       query.setResult("tableName");

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
index bbd47b8..a3dd4e5 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -28,6 +28,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
@@ -155,6 +156,9 @@ public interface RawStore extends Configurable {
   public List<String> getTables(String dbName, String pattern)
       throws MetaException;
 
+  public List<String> getTables(String dbName, String pattern, TableType tableType)
+      throws MetaException;
+
   public List<TableMeta> getTableMeta(
       String dbNames, String tableNames, List<String> tableTypes) throws MetaException;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index c65c7a4..f9fad4c 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hive.metastore.HiveMetaStore;
 import org.apache.hadoop.hive.metastore.PartFilterExprUtil;
 import org.apache.hadoop.hive.metastore.PartitionExpressionProxy;
 import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
@@ -530,6 +531,11 @@ public class HBaseStore implements RawStore {
     }
   }
 
+  @Override
+  public List<String> getTables(String dbName, String pattern, TableType tableType) throws MetaException {
+    throw new UnsupportedOperationException();
+  }
+
   private List<String> getTableNamesInTx(String dbName, String pattern) throws IOException {
     List<Table> tables = getHBase().scanTables(HiveStringUtils.normalizeIdentifier(dbName),
         pattern==null?null:HiveStringUtils.normalizeIdentifier(likeToRegex(pattern)));

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index 1ea72a0..f64b08d 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -26,6 +26,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
@@ -223,6 +224,11 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
+  public List<String> getTables(String dbName, String pattern, TableType tableType) throws MetaException {
+    return objectStore.getTables(dbName, pattern, tableType);
+  }
+
+  @Override
   public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
       throws MetaException {
     return objectStore.getTableMeta(dbNames, tableNames, tableTypes);

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index 3e6acc7..2682886 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -27,6 +27,7 @@ import junit.framework.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
@@ -227,6 +228,11 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
+  public List<String> getTables(String dbName, String pattern, TableType tableType) throws MetaException {
+    return Collections.emptyList();
+  }
+
+  @Override
   public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
       throws MetaException {
     return Collections.emptyList();

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 ec31cf4..cea8ce8 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
@@ -427,7 +427,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
       ShowTablesDesc showTbls = work.getShowTblsDesc();
       if (showTbls != null) {
-        return showTables(db, showTbls);
+        return showTablesOrViews(db, showTbls);
       }
 
       ShowColumnsDesc showCols = work.getShowColumnsDesc();
@@ -2380,37 +2380,44 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
   }
 
   /**
-   * Write a list of the tables in the database to a file.
+   * Write a list of the tables/views in the database to a file.
    *
    * @param db
-   *          The database in question.
-   * @param showTbls
-   *          These are the tables we're interested in.
+   *          The database in context.
+   * @param showDesc
+   *        A ShowTablesDesc for tables or views we're interested in.
    * @return Returns 0 when execution succeeds and above 0 if it fails.
    * @throws HiveException
    *           Throws this exception if an unexpected error occurs.
    */
-  private int showTables(Hive db, ShowTablesDesc showTbls) throws HiveException {
-    // get the tables for the desired pattern - populate the output stream
-    List<String> tbls = null;
-    String dbName = showTbls.getDbName();
+  private int showTablesOrViews(Hive db, ShowTablesDesc showDesc) throws HiveException {
+    // get the tables/views for the desired pattern - populate the output stream
+    List<String> tablesOrViews = null;
+
+    String dbName      = showDesc.getDbName();
+    String pattern     = showDesc.getPattern(); // if null, all tables/views are returned
+    String resultsFile = showDesc.getResFile();
+    TableType type     = showDesc.getType(); // will be null for tables, VIRTUAL_VIEW for views
 
     if (!db.databaseExists(dbName)) {
       throw new HiveException(ErrorMsg.DATABASE_NOT_EXISTS, dbName);
     }
-    if (showTbls.getPattern() != null) {
-      LOG.info("pattern: " + showTbls.getPattern());
-      tbls = db.getTablesByPattern(dbName, showTbls.getPattern());
-      LOG.info("results : " + tbls.size());
-    } else {
-      tbls = db.getAllTables(dbName);
-    }
+
+    LOG.debug("pattern: " + pattern);
+    tablesOrViews = db.getTablesByType(dbName, pattern, type);
+    LOG.debug("results : " + tablesOrViews.size());
 
     // write the results in the file
-    DataOutputStream outStream = getOutputStream(showTbls.getResFile());
+    DataOutputStream outStream = null;
     try {
-      SortedSet<String> sortedTbls = new TreeSet<String>(tbls);
-      formatter.showTables(outStream, sortedTbls);
+      Path resFile = new Path(resultsFile);
+      FileSystem fs = resFile.getFileSystem(conf);
+      outStream = fs.create(resFile);
+
+      SortedSet<String> sortedSet = new TreeSet<String>(tablesOrViews);
+      formatter.showTables(outStream, sortedSet);
+      outStream.close();
+      outStream = null;
     } catch (Exception e) {
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR, "in database" + dbName);
     } finally {


[12/15] hive git commit: HIVE-14671 : merge master into hive-14535 (Sergey Shelukhin)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/754443e6/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --cc metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 4134483,d827d6c..6fdd29a
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@@ -48379,8 -48004,1032 +48532,1032 @@@ public class ThriftHiveMetastore 
      public boolean equals(Object that) {
        if (that == null)
          return false;
-       if (that instanceof get_tables_args)
-         return this.equals((get_tables_args)that);
+       if (that instanceof get_tables_args)
+         return this.equals((get_tables_args)that);
+       return false;
+     }
+ 
+     public boolean equals(get_tables_args that) {
+       if (that == null)
+         return false;
+ 
+       boolean this_present_db_name = true && this.isSetDb_name();
+       boolean that_present_db_name = true && that.isSetDb_name();
+       if (this_present_db_name || that_present_db_name) {
+         if (!(this_present_db_name && that_present_db_name))
+           return false;
+         if (!this.db_name.equals(that.db_name))
+           return false;
+       }
+ 
+       boolean this_present_pattern = true && this.isSetPattern();
+       boolean that_present_pattern = true && that.isSetPattern();
+       if (this_present_pattern || that_present_pattern) {
+         if (!(this_present_pattern && that_present_pattern))
+           return false;
+         if (!this.pattern.equals(that.pattern))
+           return false;
+       }
+ 
+       return true;
+     }
+ 
+     @Override
+     public int hashCode() {
+       List<Object> list = new ArrayList<Object>();
+ 
+       boolean present_db_name = true && (isSetDb_name());
+       list.add(present_db_name);
+       if (present_db_name)
+         list.add(db_name);
+ 
+       boolean present_pattern = true && (isSetPattern());
+       list.add(present_pattern);
+       if (present_pattern)
+         list.add(pattern);
+ 
+       return list.hashCode();
+     }
+ 
+     @Override
+     public int compareTo(get_tables_args other) {
+       if (!getClass().equals(other.getClass())) {
+         return getClass().getName().compareTo(other.getClass().getName());
+       }
+ 
+       int lastComparison = 0;
+ 
+       lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(other.isSetDb_name());
+       if (lastComparison != 0) {
+         return lastComparison;
+       }
+       if (isSetDb_name()) {
+         lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_name, other.db_name);
+         if (lastComparison != 0) {
+           return lastComparison;
+         }
+       }
+       lastComparison = Boolean.valueOf(isSetPattern()).compareTo(other.isSetPattern());
+       if (lastComparison != 0) {
+         return lastComparison;
+       }
+       if (isSetPattern()) {
+         lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pattern, other.pattern);
+         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("get_tables_args(");
+       boolean first = true;
+ 
+       sb.append("db_name:");
+       if (this.db_name == null) {
+         sb.append("null");
+       } else {
+         sb.append(this.db_name);
+       }
+       first = false;
+       if (!first) sb.append(", ");
+       sb.append("pattern:");
+       if (this.pattern == null) {
+         sb.append("null");
+       } else {
+         sb.append(this.pattern);
+       }
+       first = false;
+       sb.append(")");
+       return sb.toString();
+     }
+ 
+     public void validate() throws org.apache.thrift.TException {
+       // check for required fields
+       // 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 {
+         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 get_tables_argsStandardSchemeFactory implements SchemeFactory {
+       public get_tables_argsStandardScheme getScheme() {
+         return new get_tables_argsStandardScheme();
+       }
+     }
+ 
+     private static class get_tables_argsStandardScheme extends StandardScheme<get_tables_args> {
+ 
+       public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_args struct) throws org.apache.thrift.TException {
+         org.apache.thrift.protocol.TField schemeField;
+         iprot.readStructBegin();
+         while (true)
+         {
+           schemeField = iprot.readFieldBegin();
+           if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+             break;
+           }
+           switch (schemeField.id) {
+             case 1: // DB_NAME
+               if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                 struct.db_name = iprot.readString();
+                 struct.setDb_nameIsSet(true);
+               } else { 
+                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+               }
+               break;
+             case 2: // PATTERN
+               if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                 struct.pattern = iprot.readString();
+                 struct.setPatternIsSet(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, get_tables_args struct) throws org.apache.thrift.TException {
+         struct.validate();
+ 
+         oprot.writeStructBegin(STRUCT_DESC);
+         if (struct.db_name != null) {
+           oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
+           oprot.writeString(struct.db_name);
+           oprot.writeFieldEnd();
+         }
+         if (struct.pattern != null) {
+           oprot.writeFieldBegin(PATTERN_FIELD_DESC);
+           oprot.writeString(struct.pattern);
+           oprot.writeFieldEnd();
+         }
+         oprot.writeFieldStop();
+         oprot.writeStructEnd();
+       }
+ 
+     }
+ 
+     private static class get_tables_argsTupleSchemeFactory implements SchemeFactory {
+       public get_tables_argsTupleScheme getScheme() {
+         return new get_tables_argsTupleScheme();
+       }
+     }
+ 
+     private static class get_tables_argsTupleScheme extends TupleScheme<get_tables_args> {
+ 
+       @Override
+       public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_args struct) throws org.apache.thrift.TException {
+         TTupleProtocol oprot = (TTupleProtocol) prot;
+         BitSet optionals = new BitSet();
+         if (struct.isSetDb_name()) {
+           optionals.set(0);
+         }
+         if (struct.isSetPattern()) {
+           optionals.set(1);
+         }
+         oprot.writeBitSet(optionals, 2);
+         if (struct.isSetDb_name()) {
+           oprot.writeString(struct.db_name);
+         }
+         if (struct.isSetPattern()) {
+           oprot.writeString(struct.pattern);
+         }
+       }
+ 
+       @Override
+       public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_args struct) throws org.apache.thrift.TException {
+         TTupleProtocol iprot = (TTupleProtocol) prot;
+         BitSet incoming = iprot.readBitSet(2);
+         if (incoming.get(0)) {
+           struct.db_name = iprot.readString();
+           struct.setDb_nameIsSet(true);
+         }
+         if (incoming.get(1)) {
+           struct.pattern = iprot.readString();
+           struct.setPatternIsSet(true);
+         }
+       }
+     }
+ 
+   }
+ 
+   public static class get_tables_result implements org.apache.thrift.TBase<get_tables_result, get_tables_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_result>   {
+     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_result");
+ 
+     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
+     private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ 
+     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+     static {
+       schemes.put(StandardScheme.class, new get_tables_resultStandardSchemeFactory());
+       schemes.put(TupleScheme.class, new get_tables_resultTupleSchemeFactory());
+     }
+ 
+     private List<String> success; // required
+     private MetaException o1; // required
+ 
+     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+       SUCCESS((short)0, "success"),
+       O1((short)1, "o1");
+ 
+       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 0: // SUCCESS
+             return SUCCESS;
+           case 1: // O1
+             return O1;
+           default:
+             return null;
+         }
+       }
+ 
+       /**
+        * Find the _Fields constant that matches fieldId, throwing an exception
+        * if it is not found.
+        */
+       public static _Fields findByThriftIdOrThrow(int fieldId) {
+         _Fields fields = findByThriftId(fieldId);
+         if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+         return fields;
+       }
+ 
+       /**
+        * Find the _Fields constant that matches name, or null if its not found.
+        */
+       public static _Fields findByName(String name) {
+         return byName.get(name);
+       }
+ 
+       private final short _thriftId;
+       private final String _fieldName;
+ 
+       _Fields(short thriftId, String fieldName) {
+         _thriftId = thriftId;
+         _fieldName = fieldName;
+       }
+ 
+       public short getThriftFieldId() {
+         return _thriftId;
+       }
+ 
+       public String getFieldName() {
+         return _fieldName;
+       }
+     }
+ 
+     // isset id assignments
+     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+     static {
+       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+       tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+           new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+       tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+       metaDataMap = Collections.unmodifiableMap(tmpMap);
+       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_tables_result.class, metaDataMap);
+     }
+ 
+     public get_tables_result() {
+     }
+ 
+     public get_tables_result(
+       List<String> success,
+       MetaException o1)
+     {
+       this();
+       this.success = success;
+       this.o1 = o1;
+     }
+ 
+     /**
+      * Performs a deep copy on <i>other</i>.
+      */
+     public get_tables_result(get_tables_result other) {
+       if (other.isSetSuccess()) {
+         List<String> __this__success = new ArrayList<String>(other.success);
+         this.success = __this__success;
+       }
+       if (other.isSetO1()) {
+         this.o1 = new MetaException(other.o1);
+       }
+     }
+ 
+     public get_tables_result deepCopy() {
+       return new get_tables_result(this);
+     }
+ 
+     @Override
+     public void clear() {
+       this.success = null;
+       this.o1 = null;
+     }
+ 
+     public int getSuccessSize() {
+       return (this.success == null) ? 0 : this.success.size();
+     }
+ 
+     public java.util.Iterator<String> getSuccessIterator() {
+       return (this.success == null) ? null : this.success.iterator();
+     }
+ 
+     public void addToSuccess(String elem) {
+       if (this.success == null) {
+         this.success = new ArrayList<String>();
+       }
+       this.success.add(elem);
+     }
+ 
+     public List<String> getSuccess() {
+       return this.success;
+     }
+ 
+     public void setSuccess(List<String> success) {
+       this.success = success;
+     }
+ 
+     public void unsetSuccess() {
+       this.success = null;
+     }
+ 
+     /** Returns true if field success is set (has been assigned a value) and false otherwise */
+     public boolean isSetSuccess() {
+       return this.success != null;
+     }
+ 
+     public void setSuccessIsSet(boolean value) {
+       if (!value) {
+         this.success = null;
+       }
+     }
+ 
+     public MetaException getO1() {
+       return this.o1;
+     }
+ 
+     public void setO1(MetaException o1) {
+       this.o1 = o1;
+     }
+ 
+     public void unsetO1() {
+       this.o1 = null;
+     }
+ 
+     /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
+     public boolean isSetO1() {
+       return this.o1 != null;
+     }
+ 
+     public void setO1IsSet(boolean value) {
+       if (!value) {
+         this.o1 = null;
+       }
+     }
+ 
+     public void setFieldValue(_Fields field, Object value) {
+       switch (field) {
+       case SUCCESS:
+         if (value == null) {
+           unsetSuccess();
+         } else {
+           setSuccess((List<String>)value);
+         }
+         break;
+ 
+       case O1:
+         if (value == null) {
+           unsetO1();
+         } else {
+           setO1((MetaException)value);
+         }
+         break;
+ 
+       }
+     }
+ 
+     public Object getFieldValue(_Fields field) {
+       switch (field) {
+       case SUCCESS:
+         return getSuccess();
+ 
+       case O1:
+         return getO1();
+ 
+       }
+       throw new IllegalStateException();
+     }
+ 
+     /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+     public boolean isSet(_Fields field) {
+       if (field == null) {
+         throw new IllegalArgumentException();
+       }
+ 
+       switch (field) {
+       case SUCCESS:
+         return isSetSuccess();
+       case O1:
+         return isSetO1();
+       }
+       throw new IllegalStateException();
+     }
+ 
+     @Override
+     public boolean equals(Object that) {
+       if (that == null)
+         return false;
+       if (that instanceof get_tables_result)
+         return this.equals((get_tables_result)that);
+       return false;
+     }
+ 
+     public boolean equals(get_tables_result that) {
+       if (that == null)
+         return false;
+ 
+       boolean this_present_success = true && this.isSetSuccess();
+       boolean that_present_success = true && that.isSetSuccess();
+       if (this_present_success || that_present_success) {
+         if (!(this_present_success && that_present_success))
+           return false;
+         if (!this.success.equals(that.success))
+           return false;
+       }
+ 
+       boolean this_present_o1 = true && this.isSetO1();
+       boolean that_present_o1 = true && that.isSetO1();
+       if (this_present_o1 || that_present_o1) {
+         if (!(this_present_o1 && that_present_o1))
+           return false;
+         if (!this.o1.equals(that.o1))
+           return false;
+       }
+ 
+       return true;
+     }
+ 
+     @Override
+     public int hashCode() {
+       List<Object> list = new ArrayList<Object>();
+ 
+       boolean present_success = true && (isSetSuccess());
+       list.add(present_success);
+       if (present_success)
+         list.add(success);
+ 
+       boolean present_o1 = true && (isSetO1());
+       list.add(present_o1);
+       if (present_o1)
+         list.add(o1);
+ 
+       return list.hashCode();
+     }
+ 
+     @Override
+     public int compareTo(get_tables_result other) {
+       if (!getClass().equals(other.getClass())) {
+         return getClass().getName().compareTo(other.getClass().getName());
+       }
+ 
+       int lastComparison = 0;
+ 
+       lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+       if (lastComparison != 0) {
+         return lastComparison;
+       }
+       if (isSetSuccess()) {
+         lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+         if (lastComparison != 0) {
+           return lastComparison;
+         }
+       }
+       lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
+       if (lastComparison != 0) {
+         return lastComparison;
+       }
+       if (isSetO1()) {
+         lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, other.o1);
+         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("get_tables_result(");
+       boolean first = true;
+ 
+       sb.append("success:");
+       if (this.success == null) {
+         sb.append("null");
+       } else {
+         sb.append(this.success);
+       }
+       first = false;
+       if (!first) sb.append(", ");
+       sb.append("o1:");
+       if (this.o1 == null) {
+         sb.append("null");
+       } else {
+         sb.append(this.o1);
+       }
+       first = false;
+       sb.append(")");
+       return sb.toString();
+     }
+ 
+     public void validate() throws org.apache.thrift.TException {
+       // check for required fields
+       // 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 {
+         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 get_tables_resultStandardSchemeFactory implements SchemeFactory {
+       public get_tables_resultStandardScheme getScheme() {
+         return new get_tables_resultStandardScheme();
+       }
+     }
+ 
+     private static class get_tables_resultStandardScheme extends StandardScheme<get_tables_result> {
+ 
+       public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_result struct) throws org.apache.thrift.TException {
+         org.apache.thrift.protocol.TField schemeField;
+         iprot.readStructBegin();
+         while (true)
+         {
+           schemeField = iprot.readFieldBegin();
+           if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+             break;
+           }
+           switch (schemeField.id) {
+             case 0: // SUCCESS
+               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                 {
 -                  org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
 -                  struct.success = new ArrayList<String>(_list716.size);
 -                  String _elem717;
 -                  for (int _i718 = 0; _i718 < _list716.size; ++_i718)
++                  org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
++                  struct.success = new ArrayList<String>(_list724.size);
++                  String _elem725;
++                  for (int _i726 = 0; _i726 < _list724.size; ++_i726)
+                   {
 -                    _elem717 = iprot.readString();
 -                    struct.success.add(_elem717);
++                    _elem725 = iprot.readString();
++                    struct.success.add(_elem725);
+                   }
+                   iprot.readListEnd();
+                 }
+                 struct.setSuccessIsSet(true);
+               } else { 
+                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+               }
+               break;
+             case 1: // O1
+               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                 struct.o1 = new MetaException();
+                 struct.o1.read(iprot);
+                 struct.setO1IsSet(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, get_tables_result struct) throws org.apache.thrift.TException {
+         struct.validate();
+ 
+         oprot.writeStructBegin(STRUCT_DESC);
+         if (struct.success != null) {
+           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+           {
+             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
 -            for (String _iter719 : struct.success)
++            for (String _iter727 : struct.success)
+             {
 -              oprot.writeString(_iter719);
++              oprot.writeString(_iter727);
+             }
+             oprot.writeListEnd();
+           }
+           oprot.writeFieldEnd();
+         }
+         if (struct.o1 != null) {
+           oprot.writeFieldBegin(O1_FIELD_DESC);
+           struct.o1.write(oprot);
+           oprot.writeFieldEnd();
+         }
+         oprot.writeFieldStop();
+         oprot.writeStructEnd();
+       }
+ 
+     }
+ 
+     private static class get_tables_resultTupleSchemeFactory implements SchemeFactory {
+       public get_tables_resultTupleScheme getScheme() {
+         return new get_tables_resultTupleScheme();
+       }
+     }
+ 
+     private static class get_tables_resultTupleScheme extends TupleScheme<get_tables_result> {
+ 
+       @Override
+       public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_result struct) throws org.apache.thrift.TException {
+         TTupleProtocol oprot = (TTupleProtocol) prot;
+         BitSet optionals = new BitSet();
+         if (struct.isSetSuccess()) {
+           optionals.set(0);
+         }
+         if (struct.isSetO1()) {
+           optionals.set(1);
+         }
+         oprot.writeBitSet(optionals, 2);
+         if (struct.isSetSuccess()) {
+           {
+             oprot.writeI32(struct.success.size());
 -            for (String _iter720 : struct.success)
++            for (String _iter728 : struct.success)
+             {
 -              oprot.writeString(_iter720);
++              oprot.writeString(_iter728);
+             }
+           }
+         }
+         if (struct.isSetO1()) {
+           struct.o1.write(oprot);
+         }
+       }
+ 
+       @Override
+       public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_result struct) throws org.apache.thrift.TException {
+         TTupleProtocol iprot = (TTupleProtocol) prot;
+         BitSet incoming = iprot.readBitSet(2);
+         if (incoming.get(0)) {
+           {
 -            org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
 -            struct.success = new ArrayList<String>(_list721.size);
 -            String _elem722;
 -            for (int _i723 = 0; _i723 < _list721.size; ++_i723)
++            org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.success = new ArrayList<String>(_list729.size);
++            String _elem730;
++            for (int _i731 = 0; _i731 < _list729.size; ++_i731)
+             {
 -              _elem722 = iprot.readString();
 -              struct.success.add(_elem722);
++              _elem730 = iprot.readString();
++              struct.success.add(_elem730);
+             }
+           }
+           struct.setSuccessIsSet(true);
+         }
+         if (incoming.get(1)) {
+           struct.o1 = new MetaException();
+           struct.o1.read(iprot);
+           struct.setO1IsSet(true);
+         }
+       }
+     }
+ 
+   }
+ 
+   public static class get_tables_by_type_args implements org.apache.thrift.TBase<get_tables_by_type_args, get_tables_by_type_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_by_type_args>   {
+     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_by_type_args");
+ 
+     private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1);
+     private static final org.apache.thrift.protocol.TField PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("pattern", org.apache.thrift.protocol.TType.STRING, (short)2);
+     private static final org.apache.thrift.protocol.TField TABLE_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tableType", org.apache.thrift.protocol.TType.STRING, (short)3);
+ 
+     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+     static {
+       schemes.put(StandardScheme.class, new get_tables_by_type_argsStandardSchemeFactory());
+       schemes.put(TupleScheme.class, new get_tables_by_type_argsTupleSchemeFactory());
+     }
+ 
+     private String db_name; // required
+     private String pattern; // required
+     private String tableType; // required
+ 
+     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+       DB_NAME((short)1, "db_name"),
+       PATTERN((short)2, "pattern"),
+       TABLE_TYPE((short)3, "tableType");
+ 
+       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: // DB_NAME
+             return DB_NAME;
+           case 2: // PATTERN
+             return PATTERN;
+           case 3: // TABLE_TYPE
+             return TABLE_TYPE;
+           default:
+             return null;
+         }
+       }
+ 
+       /**
+        * Find the _Fields constant that matches fieldId, throwing an exception
+        * if it is not found.
+        */
+       public static _Fields findByThriftIdOrThrow(int fieldId) {
+         _Fields fields = findByThriftId(fieldId);
+         if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+         return fields;
+       }
+ 
+       /**
+        * Find the _Fields constant that matches name, or null if its not found.
+        */
+       public static _Fields findByName(String name) {
+         return byName.get(name);
+       }
+ 
+       private final short _thriftId;
+       private final String _fieldName;
+ 
+       _Fields(short thriftId, String fieldName) {
+         _thriftId = thriftId;
+         _fieldName = fieldName;
+       }
+ 
+       public short getThriftFieldId() {
+         return _thriftId;
+       }
+ 
+       public String getFieldName() {
+         return _fieldName;
+       }
+     }
+ 
+     // isset id assignments
+     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+     static {
+       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+       tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+       tmpMap.put(_Fields.PATTERN, new org.apache.thrift.meta_data.FieldMetaData("pattern", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+       tmpMap.put(_Fields.TABLE_TYPE, new org.apache.thrift.meta_data.FieldMetaData("tableType", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+       metaDataMap = Collections.unmodifiableMap(tmpMap);
+       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_tables_by_type_args.class, metaDataMap);
+     }
+ 
+     public get_tables_by_type_args() {
+     }
+ 
+     public get_tables_by_type_args(
+       String db_name,
+       String pattern,
+       String tableType)
+     {
+       this();
+       this.db_name = db_name;
+       this.pattern = pattern;
+       this.tableType = tableType;
+     }
+ 
+     /**
+      * Performs a deep copy on <i>other</i>.
+      */
+     public get_tables_by_type_args(get_tables_by_type_args other) {
+       if (other.isSetDb_name()) {
+         this.db_name = other.db_name;
+       }
+       if (other.isSetPattern()) {
+         this.pattern = other.pattern;
+       }
+       if (other.isSetTableType()) {
+         this.tableType = other.tableType;
+       }
+     }
+ 
+     public get_tables_by_type_args deepCopy() {
+       return new get_tables_by_type_args(this);
+     }
+ 
+     @Override
+     public void clear() {
+       this.db_name = null;
+       this.pattern = null;
+       this.tableType = null;
+     }
+ 
+     public String getDb_name() {
+       return this.db_name;
+     }
+ 
+     public void setDb_name(String db_name) {
+       this.db_name = db_name;
+     }
+ 
+     public void unsetDb_name() {
+       this.db_name = null;
+     }
+ 
+     /** Returns true if field db_name is set (has been assigned a value) and false otherwise */
+     public boolean isSetDb_name() {
+       return this.db_name != null;
+     }
+ 
+     public void setDb_nameIsSet(boolean value) {
+       if (!value) {
+         this.db_name = null;
+       }
+     }
+ 
+     public String getPattern() {
+       return this.pattern;
+     }
+ 
+     public void setPattern(String pattern) {
+       this.pattern = pattern;
+     }
+ 
+     public void unsetPattern() {
+       this.pattern = null;
+     }
+ 
+     /** Returns true if field pattern is set (has been assigned a value) and false otherwise */
+     public boolean isSetPattern() {
+       return this.pattern != null;
+     }
+ 
+     public void setPatternIsSet(boolean value) {
+       if (!value) {
+         this.pattern = null;
+       }
+     }
+ 
+     public String getTableType() {
+       return this.tableType;
+     }
+ 
+     public void setTableType(String tableType) {
+       this.tableType = tableType;
+     }
+ 
+     public void unsetTableType() {
+       this.tableType = null;
+     }
+ 
+     /** Returns true if field tableType is set (has been assigned a value) and false otherwise */
+     public boolean isSetTableType() {
+       return this.tableType != null;
+     }
+ 
+     public void setTableTypeIsSet(boolean value) {
+       if (!value) {
+         this.tableType = null;
+       }
+     }
+ 
+     public void setFieldValue(_Fields field, Object value) {
+       switch (field) {
+       case DB_NAME:
+         if (value == null) {
+           unsetDb_name();
+         } else {
+           setDb_name((String)value);
+         }
+         break;
+ 
+       case PATTERN:
+         if (value == null) {
+           unsetPattern();
+         } else {
+           setPattern((String)value);
+         }
+         break;
+ 
+       case TABLE_TYPE:
+         if (value == null) {
+           unsetTableType();
+         } else {
+           setTableType((String)value);
+         }
+         break;
+ 
+       }
+     }
+ 
+     public Object getFieldValue(_Fields field) {
+       switch (field) {
+       case DB_NAME:
+         return getDb_name();
+ 
+       case PATTERN:
+         return getPattern();
+ 
+       case TABLE_TYPE:
+         return getTableType();
+ 
+       }
+       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 DB_NAME:
+         return isSetDb_name();
+       case PATTERN:
+         return isSetPattern();
+       case TABLE_TYPE:
+         return isSetTableType();
+       }
+       throw new IllegalStateException();
+     }
+ 
+     @Override
+     public boolean equals(Object that) {
+       if (that == null)
+         return false;
+       if (that instanceof get_tables_by_type_args)
+         return this.equals((get_tables_by_type_args)that);
        return false;
      }
  
@@@ -49014,13 -49718,13 +50246,13 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
--                  struct.success = new ArrayList<String>(_list724.size);
--                  String _elem725;
--                  for (int _i726 = 0; _i726 < _list724.size; ++_i726)
++                  org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
++                  struct.success = new ArrayList<String>(_list732.size);
++                  String _elem733;
++                  for (int _i734 = 0; _i734 < _list732.size; ++_i734)
                    {
--                    _elem725 = iprot.readString();
--                    struct.success.add(_elem725);
++                    _elem733 = iprot.readString();
++                    struct.success.add(_elem733);
                    }
                    iprot.readListEnd();
                  }
@@@ -49055,9 -49759,9 +50287,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
--            for (String _iter727 : struct.success)
++            for (String _iter735 : struct.success)
              {
--              oprot.writeString(_iter727);
++              oprot.writeString(_iter735);
              }
              oprot.writeListEnd();
            }
@@@ -49096,9 -49800,9 +50328,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
--            for (String _iter728 : struct.success)
++            for (String _iter736 : struct.success)
              {
--              oprot.writeString(_iter728);
++              oprot.writeString(_iter736);
              }
            }
          }
@@@ -49113,13 -49817,13 +50345,13 @@@
          BitSet incoming = iprot.readBitSet(2);
          if (incoming.get(0)) {
            {
--            org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
--            struct.success = new ArrayList<String>(_list729.size);
--            String _elem730;
--            for (int _i731 = 0; _i731 < _list729.size; ++_i731)
++            org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.success = new ArrayList<String>(_list737.size);
++            String _elem738;
++            for (int _i739 = 0; _i739 < _list737.size; ++_i739)
              {
--              _elem730 = iprot.readString();
--              struct.success.add(_elem730);
++              _elem738 = iprot.readString();
++              struct.success.add(_elem738);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -49624,13 -50328,13 +50856,13 @@@
              case 3: // TBL_TYPES
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
--                  struct.tbl_types = new ArrayList<String>(_list732.size);
--                  String _elem733;
--                  for (int _i734 = 0; _i734 < _list732.size; ++_i734)
++                  org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
++                  struct.tbl_types = new ArrayList<String>(_list740.size);
++                  String _elem741;
++                  for (int _i742 = 0; _i742 < _list740.size; ++_i742)
                    {
--                    _elem733 = iprot.readString();
--                    struct.tbl_types.add(_elem733);
++                    _elem741 = iprot.readString();
++                    struct.tbl_types.add(_elem741);
                    }
                    iprot.readListEnd();
                  }
@@@ -49666,9 -50370,9 +50898,9 @@@
            oprot.writeFieldBegin(TBL_TYPES_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_types.size()));
--            for (String _iter735 : struct.tbl_types)
++            for (String _iter743 : struct.tbl_types)
              {
--              oprot.writeString(_iter735);
++              oprot.writeString(_iter743);
              }
              oprot.writeListEnd();
            }
@@@ -49711,9 -50415,9 +50943,9 @@@
          if (struct.isSetTbl_types()) {
            {
              oprot.writeI32(struct.tbl_types.size());
--            for (String _iter736 : struct.tbl_types)
++            for (String _iter744 : struct.tbl_types)
              {
--              oprot.writeString(_iter736);
++              oprot.writeString(_iter744);
              }
            }
          }
@@@ -49733,13 -50437,13 +50965,13 @@@
          }
          if (incoming.get(2)) {
            {
--            org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
--            struct.tbl_types = new ArrayList<String>(_list737.size);
--            String _elem738;
--            for (int _i739 = 0; _i739 < _list737.size; ++_i739)
++            org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.tbl_types = new ArrayList<String>(_list745.size);
++            String _elem746;
++            for (int _i747 = 0; _i747 < _list745.size; ++_i747)
              {
--              _elem738 = iprot.readString();
--              struct.tbl_types.add(_elem738);
++              _elem746 = iprot.readString();
++              struct.tbl_types.add(_elem746);
              }
            }
            struct.setTbl_typesIsSet(true);
@@@ -50145,14 -50849,14 +51377,14 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
--                  struct.success = new ArrayList<TableMeta>(_list740.size);
--                  TableMeta _elem741;
--                  for (int _i742 = 0; _i742 < _list740.size; ++_i742)
++                  org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
++                  struct.success = new ArrayList<TableMeta>(_list748.size);
++                  TableMeta _elem749;
++                  for (int _i750 = 0; _i750 < _list748.size; ++_i750)
                    {
--                    _elem741 = new TableMeta();
--                    _elem741.read(iprot);
--                    struct.success.add(_elem741);
++                    _elem749 = new TableMeta();
++                    _elem749.read(iprot);
++                    struct.success.add(_elem749);
                    }
                    iprot.readListEnd();
                  }
@@@ -50187,9 -50891,9 +51419,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
--            for (TableMeta _iter743 : struct.success)
++            for (TableMeta _iter751 : struct.success)
              {
--              _iter743.write(oprot);
++              _iter751.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -50228,9 -50932,9 +51460,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
--            for (TableMeta _iter744 : struct.success)
++            for (TableMeta _iter752 : struct.success)
              {
--              _iter744.write(oprot);
++              _iter752.write(oprot);
              }
            }
          }
@@@ -50245,14 -50949,14 +51477,14 @@@
          BitSet incoming = iprot.readBitSet(2);
          if (incoming.get(0)) {
            {
--            org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
--            struct.success = new ArrayList<TableMeta>(_list745.size);
--            TableMeta _elem746;
--            for (int _i747 = 0; _i747 < _list745.size; ++_i747)
++            org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.success = new ArrayList<TableMeta>(_list753.size);
++            TableMeta _elem754;
++            for (int _i755 = 0; _i755 < _list753.size; ++_i755)
              {
--              _elem746 = new TableMeta();
--              _elem746.read(iprot);
--              struct.success.add(_elem746);
++              _elem754 = new TableMeta();
++              _elem754.read(iprot);
++              struct.success.add(_elem754);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -51018,13 -51722,13 +52250,13 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
--                  struct.success = new ArrayList<String>(_list748.size);
--                  String _elem749;
--                  for (int _i750 = 0; _i750 < _list748.size; ++_i750)
++                  org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
++                  struct.success = new ArrayList<String>(_list756.size);
++                  String _elem757;
++                  for (int _i758 = 0; _i758 < _list756.size; ++_i758)
                    {
--                    _elem749 = iprot.readString();
--                    struct.success.add(_elem749);
++                    _elem757 = iprot.readString();
++                    struct.success.add(_elem757);
                    }
                    iprot.readListEnd();
                  }
@@@ -51059,9 -51763,9 +52291,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
--            for (String _iter751 : struct.success)
++            for (String _iter759 : struct.success)
              {
--              oprot.writeString(_iter751);
++              oprot.writeString(_iter759);
              }
              oprot.writeListEnd();
            }
@@@ -51100,9 -51804,9 +52332,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
--            for (String _iter752 : struct.success)
++            for (String _iter760 : struct.success)
              {
--              oprot.writeString(_iter752);
++              oprot.writeString(_iter760);
              }
            }
          }
@@@ -51117,13 -51821,13 +52349,13 @@@
          BitSet incoming = iprot.readBitSet(2);
          if (incoming.get(0)) {
            {
--            org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
--            struct.success = new ArrayList<String>(_list753.size);
--            String _elem754;
--            for (int _i755 = 0; _i755 < _list753.size; ++_i755)
++            org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.success = new ArrayList<String>(_list761.size);
++            String _elem762;
++            for (int _i763 = 0; _i763 < _list761.size; ++_i763)
              {
--              _elem754 = iprot.readString();
--              struct.success.add(_elem754);
++              _elem762 = iprot.readString();
++              struct.success.add(_elem762);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -52576,13 -53280,13 +53808,13 @@@
              case 2: // TBL_NAMES
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
--                  struct.tbl_names = new ArrayList<String>(_list756.size);
--                  String _elem757;
--                  for (int _i758 = 0; _i758 < _list756.size; ++_i758)
++                  org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
++                  struct.tbl_names = new ArrayList<String>(_list764.size);
++                  String _elem765;
++                  for (int _i766 = 0; _i766 < _list764.size; ++_i766)
                    {
--                    _elem757 = iprot.readString();
--                    struct.tbl_names.add(_elem757);
++                    _elem765 = iprot.readString();
++                    struct.tbl_names.add(_elem765);
                    }
                    iprot.readListEnd();
                  }
@@@ -52613,9 -53317,9 +53845,9 @@@
            oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
--            for (String _iter759 : struct.tbl_names)
++            for (String _iter767 : struct.tbl_names)
              {
--              oprot.writeString(_iter759);
++              oprot.writeString(_iter767);
              }
              oprot.writeListEnd();
            }
@@@ -52652,9 -53356,9 +53884,9 @@@
          if (struct.isSetTbl_names()) {
            {
              oprot.writeI32(struct.tbl_names.size());
--            for (String _iter760 : struct.tbl_names)
++            for (String _iter768 : struct.tbl_names)
              {
--              oprot.writeString(_iter760);
++              oprot.writeString(_iter768);
              }
            }
          }
@@@ -52670,13 -53374,13 +53902,13 @@@
          }
          if (incoming.get(1)) {
            {
--            org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
--            struct.tbl_names = new ArrayList<String>(_list761.size);
--            String _elem762;
--            for (int _i763 = 0; _i763 < _list761.size; ++_i763)
++            org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.tbl_names = new ArrayList<String>(_list769.size);
++            String _elem770;
++            for (int _i771 = 0; _i771 < _list769.size; ++_i771)
              {
--              _elem762 = iprot.readString();
--              struct.tbl_names.add(_elem762);
++              _elem770 = iprot.readString();
++              struct.tbl_names.add(_elem770);
              }
            }
            struct.setTbl_namesIsSet(true);
@@@ -53244,14 -53948,14 +54476,14 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
--                  struct.success = new ArrayList<Table>(_list764.size);
--                  Table _elem765;
--                  for (int _i766 = 0; _i766 < _list764.size; ++_i766)
++                  org.apache.thrift.protocol.TList _list772 = iprot.readListBegin();
++                  struct.success = new ArrayList<Table>(_list772.size);
++                  Table _elem773;
++                  for (int _i774 = 0; _i774 < _list772.size; ++_i774)
                    {
--                    _elem765 = new Table();
--                    _elem765.read(iprot);
--                    struct.success.add(_elem765);
++                    _elem773 = new Table();
++                    _elem773.read(iprot);
++                    struct.success.add(_elem773);
                    }
                    iprot.readListEnd();
                  }
@@@ -53304,9 -54008,9 +54536,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
--            for (Table _iter767 : struct.success)
++            for (Table _iter775 : struct.success)
              {
--              _iter767.write(oprot);
++              _iter775.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -53361,9 -54065,9 +54593,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
--            for (Table _iter768 : struct.success)
++            for (Table _iter776 : struct.success)
              {
--              _iter768.write(oprot);
++              _iter776.write(oprot);
              }
            }
          }
@@@ -53384,14 -54088,14 +54616,14 @@@
          BitSet incoming = iprot.readBitSet(4);
          if (incoming.get(0)) {
            {
--            org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
--            struct.success = new ArrayList<Table>(_list769.size);
--            Table _elem770;
--            for (int _i771 = 0; _i771 < _list769.size; ++_i771)
++            org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.success = new ArrayList<Table>(_list777.size);
++            Table _elem778;
++            for (int _i779 = 0; _i779 < _list777.size; ++_i779)
              {
--              _elem770 = new Table();
--              _elem770.read(iprot);
--              struct.success.add(_elem770);
++              _elem778 = new Table();
++              _elem778.read(iprot);
++              struct.success.add(_elem778);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -54537,13 -55241,13 +55769,13 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list772 = iprot.readListBegin();
--                  struct.success = new ArrayList<String>(_list772.size);
--                  String _elem773;
--                  for (int _i774 = 0; _i774 < _list772.size; ++_i774)
++                  org.apache.thrift.protocol.TList _list780 = iprot.readListBegin();
++                  struct.success = new ArrayList<String>(_list780.size);
++                  String _elem781;
++                  for (int _i782 = 0; _i782 < _list780.size; ++_i782)
                    {
--                    _elem773 = iprot.readString();
--                    struct.success.add(_elem773);
++                    _elem781 = iprot.readString();
++                    struct.success.add(_elem781);
                    }
                    iprot.readListEnd();
                  }
@@@ -54596,9 -55300,9 +55828,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
--            for (String _iter775 : struct.success)
++            for (String _iter783 : struct.success)
              {
--              oprot.writeString(_iter775);
++              oprot.writeString(_iter783);
              }
              oprot.writeListEnd();
            }
@@@ -54653,9 -55357,9 +55885,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
--            for (String _iter776 : struct.success)
++            for (String _iter784 : struct.success)
              {
--              oprot.writeString(_iter776);
++              oprot.writeString(_iter784);
              }
            }
          }
@@@ -54676,13 -55380,13 +55908,13 @@@
          BitSet incoming = iprot.readBitSet(4);
          if (incoming.get(0)) {
            {
--            org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
--            struct.success = new ArrayList<String>(_list777.size);
--            String _elem778;
--            for (int _i779 = 0; _i779 < _list777.size; ++_i779)
++            org.apache.thrift.protocol.TList _list785 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.success = new ArrayList<String>(_list785.size);
++            String _elem786;
++            for (int _i787 = 0; _i787 < _list785.size; ++_i787)
              {
--              _elem778 = iprot.readString();
--              struct.success.add(_elem778);
++              _elem786 = iprot.readString();
++              struct.success.add(_elem786);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -60541,14 -61245,14 +61773,14 @@@
              case 1: // NEW_PARTS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list780 = iprot.readListBegin();
--                  struct.new_parts = new ArrayList<Partition>(_list780.size);
--                  Partition _elem781;
--                  for (int _i782 = 0; _i782 < _list780.size; ++_i782)
++                  org.apache.thrift.protocol.TList _list788 = iprot.readListBegin();
++                  struct.new_parts = new ArrayList<Partition>(_list788.size);
++                  Partition _elem789;
++                  for (int _i790 = 0; _i790 < _list788.size; ++_i790)
                    {
--                    _elem781 = new Partition();
--                    _elem781.read(iprot);
--                    struct.new_parts.add(_elem781);
++                    _elem789 = new Partition();
++                    _elem789.read(iprot);
++                    struct.new_parts.add(_elem789);
                    }
                    iprot.readListEnd();
                  }
@@@ -60574,9 -61278,9 +61806,9 @@@
            oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
--            for (Partition _iter783 : struct.new_parts)
++            for (Partition _iter791 : struct.new_parts)
              {
--              _iter783.write(oprot);
++              _iter791.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -60607,9 -61311,9 +61839,9 @@@
          if (struct.isSetNew_parts()) {
            {
              oprot.writeI32(struct.new_parts.size());
--            for (Partition _iter784 : struct.new_parts)
++            for (Partition _iter792 : struct.new_parts)
              {
--              _iter784.write(oprot);
++              _iter792.write(oprot);
              }
            }
          }
@@@ -60621,14 -61325,14 +61853,14 @@@
          BitSet incoming = iprot.readBitSet(1);
          if (incoming.get(0)) {
            {
--            org.apache.thrift.protocol.TList _list785 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
--            struct.new_parts = new ArrayList<Partition>(_list785.size);
--            Partition _elem786;
--            for (int _i787 = 0; _i787 < _list785.size; ++_i787)
++            org.apache.thrift.protocol.TList _list793 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.new_parts = new ArrayList<Partition>(_list793.size);
++            Partition _elem794;
++            for (int _i795 = 0; _i795 < _list793.size; ++_i795)
              {
--              _elem786 = new Partition();
--              _elem786.read(iprot);
--              struct.new_parts.add(_elem786);
++              _elem794 = new Partition();
++              _elem794.read(iprot);
++              struct.new_parts.add(_elem794);
              }
            }
            struct.setNew_partsIsSet(true);
@@@ -61629,14 -62333,14 +62861,14 @@@
              case 1: // NEW_PARTS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list788 = iprot.readListBegin();
--                  struct.new_parts = new ArrayList<PartitionSpec>(_list788.size);
--                  PartitionSpec _elem789;
--                  for (int _i790 = 0; _i790 < _list788.size; ++_i790)
++                  org.apache.thrift.protocol.TList _list796 = iprot.readListBegin();
++                  struct.new_parts = new ArrayList<PartitionSpec>(_list796.size);
++                  PartitionSpec _elem797;
++                  for (int _i798 = 0; _i798 < _list796.size; ++_i798)
                    {
--                    _elem789 = new PartitionSpec();
--                    _elem789.read(iprot);
--                    struct.new_parts.add(_elem789);
++                    _elem797 = new PartitionSpec();
++                    _elem797.read(iprot);
++                    struct.new_parts.add(_elem797);
                    }
                    iprot.readListEnd();
                  }
@@@ -61662,9 -62366,9 +62894,9 @@@
            oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
--            for (PartitionSpec _iter791 : struct.new_parts)
++            for (PartitionSpec _iter799 : struct.new_parts)
              {
--              _iter791.write(oprot);
++              _iter799.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -61695,9 -62399,9 +62927,9 @@@
          if (struct.isSetNew_parts()) {
            {
              oprot.writeI32(struct.new_parts.size());
--            for (PartitionSpec _iter792 : struct.new_parts)
++            for (PartitionSpec _iter800 : struct.new_parts)
              {
--              _iter792.write(oprot);
++              _iter800.write(oprot);
              }
            }
          }
@@@ -61709,14 -62413,14 +62941,14 @@@
          BitSet incoming = iprot.readBitSet(1);
          if (incoming.get(0)) {
            {
--            org.apache.thrift.protocol.TList _list793 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
--            struct.new_parts = new ArrayList<PartitionSpec>(_list793.size);
--            PartitionSpec _elem794;
--            for (int _i795 = 0; _i795 < _list793.size; ++_i795)
++            org.apache.thrift.protocol.TList _list801 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.new_parts = new ArrayList<PartitionSpec>(_list801.size);
++            PartitionSpec _elem802;
++            for (int _i803 = 0; _i803 < _list801.size; ++_i803)
              {
--              _elem794 = new PartitionSpec();
--              _elem794.read(iprot);
--              struct.new_parts.add(_elem794);
++              _elem802 = new PartitionSpec();
++              _elem802.read(iprot);
++              struct.new_parts.add(_elem802);
              }
            }
            struct.setNew_partsIsSet(true);
@@@ -62892,13 -63596,13 +64124,13 @@@
              case 3: // PART_VALS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list796 = iprot.readListBegin();
--                  struct.part_vals = new ArrayList<String>(_list796.size);
--                  String _elem797;
--                  for (int _i798 = 0; _i798 < _list796.size; ++_i798)
++                  org.apache.thrift.protocol.TList _list804 = iprot.readListBegin();
++                  struct.part_vals = new ArrayList<String>(_list804.size);
++                  String _elem805;
++                  for (int _i806 = 0; _i806 < _list804.size; ++_i806)
                    {
--                    _elem797 = iprot.readString();
--                    struct.part_vals.add(_elem797);
++                    _elem805 = iprot.readString();
++                    struct.part_vals.add(_elem805);
                    }
                    iprot.readListEnd();
                  }
@@@ -62934,9 -63638,9 +64166,9 @@@
            oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
--            for (String _iter799 : struct.part_vals)
++            for (String _iter807 : struct.part_vals)
              {
--              oprot.writeString(_iter799);
++              oprot.writeString(_iter807);
              }
              oprot.writeListEnd();
            }
@@@ -62979,9 -63683,9 +64211,9 @@@
          if (struct.isSetPart_vals()) {
            {
              oprot.writeI32(struct.part_vals.size());
--            for (String _iter800 : struct.part_vals)
++            for (String _iter808 : struct.part_vals)
              {
--              oprot.writeString(_iter800);
++              oprot.writeString(_iter808);
              }
            }
          }
@@@ -63001,13 -63705,13 +64233,13 @@@
          }
          if (incoming.get(2)) {
            {
--            org.apache.thrift.protocol.TList _list801 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
--            struct.part_vals = new ArrayList<String>(_list801.size);
--            String _elem802;
--            for (int _i803 = 0; _i803 < _list801.size; ++_i803)
++            org.apache.thrift.protocol.TList _list809 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.part_vals = new ArrayList<String>(_list809.size);
++            String _elem810;
++            for (int _i811 = 0; _i811 < _list809.size; ++_i811)
              {
--              _elem802 = iprot.readString();
--              struct.part_vals.add(_elem802);
++              _elem810 = iprot.readString();
++              struct.part_vals.add(_elem810);
              }
            }
            struct.setPart_valsIsSet(true);
@@@ -65316,13 -66020,13 +66548,13 @@@
              case 3: // PART_VALS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list804 = iprot.readListBegin();
--                  struct.part_vals = new ArrayList<String>(_list804.size);
--                  String _elem805;
--                  for (int _i806 = 0; _i806 < _list804.size; ++_i806)
++                  org.apache.thrift.protocol.TList _list812 = iprot.readListBegin();
++                  struct.part_vals = new ArrayList<String>(_list812.size);
++                  String _elem813;
++                  for (int _i814 = 0; _i814 < _list812.size; ++_i814)
                    {
--                    _elem805 = iprot.readString();
--                    struct.part_vals.add(_elem805);
++                    _elem813 = iprot.readString();
++                    struct.part_vals.add(_elem813);
                    }
                    iprot.readListEnd();
                  }
@@@ -65367,9 -66071,9 +66599,9 @@@
            oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
--            for (String _iter807 : struct.part_vals)
++            for (String _iter815 : struct.part_vals)
              {
--              oprot.writeString(_iter807);
++              oprot.writeString(_iter815);
              }
              oprot.writeListEnd();
            }
@@@ -65420,9 -66124,9 +66652,9 @@@
          if (struct.isSetPart_vals()) {
            {
              oprot.writeI32(struct.part_vals.size());
--            for (String _iter808 : struct.part_vals)
++            for (String _iter816 : struct.part_vals)
              {
--              oprot.writeString(_iter808);
++              oprot.writeString(_iter816);
              }
            }
          }
@@@ -65445,13 -66149,13 +66677,13 @@@
          }
          if (incoming.get(2)) {
            {
--            org.apache.thrift.protocol.TList _list809 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
--            struct.part_vals = new ArrayList<String>(_list809.size);
--            String _elem810;
--            for (int _i811 = 0; _i811 < _list809.size; ++_i811)
++            org.apache.thrift.protocol.TList _list817 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.part_vals = new ArrayList<String>(_list817.size);
++            String _elem818;
++            for (int _i819 = 0; _i819 < _list817.size; ++_i819)
              {
--              _elem810 = iprot.readString();
--              struct.part_vals.add(_elem810);
++              _elem818 = iprot.readString();
++              struct.part_vals.add(_elem818);
              }
            }
            struct.setPart_valsIsSet(true);
@@@ -69321,13 -70025,13 +70553,13 @@@
              case 3: // PART_VALS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list812 = iprot.readListBegin();
--                  struct.part_vals = new ArrayList<String>(_list812.size);
--                  String _elem813;
--                  for (int _i814 = 0; _i814 < _list812.size; ++_i814)
++                  org.apache.thrift.protocol.TList _list820 = iprot.readListBegin();
++                  struct.part_vals = new ArrayList<String>(_list820.size);
++                  String _elem821;
++                  for (int _i822 = 0; _i822 < _list820.size; ++_i822)
                    {
--                    _elem813 = iprot.readString();
--                    struct.part_vals.add(_elem813);
++                    _elem821 = iprot.readString();
++                    struct.part_vals.add(_elem821);
                    }
                    iprot.readListEnd();
                  }
@@@ -69371,9 -70075,9 +70603,9 @@@
            oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
--            for (String _iter815 : struct.part_vals)
++            for (String _iter823 : struct.part_vals)
              {
--              oprot.writeString(_iter815);
++              oprot.writeString(_iter823);
              }
              oprot.writeListEnd();
            }
@@@ -69422,9 -70126,9 +70654,9 @@@
          if (struct.isSetPart_vals()) {
            {
              oprot.writeI32(struct.part_vals.size());
--            for (String _iter816 : struct.part_vals)
++            for (String _iter824 : struct.part_vals)
              {
--              oprot.writeString(_iter816);
++              oprot.writeString(_iter824);
              }
            }
          }
@@@ -69447,13 -70151,13 +70679,13 @@@
          }
          if (incoming.get(2)) {
            {
--            org.apache.thrift.protocol.TList _list817 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
--            struct.part_vals = new ArrayList<String>(_list817.size);
--            String _elem818;
--            for (int _i819 = 0; _i819 < _list817.size; ++_i819)
++            org.apache.thrift.protocol.TList _list825 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.part_vals = new ArrayList<String>(_list825.size);
++            String _elem826;
++            for (int _i827 = 0; _i827 < _list825.size; ++_i827)
              {
--              _elem818 = iprot.readString();
--              struct.part_vals.add(_elem818);
++              _elem826 = iprot.readString();
++              struct.part_vals.add(_elem826);
              }
            }
            struct.setPart_valsIsSet(true);
@@@ -70692,13 -71396,13 +71924,13 @@@
              case 3: // PART_VALS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list820 = iprot.readListBegin();
--                  struct.part_vals = new ArrayList<String>(_list820.size);
--                  String _elem821;
--                  for (int _i822 = 0; _i822 < _list820.size; ++_i822)
++                  org.apache.thrift.protocol.TList _list828 = iprot.readListBegin();
++                  struct.part_vals = new ArrayList<String>(_list828.size);
++                  String _elem829;
++                  for (int _i830 = 0; _i830 < _list828.size; ++_i830)
                    {
--                    _elem821 = iprot.readString();
--                    struct.part_vals.add(_elem821);
++                    _elem829 = iprot.readString();
++                    struct.part_vals.add(_elem829);
                    }
                    iprot.readListEnd();
                  }
@@@ -70751,9 -71455,9 +71983,9 @@@
            oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
--            for (String _iter823 : struct.part_vals)
++            for (String _iter831 : struct.part_vals)
              {
--              oprot.writeString(_iter823);
++              oprot.writeString(_iter831);
              }
              oprot.writeListEnd();
            }
@@@ -70810,9 -71514,9 +72042,9 @@@
          if (struct.isSetPart_vals()) {
            {
              oprot.writeI32(struct.part_vals.size());
--            for (String _iter824 : struct.part_vals)
++            for (String _iter832 : struct.part_vals)
              {
--              oprot.writeString(_iter824);
++              oprot.writeString(_iter832);
              }
            }
          }
@@@ -70838,13 -71542,13 +72070,13 @@@
          }
          if (incoming.get(2)) {
            {
--            org.apache.thrift.protocol.TList _list825 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
--            struct.part_vals = new ArrayList<String>(_list825.size);
--            String _elem826;
--            for (int _i827 = 0; _i827 < _list825.size; ++_i827)
++            org.apache.thrift.protocol.TList _list833 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.part_vals = new ArrayList<String>(_list833.size);
++            String _elem834;
++            for (int _i835 = 0; _i835 < _list833.size; ++_i835)
              {
--              _elem826 = iprot.readString();
--              struct.part_vals.add(_elem826);
++              _elem834 = iprot.readString();
++              struct.part_vals.add(_elem834);
              }
            }
            struct.setPart_valsIsSet(true);
@@@ -75446,13 -76150,13 +76678,13 @@@
              case 3: // PART_VALS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list828 = iprot.readListBegin();
--                  struct.part_vals = new ArrayList<String>(_list828.size);
--                  String _elem829;
--                  for (int _i830 = 0; _i830 < _list828.size; ++_i830)
++                  org.apache.thrift.protocol.TList _list836 = iprot.readListBegin();
++                  struct.part_vals = new ArrayList<String>(_list836.size);
++                  String _elem837;
++                  for (int _i838 = 0; _i838 < _list836.size; ++_i838)
                    {
--                    _elem829 = iprot.readString();
--                    struct.part_vals.add(_elem829);
++                    _elem837 = iprot.readString();
++                    struct.part_vals.add(_elem837);
                    }
                    iprot.readListEnd();
                  }
@@@ -75488,9 -76192,9 +76720,9 @@@
            oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
--            for (String _iter831 : struct.part_vals)
++            for (String _iter839 : struct.part_vals)
              {
--              oprot.writeString(_iter831);
++              oprot.writeString(_iter839);
              }
              oprot.writeListEnd();
            }
@@@ -75533,9 -76237,9 +76765,9 @@@
          if (struct.isSetPart_vals()) {
            {
              oprot.writeI32(struct.part_vals.size());
--            for (String _iter832 : struct.part_vals)
++            for (String _iter840 : struct.part_vals)
              {
--              oprot.writeString(_iter832);
++              oprot.writeString(_iter840);
              }
            }
          }
@@@ -75555,13 -76259,13 +76787,13 @@@
          }
          if (incoming.get(2)) {
            {
--            org.apache.thrift.protocol.TList _list833 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
--            struct.part_vals = new ArrayList<String>(_list833.size);
--            String _elem834;
--            for (int _i835 = 0; _i835 < _list833.size; ++_i835)
++            org.apache.thrift.protocol.TList _list841 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.part_vals = new ArrayList<String>(_list841.size);
++            String _elem842;
++            for (int _i843 = 0; _i843 < _list841.size; ++_i843)
              {
--              _elem834 = iprot.readString();
--              struct.part_vals.add(_elem834);
++              _elem842 = iprot.readString();
++              struct.part_vals.add(_elem842);
              }
            }
            struct.setPart_valsIsSet(true);
@@@ -76779,15 -77483,15 +78011,15 @@@
              case 1: // PARTITION_SPECS
                if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                  {
--                  org.apache.thrift.protocol.TMap _map836 = iprot.readMapBegin();
--                  struct.partitionSpecs = new HashMap<String,String>(2*_map836.size);
--                  String _key837;
--                  String _val838;
--                  for (int _i839 = 0; _i839 < _map836.size; ++_i839)
++                  org.apache.thrift.protocol.TMap _map844 = iprot.readMapBegin();
++                  struct.partitionSpecs = new HashMap<String,String>(2*_map844.size);
++                  String _key845;
++                  String _val846;
++                  for (int _i847 = 0; _i847 < _map844.size; ++_i847)
                    {
--                    _key837 = iprot.readString();
--                    _val838 = iprot.readString();
--                    struct.partitionSpecs.put(_key837, _val838);
++                    _key845 = iprot.readString();
++                    _val846 = iprot.readString();
++                    struct.partitionSpecs.put(_key845, _val846);
                    }
                    iprot.readMapEnd();
                  }
@@@ -76845,10 -77549,10 +78077,10 @@@
            oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC);
            {
              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size()));
--            for (Map.Entry<String, String> _iter840 : struct.partitionSpecs.entrySet())
++            for (Map.Entry<String, String> _iter848 : struct.partitionSpecs.entrySet())
              {
--              oprot.writeString(_iter840.getKey());
--              oprot.writeString(_iter840.getValue());
++              oprot.writeString(_iter848.getKey());
++              oprot.writeString(_iter848.getValue());
              }
              oprot.writeMapEnd();
            }
@@@ -76911,10 -77615,10 +78143,10 @@@
          if (struct.isSetPartitionSpecs()) {
            {
              oprot.writeI32(struct.partitionSpecs.size());
--            for (Map.Entry<String, String> _iter841 : struct.partitionSpecs.entrySet())
++            for (Map.Entry<String, String> _iter849 : struct.partitionSpecs.entrySet())
              {
--              oprot.writeString(_iter841.getKey());
--              oprot.writeString(_iter841.getValue());
++              oprot.writeString(_iter849.getKey());
++              oprot.writeString(_iter849.getValue());
              }
            }
          }
@@@ -76938,15 -77642,15 +78170,15 @@@
          BitSet incoming = iprot.readBitSet(5);
          if (incoming.get(0)) {
            {
--            org.apache.thrift.protocol.TMap _map842 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
--            struct.partitionSpecs = new HashMap<String,String>(2*_map842.size);
--            String _key843;
--            String _val844;
--            for (int _i845 = 0; _i845 < _map842.size; ++_i845)
++            org.apache.thrift.protocol.TMap _map850 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.partitionSpecs = new HashMap<String,String>(2*_map850.size);
++            String _key851;
++            String _val852;
++            for (int _i853 = 0; _i853 < _map850.size; ++_i853)
              {
--              _key843 = iprot.readString();
--              _val844 = iprot.readString();
--              struct.partitionSpecs.put(_key843, _val844);
++              _key851 = iprot.readString();
++              _val852 = iprot.readString();
++              struct.partitionSpecs.put(_key851, _val852);
              }
            }
            struct.setPartitionSpecsIsSet(true);
@@@ -78392,15 -79096,15 +79624,15 @@@
              case 1: // PARTITION_SPECS
                if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                  {
--                  org.apache.thrift.protocol.TMap _map846 = iprot.readMapBegin();
--                  struct.partitionSpecs = new HashMap<String,String>(2*_map846.size);
--                  String _key847;
--                  String _val848;
--                  for (int _i849 = 0; _i849 < _map846.size; ++_i849)
++                  org.apache.thrift.protocol.TMap _map854 = iprot.readMapBegin();
++                  struct.partitionSpecs = new HashMap<String,String>(2*_map854.size);
++                  String _key855;
++                  String _val856;
++                  for (int _i857 = 0; _i857 < _map854.size; ++_i857)
                    {
--                    _key847 = iprot.readString();
--                    _val848 = iprot.readString();
--                    struct.partitionSpecs.put(_key847, _val848);
++                    _key855 = iprot.readString();
++                    _val856 = iprot.readString();
++                    struct.partitionSpecs.put(_key855, _val856);
                    }
                    iprot.readMapEnd();
                  }
@@@ -78458,10 -79162,10 +79690,10 @@@
            oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC);
            {
              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size()));
--            for (Map.Entry<String, String> _iter850 : struct.partitionSpecs.entrySet())
++            for (Map.Entry<String, String> _iter858 : struct.partitionSpecs.entrySet())
              {
--              oprot.writeString(_iter850.getKey());
--              oprot.writeString(_iter850.getValue());
++              oprot.writeString(_iter858.getKey());
++              oprot.writeString(_iter858.getValue());
              }
              oprot.writeMapEnd();
            }
@@@ -78524,10 -79228,10 +79756,10 @@@
          if (struct.isSetPartitionSpecs()) {
            {
              oprot.writeI32(struct.partitionSpecs.size());
--            for (Map.Entry<String, String> _iter851 : struct.partitionSpecs.entrySet())
++            for (Map.Entry<String, String> _iter859 : struct.partitionSpecs.entrySet())
              {
--              oprot.writeString(_iter851.getKey());
--              oprot.writeString(_iter851.getValue());
++              oprot.writeString(_iter859.getKey());
++              oprot.writeString(_iter859.getValue());
              }
            }
          }
@@@ -78551,15 -79255,15 +79783,15 @@@
          BitSet incoming = iprot.readBitSet(5);
          if (incoming.get(0)) {
            {
--            org.apache.thrift.protocol.TMap _map852 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
--            struct.partitionSpecs = new HashMap<String,String>(2*_map852.size);
--            String _key853;
--            String _val854;
--            for (int _i855 = 0; _i855 < _map852.size; ++_i855)
++            org.apache.thrift.protocol.TMap _map860 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.partitionSpecs = new HashMap<String,String>(2*_map860.size);
++            String _key861;
++            String _val862;
++            for (int _i863 = 0; _i863 < _map860.size; ++_i863)
              {
--              _key853 = iprot.readString();
--              _val854 = iprot.readString();
--              struct.partitionSpecs.put(_key853, _val854);
++              _key861 = iprot.readString();
++              _val862 = iprot.readString();
++              struct.partitionSpecs.put(_key861, _val862);
              }
            }
            struct.setPartitionSpecsIsSet(true);
@@@ -79224,14 -79928,14 +80456,14 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
--                  org.apache.thrift.protocol.TList _list856 = iprot.readListBegin();
--                  struct.success = new ArrayList<Partition>(_list856.size);
--                  Partition _elem857;
--                  for (int _i858 = 0; _i858 < _list856.size; ++_i858)
++                  org.apache.thrift.protocol.TList _list864 = iprot.readListBegin();
++                  struct.success = new ArrayList<Partition>(_list864.size);
++                  Partition _elem865;
++                  for (int _i866 = 0; _i866 < _list864.size; ++_i866)
                    {
--                    _elem857 = new Partition();
--                    _elem857.read(iprot);
--                    struct.success.add(_elem857);
++                    _elem865 = new Partition();
++                    _elem865.read(iprot);
++                    struct.success.add(_elem865);
                    }
                    iprot.readListEnd();
                  }
@@@ -79293,9 -79997,9 +80525,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
--            for (Partition _iter859 : struct.success)
++            for (Partition _iter867 : struct.success)
              {
--              _iter859.write(oprot);
++              _iter867.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -79358,9 -80062,9 +80590,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
--            for (Partition _iter860 : struct.success)
++            for (Partition _iter868 : struct.success)
              {
--              _iter860.write(oprot);
++              _iter868.write(oprot);
              }
            }
          }
@@@ -79384,14 -80088,14 +80616,14 @@@
          BitSet incoming = iprot.readBitSet(5);
          if (incoming.get(0)) {
            {
--            org.apache.thrift.protocol.TList _list861 = new org.apache.thrif

<TRUNCATED>

[04/15] hive git commit: HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 2d82c92..24b3ba1 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -209,6 +209,14 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function get_tables($db_name, $pattern);
   /**
+   * @param string $db_name
+   * @param string $pattern
+   * @param string $tableType
+   * @return string[]
+   * @throws \metastore\MetaException
+   */
+  public function get_tables_by_type($db_name, $pattern, $tableType);
+  /**
    * @param string $db_patterns
    * @param string $tbl_patterns
    * @param string[] $tbl_types
@@ -2598,6 +2606,62 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("get_tables failed: unknown result");
   }
 
+  public function get_tables_by_type($db_name, $pattern, $tableType)
+  {
+    $this->send_get_tables_by_type($db_name, $pattern, $tableType);
+    return $this->recv_get_tables_by_type();
+  }
+
+  public function send_get_tables_by_type($db_name, $pattern, $tableType)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_tables_by_type_args();
+    $args->db_name = $db_name;
+    $args->pattern = $pattern;
+    $args->tableType = $tableType;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_tables_by_type', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_tables_by_type', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_tables_by_type()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_tables_by_type_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_get_tables_by_type_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    throw new \Exception("get_tables_by_type failed: unknown result");
+  }
+
   public function get_table_meta($db_patterns, $tbl_patterns, array $tbl_types)
   {
     $this->send_get_table_meta($db_patterns, $tbl_patterns, $tbl_types);
@@ -15313,6 +15377,253 @@ class ThriftHiveMetastore_get_tables_result {
 
 }
 
+class ThriftHiveMetastore_get_tables_by_type_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $db_name = null;
+  /**
+   * @var string
+   */
+  public $pattern = null;
+  /**
+   * @var string
+   */
+  public $tableType = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'db_name',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'pattern',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'tableType',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['db_name'])) {
+        $this->db_name = $vals['db_name'];
+      }
+      if (isset($vals['pattern'])) {
+        $this->pattern = $vals['pattern'];
+      }
+      if (isset($vals['tableType'])) {
+        $this->tableType = $vals['tableType'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_tables_by_type_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->db_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->pattern);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tableType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_tables_by_type_args');
+    if ($this->db_name !== null) {
+      $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1);
+      $xfer += $output->writeString($this->db_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->pattern !== null) {
+      $xfer += $output->writeFieldBegin('pattern', TType::STRING, 2);
+      $xfer += $output->writeString($this->pattern);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tableType !== null) {
+      $xfer += $output->writeFieldBegin('tableType', TType::STRING, 3);
+      $xfer += $output->writeString($this->tableType);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_tables_by_type_result {
+  static $_TSPEC;
+
+  /**
+   * @var string[]
+   */
+  public $success = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o1 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_tables_by_type_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::LST) {
+            $this->success = array();
+            $_size641 = 0;
+            $_etype644 = 0;
+            $xfer += $input->readListBegin($_etype644, $_size641);
+            for ($_i645 = 0; $_i645 < $_size641; ++$_i645)
+            {
+              $elem646 = null;
+              $xfer += $input->readString($elem646);
+              $this->success []= $elem646;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\MetaException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_tables_by_type_result');
+    if ($this->success !== null) {
+      if (!is_array($this->success)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('success', TType::LST, 0);
+      {
+        $output->writeListBegin(TType::STRING, count($this->success));
+        {
+          foreach ($this->success as $iter647)
+          {
+            $xfer += $output->writeString($iter647);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class ThriftHiveMetastore_get_table_meta_args {
   static $_TSPEC;
 
@@ -15399,14 +15710,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size641 = 0;
-            $_etype644 = 0;
-            $xfer += $input->readListBegin($_etype644, $_size641);
-            for ($_i645 = 0; $_i645 < $_size641; ++$_i645)
+            $_size648 = 0;
+            $_etype651 = 0;
+            $xfer += $input->readListBegin($_etype651, $_size648);
+            for ($_i652 = 0; $_i652 < $_size648; ++$_i652)
             {
-              $elem646 = null;
-              $xfer += $input->readString($elem646);
-              $this->tbl_types []= $elem646;
+              $elem653 = null;
+              $xfer += $input->readString($elem653);
+              $this->tbl_types []= $elem653;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15444,9 +15755,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter647)
+          foreach ($this->tbl_types as $iter654)
           {
-            $xfer += $output->writeString($iter647);
+            $xfer += $output->writeString($iter654);
           }
         }
         $output->writeListEnd();
@@ -15523,15 +15834,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size648 = 0;
-            $_etype651 = 0;
-            $xfer += $input->readListBegin($_etype651, $_size648);
-            for ($_i652 = 0; $_i652 < $_size648; ++$_i652)
+            $_size655 = 0;
+            $_etype658 = 0;
+            $xfer += $input->readListBegin($_etype658, $_size655);
+            for ($_i659 = 0; $_i659 < $_size655; ++$_i659)
             {
-              $elem653 = null;
-              $elem653 = new \metastore\TableMeta();
-              $xfer += $elem653->read($input);
-              $this->success []= $elem653;
+              $elem660 = null;
+              $elem660 = new \metastore\TableMeta();
+              $xfer += $elem660->read($input);
+              $this->success []= $elem660;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15567,9 +15878,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter654)
+          foreach ($this->success as $iter661)
           {
-            $xfer += $iter654->write($output);
+            $xfer += $iter661->write($output);
           }
         }
         $output->writeListEnd();
@@ -15725,14 +16036,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size655 = 0;
-            $_etype658 = 0;
-            $xfer += $input->readListBegin($_etype658, $_size655);
-            for ($_i659 = 0; $_i659 < $_size655; ++$_i659)
+            $_size662 = 0;
+            $_etype665 = 0;
+            $xfer += $input->readListBegin($_etype665, $_size662);
+            for ($_i666 = 0; $_i666 < $_size662; ++$_i666)
             {
-              $elem660 = null;
-              $xfer += $input->readString($elem660);
-              $this->success []= $elem660;
+              $elem667 = null;
+              $xfer += $input->readString($elem667);
+              $this->success []= $elem667;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15768,9 +16079,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter661)
+          foreach ($this->success as $iter668)
           {
-            $xfer += $output->writeString($iter661);
+            $xfer += $output->writeString($iter668);
           }
         }
         $output->writeListEnd();
@@ -16085,14 +16396,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size662 = 0;
-            $_etype665 = 0;
-            $xfer += $input->readListBegin($_etype665, $_size662);
-            for ($_i666 = 0; $_i666 < $_size662; ++$_i666)
+            $_size669 = 0;
+            $_etype672 = 0;
+            $xfer += $input->readListBegin($_etype672, $_size669);
+            for ($_i673 = 0; $_i673 < $_size669; ++$_i673)
             {
-              $elem667 = null;
-              $xfer += $input->readString($elem667);
-              $this->tbl_names []= $elem667;
+              $elem674 = null;
+              $xfer += $input->readString($elem674);
+              $this->tbl_names []= $elem674;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16125,9 +16436,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter668)
+          foreach ($this->tbl_names as $iter675)
           {
-            $xfer += $output->writeString($iter668);
+            $xfer += $output->writeString($iter675);
           }
         }
         $output->writeListEnd();
@@ -16228,15 +16539,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size669 = 0;
-            $_etype672 = 0;
-            $xfer += $input->readListBegin($_etype672, $_size669);
-            for ($_i673 = 0; $_i673 < $_size669; ++$_i673)
+            $_size676 = 0;
+            $_etype679 = 0;
+            $xfer += $input->readListBegin($_etype679, $_size676);
+            for ($_i680 = 0; $_i680 < $_size676; ++$_i680)
             {
-              $elem674 = null;
-              $elem674 = new \metastore\Table();
-              $xfer += $elem674->read($input);
-              $this->success []= $elem674;
+              $elem681 = null;
+              $elem681 = new \metastore\Table();
+              $xfer += $elem681->read($input);
+              $this->success []= $elem681;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16288,9 +16599,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter675)
+          foreach ($this->success as $iter682)
           {
-            $xfer += $iter675->write($output);
+            $xfer += $iter682->write($output);
           }
         }
         $output->writeListEnd();
@@ -16526,14 +16837,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size676 = 0;
-            $_etype679 = 0;
-            $xfer += $input->readListBegin($_etype679, $_size676);
-            for ($_i680 = 0; $_i680 < $_size676; ++$_i680)
+            $_size683 = 0;
+            $_etype686 = 0;
+            $xfer += $input->readListBegin($_etype686, $_size683);
+            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
             {
-              $elem681 = null;
-              $xfer += $input->readString($elem681);
-              $this->success []= $elem681;
+              $elem688 = null;
+              $xfer += $input->readString($elem688);
+              $this->success []= $elem688;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16585,9 +16896,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter682)
+          foreach ($this->success as $iter689)
           {
-            $xfer += $output->writeString($iter682);
+            $xfer += $output->writeString($iter689);
           }
         }
         $output->writeListEnd();
@@ -17900,15 +18211,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size683 = 0;
-            $_etype686 = 0;
-            $xfer += $input->readListBegin($_etype686, $_size683);
-            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
+            $_size690 = 0;
+            $_etype693 = 0;
+            $xfer += $input->readListBegin($_etype693, $_size690);
+            for ($_i694 = 0; $_i694 < $_size690; ++$_i694)
             {
-              $elem688 = null;
-              $elem688 = new \metastore\Partition();
-              $xfer += $elem688->read($input);
-              $this->new_parts []= $elem688;
+              $elem695 = null;
+              $elem695 = new \metastore\Partition();
+              $xfer += $elem695->read($input);
+              $this->new_parts []= $elem695;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17936,9 +18247,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter689)
+          foreach ($this->new_parts as $iter696)
           {
-            $xfer += $iter689->write($output);
+            $xfer += $iter696->write($output);
           }
         }
         $output->writeListEnd();
@@ -18153,15 +18464,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size690 = 0;
-            $_etype693 = 0;
-            $xfer += $input->readListBegin($_etype693, $_size690);
-            for ($_i694 = 0; $_i694 < $_size690; ++$_i694)
+            $_size697 = 0;
+            $_etype700 = 0;
+            $xfer += $input->readListBegin($_etype700, $_size697);
+            for ($_i701 = 0; $_i701 < $_size697; ++$_i701)
             {
-              $elem695 = null;
-              $elem695 = new \metastore\PartitionSpec();
-              $xfer += $elem695->read($input);
-              $this->new_parts []= $elem695;
+              $elem702 = null;
+              $elem702 = new \metastore\PartitionSpec();
+              $xfer += $elem702->read($input);
+              $this->new_parts []= $elem702;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18189,9 +18500,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter696)
+          foreach ($this->new_parts as $iter703)
           {
-            $xfer += $iter696->write($output);
+            $xfer += $iter703->write($output);
           }
         }
         $output->writeListEnd();
@@ -18441,14 +18752,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size697 = 0;
-            $_etype700 = 0;
-            $xfer += $input->readListBegin($_etype700, $_size697);
-            for ($_i701 = 0; $_i701 < $_size697; ++$_i701)
+            $_size704 = 0;
+            $_etype707 = 0;
+            $xfer += $input->readListBegin($_etype707, $_size704);
+            for ($_i708 = 0; $_i708 < $_size704; ++$_i708)
             {
-              $elem702 = null;
-              $xfer += $input->readString($elem702);
-              $this->part_vals []= $elem702;
+              $elem709 = null;
+              $xfer += $input->readString($elem709);
+              $this->part_vals []= $elem709;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18486,9 +18797,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter703)
+          foreach ($this->part_vals as $iter710)
           {
-            $xfer += $output->writeString($iter703);
+            $xfer += $output->writeString($iter710);
           }
         }
         $output->writeListEnd();
@@ -18990,14 +19301,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size704 = 0;
-            $_etype707 = 0;
-            $xfer += $input->readListBegin($_etype707, $_size704);
-            for ($_i708 = 0; $_i708 < $_size704; ++$_i708)
+            $_size711 = 0;
+            $_etype714 = 0;
+            $xfer += $input->readListBegin($_etype714, $_size711);
+            for ($_i715 = 0; $_i715 < $_size711; ++$_i715)
             {
-              $elem709 = null;
-              $xfer += $input->readString($elem709);
-              $this->part_vals []= $elem709;
+              $elem716 = null;
+              $xfer += $input->readString($elem716);
+              $this->part_vals []= $elem716;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19043,9 +19354,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter710)
+          foreach ($this->part_vals as $iter717)
           {
-            $xfer += $output->writeString($iter710);
+            $xfer += $output->writeString($iter717);
           }
         }
         $output->writeListEnd();
@@ -19899,14 +20210,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size711 = 0;
-            $_etype714 = 0;
-            $xfer += $input->readListBegin($_etype714, $_size711);
-            for ($_i715 = 0; $_i715 < $_size711; ++$_i715)
+            $_size718 = 0;
+            $_etype721 = 0;
+            $xfer += $input->readListBegin($_etype721, $_size718);
+            for ($_i722 = 0; $_i722 < $_size718; ++$_i722)
             {
-              $elem716 = null;
-              $xfer += $input->readString($elem716);
-              $this->part_vals []= $elem716;
+              $elem723 = null;
+              $xfer += $input->readString($elem723);
+              $this->part_vals []= $elem723;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19951,9 +20262,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter717)
+          foreach ($this->part_vals as $iter724)
           {
-            $xfer += $output->writeString($iter717);
+            $xfer += $output->writeString($iter724);
           }
         }
         $output->writeListEnd();
@@ -20206,14 +20517,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size718 = 0;
-            $_etype721 = 0;
-            $xfer += $input->readListBegin($_etype721, $_size718);
-            for ($_i722 = 0; $_i722 < $_size718; ++$_i722)
+            $_size725 = 0;
+            $_etype728 = 0;
+            $xfer += $input->readListBegin($_etype728, $_size725);
+            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
             {
-              $elem723 = null;
-              $xfer += $input->readString($elem723);
-              $this->part_vals []= $elem723;
+              $elem730 = null;
+              $xfer += $input->readString($elem730);
+              $this->part_vals []= $elem730;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20266,9 +20577,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter724)
+          foreach ($this->part_vals as $iter731)
           {
-            $xfer += $output->writeString($iter724);
+            $xfer += $output->writeString($iter731);
           }
         }
         $output->writeListEnd();
@@ -21282,14 +21593,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size725 = 0;
-            $_etype728 = 0;
-            $xfer += $input->readListBegin($_etype728, $_size725);
-            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
+            $_size732 = 0;
+            $_etype735 = 0;
+            $xfer += $input->readListBegin($_etype735, $_size732);
+            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
             {
-              $elem730 = null;
-              $xfer += $input->readString($elem730);
-              $this->part_vals []= $elem730;
+              $elem737 = null;
+              $xfer += $input->readString($elem737);
+              $this->part_vals []= $elem737;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21327,9 +21638,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter731)
+          foreach ($this->part_vals as $iter738)
           {
-            $xfer += $output->writeString($iter731);
+            $xfer += $output->writeString($iter738);
           }
         }
         $output->writeListEnd();
@@ -21571,17 +21882,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size732 = 0;
-            $_ktype733 = 0;
-            $_vtype734 = 0;
-            $xfer += $input->readMapBegin($_ktype733, $_vtype734, $_size732);
-            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
+            $_size739 = 0;
+            $_ktype740 = 0;
+            $_vtype741 = 0;
+            $xfer += $input->readMapBegin($_ktype740, $_vtype741, $_size739);
+            for ($_i743 = 0; $_i743 < $_size739; ++$_i743)
             {
-              $key737 = '';
-              $val738 = '';
-              $xfer += $input->readString($key737);
-              $xfer += $input->readString($val738);
-              $this->partitionSpecs[$key737] = $val738;
+              $key744 = '';
+              $val745 = '';
+              $xfer += $input->readString($key744);
+              $xfer += $input->readString($val745);
+              $this->partitionSpecs[$key744] = $val745;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -21637,10 +21948,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter739 => $viter740)
+          foreach ($this->partitionSpecs as $kiter746 => $viter747)
           {
-            $xfer += $output->writeString($kiter739);
-            $xfer += $output->writeString($viter740);
+            $xfer += $output->writeString($kiter746);
+            $xfer += $output->writeString($viter747);
           }
         }
         $output->writeMapEnd();
@@ -21952,17 +22263,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size741 = 0;
-            $_ktype742 = 0;
-            $_vtype743 = 0;
-            $xfer += $input->readMapBegin($_ktype742, $_vtype743, $_size741);
-            for ($_i745 = 0; $_i745 < $_size741; ++$_i745)
+            $_size748 = 0;
+            $_ktype749 = 0;
+            $_vtype750 = 0;
+            $xfer += $input->readMapBegin($_ktype749, $_vtype750, $_size748);
+            for ($_i752 = 0; $_i752 < $_size748; ++$_i752)
             {
-              $key746 = '';
-              $val747 = '';
-              $xfer += $input->readString($key746);
-              $xfer += $input->readString($val747);
-              $this->partitionSpecs[$key746] = $val747;
+              $key753 = '';
+              $val754 = '';
+              $xfer += $input->readString($key753);
+              $xfer += $input->readString($val754);
+              $this->partitionSpecs[$key753] = $val754;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -22018,10 +22329,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter748 => $viter749)
+          foreach ($this->partitionSpecs as $kiter755 => $viter756)
           {
-            $xfer += $output->writeString($kiter748);
-            $xfer += $output->writeString($viter749);
+            $xfer += $output->writeString($kiter755);
+            $xfer += $output->writeString($viter756);
           }
         }
         $output->writeMapEnd();
@@ -22154,15 +22465,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size750 = 0;
-            $_etype753 = 0;
-            $xfer += $input->readListBegin($_etype753, $_size750);
-            for ($_i754 = 0; $_i754 < $_size750; ++$_i754)
+            $_size757 = 0;
+            $_etype760 = 0;
+            $xfer += $input->readListBegin($_etype760, $_size757);
+            for ($_i761 = 0; $_i761 < $_size757; ++$_i761)
             {
-              $elem755 = null;
-              $elem755 = new \metastore\Partition();
-              $xfer += $elem755->read($input);
-              $this->success []= $elem755;
+              $elem762 = null;
+              $elem762 = new \metastore\Partition();
+              $xfer += $elem762->read($input);
+              $this->success []= $elem762;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22222,9 +22533,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter756)
+          foreach ($this->success as $iter763)
           {
-            $xfer += $iter756->write($output);
+            $xfer += $iter763->write($output);
           }
         }
         $output->writeListEnd();
@@ -22370,14 +22681,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size757 = 0;
-            $_etype760 = 0;
-            $xfer += $input->readListBegin($_etype760, $_size757);
-            for ($_i761 = 0; $_i761 < $_size757; ++$_i761)
+            $_size764 = 0;
+            $_etype767 = 0;
+            $xfer += $input->readListBegin($_etype767, $_size764);
+            for ($_i768 = 0; $_i768 < $_size764; ++$_i768)
             {
-              $elem762 = null;
-              $xfer += $input->readString($elem762);
-              $this->part_vals []= $elem762;
+              $elem769 = null;
+              $xfer += $input->readString($elem769);
+              $this->part_vals []= $elem769;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22394,14 +22705,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size763 = 0;
-            $_etype766 = 0;
-            $xfer += $input->readListBegin($_etype766, $_size763);
-            for ($_i767 = 0; $_i767 < $_size763; ++$_i767)
+            $_size770 = 0;
+            $_etype773 = 0;
+            $xfer += $input->readListBegin($_etype773, $_size770);
+            for ($_i774 = 0; $_i774 < $_size770; ++$_i774)
             {
-              $elem768 = null;
-              $xfer += $input->readString($elem768);
-              $this->group_names []= $elem768;
+              $elem775 = null;
+              $xfer += $input->readString($elem775);
+              $this->group_names []= $elem775;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22439,9 +22750,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter769)
+          foreach ($this->part_vals as $iter776)
           {
-            $xfer += $output->writeString($iter769);
+            $xfer += $output->writeString($iter776);
           }
         }
         $output->writeListEnd();
@@ -22461,9 +22772,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter770)
+          foreach ($this->group_names as $iter777)
           {
-            $xfer += $output->writeString($iter770);
+            $xfer += $output->writeString($iter777);
           }
         }
         $output->writeListEnd();
@@ -23054,15 +23365,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size771 = 0;
-            $_etype774 = 0;
-            $xfer += $input->readListBegin($_etype774, $_size771);
-            for ($_i775 = 0; $_i775 < $_size771; ++$_i775)
+            $_size778 = 0;
+            $_etype781 = 0;
+            $xfer += $input->readListBegin($_etype781, $_size778);
+            for ($_i782 = 0; $_i782 < $_size778; ++$_i782)
             {
-              $elem776 = null;
-              $elem776 = new \metastore\Partition();
-              $xfer += $elem776->read($input);
-              $this->success []= $elem776;
+              $elem783 = null;
+              $elem783 = new \metastore\Partition();
+              $xfer += $elem783->read($input);
+              $this->success []= $elem783;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23106,9 +23417,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter777)
+          foreach ($this->success as $iter784)
           {
-            $xfer += $iter777->write($output);
+            $xfer += $iter784->write($output);
           }
         }
         $output->writeListEnd();
@@ -23254,14 +23565,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size778 = 0;
-            $_etype781 = 0;
-            $xfer += $input->readListBegin($_etype781, $_size778);
-            for ($_i782 = 0; $_i782 < $_size778; ++$_i782)
+            $_size785 = 0;
+            $_etype788 = 0;
+            $xfer += $input->readListBegin($_etype788, $_size785);
+            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
             {
-              $elem783 = null;
-              $xfer += $input->readString($elem783);
-              $this->group_names []= $elem783;
+              $elem790 = null;
+              $xfer += $input->readString($elem790);
+              $this->group_names []= $elem790;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23309,9 +23620,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter784)
+          foreach ($this->group_names as $iter791)
           {
-            $xfer += $output->writeString($iter784);
+            $xfer += $output->writeString($iter791);
           }
         }
         $output->writeListEnd();
@@ -23400,15 +23711,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size785 = 0;
-            $_etype788 = 0;
-            $xfer += $input->readListBegin($_etype788, $_size785);
-            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
+            $_size792 = 0;
+            $_etype795 = 0;
+            $xfer += $input->readListBegin($_etype795, $_size792);
+            for ($_i796 = 0; $_i796 < $_size792; ++$_i796)
             {
-              $elem790 = null;
-              $elem790 = new \metastore\Partition();
-              $xfer += $elem790->read($input);
-              $this->success []= $elem790;
+              $elem797 = null;
+              $elem797 = new \metastore\Partition();
+              $xfer += $elem797->read($input);
+              $this->success []= $elem797;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23452,9 +23763,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter791)
+          foreach ($this->success as $iter798)
           {
-            $xfer += $iter791->write($output);
+            $xfer += $iter798->write($output);
           }
         }
         $output->writeListEnd();
@@ -23674,15 +23985,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size792 = 0;
-            $_etype795 = 0;
-            $xfer += $input->readListBegin($_etype795, $_size792);
-            for ($_i796 = 0; $_i796 < $_size792; ++$_i796)
+            $_size799 = 0;
+            $_etype802 = 0;
+            $xfer += $input->readListBegin($_etype802, $_size799);
+            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
             {
-              $elem797 = null;
-              $elem797 = new \metastore\PartitionSpec();
-              $xfer += $elem797->read($input);
-              $this->success []= $elem797;
+              $elem804 = null;
+              $elem804 = new \metastore\PartitionSpec();
+              $xfer += $elem804->read($input);
+              $this->success []= $elem804;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23726,9 +24037,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter798)
+          foreach ($this->success as $iter805)
           {
-            $xfer += $iter798->write($output);
+            $xfer += $iter805->write($output);
           }
         }
         $output->writeListEnd();
@@ -23935,14 +24246,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size799 = 0;
-            $_etype802 = 0;
-            $xfer += $input->readListBegin($_etype802, $_size799);
-            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
+            $_size806 = 0;
+            $_etype809 = 0;
+            $xfer += $input->readListBegin($_etype809, $_size806);
+            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
             {
-              $elem804 = null;
-              $xfer += $input->readString($elem804);
-              $this->success []= $elem804;
+              $elem811 = null;
+              $xfer += $input->readString($elem811);
+              $this->success []= $elem811;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23978,9 +24289,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter805)
+          foreach ($this->success as $iter812)
           {
-            $xfer += $output->writeString($iter805);
+            $xfer += $output->writeString($iter812);
           }
         }
         $output->writeListEnd();
@@ -24096,14 +24407,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size806 = 0;
-            $_etype809 = 0;
-            $xfer += $input->readListBegin($_etype809, $_size806);
-            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
+            $_size813 = 0;
+            $_etype816 = 0;
+            $xfer += $input->readListBegin($_etype816, $_size813);
+            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
             {
-              $elem811 = null;
-              $xfer += $input->readString($elem811);
-              $this->part_vals []= $elem811;
+              $elem818 = null;
+              $xfer += $input->readString($elem818);
+              $this->part_vals []= $elem818;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24148,9 +24459,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter812)
+          foreach ($this->part_vals as $iter819)
           {
-            $xfer += $output->writeString($iter812);
+            $xfer += $output->writeString($iter819);
           }
         }
         $output->writeListEnd();
@@ -24244,15 +24555,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size813 = 0;
-            $_etype816 = 0;
-            $xfer += $input->readListBegin($_etype816, $_size813);
-            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
+            $_size820 = 0;
+            $_etype823 = 0;
+            $xfer += $input->readListBegin($_etype823, $_size820);
+            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
             {
-              $elem818 = null;
-              $elem818 = new \metastore\Partition();
-              $xfer += $elem818->read($input);
-              $this->success []= $elem818;
+              $elem825 = null;
+              $elem825 = new \metastore\Partition();
+              $xfer += $elem825->read($input);
+              $this->success []= $elem825;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24296,9 +24607,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter819)
+          foreach ($this->success as $iter826)
           {
-            $xfer += $iter819->write($output);
+            $xfer += $iter826->write($output);
           }
         }
         $output->writeListEnd();
@@ -24445,14 +24756,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size820 = 0;
-            $_etype823 = 0;
-            $xfer += $input->readListBegin($_etype823, $_size820);
-            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
+            $_size827 = 0;
+            $_etype830 = 0;
+            $xfer += $input->readListBegin($_etype830, $_size827);
+            for ($_i831 = 0; $_i831 < $_size827; ++$_i831)
             {
-              $elem825 = null;
-              $xfer += $input->readString($elem825);
-              $this->part_vals []= $elem825;
+              $elem832 = null;
+              $xfer += $input->readString($elem832);
+              $this->part_vals []= $elem832;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24476,14 +24787,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size826 = 0;
-            $_etype829 = 0;
-            $xfer += $input->readListBegin($_etype829, $_size826);
-            for ($_i830 = 0; $_i830 < $_size826; ++$_i830)
+            $_size833 = 0;
+            $_etype836 = 0;
+            $xfer += $input->readListBegin($_etype836, $_size833);
+            for ($_i837 = 0; $_i837 < $_size833; ++$_i837)
             {
-              $elem831 = null;
-              $xfer += $input->readString($elem831);
-              $this->group_names []= $elem831;
+              $elem838 = null;
+              $xfer += $input->readString($elem838);
+              $this->group_names []= $elem838;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24521,9 +24832,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter832)
+          foreach ($this->part_vals as $iter839)
           {
-            $xfer += $output->writeString($iter832);
+            $xfer += $output->writeString($iter839);
           }
         }
         $output->writeListEnd();
@@ -24548,9 +24859,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter833)
+          foreach ($this->group_names as $iter840)
           {
-            $xfer += $output->writeString($iter833);
+            $xfer += $output->writeString($iter840);
           }
         }
         $output->writeListEnd();
@@ -24639,15 +24950,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size834 = 0;
-            $_etype837 = 0;
-            $xfer += $input->readListBegin($_etype837, $_size834);
-            for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
+            $_size841 = 0;
+            $_etype844 = 0;
+            $xfer += $input->readListBegin($_etype844, $_size841);
+            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
             {
-              $elem839 = null;
-              $elem839 = new \metastore\Partition();
-              $xfer += $elem839->read($input);
-              $this->success []= $elem839;
+              $elem846 = null;
+              $elem846 = new \metastore\Partition();
+              $xfer += $elem846->read($input);
+              $this->success []= $elem846;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24691,9 +25002,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter840)
+          foreach ($this->success as $iter847)
           {
-            $xfer += $iter840->write($output);
+            $xfer += $iter847->write($output);
           }
         }
         $output->writeListEnd();
@@ -24814,14 +25125,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size841 = 0;
-            $_etype844 = 0;
-            $xfer += $input->readListBegin($_etype844, $_size841);
-            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
+            $_size848 = 0;
+            $_etype851 = 0;
+            $xfer += $input->readListBegin($_etype851, $_size848);
+            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
             {
-              $elem846 = null;
-              $xfer += $input->readString($elem846);
-              $this->part_vals []= $elem846;
+              $elem853 = null;
+              $xfer += $input->readString($elem853);
+              $this->part_vals []= $elem853;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24866,9 +25177,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter847)
+          foreach ($this->part_vals as $iter854)
           {
-            $xfer += $output->writeString($iter847);
+            $xfer += $output->writeString($iter854);
           }
         }
         $output->writeListEnd();
@@ -24961,14 +25272,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size848 = 0;
-            $_etype851 = 0;
-            $xfer += $input->readListBegin($_etype851, $_size848);
-            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
+            $_size855 = 0;
+            $_etype858 = 0;
+            $xfer += $input->readListBegin($_etype858, $_size855);
+            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
             {
-              $elem853 = null;
-              $xfer += $input->readString($elem853);
-              $this->success []= $elem853;
+              $elem860 = null;
+              $xfer += $input->readString($elem860);
+              $this->success []= $elem860;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25012,9 +25323,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter854)
+          foreach ($this->success as $iter861)
           {
-            $xfer += $output->writeString($iter854);
+            $xfer += $output->writeString($iter861);
           }
         }
         $output->writeListEnd();
@@ -25257,15 +25568,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size855 = 0;
-            $_etype858 = 0;
-            $xfer += $input->readListBegin($_etype858, $_size855);
-            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
+            $_size862 = 0;
+            $_etype865 = 0;
+            $xfer += $input->readListBegin($_etype865, $_size862);
+            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
             {
-              $elem860 = null;
-              $elem860 = new \metastore\Partition();
-              $xfer += $elem860->read($input);
-              $this->success []= $elem860;
+              $elem867 = null;
+              $elem867 = new \metastore\Partition();
+              $xfer += $elem867->read($input);
+              $this->success []= $elem867;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25309,9 +25620,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter861)
+          foreach ($this->success as $iter868)
           {
-            $xfer += $iter861->write($output);
+            $xfer += $iter868->write($output);
           }
         }
         $output->writeListEnd();
@@ -25554,15 +25865,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size862 = 0;
-            $_etype865 = 0;
-            $xfer += $input->readListBegin($_etype865, $_size862);
-            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
+            $_size869 = 0;
+            $_etype872 = 0;
+            $xfer += $input->readListBegin($_etype872, $_size869);
+            for ($_i873 = 0; $_i873 < $_size869; ++$_i873)
             {
-              $elem867 = null;
-              $elem867 = new \metastore\PartitionSpec();
-              $xfer += $elem867->read($input);
-              $this->success []= $elem867;
+              $elem874 = null;
+              $elem874 = new \metastore\PartitionSpec();
+              $xfer += $elem874->read($input);
+              $this->success []= $elem874;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25606,9 +25917,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter868)
+          foreach ($this->success as $iter875)
           {
-            $xfer += $iter868->write($output);
+            $xfer += $iter875->write($output);
           }
         }
         $output->writeListEnd();
@@ -26174,14 +26485,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size869 = 0;
-            $_etype872 = 0;
-            $xfer += $input->readListBegin($_etype872, $_size869);
-            for ($_i873 = 0; $_i873 < $_size869; ++$_i873)
+            $_size876 = 0;
+            $_etype879 = 0;
+            $xfer += $input->readListBegin($_etype879, $_size876);
+            for ($_i880 = 0; $_i880 < $_size876; ++$_i880)
             {
-              $elem874 = null;
-              $xfer += $input->readString($elem874);
-              $this->names []= $elem874;
+              $elem881 = null;
+              $xfer += $input->readString($elem881);
+              $this->names []= $elem881;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26219,9 +26530,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter875)
+          foreach ($this->names as $iter882)
           {
-            $xfer += $output->writeString($iter875);
+            $xfer += $output->writeString($iter882);
           }
         }
         $output->writeListEnd();
@@ -26310,15 +26621,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size876 = 0;
-            $_etype879 = 0;
-            $xfer += $input->readListBegin($_etype879, $_size876);
-            for ($_i880 = 0; $_i880 < $_size876; ++$_i880)
+            $_size883 = 0;
+            $_etype886 = 0;
+            $xfer += $input->readListBegin($_etype886, $_size883);
+            for ($_i887 = 0; $_i887 < $_size883; ++$_i887)
             {
-              $elem881 = null;
-              $elem881 = new \metastore\Partition();
-              $xfer += $elem881->read($input);
-              $this->success []= $elem881;
+              $elem888 = null;
+              $elem888 = new \metastore\Partition();
+              $xfer += $elem888->read($input);
+              $this->success []= $elem888;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26362,9 +26673,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter882)
+          foreach ($this->success as $iter889)
           {
-            $xfer += $iter882->write($output);
+            $xfer += $iter889->write($output);
           }
         }
         $output->writeListEnd();
@@ -26703,15 +27014,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size883 = 0;
-            $_etype886 = 0;
-            $xfer += $input->readListBegin($_etype886, $_size883);
-            for ($_i887 = 0; $_i887 < $_size883; ++$_i887)
+            $_size890 = 0;
+            $_etype893 = 0;
+            $xfer += $input->readListBegin($_etype893, $_size890);
+            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
             {
-              $elem888 = null;
-              $elem888 = new \metastore\Partition();
-              $xfer += $elem888->read($input);
-              $this->new_parts []= $elem888;
+              $elem895 = null;
+              $elem895 = new \metastore\Partition();
+              $xfer += $elem895->read($input);
+              $this->new_parts []= $elem895;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26749,9 +27060,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter889)
+          foreach ($this->new_parts as $iter896)
           {
-            $xfer += $iter889->write($output);
+            $xfer += $iter896->write($output);
           }
         }
         $output->writeListEnd();
@@ -26966,15 +27277,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size890 = 0;
-            $_etype893 = 0;
-            $xfer += $input->readListBegin($_etype893, $_size890);
-            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
+            $_size897 = 0;
+            $_etype900 = 0;
+            $xfer += $input->readListBegin($_etype900, $_size897);
+            for ($_i901 = 0; $_i901 < $_size897; ++$_i901)
             {
-              $elem895 = null;
-              $elem895 = new \metastore\Partition();
-              $xfer += $elem895->read($input);
-              $this->new_parts []= $elem895;
+              $elem902 = null;
+              $elem902 = new \metastore\Partition();
+              $xfer += $elem902->read($input);
+              $this->new_parts []= $elem902;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27020,9 +27331,9 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter896)
+          foreach ($this->new_parts as $iter903)
           {
-            $xfer += $iter896->write($output);
+            $xfer += $iter903->write($output);
           }
         }
         $output->writeListEnd();
@@ -27500,14 +27811,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size897 = 0;
-            $_etype900 = 0;
-            $xfer += $input->readListBegin($_etype900, $_size897);
-            for ($_i901 = 0; $_i901 < $_size897; ++$_i901)
+            $_size904 = 0;
+            $_etype907 = 0;
+            $xfer += $input->readListBegin($_etype907, $_size904);
+            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
             {
-              $elem902 = null;
-              $xfer += $input->readString($elem902);
-              $this->part_vals []= $elem902;
+              $elem909 = null;
+              $xfer += $input->readString($elem909);
+              $this->part_vals []= $elem909;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27553,9 +27864,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter903)
+          foreach ($this->part_vals as $iter910)
           {
-            $xfer += $output->writeString($iter903);
+            $xfer += $output->writeString($iter910);
           }
         }
         $output->writeListEnd();
@@ -27740,14 +28051,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size904 = 0;
-            $_etype907 = 0;
-            $xfer += $input->readListBegin($_etype907, $_size904);
-            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
+            $_size911 = 0;
+            $_etype914 = 0;
+            $xfer += $input->readListBegin($_etype914, $_size911);
+            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
             {
-              $elem909 = null;
-              $xfer += $input->readString($elem909);
-              $this->part_vals []= $elem909;
+              $elem916 = null;
+              $xfer += $input->readString($elem916);
+              $this->part_vals []= $elem916;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27782,9 +28093,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter910)
+          foreach ($this->part_vals as $iter917)
           {
-            $xfer += $output->writeString($iter910);
+            $xfer += $output->writeString($iter917);
           }
         }
         $output->writeListEnd();
@@ -28238,14 +28549,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size911 = 0;
-            $_etype914 = 0;
-            $xfer += $input->readListBegin($_etype914, $_size911);
-            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
+            $_size918 = 0;
+            $_etype921 = 0;
+            $xfer += $input->readListBegin($_etype921, $_size918);
+            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
             {
-              $elem916 = null;
-              $xfer += $input->readString($elem916);
-              $this->success []= $elem916;
+              $elem923 = null;
+              $xfer += $input->readString($elem923);
+              $this->success []= $elem923;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28281,9 +28592,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter917)
+          foreach ($this->success as $iter924)
           {
-            $xfer += $output->writeString($iter917);
+            $xfer += $output->writeString($iter924);
           }
         }
         $output->writeListEnd();
@@ -28443,17 +28754,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size918 = 0;
-            $_ktype919 = 0;
-            $_vtype920 = 0;
-            $xfer += $input->readMapBegin($_ktype919, $_vtype920, $_size918);
-            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
+            $_size925 = 0;
+            $_ktype926 = 0;
+            $_vtype927 = 0;
+            $xfer += $input->readMapBegin($_ktype926, $_vtype927, $_size925);
+            for ($_i929 = 0; $_i929 < $_size925; ++$_i929)
             {
-              $key923 = '';
-              $val924 = '';
-              $xfer += $input->readString($key923);
-              $xfer += $input->readString($val924);
-              $this->success[$key923] = $val924;
+              $key930 = '';
+              $val931 = '';
+              $xfer += $input->readString($key930);
+              $xfer += $input->readString($val931);
+              $this->success[$key930] = $val931;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -28489,10 +28800,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter925 => $viter926)
+          foreach ($this->success as $kiter932 => $viter933)
           {
-            $xfer += $output->writeString($kiter925);
-            $xfer += $output->writeString($viter926);
+            $xfer += $output->writeString($kiter932);
+            $xfer += $output->writeString($viter933);
           }
         }
         $output->writeMapEnd();
@@ -28612,17 +28923,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size927 = 0;
-            $_ktype928 = 0;
-            $_vtype929 = 0;
-            $xfer += $input->readMapBegin($_ktype928, $_vtype929, $_size927);
-            for ($_i931 = 0; $_i931 < $_size927; ++$_i931)
+            $_size934 = 0;
+            $_ktype935 = 0;
+            $_vtype936 = 0;
+            $xfer += $input->readMapBegin($_ktype935, $_vtype936, $_size934);
+            for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
             {
-              $key932 = '';
-              $val933 = '';
-              $xfer += $input->readString($key932);
-              $xfer += $input->readString($val933);
-              $this->part_vals[$key932] = $val933;
+              $key939 = '';
+              $val940 = '';
+              $xfer += $input->readString($key939);
+              $xfer += $input->readString($val940);
+              $this->part_vals[$key939] = $val940;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -28667,10 +28978,10 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter934 => $viter935)
+          foreach ($this->part_vals as $kiter941 => $viter942)
           {
-            $xfer += $output->writeString($kiter934);
-            $xfer += $output->writeString($viter935);
+            $xfer += $output->writeString($kiter941);
+            $xfer += $output->writeString($viter942);
           }
         }
         $output->writeMapEnd();
@@ -28992,17 +29303,17 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size936 = 0;
-            $_ktype937 = 0;
-            $_vtype938 = 0;
-            $xfer += $input->readMapBegin($_ktype937, $_vtype938, $_size936);
-            for ($_i940 = 0; $_i940 < $_size936; ++$_i940)
+            $_size943 = 0;
+            $_ktype944 = 0;
+            $_vtype945 = 0;
+            $xfer += $input->readMapBegin($_ktype944, $_vtype945, $_size943);
+            for ($_i947 = 0; $_i947 < $_size943; ++$_i947)
             {
-              $key941 = '';
-              $val942 = '';
-              $xfer += $input->readString($key941);
-              $xfer += $input->readString($val942);
-              $this->part_vals[$key941] = $val942;
+              $key948 = '';
+              $val949 = '';
+              $xfer += $input->readString($key948);
+              $xfer += $input->readString($val949);
+              $this->part_vals[$key948] = $val949;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -29047,10 +29358,10 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter943 => $viter944)
+          foreach ($this->part_vals as $kiter950 => $viter951)
           {
-            $xfer += $output->writeString($kiter943);
-            $xfer += $output->writeString($viter944);
+            $xfer += $output->writeString($kiter950);
+            $xfer += $output->writeString($viter951);
           }
         }
         $output->writeMapEnd();
@@ -30524,15 +30835,15 @@ class ThriftHiveMetastore_get_indexes_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size945 = 0;
-            $_etype948 = 0;
-            $xfer += $input->readListBegin($_etype948, $_size945);
-            for ($_i949 = 0; $_i949 < $_size945; ++$_i949)
+            $_size952 = 0;
+            $_etype955 = 0;
+            $xfer += $input->readListBegin($_etype955, $_size952);
+            for ($_i956 = 0; $_i956 < $_size952; ++$_i956)
             {
-              $elem950 = null;
-              $elem950 = new \metastore\Index();
-              $xfer += $elem950->read($input);
-              $this->success []= $elem950;
+              $elem957 = null;
+              $elem957 = new \metastore\Index();
+              $xfer += $elem957->read($input);
+              $this->success []= $elem957;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30576,9 +30887,9 @@ class ThriftHiveMetastore_get_indexes_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter951)
+          foreach ($this->success as $iter958)
           {
-            $xfer += $iter951->write($output);
+            $xfer += $iter958->write($output);
           }
         }
         $output->writeListEnd();
@@ -30785,14 +31096,14 @@ class ThriftHiveMetastore_get_index_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size952 = 0;
-            $_etype955 = 0;
-            $xfer += $input->readListBegin($_etype955, $_size952);
-            for ($_i956 = 0; $_i956 < $_size952; ++$_i956)
+            $_size959 = 0;
+            $_etype962 = 0;
+            $xfer += $input->readListBegin($_etype962, $_size959);
+            for ($_i963 = 0; $_i963 < $_size959; ++$_i963)
             {
-              $elem957 = null;
-              $xfer += $input->readString($elem957);
-              $this->success []= $elem957;
+              $elem964 = null;
+              $xfer += $input->readString($elem964);
+              $this->success []= $elem964;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30828,9 +31139,9 @@ class ThriftHiveMetastore_get_index_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter958)
+          foreach ($this->success as $iter965)
           {
-            $xfer += $output->writeString($iter958);
+            $xfer += $output->writeString($iter965);
           }
         }
         $output->writeListEnd();
@@ -34724,14 +35035,14 @@ class ThriftHiveMetastore_get_functions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size959 = 0;
-            $_etype962 = 0;
-            $xfer += $input->readListBegin($_etype962, $_size959);
-            for ($_i963 = 0; $_i963 < $_size959; ++$_i963)
+            $_size966 = 0;
+            $_etype969 = 0;
+            $xfer += $input->readListBegin($_etype969, $_size966);
+            for ($_i970 = 0; $_i970 < $_size966; ++$_i970)
             {
-              $elem964 = null;
-              $xfer += $input->readString($elem964);
-              $this->success []= $elem964;
+              $elem971 = null;
+              $xfer += $input->readString($elem971);
+              $this->success []= $elem971;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34767,9 +35078,9 @@ class ThriftHiveMetastore_get_functions_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter965)
+          foreach ($this->success as $iter972)
           {
-            $xfer += $output->writeString($iter965);
+            $xfer += $output->writeString($iter972);
           }
         }
         $output->writeListEnd();
@@ -35638,14 +35949,14 @@ class ThriftHiveMetastore_get_role_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size966 = 0;
-            $_etype969 = 0;
-            $xfer += $input->readListBegin($_etype969, $_size966);
-            for ($_i970 = 0; $_i970 < $_size966; ++$_i970)
+            $_size973 = 0;
+            $_etype976 = 0;
+            $xfer += $input->readListBegin($_etype976, $_size973);
+            for ($_i977 = 0; $_i977 < $_size973; ++$_i977)
             {
-              $elem971 = null;
-              $xfer += $input->readString($elem971);
-              $this->success []= $elem971;
+              $elem978 = null;
+              $xfer += $input->readString($elem978);
+              $this->success []= $elem978;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35681,9 +35992,9 @@ class ThriftHiveMetastore_get_role_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter972)
+          foreach ($this->success as $iter979)
           {
-            $xfer += $output->writeString($iter972);
+            $xfer += $output->writeString($iter979);
           }
         }
         $output->writeListEnd();
@@ -36374,15 +36685,15 @@ class ThriftHiveMetastore_list_roles_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size973 = 0;
-            $_etype976 = 0;
-            $xfer += $input->readListBegin($_etype976, $_size973);
-            for ($_i977 = 0; $_i977 < $_size973; ++$_i977)
+            $_size980 = 0;
+            $_etype983 = 0;
+            $xfer += $input->readListBegin($_etype983, $_size980);
+            for ($_i984 = 0; $_i984 < $_size980; ++$_i984)
             {
-              $elem978 = null;
-              $elem978 = new \metastore\Role();
-              $xfer += $elem978->read($input);
-              $this->success []= $elem978;
+              $elem985 = null;
+              $elem985 = new \metastore\Role();
+              $xfer += $elem985->read($input);
+              $this->success []= $elem985;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36418,9 +36729,9 @@ class ThriftHiveMetastore_list_roles_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter979)
+          foreach ($this->success as $iter986)
           {
-            $xfer += $iter979->write($output);
+            $xfer += $iter986->write($output);
           }
         }
         $output->writeListEnd();
@@ -37082,14 +37393,14 @@ class ThriftHiveMetastore_get_privilege_set_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size980 = 0;
-            $_etype983 = 0;
-            $xfer += $input->readListBegin($_etype983, $_size980);
-            for ($_i984 = 0; $_i984 < $_size980; ++$_i984)
+            $_size987 = 0;
+            $_etype990 = 0;
+            $xfer += $input->readListBegin($_etype990, $_size987);
+            for ($_i991 = 0; $_i991 < $_size987; ++$_i991)
             {
-              $elem985 = null;
-              $xfer += $input->readString($elem985);
-              $this->group_names []= $elem985;
+              $elem992 = null;
+              $xfer += $input->readString($elem992);
+              $this->group_names []= $elem992;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -37130,9 +37441,9 @@ class ThriftHiveMetastore_get_privilege_set_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter986)
+          foreach ($this->group_names as $iter993)
           {
-            $xfer += $output->writeString($iter986);
+            $xfer += $output->writeString($iter993);
           }
         }
         $output->writeListEnd();
@@ -37440,15 +37751,15 @@ class ThriftHiveMetastore_list_privileges_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size987 = 0;
-            $_etype990 = 0;
-            $xfer += $input->readListBegin($_etype990, $_size987);
-            for ($_i991 = 0; $_i991 < $_size987; ++$_i991)
+            $_size994 = 0;
+            $_etype997 = 0;
+            $xfer += $input->readListBegin($_etype997, $_size994);
+            for ($_i998 = 0; $_i998 < $_size994; ++$_i998)
             {
-              $elem992 = null;
-              $elem992 = new \metastore\HiveObjectPrivilege();
-              $xfer += $elem992->read($input);
-              $this->success []= $elem992;
+              $elem999 = null;
+              $elem999 = new \metastore\HiveObjectPrivilege();
+              $xfer += $elem999->read($input);
+              $this->success []= $elem999;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -37484,9 +37795,9 @@ class ThriftHiveMetastore_list_privileges_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter993)
+          foreach ($this->success as $iter1000)
           {
-            $xfer += $iter993->write($output);
+            $xfer += $iter1000->write($output);
           }
         }
         $output->writeListEnd();
@@ -38118,14 +38429,14 @@ class ThriftHiveMetastore_set_ugi_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size994 = 0;
-            $_etype997 = 0;
-            $xfer += $input->readListBegin($_etype997, $_size994);
-            for ($_i998 = 0; $_i998 < $_size994; ++$_i998)
+            $_size1001 = 0;
+            $_etype1004 = 0;
+            $xfer += $input->readListBegin($_etype1004, $_size1001);
+            for ($_i1005 = 0; $_i1005 < $_size1001; ++$_i1005)
             {
-              $elem999 = null;
-              $xfer += $input->readString($elem999);
-              $this->group_names []= $elem999;
+              $elem1006 = null;
+              $xfer += $input->readString($elem1006);
+              $this->group_names []= $elem1006;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -38158,9 +38469,9 @@ class ThriftHiveMetastore_set_ugi_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1000)
+          foreach ($this->group_names as $iter1007)
           {
-            $xfer += $output->writeString($iter1000);
+            $xfer += $output->writeString($iter1007);
           }
         }
         $output->writeListEnd();
@@ -38236,14 +38547,14 @@ class ThriftHiveMetastore_set_ugi_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1001 = 0;
-            $_etype1004 = 0;
-            $xfer += $input->readListBegin($_etype1004, $_size1001);
-            for ($_i1005 = 0; $_i1005 < $_size1001; ++$_i1005)
+            $_size1008 = 0;
+            $_etype1011 = 0;
+            $xfer += $input->readListBegin($_etype1011, $_size1008);
+            for ($_i1012 = 0; $_i1012 < $_size1008; ++$_i1012)
             {
-              $elem1006 = null;
-              $xfer += $input->readString($elem1006);
-              $this->success []= $elem1006;
+              $elem1013 = null;
+              $xfer += $input->readString($elem1013);
+              $this->success []= $elem1013;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -38279,9 +38590,9 @@ class ThriftHiveMetastore_set_ugi_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1007)
+          foreach ($this->success as $iter1014)
           {
-            $xfer += $output->writeString($iter1007);
+            $xfer += $output->writeString($iter1014);
           }
         }
         $output->writeListEnd();
@@ -39398,14 +39709,14 @@ class ThriftHiveMetastore_get_all_token_identifiers_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1008 = 0;
-            $_etype1011 = 0;
-            $xfer += $input->readListBegin($_etype1011, $_size1008);
-            for ($_i1012 = 0; $_i1012 < $_size1008; ++$_i1012)
+            $_size1015 = 0;
+            $_etype1018 = 0;
+            $xfer += $input->readListBegin($_etype1018, $_size1015);
+            for ($_i1019 = 0; $_i1019 < $_size1015; ++$_i1019)
             {
-              $elem1013 = null;
-              $xfer += $input->readString($elem1013);
-              $this->success []= $elem1013;
+              $elem1020 = null;
+              $xfer += $input->readString($elem1020);
+              $this->success []= $elem1020;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -39433,9 +39744,9 @@ class ThriftHiveMetastore_get_all_token_identifiers_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1014)
+          foreach ($this->success as $iter1021)
           {
-            $xfer += $output->writeString($iter1014);
+            $xfer += $output->writeString($iter1021);
           }
         }
         $output->writeListEnd();
@@ -40074,14 +40385,14 @@ class ThriftHiveMetastore_get_master_keys_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1015 = 0;
-            $_etype1018 = 0;
-            $xfer += $input->readListBegin($_etype1018, $_size1015);
-            for ($_i1019 = 0; $_i1019 < $_size1015; ++$_i1019)
+            $_size1022 = 0;
+            $_etype1025 = 0;
+            $xfer += $input->readListBegin($_etype1025, $_size1022);
+            for ($_i1026 = 0; $_i1026 < $_size1022; ++$_i1026)
             {
-              $elem1020 = null;
-              $xfer += $input->readString($elem1020);
-              $this->success []= $elem1020;
+              $elem1027 = null;
+              $xfer += $input->readString($elem1027);
+              $this->success []= $elem1027;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -40109,9 +40420,9 @@ class ThriftHiveMetastore_get_master_keys_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1021)
+          foreach ($this->success as $iter1028)
           {
-            $xfer += $output->writeString($iter1021);
+            $xfer += $output->writeString($iter1028);
           }
         }
         $output->writeListEnd();

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index c58ccf2..8f505f1 100755
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -49,6 +49,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  void drop_table(string dbname, string name, bool deleteData)')
   print('  void drop_table_with_environment_context(string dbname, string name, bool deleteData, EnvironmentContext environment_context)')
   print('   get_tables(string db_name, string pattern)')
+  print('   get_tables_by_type(string db_name, string pattern, string tableType)')
   print('   get_table_meta(string db_patterns, string tbl_patterns,  tbl_types)')
   print('   get_all_tables(string db_name)')
   print('  Table get_table(string dbname, string tbl_name)')
@@ -392,6 +393,12 @@ elif cmd == 'get_tables':
     sys.exit(1)
   pp.pprint(client.get_tables(args[0],args[1],))
 
+elif cmd == 'get_tables_by_type':
+  if len(args) != 3:
+    print('get_tables_by_type requires 3 args')
+    sys.exit(1)
+  pp.pprint(client.get_tables_by_type(args[0],args[1],args[2],))
+
 elif cmd == 'get_table_meta':
   if len(args) != 3:
     print('get_table_meta requires 3 args')


[02/15] hive git commit: HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 de6adb5..691c3a8 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
@@ -1313,7 +1313,7 @@ public class Hive {
    * @throws HiveException
    */
   public List<String> getAllTables() throws HiveException {
-    return getAllTables(SessionState.get().getCurrentDatabase());
+    return getTablesByType(SessionState.get().getCurrentDatabase(), null, null);
   }
 
   /**
@@ -1323,7 +1323,7 @@ public class Hive {
    * @throws HiveException
    */
   public List<String> getAllTables(String dbName) throws HiveException {
-    return getTablesByPattern(dbName, ".*");
+    return getTablesByType(dbName, ".*", null);
   }
 
   /**
@@ -1336,8 +1336,8 @@ public class Hive {
    * @throws HiveException
    */
   public List<String> getTablesByPattern(String tablePattern) throws HiveException {
-    return getTablesByPattern(SessionState.get().getCurrentDatabase(),
-        tablePattern);
+    return getTablesByType(SessionState.get().getCurrentDatabase(),
+        tablePattern, null);
   }
 
   /**
@@ -1349,11 +1349,7 @@ public class Hive {
    * @throws HiveException
    */
   public List<String> getTablesByPattern(String dbName, String tablePattern) throws HiveException {
-    try {
-      return getMSC().getTables(dbName, tablePattern);
-    } catch (Exception e) {
-      throw new HiveException(e);
-    }
+    return getTablesByType(dbName, tablePattern, null);
   }
 
   /**
@@ -1369,8 +1365,38 @@ public class Hive {
    */
   public List<String> getTablesForDb(String database, String tablePattern)
       throws HiveException {
+    return getTablesByType(database, tablePattern, null);
+  }
+
+  /**
+   * Returns all existing tables of a type (VIRTUAL_VIEW|EXTERNAL_TABLE|MANAGED_TABLE) from the specified
+   * database which match the given pattern. The matching occurs as per Java regular expressions.
+   * @param dbName Database name to find the tables in. if null, uses the current database in this session.
+   * @param pattern A pattern to match for the table names.If null, returns all names from this DB.
+   * @param type The type of tables to return. VIRTUAL_VIEWS for views. If null, returns all tables and views.
+   * @return list of table names that match the pattern.
+   * @throws HiveException
+   */
+  public List<String> getTablesByType(String dbName, String pattern, TableType type)
+      throws HiveException {
+    List<String> retList = new ArrayList<String>();
+    if (dbName == null)
+      dbName = SessionState.get().getCurrentDatabase();
+
     try {
-      return getMSC().getTables(database, tablePattern);
+      if (type != null) {
+        if (pattern != null) {
+          return getMSC().getTables(dbName, pattern, type);
+        } else {
+          return getMSC().getTables(dbName, ".*", type);
+        }
+      } else {
+        if (pattern != null) {
+          return getMSC().getTables(dbName, pattern);
+        } else {
+          return getMSC().getTables(dbName, ".*");
+        }
+      }
     } catch (Exception e) {
       throw new HiveException(e);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/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 8b0db4a..a264c4d 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
@@ -394,6 +394,10 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       ctx.setResFile(ctx.getLocalTmpPath());
       analyzeShowConf(ast);
       break;
+    case HiveParser.TOK_SHOWVIEWS:
+      ctx.setResFile(ctx.getLocalTmpPath());
+      analyzeShowViews(ast);
+      break;
     case HiveParser.TOK_DESCFUNCTION:
       ctx.setResFile(ctx.getLocalTmpPath());
       analyzeDescFunction(ast);
@@ -2402,6 +2406,45 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     setFetchTask(createFetchTask(showConfDesc.getSchema()));
   }
 
+  private void analyzeShowViews(ASTNode ast) throws SemanticException {
+    ShowTablesDesc showViewsDesc;
+    String dbName = SessionState.get().getCurrentDatabase();
+    String viewNames = null;
+
+    if (ast.getChildCount() > 3) {
+      throw new SemanticException(ErrorMsg.GENERIC_ERROR.getMsg());
+    }
+
+    switch (ast.getChildCount()) {
+    case 1: // Uses a pattern
+      viewNames = unescapeSQLString(ast.getChild(0).getText());
+      showViewsDesc = new ShowTablesDesc(ctx.getResFile(), dbName, viewNames, TableType.VIRTUAL_VIEW);
+      break;
+    case 2: // Specifies a DB
+      assert (ast.getChild(0).getType() == HiveParser.TOK_FROM);
+      dbName = unescapeIdentifier(ast.getChild(1).getText());
+      validateDatabase(dbName);
+      showViewsDesc = new ShowTablesDesc(ctx.getResFile(), dbName);
+      showViewsDesc.setType(TableType.VIRTUAL_VIEW);
+      break;
+    case 3: // Uses a pattern and specifies a DB
+      assert (ast.getChild(0).getType() == HiveParser.TOK_FROM);
+      dbName = unescapeIdentifier(ast.getChild(1).getText());
+      viewNames = unescapeSQLString(ast.getChild(2).getText());
+      validateDatabase(dbName);
+      showViewsDesc = new ShowTablesDesc(ctx.getResFile(), dbName, viewNames, TableType.VIRTUAL_VIEW);
+      break;
+    default: // No pattern or DB
+      showViewsDesc = new ShowTablesDesc(ctx.getResFile(), dbName);
+      showViewsDesc.setType(TableType.VIRTUAL_VIEW);
+      break;
+    }
+
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
+        showViewsDesc), conf));
+    setFetchTask(createFetchTask(showViewsDesc.getSchema()));
+  }
+
   /**
    * Add the task according to the parsed command tree. This is used for the CLI
    * command "LOCK TABLE ..;".

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index b623187..58e76c8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -216,6 +216,7 @@ KW_MINUS: 'MINUS';
 KW_FETCH: 'FETCH';
 KW_INTERSECT: 'INTERSECT';
 KW_VIEW: 'VIEW';
+KW_VIEWS: 'VIEWS';
 KW_IN: 'IN';
 KW_DATABASE: 'DATABASE';
 KW_DATABASES: 'DATABASES';

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 62bbcc6..bf78545 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -368,6 +368,7 @@ TOK_ANONYMOUS;
 TOK_COL_NAME;
 TOK_URI_TYPE;
 TOK_SERVER_TYPE;
+TOK_SHOWVIEWS;
 TOK_START_TRANSACTION;
 TOK_ISOLATION_LEVEL;
 TOK_ISOLATION_SNAPSHOT;
@@ -1443,6 +1444,7 @@ showStatement
 @after { popMsg(state); }
     : KW_SHOW (KW_DATABASES|KW_SCHEMAS) (KW_LIKE showStmtIdentifier)? -> ^(TOK_SHOWDATABASES showStmtIdentifier?)
     | KW_SHOW KW_TABLES ((KW_FROM|KW_IN) db_name=identifier)? (KW_LIKE showStmtIdentifier|showStmtIdentifier)?  -> ^(TOK_SHOWTABLES (TOK_FROM $db_name)? showStmtIdentifier?)
+    | KW_SHOW KW_VIEWS ((KW_FROM|KW_IN) db_name=identifier)? (KW_LIKE showStmtIdentifier|showStmtIdentifier)?  -> ^(TOK_SHOWVIEWS (TOK_FROM $db_name)? showStmtIdentifier?)
     | KW_SHOW KW_COLUMNS (KW_FROM|KW_IN) tableName ((KW_FROM|KW_IN) db_name=identifier)?
     -> ^(TOK_SHOWCOLUMNS tableName $db_name?)
     | KW_SHOW KW_FUNCTIONS (KW_LIKE showFunctionIdentifier|showFunctionIdentifier)?  -> ^(TOK_SHOWFUNCTIONS KW_LIKE? showFunctionIdentifier?)

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
index 18439f9..4f0ead0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
@@ -78,6 +78,7 @@ public final class SemanticAnalyzerFactory {
     commandType.put(HiveParser.TOK_SHOWLOCKS, HiveOperation.SHOWLOCKS);
     commandType.put(HiveParser.TOK_SHOWDBLOCKS, HiveOperation.SHOWLOCKS);
     commandType.put(HiveParser.TOK_SHOWCONF, HiveOperation.SHOWCONF);
+    commandType.put(HiveParser.TOK_SHOWVIEWS, HiveOperation.SHOWVIEWS);
     commandType.put(HiveParser.TOK_CREATEFUNCTION, HiveOperation.CREATEFUNCTION);
     commandType.put(HiveParser.TOK_DROPFUNCTION, HiveOperation.DROPFUNCTION);
     commandType.put(HiveParser.TOK_RELOADFUNCTION, HiveOperation.RELOADFUNCTION);
@@ -256,6 +257,7 @@ public final class SemanticAnalyzerFactory {
       case HiveParser.TOK_SHOW_TRANSACTIONS:
       case HiveParser.TOK_ABORT_TRANSACTIONS:
       case HiveParser.TOK_SHOWCONF:
+      case HiveParser.TOK_SHOWVIEWS:
       case HiveParser.TOK_CREATEINDEX:
       case HiveParser.TOK_DROPINDEX:
       case HiveParser.TOK_ALTERTABLE_CLUSTER_SORT:

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
index 8b54609..e069acd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
@@ -524,7 +524,7 @@ public class DDLWork implements Serializable {
     this.cacheMetadataDesc = cacheMetadataDesc;
   }
 
-    /**
+  /**
    * @return Create Database descriptor
    */
   public CreateDatabaseDesc getCreateDatabaseDesc() {

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
index 6fd4df4..d333f91 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
@@ -72,6 +72,7 @@ public enum HiveOperation {
   SHOWPARTITIONS("SHOWPARTITIONS", null, null),
   SHOWLOCKS("SHOWLOCKS", null, null),
   SHOWCONF("SHOWCONF", null, null),
+  SHOWVIEWS("SHOWVIEWS", null, null),
   CREATEFUNCTION("CREATEFUNCTION", null, null),
   DROPFUNCTION("DROPFUNCTION", null, null),
   RELOADFUNCTION("RELOADFUNCTION", null, null),

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTablesDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTablesDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTablesDesc.java
index 9e9d639..c9193b7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTablesDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTablesDesc.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.plan;
 import java.io.Serializable;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
 
 
@@ -34,6 +35,7 @@ public class ShowTablesDesc extends DDLDesc implements Serializable {
   String pattern;
   String dbName;
   String resFile;
+  TableType type;
   /**
    * table name for the result of show tables.
    */
@@ -82,6 +84,17 @@ public class ShowTablesDesc extends DDLDesc implements Serializable {
   }
 
   /**
+   * @param type
+   *          type of the tables to show
+   */
+  public ShowTablesDesc(Path resFile, String dbName, String pattern, TableType type) {
+    this.resFile = resFile.toString();
+    this.dbName = dbName;
+    this.pattern = pattern;
+    this.type    = type;
+  }
+
+  /**
    * @return the pattern
    */
   @Explain(displayName = "pattern")
@@ -98,6 +111,22 @@ public class ShowTablesDesc extends DDLDesc implements Serializable {
   }
 
   /**
+   * @return the table type to be fetched
+   */
+  @Explain(displayName = "type")
+  public TableType getType() {
+    return type;
+  }
+
+  /**
+   * @param type
+   *          the table type to set
+   */
+  public void setType(TableType type) {
+    this.type = type;
+  }
+
+  /**
    * @return the resFile
    */
   @Explain(displayName = "result file", explainLevels = { Level.EXTENDED })

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
index 2da4249..7da44e8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
@@ -76,6 +76,7 @@ public enum HiveOperationType {
   SHOWPARTITIONS,
   SHOWLOCKS,
   SHOWCONF,
+  SHOWVIEWS,
   CREATEFUNCTION,
   DROPFUNCTION,
   RELOADFUNCTION,

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
index e5389bd..e7bbd54 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
@@ -360,6 +360,8 @@ public class Operation2Privilege {
 (null, null));
     op2Priv.put(HiveOperationType.SHOWCONF, PrivRequirement.newIOPrivRequirement
 (null, null));
+    op2Priv.put(HiveOperationType.SHOWVIEWS, PrivRequirement.newIOPrivRequirement
+(null, null));
 
     op2Priv.put(HiveOperationType.LOCKTABLE, PrivRequirement.newIOPrivRequirement
 (null, null));

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/test/queries/clientpositive/show_views.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/show_views.q b/ql/src/test/queries/clientpositive/show_views.q
new file mode 100644
index 0000000..61aad43
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/show_views.q
@@ -0,0 +1,56 @@
+CREATE DATABASE test1;
+CREATE DATABASE test2;
+
+USE test1;
+CREATE TABLE shtb_test1(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE;
+CREATE VIEW shtb_test1_view1 AS SELECT * FROM shtb_test1 where KEY > 1000 and KEY < 2000;
+CREATE VIEW shtb_test1_view2 AS SELECT * FROM shtb_test1 where KEY > 100 and KEY < 200;
+CREATE VIEW shtb_full_view2 AS SELECT * FROM shtb_test1;
+USE test2;
+CREATE TABLE shtb_test1(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE;
+CREATE TABLE shtb_test2(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE;
+CREATE VIEW shtb_test1_view1 AS SELECT * FROM shtb_test1 where KEY > 1000 and KEY < 2000;
+CREATE VIEW shtb_test2_view2 AS SELECT * FROM shtb_test2 where KEY > 100 and KEY < 200;
+
+USE test1;
+SHOW VIEWS;
+SHOW VIEWS 'test_*';
+SHOW VIEWS '*view2';
+SHOW VIEWS LIKE 'test_view1|test_view2';
+
+USE test2;
+SHOW VIEWS 'shtb_*';
+
+-- SHOW VIEWS basic syntax tests
+USE default;
+SHOW VIEWS FROM test1;
+SHOW VIEWS FROM test2;
+SHOW VIEWS IN test1;
+SHOW VIEWS IN default;
+SHOW VIEWS IN test1 "shtb_test_*";
+SHOW VIEWS IN test2 LIKE "nomatch";
+
+-- SHOW VIEWS from a database with a name that requires escaping
+CREATE DATABASE `database`;
+USE `database`;
+CREATE TABLE foo(a INT);
+CREATE VIEW fooview AS SELECT * FROM foo;
+USE default;
+SHOW VIEWS FROM `database` LIKE "fooview";
+
+DROP VIEW fooview;
+DROP TABLE foo;
+
+USE test1;
+DROP VIEW shtb_test1_view1;
+DROP VIEW shtb_test1_view2;
+DROP VIEW shtb_full_view2;
+DROP TABLE shtb_test1;
+DROP DATABASE test1;
+
+USE test2;
+DROP VIEW shtb_test1_view1;
+DROP VIEW shtb_test2_view2;
+DROP TABLE shtb_test1;
+DROP TABLE shtb_test2;
+DROP DATABASE test2;

http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/ql/src/test/results/clientpositive/show_views.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/show_views.q.out b/ql/src/test/results/clientpositive/show_views.q.out
new file mode 100644
index 0000000..61b5f1d
--- /dev/null
+++ b/ql/src/test/results/clientpositive/show_views.q.out
@@ -0,0 +1,320 @@
+PREHOOK: query: CREATE DATABASE test1
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:test1
+POSTHOOK: query: CREATE DATABASE test1
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:test1
+PREHOOK: query: CREATE DATABASE test2
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:test2
+POSTHOOK: query: CREATE DATABASE test2
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:test2
+PREHOOK: query: USE test1
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:test1
+POSTHOOK: query: USE test1
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:test1
+PREHOOK: query: CREATE TABLE shtb_test1(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:test1
+PREHOOK: Output: test1@shtb_test1
+POSTHOOK: query: CREATE TABLE shtb_test1(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:test1
+POSTHOOK: Output: test1@shtb_test1
+PREHOOK: query: CREATE VIEW shtb_test1_view1 AS SELECT * FROM shtb_test1 where KEY > 1000 and KEY < 2000
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: test1@shtb_test1
+PREHOOK: Output: database:test1
+PREHOOK: Output: test1@shtb_test1_view1
+POSTHOOK: query: CREATE VIEW shtb_test1_view1 AS SELECT * FROM shtb_test1 where KEY > 1000 and KEY < 2000
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: test1@shtb_test1
+POSTHOOK: Output: database:test1
+POSTHOOK: Output: test1@shtb_test1_view1
+PREHOOK: query: CREATE VIEW shtb_test1_view2 AS SELECT * FROM shtb_test1 where KEY > 100 and KEY < 200
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: test1@shtb_test1
+PREHOOK: Output: database:test1
+PREHOOK: Output: test1@shtb_test1_view2
+POSTHOOK: query: CREATE VIEW shtb_test1_view2 AS SELECT * FROM shtb_test1 where KEY > 100 and KEY < 200
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: test1@shtb_test1
+POSTHOOK: Output: database:test1
+POSTHOOK: Output: test1@shtb_test1_view2
+PREHOOK: query: CREATE VIEW shtb_full_view2 AS SELECT * FROM shtb_test1
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: test1@shtb_test1
+PREHOOK: Output: database:test1
+PREHOOK: Output: test1@shtb_full_view2
+POSTHOOK: query: CREATE VIEW shtb_full_view2 AS SELECT * FROM shtb_test1
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: test1@shtb_test1
+POSTHOOK: Output: database:test1
+POSTHOOK: Output: test1@shtb_full_view2
+PREHOOK: query: USE test2
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:test2
+POSTHOOK: query: USE test2
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:test2
+PREHOOK: query: CREATE TABLE shtb_test1(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:test2
+PREHOOK: Output: test2@shtb_test1
+POSTHOOK: query: CREATE TABLE shtb_test1(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:test2
+POSTHOOK: Output: test2@shtb_test1
+PREHOOK: query: CREATE TABLE shtb_test2(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:test2
+PREHOOK: Output: test2@shtb_test2
+POSTHOOK: query: CREATE TABLE shtb_test2(KEY INT, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:test2
+POSTHOOK: Output: test2@shtb_test2
+PREHOOK: query: CREATE VIEW shtb_test1_view1 AS SELECT * FROM shtb_test1 where KEY > 1000 and KEY < 2000
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: test2@shtb_test1
+PREHOOK: Output: database:test2
+PREHOOK: Output: test2@shtb_test1_view1
+POSTHOOK: query: CREATE VIEW shtb_test1_view1 AS SELECT * FROM shtb_test1 where KEY > 1000 and KEY < 2000
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: test2@shtb_test1
+POSTHOOK: Output: database:test2
+POSTHOOK: Output: test2@shtb_test1_view1
+PREHOOK: query: CREATE VIEW shtb_test2_view2 AS SELECT * FROM shtb_test2 where KEY > 100 and KEY < 200
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: test2@shtb_test2
+PREHOOK: Output: database:test2
+PREHOOK: Output: test2@shtb_test2_view2
+POSTHOOK: query: CREATE VIEW shtb_test2_view2 AS SELECT * FROM shtb_test2 where KEY > 100 and KEY < 200
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: test2@shtb_test2
+POSTHOOK: Output: database:test2
+POSTHOOK: Output: test2@shtb_test2_view2
+PREHOOK: query: USE test1
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:test1
+POSTHOOK: query: USE test1
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:test1
+PREHOOK: query: SHOW VIEWS
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS
+POSTHOOK: type: SHOWVIEWS
+shtb_full_view2
+shtb_test1_view1
+shtb_test1_view2
+PREHOOK: query: SHOW VIEWS 'test_*'
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS 'test_*'
+POSTHOOK: type: SHOWVIEWS
+PREHOOK: query: SHOW VIEWS '*view2'
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS '*view2'
+POSTHOOK: type: SHOWVIEWS
+shtb_full_view2
+shtb_test1_view2
+PREHOOK: query: SHOW VIEWS LIKE 'test_view1|test_view2'
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS LIKE 'test_view1|test_view2'
+POSTHOOK: type: SHOWVIEWS
+PREHOOK: query: USE test2
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:test2
+POSTHOOK: query: USE test2
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:test2
+PREHOOK: query: SHOW VIEWS 'shtb_*'
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS 'shtb_*'
+POSTHOOK: type: SHOWVIEWS
+shtb_test1_view1
+shtb_test2_view2
+PREHOOK: query: -- SHOW VIEWS basic syntax tests
+USE default
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:default
+POSTHOOK: query: -- SHOW VIEWS basic syntax tests
+USE default
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:default
+PREHOOK: query: SHOW VIEWS FROM test1
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS FROM test1
+POSTHOOK: type: SHOWVIEWS
+shtb_full_view2
+shtb_test1_view1
+shtb_test1_view2
+PREHOOK: query: SHOW VIEWS FROM test2
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS FROM test2
+POSTHOOK: type: SHOWVIEWS
+shtb_test1_view1
+shtb_test2_view2
+PREHOOK: query: SHOW VIEWS IN test1
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS IN test1
+POSTHOOK: type: SHOWVIEWS
+shtb_full_view2
+shtb_test1_view1
+shtb_test1_view2
+PREHOOK: query: SHOW VIEWS IN default
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS IN default
+POSTHOOK: type: SHOWVIEWS
+PREHOOK: query: SHOW VIEWS IN test1 "shtb_test_*"
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS IN test1 "shtb_test_*"
+POSTHOOK: type: SHOWVIEWS
+PREHOOK: query: SHOW VIEWS IN test2 LIKE "nomatch"
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS IN test2 LIKE "nomatch"
+POSTHOOK: type: SHOWVIEWS
+PREHOOK: query: -- SHOW VIEWS from a database with a name that requires escaping
+CREATE DATABASE `database`
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:database
+POSTHOOK: query: -- SHOW VIEWS from a database with a name that requires escaping
+CREATE DATABASE `database`
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:database
+PREHOOK: query: USE `database`
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:database
+POSTHOOK: query: USE `database`
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:database
+PREHOOK: query: CREATE TABLE foo(a INT)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:database
+PREHOOK: Output: database@foo
+POSTHOOK: query: CREATE TABLE foo(a INT)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:database
+POSTHOOK: Output: database@foo
+PREHOOK: query: CREATE VIEW fooview AS SELECT * FROM foo
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: database@foo
+PREHOOK: Output: database:database
+PREHOOK: Output: database@fooview
+POSTHOOK: query: CREATE VIEW fooview AS SELECT * FROM foo
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: database@foo
+POSTHOOK: Output: database:database
+POSTHOOK: Output: database@fooview
+PREHOOK: query: USE default
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:default
+POSTHOOK: query: USE default
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:default
+PREHOOK: query: SHOW VIEWS FROM `database` LIKE "fooview"
+PREHOOK: type: SHOWVIEWS
+POSTHOOK: query: SHOW VIEWS FROM `database` LIKE "fooview"
+POSTHOOK: type: SHOWVIEWS
+fooview
+PREHOOK: query: DROP VIEW fooview
+PREHOOK: type: DROPVIEW
+POSTHOOK: query: DROP VIEW fooview
+POSTHOOK: type: DROPVIEW
+PREHOOK: query: DROP TABLE foo
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE foo
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: USE test1
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:test1
+POSTHOOK: query: USE test1
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:test1
+PREHOOK: query: DROP VIEW shtb_test1_view1
+PREHOOK: type: DROPVIEW
+PREHOOK: Input: test1@shtb_test1_view1
+PREHOOK: Output: test1@shtb_test1_view1
+POSTHOOK: query: DROP VIEW shtb_test1_view1
+POSTHOOK: type: DROPVIEW
+POSTHOOK: Input: test1@shtb_test1_view1
+POSTHOOK: Output: test1@shtb_test1_view1
+PREHOOK: query: DROP VIEW shtb_test1_view2
+PREHOOK: type: DROPVIEW
+PREHOOK: Input: test1@shtb_test1_view2
+PREHOOK: Output: test1@shtb_test1_view2
+POSTHOOK: query: DROP VIEW shtb_test1_view2
+POSTHOOK: type: DROPVIEW
+POSTHOOK: Input: test1@shtb_test1_view2
+POSTHOOK: Output: test1@shtb_test1_view2
+PREHOOK: query: DROP VIEW shtb_full_view2
+PREHOOK: type: DROPVIEW
+PREHOOK: Input: test1@shtb_full_view2
+PREHOOK: Output: test1@shtb_full_view2
+POSTHOOK: query: DROP VIEW shtb_full_view2
+POSTHOOK: type: DROPVIEW
+POSTHOOK: Input: test1@shtb_full_view2
+POSTHOOK: Output: test1@shtb_full_view2
+PREHOOK: query: DROP TABLE shtb_test1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: test1@shtb_test1
+PREHOOK: Output: test1@shtb_test1
+POSTHOOK: query: DROP TABLE shtb_test1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: test1@shtb_test1
+POSTHOOK: Output: test1@shtb_test1
+PREHOOK: query: DROP DATABASE test1
+PREHOOK: type: DROPDATABASE
+PREHOOK: Input: database:test1
+PREHOOK: Output: database:test1
+POSTHOOK: query: DROP DATABASE test1
+POSTHOOK: type: DROPDATABASE
+POSTHOOK: Input: database:test1
+POSTHOOK: Output: database:test1
+PREHOOK: query: USE test2
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:test2
+POSTHOOK: query: USE test2
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:test2
+PREHOOK: query: DROP VIEW shtb_test1_view1
+PREHOOK: type: DROPVIEW
+PREHOOK: Input: test2@shtb_test1_view1
+PREHOOK: Output: test2@shtb_test1_view1
+POSTHOOK: query: DROP VIEW shtb_test1_view1
+POSTHOOK: type: DROPVIEW
+POSTHOOK: Input: test2@shtb_test1_view1
+POSTHOOK: Output: test2@shtb_test1_view1
+PREHOOK: query: DROP VIEW shtb_test2_view2
+PREHOOK: type: DROPVIEW
+PREHOOK: Input: test2@shtb_test2_view2
+PREHOOK: Output: test2@shtb_test2_view2
+POSTHOOK: query: DROP VIEW shtb_test2_view2
+POSTHOOK: type: DROPVIEW
+POSTHOOK: Input: test2@shtb_test2_view2
+POSTHOOK: Output: test2@shtb_test2_view2
+PREHOOK: query: DROP TABLE shtb_test1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: test2@shtb_test1
+PREHOOK: Output: test2@shtb_test1
+POSTHOOK: query: DROP TABLE shtb_test1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: test2@shtb_test1
+POSTHOOK: Output: test2@shtb_test1
+PREHOOK: query: DROP TABLE shtb_test2
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: test2@shtb_test2
+PREHOOK: Output: test2@shtb_test2
+POSTHOOK: query: DROP TABLE shtb_test2
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: test2@shtb_test2
+POSTHOOK: Output: test2@shtb_test2
+PREHOOK: query: DROP DATABASE test2
+PREHOOK: type: DROPDATABASE
+PREHOOK: Input: database:test2
+PREHOOK: Output: database:test2
+POSTHOOK: query: DROP DATABASE test2
+POSTHOOK: type: DROPDATABASE
+POSTHOOK: Input: database:test2
+POSTHOOK: Output: database:test2


[05/15] hive git commit: HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index cb5dec9..d827d6c 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -92,6 +92,8 @@ public class ThriftHiveMetastore {
 
     public List<String> get_tables(String db_name, String pattern) throws MetaException, org.apache.thrift.TException;
 
+    public List<String> get_tables_by_type(String db_name, String pattern, String tableType) throws MetaException, org.apache.thrift.TException;
+
     public List<TableMeta> get_table_meta(String db_patterns, String tbl_patterns, List<String> tbl_types) throws MetaException, org.apache.thrift.TException;
 
     public List<String> get_all_tables(String db_name) throws MetaException, org.apache.thrift.TException;
@@ -394,6 +396,8 @@ public class ThriftHiveMetastore {
 
     public void get_tables(String db_name, String pattern, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void get_tables_by_type(String db_name, String pattern, String tableType, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void get_table_meta(String db_patterns, String tbl_patterns, List<String> tbl_types, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void get_all_tables(String db_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -1392,6 +1396,34 @@ public class ThriftHiveMetastore {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_tables failed: unknown result");
     }
 
+    public List<String> get_tables_by_type(String db_name, String pattern, String tableType) throws MetaException, org.apache.thrift.TException
+    {
+      send_get_tables_by_type(db_name, pattern, tableType);
+      return recv_get_tables_by_type();
+    }
+
+    public void send_get_tables_by_type(String db_name, String pattern, String tableType) throws org.apache.thrift.TException
+    {
+      get_tables_by_type_args args = new get_tables_by_type_args();
+      args.setDb_name(db_name);
+      args.setPattern(pattern);
+      args.setTableType(tableType);
+      sendBase("get_tables_by_type", args);
+    }
+
+    public List<String> recv_get_tables_by_type() throws MetaException, org.apache.thrift.TException
+    {
+      get_tables_by_type_result result = new get_tables_by_type_result();
+      receiveBase(result, "get_tables_by_type");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_tables_by_type failed: unknown result");
+    }
+
     public List<TableMeta> get_table_meta(String db_patterns, String tbl_patterns, List<String> tbl_types) throws MetaException, org.apache.thrift.TException
     {
       send_get_table_meta(db_patterns, tbl_patterns, tbl_types);
@@ -5835,6 +5867,44 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void get_tables_by_type(String db_name, String pattern, String tableType, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_tables_by_type_call method_call = new get_tables_by_type_call(db_name, pattern, tableType, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class get_tables_by_type_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String db_name;
+      private String pattern;
+      private String tableType;
+      public get_tables_by_type_call(String db_name, String pattern, String tableType, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.db_name = db_name;
+        this.pattern = pattern;
+        this.tableType = tableType;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_tables_by_type", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_tables_by_type_args args = new get_tables_by_type_args();
+        args.setDb_name(db_name);
+        args.setPattern(pattern);
+        args.setTableType(tableType);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public List<String> getResult() throws MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_tables_by_type();
+      }
+    }
+
     public void get_table_meta(String db_patterns, String tbl_patterns, List<String> tbl_types, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       get_table_meta_call method_call = new get_table_meta_call(db_patterns, tbl_patterns, tbl_types, resultHandler, this, ___protocolFactory, ___transport);
@@ -10270,6 +10340,7 @@ public class ThriftHiveMetastore {
       processMap.put("drop_table", new drop_table());
       processMap.put("drop_table_with_environment_context", new drop_table_with_environment_context());
       processMap.put("get_tables", new get_tables());
+      processMap.put("get_tables_by_type", new get_tables_by_type());
       processMap.put("get_table_meta", new get_table_meta());
       processMap.put("get_all_tables", new get_all_tables());
       processMap.put("get_table", new get_table());
@@ -11063,6 +11134,30 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_tables_by_type<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_tables_by_type_args> {
+      public get_tables_by_type() {
+        super("get_tables_by_type");
+      }
+
+      public get_tables_by_type_args getEmptyArgsInstance() {
+        return new get_tables_by_type_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_tables_by_type_result getResult(I iface, get_tables_by_type_args args) throws org.apache.thrift.TException {
+        get_tables_by_type_result result = new get_tables_by_type_result();
+        try {
+          result.success = iface.get_tables_by_type(args.db_name, args.pattern, args.tableType);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        }
+        return result;
+      }
+    }
+
     public static class get_table_meta<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_table_meta_args> {
       public get_table_meta() {
         super("get_table_meta");
@@ -14241,6 +14336,7 @@ public class ThriftHiveMetastore {
       processMap.put("drop_table", new drop_table());
       processMap.put("drop_table_with_environment_context", new drop_table_with_environment_context());
       processMap.put("get_tables", new get_tables());
+      processMap.put("get_tables_by_type", new get_tables_by_type());
       processMap.put("get_table_meta", new get_table_meta());
       processMap.put("get_all_tables", new get_all_tables());
       processMap.put("get_table", new get_table());
@@ -15943,6 +16039,63 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_tables_by_type<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_tables_by_type_args, List<String>> {
+      public get_tables_by_type() {
+        super("get_tables_by_type");
+      }
+
+      public get_tables_by_type_args getEmptyArgsInstance() {
+        return new get_tables_by_type_args();
+      }
+
+      public AsyncMethodCallback<List<String>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<List<String>>() { 
+          public void onComplete(List<String> o) {
+            get_tables_by_type_result result = new get_tables_by_type_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            get_tables_by_type_result result = new get_tables_by_type_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, get_tables_by_type_args args, org.apache.thrift.async.AsyncMethodCallback<List<String>> resultHandler) throws TException {
+        iface.get_tables_by_type(args.db_name, args.pattern, args.tableType,resultHandler);
+      }
+    }
+
     public static class get_table_meta<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_meta_args, List<TableMeta>> {
       public get_table_meta() {
         super("get_table_meta");
@@ -47851,12 +48004,1036 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_tables_args)
-        return this.equals((get_tables_args)that);
+      if (that instanceof get_tables_args)
+        return this.equals((get_tables_args)that);
+      return false;
+    }
+
+    public boolean equals(get_tables_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_db_name = true && this.isSetDb_name();
+      boolean that_present_db_name = true && that.isSetDb_name();
+      if (this_present_db_name || that_present_db_name) {
+        if (!(this_present_db_name && that_present_db_name))
+          return false;
+        if (!this.db_name.equals(that.db_name))
+          return false;
+      }
+
+      boolean this_present_pattern = true && this.isSetPattern();
+      boolean that_present_pattern = true && that.isSetPattern();
+      if (this_present_pattern || that_present_pattern) {
+        if (!(this_present_pattern && that_present_pattern))
+          return false;
+        if (!this.pattern.equals(that.pattern))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_db_name = true && (isSetDb_name());
+      list.add(present_db_name);
+      if (present_db_name)
+        list.add(db_name);
+
+      boolean present_pattern = true && (isSetPattern());
+      list.add(present_pattern);
+      if (present_pattern)
+        list.add(pattern);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(get_tables_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(other.isSetDb_name());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetDb_name()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_name, other.db_name);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetPattern()).compareTo(other.isSetPattern());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetPattern()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pattern, other.pattern);
+        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("get_tables_args(");
+      boolean first = true;
+
+      sb.append("db_name:");
+      if (this.db_name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.db_name);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("pattern:");
+      if (this.pattern == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.pattern);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // 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 {
+        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 get_tables_argsStandardSchemeFactory implements SchemeFactory {
+      public get_tables_argsStandardScheme getScheme() {
+        return new get_tables_argsStandardScheme();
+      }
+    }
+
+    private static class get_tables_argsStandardScheme extends StandardScheme<get_tables_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // DB_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.db_name = iprot.readString();
+                struct.setDb_nameIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // PATTERN
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.pattern = iprot.readString();
+                struct.setPatternIsSet(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, get_tables_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.db_name != null) {
+          oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
+          oprot.writeString(struct.db_name);
+          oprot.writeFieldEnd();
+        }
+        if (struct.pattern != null) {
+          oprot.writeFieldBegin(PATTERN_FIELD_DESC);
+          oprot.writeString(struct.pattern);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class get_tables_argsTupleSchemeFactory implements SchemeFactory {
+      public get_tables_argsTupleScheme getScheme() {
+        return new get_tables_argsTupleScheme();
+      }
+    }
+
+    private static class get_tables_argsTupleScheme extends TupleScheme<get_tables_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetDb_name()) {
+          optionals.set(0);
+        }
+        if (struct.isSetPattern()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetDb_name()) {
+          oprot.writeString(struct.db_name);
+        }
+        if (struct.isSetPattern()) {
+          oprot.writeString(struct.pattern);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.db_name = iprot.readString();
+          struct.setDb_nameIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.pattern = iprot.readString();
+          struct.setPatternIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class get_tables_result implements org.apache.thrift.TBase<get_tables_result, get_tables_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
+    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_tables_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_tables_resultTupleSchemeFactory());
+    }
+
+    private List<String> success; // required
+    private MetaException o1; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      O1((short)1, "o1");
+
+      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 0: // SUCCESS
+            return SUCCESS;
+          case 1: // O1
+            return O1;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+      tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_tables_result.class, metaDataMap);
+    }
+
+    public get_tables_result() {
+    }
+
+    public get_tables_result(
+      List<String> success,
+      MetaException o1)
+    {
+      this();
+      this.success = success;
+      this.o1 = o1;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_tables_result(get_tables_result other) {
+      if (other.isSetSuccess()) {
+        List<String> __this__success = new ArrayList<String>(other.success);
+        this.success = __this__success;
+      }
+      if (other.isSetO1()) {
+        this.o1 = new MetaException(other.o1);
+      }
+    }
+
+    public get_tables_result deepCopy() {
+      return new get_tables_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.o1 = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    public java.util.Iterator<String> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(String elem) {
+      if (this.success == null) {
+        this.success = new ArrayList<String>();
+      }
+      this.success.add(elem);
+    }
+
+    public List<String> getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(List<String> success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public MetaException getO1() {
+      return this.o1;
+    }
+
+    public void setO1(MetaException o1) {
+      this.o1 = o1;
+    }
+
+    public void unsetO1() {
+      this.o1 = null;
+    }
+
+    /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
+    public boolean isSetO1() {
+      return this.o1 != null;
+    }
+
+    public void setO1IsSet(boolean value) {
+      if (!value) {
+        this.o1 = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((List<String>)value);
+        }
+        break;
+
+      case O1:
+        if (value == null) {
+          unsetO1();
+        } else {
+          setO1((MetaException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      case O1:
+        return getO1();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      case O1:
+        return isSetO1();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_tables_result)
+        return this.equals((get_tables_result)that);
+      return false;
+    }
+
+    public boolean equals(get_tables_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_o1 = true && this.isSetO1();
+      boolean that_present_o1 = true && that.isSetO1();
+      if (this_present_o1 || that_present_o1) {
+        if (!(this_present_o1 && that_present_o1))
+          return false;
+        if (!this.o1.equals(that.o1))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (isSetSuccess());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      boolean present_o1 = true && (isSetO1());
+      list.add(present_o1);
+      if (present_o1)
+        list.add(o1);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(get_tables_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetO1()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, other.o1);
+        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("get_tables_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("o1:");
+      if (this.o1 == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.o1);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // 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 {
+        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 get_tables_resultStandardSchemeFactory implements SchemeFactory {
+      public get_tables_resultStandardScheme getScheme() {
+        return new get_tables_resultStandardScheme();
+      }
+    }
+
+    private static class get_tables_resultStandardScheme extends StandardScheme<get_tables_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list716.size);
+                  String _elem717;
+                  for (int _i718 = 0; _i718 < _list716.size; ++_i718)
+                  {
+                    _elem717 = iprot.readString();
+                    struct.success.add(_elem717);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // O1
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.o1 = new MetaException();
+                struct.o1.read(iprot);
+                struct.setO1IsSet(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, get_tables_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
+            for (String _iter719 : struct.success)
+            {
+              oprot.writeString(_iter719);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.o1 != null) {
+          oprot.writeFieldBegin(O1_FIELD_DESC);
+          struct.o1.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class get_tables_resultTupleSchemeFactory implements SchemeFactory {
+      public get_tables_resultTupleScheme getScheme() {
+        return new get_tables_resultTupleScheme();
+      }
+    }
+
+    private static class get_tables_resultTupleScheme extends TupleScheme<get_tables_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        if (struct.isSetO1()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (String _iter720 : struct.success)
+            {
+              oprot.writeString(_iter720);
+            }
+          }
+        }
+        if (struct.isSetO1()) {
+          struct.o1.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          {
+            org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list721.size);
+            String _elem722;
+            for (int _i723 = 0; _i723 < _list721.size; ++_i723)
+            {
+              _elem722 = iprot.readString();
+              struct.success.add(_elem722);
+            }
+          }
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.o1 = new MetaException();
+          struct.o1.read(iprot);
+          struct.setO1IsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class get_tables_by_type_args implements org.apache.thrift.TBase<get_tables_by_type_args, get_tables_by_type_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_by_type_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_by_type_args");
+
+    private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("pattern", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField TABLE_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tableType", org.apache.thrift.protocol.TType.STRING, (short)3);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_tables_by_type_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_tables_by_type_argsTupleSchemeFactory());
+    }
+
+    private String db_name; // required
+    private String pattern; // required
+    private String tableType; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      DB_NAME((short)1, "db_name"),
+      PATTERN((short)2, "pattern"),
+      TABLE_TYPE((short)3, "tableType");
+
+      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: // DB_NAME
+            return DB_NAME;
+          case 2: // PATTERN
+            return PATTERN;
+          case 3: // TABLE_TYPE
+            return TABLE_TYPE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.PATTERN, new org.apache.thrift.meta_data.FieldMetaData("pattern", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.TABLE_TYPE, new org.apache.thrift.meta_data.FieldMetaData("tableType", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_tables_by_type_args.class, metaDataMap);
+    }
+
+    public get_tables_by_type_args() {
+    }
+
+    public get_tables_by_type_args(
+      String db_name,
+      String pattern,
+      String tableType)
+    {
+      this();
+      this.db_name = db_name;
+      this.pattern = pattern;
+      this.tableType = tableType;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_tables_by_type_args(get_tables_by_type_args other) {
+      if (other.isSetDb_name()) {
+        this.db_name = other.db_name;
+      }
+      if (other.isSetPattern()) {
+        this.pattern = other.pattern;
+      }
+      if (other.isSetTableType()) {
+        this.tableType = other.tableType;
+      }
+    }
+
+    public get_tables_by_type_args deepCopy() {
+      return new get_tables_by_type_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.db_name = null;
+      this.pattern = null;
+      this.tableType = null;
+    }
+
+    public String getDb_name() {
+      return this.db_name;
+    }
+
+    public void setDb_name(String db_name) {
+      this.db_name = db_name;
+    }
+
+    public void unsetDb_name() {
+      this.db_name = null;
+    }
+
+    /** Returns true if field db_name is set (has been assigned a value) and false otherwise */
+    public boolean isSetDb_name() {
+      return this.db_name != null;
+    }
+
+    public void setDb_nameIsSet(boolean value) {
+      if (!value) {
+        this.db_name = null;
+      }
+    }
+
+    public String getPattern() {
+      return this.pattern;
+    }
+
+    public void setPattern(String pattern) {
+      this.pattern = pattern;
+    }
+
+    public void unsetPattern() {
+      this.pattern = null;
+    }
+
+    /** Returns true if field pattern is set (has been assigned a value) and false otherwise */
+    public boolean isSetPattern() {
+      return this.pattern != null;
+    }
+
+    public void setPatternIsSet(boolean value) {
+      if (!value) {
+        this.pattern = null;
+      }
+    }
+
+    public String getTableType() {
+      return this.tableType;
+    }
+
+    public void setTableType(String tableType) {
+      this.tableType = tableType;
+    }
+
+    public void unsetTableType() {
+      this.tableType = null;
+    }
+
+    /** Returns true if field tableType is set (has been assigned a value) and false otherwise */
+    public boolean isSetTableType() {
+      return this.tableType != null;
+    }
+
+    public void setTableTypeIsSet(boolean value) {
+      if (!value) {
+        this.tableType = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case DB_NAME:
+        if (value == null) {
+          unsetDb_name();
+        } else {
+          setDb_name((String)value);
+        }
+        break;
+
+      case PATTERN:
+        if (value == null) {
+          unsetPattern();
+        } else {
+          setPattern((String)value);
+        }
+        break;
+
+      case TABLE_TYPE:
+        if (value == null) {
+          unsetTableType();
+        } else {
+          setTableType((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case DB_NAME:
+        return getDb_name();
+
+      case PATTERN:
+        return getPattern();
+
+      case TABLE_TYPE:
+        return getTableType();
+
+      }
+      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 DB_NAME:
+        return isSetDb_name();
+      case PATTERN:
+        return isSetPattern();
+      case TABLE_TYPE:
+        return isSetTableType();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_tables_by_type_args)
+        return this.equals((get_tables_by_type_args)that);
       return false;
     }
 
-    public boolean equals(get_tables_args that) {
+    public boolean equals(get_tables_by_type_args that) {
       if (that == null)
         return false;
 
@@ -47878,6 +49055,15 @@ public class ThriftHiveMetastore {
           return false;
       }
 
+      boolean this_present_tableType = true && this.isSetTableType();
+      boolean that_present_tableType = true && that.isSetTableType();
+      if (this_present_tableType || that_present_tableType) {
+        if (!(this_present_tableType && that_present_tableType))
+          return false;
+        if (!this.tableType.equals(that.tableType))
+          return false;
+      }
+
       return true;
     }
 
@@ -47895,11 +49081,16 @@ public class ThriftHiveMetastore {
       if (present_pattern)
         list.add(pattern);
 
+      boolean present_tableType = true && (isSetTableType());
+      list.add(present_tableType);
+      if (present_tableType)
+        list.add(tableType);
+
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(get_tables_args other) {
+    public int compareTo(get_tables_by_type_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -47926,6 +49117,16 @@ public class ThriftHiveMetastore {
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(isSetTableType()).compareTo(other.isSetTableType());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTableType()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableType, other.tableType);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -47943,7 +49144,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_tables_args(");
+      StringBuilder sb = new StringBuilder("get_tables_by_type_args(");
       boolean first = true;
 
       sb.append("db_name:");
@@ -47961,6 +49162,14 @@ public class ThriftHiveMetastore {
         sb.append(this.pattern);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("tableType:");
+      if (this.tableType == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tableType);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -47986,15 +49195,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class get_tables_argsStandardSchemeFactory implements SchemeFactory {
-      public get_tables_argsStandardScheme getScheme() {
-        return new get_tables_argsStandardScheme();
+    private static class get_tables_by_type_argsStandardSchemeFactory implements SchemeFactory {
+      public get_tables_by_type_argsStandardScheme getScheme() {
+        return new get_tables_by_type_argsStandardScheme();
       }
     }
 
-    private static class get_tables_argsStandardScheme extends StandardScheme<get_tables_args> {
+    private static class get_tables_by_type_argsStandardScheme extends StandardScheme<get_tables_by_type_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_by_type_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -48020,6 +49229,14 @@ public class ThriftHiveMetastore {
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
+            case 3: // TABLE_TYPE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.tableType = iprot.readString();
+                struct.setTableTypeIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -48029,7 +49246,7 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_by_type_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -48043,22 +49260,27 @@ public class ThriftHiveMetastore {
           oprot.writeString(struct.pattern);
           oprot.writeFieldEnd();
         }
+        if (struct.tableType != null) {
+          oprot.writeFieldBegin(TABLE_TYPE_FIELD_DESC);
+          oprot.writeString(struct.tableType);
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class get_tables_argsTupleSchemeFactory implements SchemeFactory {
-      public get_tables_argsTupleScheme getScheme() {
-        return new get_tables_argsTupleScheme();
+    private static class get_tables_by_type_argsTupleSchemeFactory implements SchemeFactory {
+      public get_tables_by_type_argsTupleScheme getScheme() {
+        return new get_tables_by_type_argsTupleScheme();
       }
     }
 
-    private static class get_tables_argsTupleScheme extends TupleScheme<get_tables_args> {
+    private static class get_tables_by_type_argsTupleScheme extends TupleScheme<get_tables_by_type_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_by_type_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetDb_name()) {
@@ -48067,19 +49289,25 @@ public class ThriftHiveMetastore {
         if (struct.isSetPattern()) {
           optionals.set(1);
         }
-        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTableType()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
         if (struct.isSetDb_name()) {
           oprot.writeString(struct.db_name);
         }
         if (struct.isSetPattern()) {
           oprot.writeString(struct.pattern);
         }
+        if (struct.isSetTableType()) {
+          oprot.writeString(struct.tableType);
+        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_by_type_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(2);
+        BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           struct.db_name = iprot.readString();
           struct.setDb_nameIsSet(true);
@@ -48088,21 +49316,25 @@ public class ThriftHiveMetastore {
           struct.pattern = iprot.readString();
           struct.setPatternIsSet(true);
         }
+        if (incoming.get(2)) {
+          struct.tableType = iprot.readString();
+          struct.setTableTypeIsSet(true);
+        }
       }
     }
 
   }
 
-  public static class get_tables_result implements org.apache.thrift.TBase<get_tables_result, get_tables_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_result");
+  public static class get_tables_by_type_result implements org.apache.thrift.TBase<get_tables_by_type_result, get_tables_by_type_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_tables_by_type_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_tables_by_type_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
     private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_tables_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_tables_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new get_tables_by_type_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_tables_by_type_resultTupleSchemeFactory());
     }
 
     private List<String> success; // required
@@ -48179,13 +49411,13 @@ public class ThriftHiveMetastore {
       tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_tables_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_tables_by_type_result.class, metaDataMap);
     }
 
-    public get_tables_result() {
+    public get_tables_by_type_result() {
     }
 
-    public get_tables_result(
+    public get_tables_by_type_result(
       List<String> success,
       MetaException o1)
     {
@@ -48197,7 +49429,7 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_tables_result(get_tables_result other) {
+    public get_tables_by_type_result(get_tables_by_type_result other) {
       if (other.isSetSuccess()) {
         List<String> __this__success = new ArrayList<String>(other.success);
         this.success = __this__success;
@@ -48207,8 +49439,8 @@ public class ThriftHiveMetastore {
       }
     }
 
-    public get_tables_result deepCopy() {
-      return new get_tables_result(this);
+    public get_tables_by_type_result deepCopy() {
+      return new get_tables_by_type_result(this);
     }
 
     @Override
@@ -48330,12 +49562,12 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_tables_result)
-        return this.equals((get_tables_result)that);
+      if (that instanceof get_tables_by_type_result)
+        return this.equals((get_tables_by_type_result)that);
       return false;
     }
 
-    public boolean equals(get_tables_result that) {
+    public boolean equals(get_tables_by_type_result that) {
       if (that == null)
         return false;
 
@@ -48378,7 +49610,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(get_tables_result other) {
+    public int compareTo(get_tables_by_type_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -48422,7 +49654,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_tables_result(");
+      StringBuilder sb = new StringBuilder("get_tables_by_type_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -48465,15 +49697,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class get_tables_resultStandardSchemeFactory implements SchemeFactory {
-      public get_tables_resultStandardScheme getScheme() {
-        return new get_tables_resultStandardScheme();
+    private static class get_tables_by_type_resultStandardSchemeFactory implements SchemeFactory {
+      public get_tables_by_type_resultStandardScheme getScheme() {
+        return new get_tables_by_type_resultStandardScheme();
       }
     }
 
-    private static class get_tables_resultStandardScheme extends StandardScheme<get_tables_result> {
+    private static class get_tables_by_type_resultStandardScheme extends StandardScheme<get_tables_by_type_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_by_type_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -48486,13 +49718,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list716.size);
-                  String _elem717;
-                  for (int _i718 = 0; _i718 < _list716.size; ++_i718)
+                  org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list724.size);
+                  String _elem725;
+                  for (int _i726 = 0; _i726 < _list724.size; ++_i726)
                   {
-                    _elem717 = iprot.readString();
-                    struct.success.add(_elem717);
+                    _elem725 = iprot.readString();
+                    struct.success.add(_elem725);
                   }
                   iprot.readListEnd();
                 }
@@ -48519,7 +49751,7 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_by_type_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -48527,9 +49759,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter719 : struct.success)
+            for (String _iter727 : struct.success)
             {
-              oprot.writeString(_iter719);
+              oprot.writeString(_iter727);
             }
             oprot.writeListEnd();
           }
@@ -48546,16 +49778,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class get_tables_resultTupleSchemeFactory implements SchemeFactory {
-      public get_tables_resultTupleScheme getScheme() {
-        return new get_tables_resultTupleScheme();
+    private static class get_tables_by_type_resultTupleSchemeFactory implements SchemeFactory {
+      public get_tables_by_type_resultTupleScheme getScheme() {
+        return new get_tables_by_type_resultTupleScheme();
       }
     }
 
-    private static class get_tables_resultTupleScheme extends TupleScheme<get_tables_result> {
+    private static class get_tables_by_type_resultTupleScheme extends TupleScheme<get_tables_by_type_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_by_type_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSuccess()) {
@@ -48568,9 +49800,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter720 : struct.success)
+            for (String _iter728 : struct.success)
             {
-              oprot.writeString(_iter720);
+              oprot.writeString(_iter728);
             }
           }
         }
@@ -48580,18 +49812,18 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_by_type_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list721.size);
-            String _elem722;
-            for (int _i723 = 0; _i723 < _list721.size; ++_i723)
+            org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list729.size);
+            String _elem730;
+            for (int _i731 = 0; _i731 < _list729.size; ++_i731)
             {
-              _elem722 = iprot.readString();
-              struct.success.add(_elem722);
+              _elem730 = iprot.readString();
+              struct.success.add(_elem730);
             }
           }
           struct.setSuccessIsSet(true);
@@ -49096,13 +50328,13 @@ public class ThriftHiveMetastore {
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list724.size);
-                  String _elem725;
-                  for (int _i726 = 0; _i726 < _list724.size; ++_i726)
+                  org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list732.size);
+                  String _elem733;
+                  for (int _i734 = 0; _i734 < _list732.size; ++_i734)
                   {
-                    _elem725 = iprot.readString();
-                    struct.tbl_types.add(_elem725);
+                    _elem733 = iprot.readString();
+                    struct.tbl_types.add(_elem733);
                   }
                   iprot.readListEnd();
                 }
@@ -49138,9 +50370,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(TBL_TYPES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_types.size()));
-            for (String _iter727 : struct.tbl_types)
+            for (String _iter735 : struct.tbl_types)
             {
-              oprot.writeString(_iter727);
+              oprot.writeString(_iter735);
             }
             oprot.writeListEnd();
           }
@@ -49183,9 +50415,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter728 : struct.tbl_types)
+            for (String _iter736 : struct.tbl_types)
             {
-              oprot.writeString(_iter728);
+              oprot.writeString(_iter736);
             }
           }
         }
@@ -49205,13 +50437,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list729.size);
-            String _elem730;
-            for (int _i731 = 0; _i731 < _list729.size; ++_i731)
+            org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list737.size);
+            String _elem738;
+            for (int _i739 = 0; _i739 < _list737.size; ++_i739)
             {
-              _elem730 = iprot.readString();
-              struct.tbl_types.add(_elem730);
+              _elem738 = iprot.readString();
+              struct.tbl_types.add(_elem738);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -49617,14 +50849,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
-                  struct.success = new ArrayList<TableMeta>(_list732.size);
-                  TableMeta _elem733;
-                  for (int _i734 = 0; _i734 < _list732.size; ++_i734)
+                  org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list740.size);
+                  TableMeta _elem741;
+                  for (int _i742 = 0; _i742 < _list740.size; ++_i742)
                   {
-                    _elem733 = new TableMeta();
-                    _elem733.read(iprot);
-                    struct.success.add(_elem733);
+                    _elem741 = new TableMeta();
+                    _elem741.read(iprot);
+                    struct.success.add(_elem741);
                   }
                   iprot.readListEnd();
                 }
@@ -49659,9 +50891,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TableMeta _iter735 : struct.success)
+            for (TableMeta _iter743 : struct.success)
             {
-              _iter735.write(oprot);
+              _iter743.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -49700,9 +50932,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter736 : struct.success)
+            for (TableMeta _iter744 : struct.success)
             {
-              _iter736.write(oprot);
+              _iter744.write(oprot);
             }
           }
         }
@@ -49717,14 +50949,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TableMeta>(_list737.size);
-            TableMeta _elem738;
-            for (int _i739 = 0; _i739 < _list737.size; ++_i739)
+            org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list745.size);
+            TableMeta _elem746;
+            for (int _i747 = 0; _i747 < _list745.size; ++_i747)
             {
-              _elem738 = new TableMeta();
-              _elem738.read(iprot);
-              struct.success.add(_elem738);
+              _elem746 = new TableMeta();
+              _elem746.read(iprot);
+              struct.success.add(_elem746);
             }
           }
           struct.setSuccessIsSet(true);
@@ -50490,13 +51722,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list740.size);
-                  String _elem741;
-                  for (int _i742 = 0; _i742 < _list740.size; ++_i742)
+                  org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list748.size);
+                  String _elem749;
+                  for (int _i750 = 0; _i750 < _list748.size; ++_i750)
                   {
-                    _elem741 = iprot.readString();
-                    struct.success.add(_elem741);
+                    _elem749 = iprot.readString();
+                    struct.success.add(_elem749);
                   }
                   iprot.readListEnd();
                 }
@@ -50531,9 +51763,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter743 : struct.success)
+            for (String _iter751 : struct.success)
             {
-              oprot.writeString(_iter743);
+              oprot.writeString(_iter751);
             }
             oprot.writeListEnd();
           }
@@ -50572,9 +51804,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter744 : struct.success)
+            for (String _iter752 : struct.success)
             {
-              oprot.writeString(_iter744);
+              oprot.writeString(_iter752);
             }
           }
         }
@@ -50589,13 +51821,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list745.size);
-            String _elem746;
-            for (int _i747 = 0; _i747 < _list745.size; ++_i747)
+            org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list753.size);
+            String _elem754;
+            for (int _i755 = 0; _i755 < _list753.size; ++_i755)
             {
-              _elem746 = iprot.readString();
-              struct.success.add(_elem746);
+              _elem754 = iprot.readString();
+              struct.success.add(_elem754);
             }
           }
           struct.setSuccessIsSet(true);
@@ -52048,13 +53280,13 @@ public class ThriftHiveMetastore {
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list748.size);
-                  String _elem749;
-                  for (int _i750 = 0; _i750 < _list748.size; ++_i750)
+                  org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list756.size);
+                  String _elem757;
+                  for (int _i758 = 0; _i758 < _list756.size; ++_i758)
                   {
-                    _elem749 = iprot.readString();
-                    struct.tbl_names.add(_elem749);
+                    _elem757 = iprot.readString();
+                    struct.tbl_names.add(_elem757);
                   }
                   iprot.readListEnd();
                 }
@@ -52085,9 +53317,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter751 : struct.tbl_names)
+            for (String _iter759 : struct.tbl_names)
             {
-              oprot.writeString(_iter751);
+              oprot.writeString(_iter759);
             }
             oprot.writeListEnd();
           }
@@ -52124,9 +53356,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter752 : struct.tbl_names)
+            for (String _iter760 : struct.tbl_names)
             {
-              oprot.writeString(_iter752);
+              oprot.writeString(_iter760);
             }
           }
         }
@@ -52142,13 +53374,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list753.size);
-            String _elem754;
-            for (int _i755 = 0; _i755 < _list753.size; ++_i755)
+            org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list761.size);
+            String _elem762;
+            for (int _i763 = 0; _i763 < _list761.size; ++_i763)
             {
-              _elem754 = iprot.readString();
-              struct.tbl_names.add(_elem754);
+              _elem762 = iprot.readString();
+              struct.tbl_names.add(_elem762);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -52716,14 +53948,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list756.size);
-                  Table _elem757;
-                  for (int _i758 = 0; _i758 < _list756.size; ++_i758)
+                  org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list764.size);
+                  Table _elem765;
+                  for (int _i766 = 0; _i766 < _list764.size; ++_i766)
                   {
-                    _elem757 = new Table();
-                    _elem757.read(iprot);
-                    struct.success.add(_elem757);
+                    _elem765 = new Table();
+                    _elem765.read(iprot);
+                    struct.success.add(_elem765);
                   }
                   iprot.readListEnd();
                 }
@@ -52776,9 +54008,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter759 : struct.success)
+            for (Table _iter767 : struct.success)
             {
-              _iter759.write(oprot);
+              _iter767.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -52833,9 +54065,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter760 : struct.success)
+            for (Table _iter768 : struct.success)
             {
-              _iter760.write(oprot);
+              _iter768.write(oprot);
             }
           }
         }
@@ -52856,14 +54088,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list761.size);
-            Table _elem762;
-            for (int _i763 = 0; _i763 < _list761.size; ++_i763)
+            org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list769.size);
+            Table _elem770;
+            for (int _i771 = 0; _i771 < _list769.size; ++_i771)
             {
-              _elem762 = new Table();
-              _elem762.read(iprot);
-              struct.success.add(_elem762);
+              _elem770 = new Table();
+              _elem770.read(iprot);
+              struct.success.add(_elem770);
             }
           }
           struct.setSuccessIsSet(true);
@@ -54009,13 +55241,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list764.size);
-                  String _elem765;
-                  for (int _i766 = 0; _i766 < _list764.size; ++_i766)
+                  org.apache.thrift.protocol.TList _list772 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list772.size);
+                  String _elem773;
+                  for (int _i774 = 0; _i774 < _list772.size; ++_i774)
                   {
-                    _elem765 = iprot.readString();
-                    struct.success.add(_elem765);
+                    _elem773 = iprot.readString();
+                    struct.success.add(_elem773);
                   }
                   iprot.readListEnd();
                 }
@@ -54068,9 +55300,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter767 : struct.success)
+            for (String _iter775 : struct.success)
             {
-              oprot.writeString(_iter767);
+              oprot.writeString(_iter775);
             }
             oprot.writeListEnd();
           }
@@ -54125,9 +55357,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter768 : struct.success)
+            for (String _iter776 : struct.success)
             {
-              oprot.writeString(_iter768);
+              oprot.writeString(_iter776);
             }
           }
         }
@@ -54148,13 +55380,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list769.size);
-            String _elem770;
-            for (int _i771 = 0; _i771 < _list769.size; ++_i771)
+            org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list777.size);
+            String _elem778;
+            for (int _i779 = 0; _i779 < _list777.size; ++_i779)
             {
-              _elem770 = iprot.readString();
-              struct.success.add(_elem770);
+              _elem778 = iprot.readString();
+              struct.success.add(_elem778);
             }
           }
           struct.setSuccessIsSet(true);
@@ -60013,14 +61245,14 @@ public class ThriftHiveMetastore {
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list772 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list772.size);
-                  Partition _elem773;
-                  for (int _i774 = 0; _i774 < _list772.size; ++_i774)
+                  org.apache.thrift.protocol.TList _list780 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list780.size);
+                  Partition _elem781;
+                  for (int _i782 = 0; _i782 < _list780.size; ++_i782)
                   {
-                    _elem773 = new Partition();
-                    _elem773.read(iprot);
-                    struct.new_parts.add(_elem773);
+                    _elem781 = new Partition();
+                    _elem781.read(iprot);
+                    struct.new_parts.add(_elem781);
                   }
                   iprot.readListEnd();
                 }
@@ -60046,9 +61278,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (Partition _iter775 : struct.new_parts)
+            for (Partition _iter783 : struct.new_parts)
             {
-              _iter775.write(oprot);
+              _iter783.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -60079,9 +61311,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter776 : struct.new_parts)
+            for (Partition _iter784 : struct.new_parts)
             {
-              _iter776.write(oprot);
+              _iter784.write(oprot);
             }
           }
         }
@@ -60093,14 +61325,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list777.size);
-            Partition _elem778;
-            for (int _i779 = 0; _i779 < _list777.size; ++_i779)
+            org.apache.thrift.protocol.TList _list785 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list785.size);
+            Partition _elem786;
+            for (int _i787 = 0; _i787 < _list785.size; ++_i787)
             {
-              _elem778 = new Partition();
-              _elem778.read(iprot);
-              struct.new_parts.add(_elem778);
+              _elem786 = new Partition();
+              _elem786.read(iprot);
+              struct.new_parts.add(_elem786);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -61101,14 +62333,14 @@ public class ThriftHiveMetastore {
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list780 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<PartitionSpec>(_list780.size);
-                  PartitionSpec _elem781;
-                  for (int _i782 = 0; _i782 < _list780.size; ++_i782)
+                  org.apache.thrift.protocol.TList _list788 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<PartitionSpec>(_list788.size);
+                  PartitionSpec _elem789;
+                  for (int _i790 = 0; _i790 < _list788.size; ++_i790)
                   {
-                    _elem781 = new PartitionSpec();
-                    _elem781.read(iprot);
-                    struct.new_parts.add(_elem781);
+                    _elem789 = new PartitionSpec();
+                    _elem789.read(iprot);
+                    struct.new_parts.add(_elem789);
                   }
                   iprot.readListEnd();
                 }
@@ -61134,9 +62366,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (PartitionSpec _iter783 : struct.new_parts)
+            for (PartitionSpec _iter791 : struct.new_parts)
             {
-              _iter783.write(oprot);
+              _iter791.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -61167,9 +62399,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (PartitionSpec _iter784 : struct.new_parts)
+            for (PartitionSpec _iter792 : struct.new_parts)
             {
-              _iter784.write(oprot);
+              _iter792.write(oprot);
             }
           }
         }
@@ -61181,14 +62413,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list785 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<PartitionSpec>(_list785.size);
-            PartitionSpec _elem786;
-            for (int _i787 = 0; _i787 < _list785.size; ++_i787)
+            org.apache.thrift.protocol.TList _list793 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<PartitionSpec>(_list793.size);
+            PartitionSpec _elem794;
+            for (int _i795 = 0; _i795 < _list793.size; ++_i795)
             {
-              _elem786 = new PartitionSpec();
-              _elem786.read(iprot);
-              struct.new_parts.add(_elem786);
+              _elem794 = new PartitionSpec();
+              _elem794.read(iprot);
+              struct.new_parts.add(_elem794);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -62364,13 +63596,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list788 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list788.size);
-                  String _elem789;
-                  for (int _i790 = 0; _i790 < _list788.size; ++_i790)
+                  org.apache.thrift.protocol.TList _list796 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list796.size);
+                  String _elem797;
+                  for (int _i798 = 0; _i798 < _list796.size; ++_i798)
                   {
-                    _elem789 = iprot.readString();
-                    struct.part_vals.add(_elem789);
+                    _elem797 = iprot.readString();
+                    struct.part_vals.add(_elem797);
                   }
                   iprot.readListEnd();
                 }
@@ -62406,9 +63638,9 @@ public class

<TRUNCATED>

[09/15] hive git commit: HIVE-14806: Support UDTF in CBO (AST return path) (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-14806: Support UDTF in CBO (AST return path) (Pengcheng Xiong, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hive-14535
Commit: efe9c84e7bd30de537b0b2c0a224cb47b16b7618
Parents: 21a0142
Author: Pengcheng Xiong <px...@apache.org>
Authored: Tue Oct 4 11:00:00 2016 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Tue Oct 4 11:00:00 2016 -0700

----------------------------------------------------------------------
 .../reloperators/HiveTableFunctionScan.java     |  77 +++++++
 .../calcite/translator/ASTConverter.java        |  83 ++++++--
 .../translator/SqlFunctionConverter.java        |   9 +
 .../hadoop/hive/ql/parse/CalcitePlanner.java    | 211 +++++++++++++++++--
 .../clientpositive/allcolref_in_udf.q.out       |  80 ++++---
 .../clientpositive/lateral_view_noalias.q.out   |  12 +-
 .../test/results/clientpositive/ppd_udtf.q.out  |   4 +-
 .../results/clientpositive/udf_inline.q.out     |  12 +-
 .../results/clientpositive/udtf_explode.q.out   |  76 ++++---
 .../clientpositive/udtf_json_tuple.q.out        |  12 +-
 .../clientpositive/udtf_parse_url_tuple.q.out   |  12 +-
 11 files changed, 478 insertions(+), 110 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/efe9c84e/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableFunctionScan.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableFunctionScan.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableFunctionScan.java
new file mode 100644
index 0000000..bf4896d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableFunctionScan.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.optimizer.calcite.reloperators;
+
+import java.lang.reflect.Type;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableFunctionScan;
+import org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
+
+public class HiveTableFunctionScan extends TableFunctionScan implements HiveRelNode {
+
+  /**
+   * @param cluster
+   *          cluster - Cluster that this relational expression belongs to
+   * @param traitSet
+   * @param inputs
+   *          inputs - 0 or more relational inputs
+   * @param rexCall
+   *          rexCall - Function invocation expression
+   * @param elementType
+   *          elementType - Element type of the collection that will implement
+   *          this table
+   * @param rowType
+   *          rowType - Row type produced by function
+   * @param columnMappings
+   *          columnMappings - Column mappings associated with this function
+   */
+  public HiveTableFunctionScan(RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs,
+      RexNode rexCall, Type elementType, RelDataType rowType, Set<RelColumnMapping> columnMappings) {
+    super(cluster, traitSet, inputs, rexCall, elementType, rowType, columnMappings);
+  }
+
+  public static HiveTableFunctionScan create(RelOptCluster cluster, RelTraitSet traitSet,
+      List<RelNode> inputs, RexNode rexCall, Type elementType, RelDataType rowType,
+      Set<RelColumnMapping> columnMappings) throws CalciteSemanticException {
+    HiveTableFunctionScan hiveTableFunctionScan = new HiveTableFunctionScan(cluster, traitSet,
+        inputs, rexCall, elementType, rowType, columnMappings);
+    return hiveTableFunctionScan;
+  }
+
+  @Override
+  public TableFunctionScan copy(RelTraitSet traitSet, List<RelNode> inputs, RexNode rexCall,
+      Type elementType, RelDataType rowType, Set<RelColumnMapping> columnMappings) {
+    HiveTableFunctionScan htfs = new HiveTableFunctionScan(getCluster(), traitSet, inputs, rexCall,
+        elementType, rowType, columnMappings);
+    return htfs;
+  }
+
+  @Override
+  public void implement(Implementor implementor) {
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/efe9c84e/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
index 9f5e733..8d738aa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
@@ -35,6 +35,7 @@ import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.SemiJoin;
 import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableFunctionScan;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.core.Union;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -60,6 +61,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidQuery;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableFunctionScan;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.SqlFunctionConverter.HiveToken;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
@@ -79,7 +81,7 @@ public class ASTConverter {
   private Filter           where;
   private Aggregate        groupBy;
   private Filter           having;
-  private Project          select;
+  private RelNode          select;
   private Sort             orderLimit;
 
   private Schema           schema;
@@ -192,25 +194,50 @@ public class ASTConverter {
      */
     ASTBuilder b = ASTBuilder.construct(HiveParser.TOK_SELECT, "TOK_SELECT");
 
-    if (select.getChildExps().isEmpty()) {
-      RexLiteral r = select.getCluster().getRexBuilder().makeExactLiteral(new BigDecimal(1));
-      ASTNode selectExpr = ASTBuilder.selectExpr(ASTBuilder.literal(r), "1");
-      b.add(selectExpr);
-    } else {
-      int i = 0;
+    if (select instanceof Project) {
+      if (select.getChildExps().isEmpty()) {
+        RexLiteral r = select.getCluster().getRexBuilder().makeExactLiteral(new BigDecimal(1));
+        ASTNode selectExpr = ASTBuilder.selectExpr(ASTBuilder.literal(r), "1");
+        b.add(selectExpr);
+      } else {
+        int i = 0;
 
-      for (RexNode r : select.getChildExps()) {
+        for (RexNode r : select.getChildExps()) {
+          if (RexUtil.isNull(r) && r.getType().getSqlTypeName() != SqlTypeName.NULL) {
+            // It is NULL value with different type, we need to introduce a CAST
+            // to keep it
+            r = select.getCluster().getRexBuilder().makeAbstractCast(r.getType(), r);
+          }
+          ASTNode expr = r.accept(new RexVisitor(schema, r instanceof RexLiteral));
+          String alias = select.getRowType().getFieldNames().get(i++);
+          ASTNode selectExpr = ASTBuilder.selectExpr(expr, alias);
+          b.add(selectExpr);
+        }
+      }
+      hiveAST.select = b.node();
+    } else {
+      // select is UDTF
+      HiveTableFunctionScan udtf = (HiveTableFunctionScan) select;
+      List<ASTNode> children = new ArrayList<>();
+      RexCall call = (RexCall) udtf.getCall();
+      for (RexNode r : call.getOperands()) {
         if (RexUtil.isNull(r) && r.getType().getSqlTypeName() != SqlTypeName.NULL) {
-          // It is NULL value with different type, we need to introduce a CAST to keep it
+          // It is NULL value with different type, we need to introduce a CAST
+          // to keep it
           r = select.getCluster().getRexBuilder().makeAbstractCast(r.getType(), r);
         }
         ASTNode expr = r.accept(new RexVisitor(schema, r instanceof RexLiteral));
-        String alias = select.getRowType().getFieldNames().get(i++);
-        ASTNode selectExpr = ASTBuilder.selectExpr(expr, alias);
-        b.add(selectExpr);
+        children.add(expr);
+      }
+      ASTBuilder sel = ASTBuilder.construct(HiveParser.TOK_SELEXPR, "TOK_SELEXPR");
+      ASTNode function = buildUDTFAST(call.getOperator().getName(), children);
+      sel.add(function);
+      for (String alias : udtf.getRowType().getFieldNames()) {
+        sel.add(HiveParser.Identifier, alias);
       }
+      b.add(sel);
+      hiveAST.select = b.node();
     }
-    hiveAST.select = b.node();
 
     /*
      * 7. Order Use in Order By from the block above. RelNode has no pointer to
@@ -224,6 +251,14 @@ public class ASTConverter {
     return hiveAST.getAST();
   }
 
+  private ASTNode buildUDTFAST(String functionName, List<ASTNode> children) {
+    ASTNode node = (ASTNode) ParseDriver.adaptor.create(HiveParser.TOK_FUNCTION, "TOK_FUNCTION");
+    node.addChild((ASTNode) ParseDriver.adaptor.create(HiveParser.Identifier, functionName));
+    for (ASTNode c : children) {
+      ParseDriver.adaptor.addChild(node, c);
+    }
+    return node;
+  }
   private void convertOrderLimitToASTNode(HiveSortLimit order) {
     if (order != null) {
       HiveSortLimit hiveSortLimit = order;
@@ -296,7 +331,11 @@ public class ASTConverter {
   }
 
   private Schema getRowSchema(String tblAlias) {
-    return new Schema(select, tblAlias);
+    if (select instanceof Project) {
+      return new Schema((Project) select, tblAlias);
+    } else {
+      return new Schema((TableFunctionScan) select, tblAlias);
+    }
   }
 
   private QueryBlockInfo convertSource(RelNode r) throws CalciteSemanticException {
@@ -375,6 +414,14 @@ public class ASTConverter {
       }
     }
 
+    public void handle(TableFunctionScan tableFunctionScan) {
+      if (ASTConverter.this.select == null) {
+        ASTConverter.this.select = tableFunctionScan;
+      } else {
+        ASTConverter.this.from = tableFunctionScan;
+      }
+    }
+
     @Override
     public void visit(RelNode node, int ordinal, RelNode parent) {
 
@@ -384,6 +431,8 @@ public class ASTConverter {
         handle((Filter) node);
       } else if (node instanceof Project) {
         handle((Project) node);
+      } else if (node instanceof TableFunctionScan) {
+        handle((TableFunctionScan) node);
       } else if (node instanceof Join) {
         ASTConverter.this.from = node;
       } else if (node instanceof Union) {
@@ -644,6 +693,12 @@ public class ASTConverter {
       }
     }
 
+    Schema(TableFunctionScan select, String alias) {
+      for (RelDataTypeField field : select.getRowType().getFieldList()) {
+        add(new ColumnInfo(alias, field.getName()));
+      }
+    }
+
     Schema(Union unionRel, String alias) {
       for (RelDataTypeField field : unionRel.getRowType().getFieldList()) {
         add(new ColumnInfo(alias, field.getName()));

http://git-wip-us.apache.org/repos/asf/hive/blob/efe9c84e/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
index 53e4a2a..f150132 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNegative;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPPositive;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF;
 import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
@@ -108,6 +109,14 @@ public class SqlFunctionConverter {
     return getCalciteFn(name, calciteArgTypes, retType, FunctionRegistry.isDeterministic(hiveUDF));
   }
 
+  public static SqlOperator getCalciteOperator(String funcTextName, GenericUDTF hiveUDTF,
+      ImmutableList<RelDataType> calciteArgTypes, RelDataType retType) throws SemanticException {
+    // We could just do toLowerCase here and let SA qualify it, but
+    // let's be proper...
+    String name = FunctionRegistry.getNormalizedFunctionName(funcTextName);
+    return getCalciteFn(name, calciteArgTypes, retType, false);
+  }
+
   public static GenericUDF getHiveUDF(SqlOperator op, RelDataType dt, int argsLength) {
     String name = reverseOperatorMap.get(op);
     if (name == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/efe9c84e/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index e2ddb14..e6ab947 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -91,6 +91,7 @@ import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlWindow;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeName;
@@ -144,6 +145,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveRelNode;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableFunctionScan;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveUnion;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveAggregateJoinTransposeRule;
@@ -208,8 +210,11 @@ import org.apache.hadoop.hive.ql.plan.SelectDesc;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.Mode;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
@@ -3017,24 +3022,70 @@ public class CalcitePlanner extends SemanticAnalyzer {
         throw new CalciteSemanticException(msg, UnsupportedFeature.Select_transform);
       }
 
-      // 5. Bailout if select involves UDTF
+      // 5. Check if select involves UDTF
+      String udtfTableAlias = null;
+      GenericUDTF genericUDTF = null;
+      String genericUDTFName = null;
+      ArrayList<String> udtfColAliases = new ArrayList<String>();
       ASTNode expr = (ASTNode) selExprList.getChild(posn).getChild(0);
       int exprType = expr.getType();
       if (exprType == HiveParser.TOK_FUNCTION || exprType == HiveParser.TOK_FUNCTIONSTAR) {
         String funcName = TypeCheckProcFactory.DefaultExprProcessor.getFunctionText(expr, true);
         FunctionInfo fi = FunctionRegistry.getFunctionInfo(funcName);
         if (fi != null && fi.getGenericUDTF() != null) {
-          String msg = String.format("UDTF " + funcName + " is currently not supported in CBO,"
-              + " turn off cbo to use UDTF " + funcName);
-          LOG.debug(msg);
-          throw new CalciteSemanticException(msg, UnsupportedFeature.UDTF);
+          LOG.debug("Find UDTF " + funcName);
+          genericUDTF = fi.getGenericUDTF();
+          genericUDTFName = funcName;
+          if (genericUDTF != null && (selectStar = exprType == HiveParser.TOK_FUNCTIONSTAR)) {
+            genColListRegex(".*", null, (ASTNode) expr.getChild(0),
+                col_list, null, inputRR, starRR, pos, out_rwsch, qb.getAliases(), false);
+          }
+        }
+      }
+
+      if (genericUDTF != null) {
+        // Only support a single expression when it's a UDTF
+        if (selExprList.getChildCount() > 1) {
+          throw new SemanticException(generateErrorMessage(
+              (ASTNode) selExprList.getChild(1),
+              ErrorMsg.UDTF_MULTIPLE_EXPR.getMsg()));
+        }
+
+        ASTNode selExpr = (ASTNode) selExprList.getChild(posn);
+
+        // Get the column / table aliases from the expression. Start from 1 as
+        // 0 is the TOK_FUNCTION
+        // column names also can be inferred from result of UDTF
+        for (int i = 1; i < selExpr.getChildCount(); i++) {
+          ASTNode selExprChild = (ASTNode) selExpr.getChild(i);
+          switch (selExprChild.getType()) {
+          case HiveParser.Identifier:
+            udtfColAliases.add(unescapeIdentifier(selExprChild.getText().toLowerCase()));
+            break;
+          case HiveParser.TOK_TABALIAS:
+            assert (selExprChild.getChildCount() == 1);
+            udtfTableAlias = unescapeIdentifier(selExprChild.getChild(0)
+                .getText());
+            qb.addAlias(udtfTableAlias);
+            break;
+          default:
+            throw new SemanticException("Find invalid token type " + selExprChild.getType()
+                + " in UDTF.");
+          }
         }
+        LOG.debug("UDTF table alias is " + udtfTableAlias);
+        LOG.debug("UDTF col aliases are " + udtfColAliases);
       }
 
       // 6. Iterate over all expression (after SELECT)
-      ASTNode exprList = selExprList;
-      int startPosn = posn;
-      List<String> tabAliasesForAllProjs = getTabAliases(starRR);
+      ASTNode exprList;
+      if (genericUDTF != null) {
+        exprList = expr;
+      } else {
+        exprList = selExprList;
+      }
+      // For UDTF's, skip the function name to get the expressions
+      int startPosn = genericUDTF != null ? posn + 1 : posn;
       for (int i = startPosn; i < exprList.getChildCount(); ++i) {
 
         // 6.1 child can be EXPR AS ALIAS, or EXPR.
@@ -3045,7 +3096,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         // This check is not needed and invalid when there is a transform b/c
         // the
         // AST's are slightly different.
-        if (child.getChildCount() > 2) {
+        if (genericUDTF == null && child.getChildCount() > 2) {
           throw new SemanticException(SemanticAnalyzer.generateErrorMessage(
               (ASTNode) child.getChild(2), ErrorMsg.INVALID_AS.getMsg()));
         }
@@ -3053,12 +3104,18 @@ public class CalcitePlanner extends SemanticAnalyzer {
         String tabAlias;
         String colAlias;
 
-        // 6.3 Get rid of TOK_SELEXPR
-        expr = (ASTNode) child.getChild(0);
-        String[] colRef = SemanticAnalyzer.getColAlias(child, getAutogenColAliasPrfxLbl(), inputRR,
-            autogenColAliasPrfxIncludeFuncName(), i);
-        tabAlias = colRef[0];
-        colAlias = colRef[1];
+        if (genericUDTF != null) {
+          tabAlias = null;
+          colAlias = getAutogenColAliasPrfxLbl() + i;
+          expr = child;
+        } else {
+          // 6.3 Get rid of TOK_SELEXPR
+          expr = (ASTNode) child.getChild(0);
+          String[] colRef = SemanticAnalyzer.getColAlias(child, getAutogenColAliasPrfxLbl(),
+              inputRR, autogenColAliasPrfxIncludeFuncName(), i);
+          tabAlias = colRef[0];
+          colAlias = colRef[1];
+        }
 
         // 6.4 Build ExprNode corresponding to colums
         if (expr.getType() == HiveParser.TOK_ALLCOLREF) {
@@ -3143,7 +3200,16 @@ public class CalcitePlanner extends SemanticAnalyzer {
       }
 
       // 8. Build Calcite Rel
-      RelNode outputRel = genSelectRelNode(calciteColLst, out_rwsch, srcRel);
+      RelNode outputRel = null;
+      if (genericUDTF != null) {
+        // The basic idea for CBO support of UDTF is to treat UDTF as a special project.
+        // In AST return path, as we just need to generate a SEL_EXPR, we just need to remember the expressions and the alias.
+        // In OP return path, we need to generate a SEL and then a UDTF following old semantic analyzer.
+        outputRel = genUDTFPlan(genericUDTF, genericUDTFName, udtfTableAlias, udtfColAliases, qb, calciteColLst, out_rwsch, srcRel);
+      }
+      else{
+        outputRel = genSelectRelNode(calciteColLst, out_rwsch, srcRel);
+      }
 
       // 9. Handle select distinct as GBY if there exist windowing functions
       if (selForWindow != null && selExprList.getToken().getType() == HiveParser.TOK_SELECTDI) {
@@ -3165,6 +3231,119 @@ public class CalcitePlanner extends SemanticAnalyzer {
       return outputRel;
     }
 
+    private RelNode genUDTFPlan(GenericUDTF genericUDTF, String genericUDTFName, String outputTableAlias,
+        ArrayList<String> colAliases, QB qb, List<RexNode> selectColLst, RowResolver selectRR, RelNode input) throws SemanticException {
+
+      // No GROUP BY / DISTRIBUTE BY / SORT BY / CLUSTER BY
+      QBParseInfo qbp = qb.getParseInfo();
+      if (!qbp.getDestToGroupBy().isEmpty()) {
+        throw new SemanticException(ErrorMsg.UDTF_NO_GROUP_BY.getMsg());
+      }
+      if (!qbp.getDestToDistributeBy().isEmpty()) {
+        throw new SemanticException(ErrorMsg.UDTF_NO_DISTRIBUTE_BY.getMsg());
+      }
+      if (!qbp.getDestToSortBy().isEmpty()) {
+        throw new SemanticException(ErrorMsg.UDTF_NO_SORT_BY.getMsg());
+      }
+      if (!qbp.getDestToClusterBy().isEmpty()) {
+        throw new SemanticException(ErrorMsg.UDTF_NO_CLUSTER_BY.getMsg());
+      }
+      if (!qbp.getAliasToLateralViews().isEmpty()) {
+        throw new SemanticException(ErrorMsg.UDTF_LATERAL_VIEW.getMsg());
+      }
+
+      LOG.debug("Table alias: " + outputTableAlias + " Col aliases: " + colAliases);
+
+      // Use the RowResolver from the input operator to generate a input
+      // ObjectInspector that can be used to initialize the UDTF. Then, the
+      // resulting output object inspector can be used to make the RowResolver
+      // for the UDTF operator
+      ArrayList<ColumnInfo> inputCols = selectRR.getColumnInfos();
+
+      // Create the object inspector for the input columns and initialize the
+      // UDTF
+      ArrayList<String> colNames = new ArrayList<String>();
+      ObjectInspector[] colOIs = new ObjectInspector[inputCols.size()];
+      for (int i = 0; i < inputCols.size(); i++) {
+        colNames.add(inputCols.get(i).getInternalName());
+        colOIs[i] = inputCols.get(i).getObjectInspector();
+      }
+      StandardStructObjectInspector rowOI = ObjectInspectorFactory
+          .getStandardStructObjectInspector(colNames, Arrays.asList(colOIs));
+      StructObjectInspector outputOI = genericUDTF.initialize(rowOI);
+
+      int numUdtfCols = outputOI.getAllStructFieldRefs().size();
+      if (colAliases.isEmpty()) {
+        // user did not specfied alias names, infer names from outputOI
+        for (StructField field : outputOI.getAllStructFieldRefs()) {
+          colAliases.add(field.getFieldName());
+        }
+      }
+      // Make sure that the number of column aliases in the AS clause matches
+      // the number of columns output by the UDTF
+      int numSuppliedAliases = colAliases.size();
+      if (numUdtfCols != numSuppliedAliases) {
+        throw new SemanticException(ErrorMsg.UDTF_ALIAS_MISMATCH.getMsg("expected " + numUdtfCols
+            + " aliases " + "but got " + numSuppliedAliases));
+      }
+
+      // Generate the output column info's / row resolver using internal names.
+      ArrayList<ColumnInfo> udtfCols = new ArrayList<ColumnInfo>();
+
+      Iterator<String> colAliasesIter = colAliases.iterator();
+      for (StructField sf : outputOI.getAllStructFieldRefs()) {
+
+        String colAlias = colAliasesIter.next();
+        assert (colAlias != null);
+
+        // Since the UDTF operator feeds into a LVJ operator that will rename
+        // all the internal names, we can just use field name from the UDTF's OI
+        // as the internal name
+        ColumnInfo col = new ColumnInfo(sf.getFieldName(),
+            TypeInfoUtils.getTypeInfoFromObjectInspector(sf.getFieldObjectInspector()),
+            outputTableAlias, false);
+        udtfCols.add(col);
+      }
+
+      // Create the row resolver for this operator from the output columns
+      RowResolver out_rwsch = new RowResolver();
+      for (int i = 0; i < udtfCols.size(); i++) {
+        out_rwsch.put(outputTableAlias, colAliases.get(i), udtfCols.get(i));
+      }
+
+      // Add the UDTFOperator to the operator DAG
+      RelTraitSet traitSet = TraitsUtil.getDefaultTraitSet(cluster);
+
+      // Build row type from field <type, name>
+      RelDataType retType = TypeConverter.getType(cluster, out_rwsch, null);
+
+      Builder<RelDataType> argTypeBldr = ImmutableList.<RelDataType> builder();
+
+      RexBuilder rexBuilder = cluster.getRexBuilder();
+      RelDataTypeFactory dtFactory = rexBuilder.getTypeFactory();
+      RowSchema rs = selectRR.getRowSchema();
+      for (ColumnInfo ci : rs.getSignature()) {
+        argTypeBldr.add(TypeConverter.convert(ci.getType(), dtFactory));
+      }
+ 
+      SqlOperator calciteOp = SqlFunctionConverter.getCalciteOperator(genericUDTFName, genericUDTF,
+             argTypeBldr.build(), retType);
+
+      // Hive UDTF only has a single input
+      List<RelNode> list = new ArrayList<>();
+      list.add(input);
+
+      RexNode rexNode = cluster.getRexBuilder().makeCall(calciteOp, selectColLst);
+
+      RelNode udtf = HiveTableFunctionScan.create(cluster, traitSet, list, rexNode, null, retType,
+          null);
+      // Add new rel & its RR to the maps
+      relToHiveColNameCalcitePosMap.put(udtf, this.buildHiveToCalciteColumnMap(out_rwsch, udtf));
+      relToHiveRR.put(udtf, out_rwsch);
+
+      return udtf;
+    }
+
     private RelNode genLogicalPlan(QBExpr qbexpr) throws SemanticException {
       if (qbexpr.getOpcode() == QBExpr.Opcode.NULLOP) {
         return genLogicalPlan(qbexpr.getQB(), false);

http://git-wip-us.apache.org/repos/asf/hive/blob/efe9c84e/ql/src/test/results/clientpositive/allcolref_in_udf.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/allcolref_in_udf.q.out b/ql/src/test/results/clientpositive/allcolref_in_udf.q.out
index eda49ed..ecd784e 100644
--- a/ql/src/test/results/clientpositive/allcolref_in_udf.q.out
+++ b/ql/src/test/results/clientpositive/allcolref_in_udf.q.out
@@ -85,33 +85,41 @@ STAGE PLANS:
             alias: a
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: ((key + 1) is not null and (key < 100)) (type: boolean)
+              predicate: (UDFToDouble(key) < 100.0) (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator
-                key expressions: (key + 1) (type: double)
-                sort order: +
-                Map-reduce partition columns: (key + 1) (type: double)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
-                value expressions: key (type: string), value (type: string)
+                Reduce Output Operator
+                  key expressions: (UDFToDouble(_col0) + 1.0) (type: double)
+                  sort order: +
+                  Map-reduce partition columns: (UDFToDouble(_col0) + 1.0) (type: double)
+                  Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: string), _col1 (type: string)
           TableScan
             alias: b
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: UDFToDouble(key) is not null (type: boolean)
+              predicate: key is not null (type: boolean)
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator
-                key expressions: UDFToDouble(key) (type: double)
-                sort order: +
-                Map-reduce partition columns: UDFToDouble(key) (type: double)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                value expressions: key (type: string), value (type: string)
+                Reduce Output Operator
+                  key expressions: UDFToDouble(_col0) (type: double)
+                  sort order: +
+                  Map-reduce partition columns: UDFToDouble(_col0) (type: double)
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: string), _col1 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Inner Join 0 to 1
           keys:
-            0 (key + 1) (type: double)
-            1 UDFToDouble(key) (type: double)
+            0 (UDFToDouble(_col0) + 1.0) (type: double)
+            1 UDFToDouble(_col0) (type: double)
           outputColumnNames: _col0, _col1, _col5, _col6
           Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
           Select Operator
@@ -121,16 +129,20 @@ STAGE PLANS:
             UDTF Operator
               Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
               function name: stack
-              Limit
-                Number of rows: 10
-                Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
-                File Output Operator
-                  compressed: false
+              Select Operator
+                expressions: col0 (type: string), col1 (type: string), col2 (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                Limit
+                  Number of rows: 10
                   Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
@@ -195,16 +207,20 @@ STAGE PLANS:
               UDTF Operator
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 function name: explode
-                Limit
-                  Number of rows: 10
-                  Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
+                Select Operator
+                  expressions: col (type: string)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Limit
+                    Number of rows: 10
                     Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/efe9c84e/ql/src/test/results/clientpositive/lateral_view_noalias.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/lateral_view_noalias.q.out b/ql/src/test/results/clientpositive/lateral_view_noalias.q.out
index 7c08b86..e8d23d4 100644
--- a/ql/src/test/results/clientpositive/lateral_view_noalias.q.out
+++ b/ql/src/test/results/clientpositive/lateral_view_noalias.q.out
@@ -81,10 +81,14 @@ STAGE PLANS:
             UDTF Operator
               Statistics: Num rows: 500 Data size: 192000 Basic stats: COMPLETE Column stats: COMPLETE
               function name: explode
-              Limit
-                Number of rows: 2
-                Statistics: Num rows: 2 Data size: 768 Basic stats: COMPLETE Column stats: COMPLETE
-                ListSink
+              Select Operator
+                expressions: key (type: string), value (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 500 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                Limit
+                  Number of rows: 2
+                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                  ListSink
 
 PREHOOK: query: SELECT explode(map('key1', 100, 'key2', 200)) from src limit 2
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/efe9c84e/ql/src/test/results/clientpositive/ppd_udtf.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ppd_udtf.q.out b/ql/src/test/results/clientpositive/ppd_udtf.q.out
index 544b80e..d008526 100644
--- a/ql/src/test/results/clientpositive/ppd_udtf.q.out
+++ b/ql/src/test/results/clientpositive/ppd_udtf.q.out
@@ -24,7 +24,7 @@ STAGE PLANS:
             alias: src
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: (key > 400) (type: boolean)
+              predicate: (UDFToDouble(key) > 400.0) (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: array(key,value) (type: array<string>)
@@ -34,7 +34,7 @@ STAGE PLANS:
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                   function name: explode
                   Filter Operator
-                    predicate: (col < 450) (type: boolean)
+                    predicate: (UDFToDouble(col) < 450.0) (type: boolean)
                     Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: col (type: string)

http://git-wip-us.apache.org/repos/asf/hive/blob/efe9c84e/ql/src/test/results/clientpositive/udf_inline.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_inline.q.out b/ql/src/test/results/clientpositive/udf_inline.q.out
index f986abf..dca41d9 100644
--- a/ql/src/test/results/clientpositive/udf_inline.q.out
+++ b/ql/src/test/results/clientpositive/udf_inline.q.out
@@ -37,10 +37,14 @@ STAGE PLANS:
             UDTF Operator
               Statistics: Num rows: 500 Data size: 32000 Basic stats: COMPLETE Column stats: COMPLETE
               function name: inline
-              Limit
-                Number of rows: 2
-                Statistics: Num rows: 2 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
-                ListSink
+              Select Operator
+                expressions: col1 (type: int), col2 (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 500 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                Limit
+                  Number of rows: 2
+                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                  ListSink
 
 PREHOOK: query: SELECT inline( 
   ARRAY(

http://git-wip-us.apache.org/repos/asf/hive/blob/efe9c84e/ql/src/test/results/clientpositive/udtf_explode.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udtf_explode.q.out b/ql/src/test/results/clientpositive/udtf_explode.q.out
index de7a2f7..f00c9f4 100644
--- a/ql/src/test/results/clientpositive/udtf_explode.q.out
+++ b/ql/src/test/results/clientpositive/udtf_explode.q.out
@@ -31,10 +31,14 @@ STAGE PLANS:
             UDTF Operator
               Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE
               function name: explode
-              Limit
-                Number of rows: 3
-                Statistics: Num rows: 3 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
-                ListSink
+              Select Operator
+                expressions: col (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 500 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                Limit
+                  Number of rows: 3
+                  Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                  ListSink
 
 PREHOOK: query: EXPLAIN EXTENDED SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3) a GROUP BY a.myCol
 PREHOOK: type: QUERY
@@ -60,16 +64,20 @@ STAGE PLANS:
               UDTF Operator
                 Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE
                 function name: explode
-                Limit
-                  Number of rows: 3
-                  Statistics: Num rows: 3 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    null sort order: 
-                    sort order: 
-                    Statistics: Num rows: 3 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
-                    tag: -1
-                    value expressions: col (type: int)
-                    auto parallelism: false
+                Select Operator
+                  expressions: col (type: int)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 500 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Limit
+                    Number of rows: 3
+                    Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      null sort order: 
+                      sort order: 
+                      Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                      tag: -1
+                      value expressions: _col0 (type: int)
+                      auto parallelism: false
       Path -> Alias:
 #### A masked pattern was here ####
       Path -> Partition:
@@ -118,7 +126,7 @@ STAGE PLANS:
               name: default.src
             name: default.src
       Truncated Path -> Alias:
-        /src [a:src]
+        /src [$hdt$_0:$hdt$_0:$hdt$_0:src]
       Needs Tagging: false
       Reduce Operator Tree:
         Select Operator
@@ -283,10 +291,14 @@ STAGE PLANS:
             UDTF Operator
               Statistics: Num rows: 500 Data size: 259500 Basic stats: COMPLETE Column stats: COMPLETE
               function name: explode
-              Limit
-                Number of rows: 3
-                Statistics: Num rows: 3 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
-                ListSink
+              Select Operator
+                expressions: key (type: int), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 500 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                Limit
+                  Number of rows: 3
+                  Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                  ListSink
 
 PREHOOK: query: EXPLAIN EXTENDED SELECT a.myKey, a.myVal, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) as (myKey,myVal) FROM src LIMIT 3) a GROUP BY a.myKey, a.myVal
 PREHOOK: type: QUERY
@@ -312,16 +324,20 @@ STAGE PLANS:
               UDTF Operator
                 Statistics: Num rows: 500 Data size: 259500 Basic stats: COMPLETE Column stats: COMPLETE
                 function name: explode
-                Limit
-                  Number of rows: 3
-                  Statistics: Num rows: 3 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    null sort order: 
-                    sort order: 
-                    Statistics: Num rows: 3 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
-                    tag: -1
-                    value expressions: key (type: int), value (type: string)
-                    auto parallelism: false
+                Select Operator
+                  expressions: key (type: int), value (type: string)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 500 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Limit
+                    Number of rows: 3
+                    Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      null sort order: 
+                      sort order: 
+                      Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                      tag: -1
+                      value expressions: _col0 (type: int), _col1 (type: string)
+                      auto parallelism: false
       Path -> Alias:
 #### A masked pattern was here ####
       Path -> Partition:
@@ -370,7 +386,7 @@ STAGE PLANS:
               name: default.src
             name: default.src
       Truncated Path -> Alias:
-        /src [a:src]
+        /src [$hdt$_0:$hdt$_0:$hdt$_0:src]
       Needs Tagging: false
       Reduce Operator Tree:
         Select Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/efe9c84e/ql/src/test/results/clientpositive/udtf_json_tuple.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udtf_json_tuple.q.out b/ql/src/test/results/clientpositive/udtf_json_tuple.q.out
index e0d4f00..5be6eb0 100644
--- a/ql/src/test/results/clientpositive/udtf_json_tuple.q.out
+++ b/ql/src/test/results/clientpositive/udtf_json_tuple.q.out
@@ -153,11 +153,15 @@ STAGE PLANS:
               UDTF Operator
                 Statistics: Num rows: 6 Data size: 236 Basic stats: COMPLETE Column stats: NONE
                 function name: json_tuple
-                Reduce Output Operator
-                  key expressions: c0 (type: string), c1 (type: string), c2 (type: string)
-                  sort order: +++
+                Select Operator
+                  expressions: c0 (type: string), c1 (type: string), c2 (type: string), c3 (type: string), c4 (type: string)
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4
                   Statistics: Num rows: 6 Data size: 236 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: c3 (type: string), c4 (type: string)
+                  Reduce Output Operator
+                    key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+                    sort order: +++
+                    Statistics: Num rows: 6 Data size: 236 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col3 (type: string), _col4 (type: string)
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string)

http://git-wip-us.apache.org/repos/asf/hive/blob/efe9c84e/ql/src/test/results/clientpositive/udtf_parse_url_tuple.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udtf_parse_url_tuple.q.out b/ql/src/test/results/clientpositive/udtf_parse_url_tuple.q.out
index ad5ea21..ab6cee7 100644
--- a/ql/src/test/results/clientpositive/udtf_parse_url_tuple.q.out
+++ b/ql/src/test/results/clientpositive/udtf_parse_url_tuple.q.out
@@ -170,11 +170,15 @@ STAGE PLANS:
               UDTF Operator
                 Statistics: Num rows: 6 Data size: 213 Basic stats: COMPLETE Column stats: NONE
                 function name: parse_url_tuple
-                Reduce Output Operator
-                  key expressions: c0 (type: string), c1 (type: string), c2 (type: string)
-                  sort order: +++
+                Select Operator
+                  expressions: c0 (type: string), c1 (type: string), c2 (type: string), c3 (type: string), c4 (type: string), c5 (type: string), c6 (type: string), c7 (type: string), c8 (type: string)
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                   Statistics: Num rows: 6 Data size: 213 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: c3 (type: string), c4 (type: string), c5 (type: string), c6 (type: string), c7 (type: string), c8 (type: string)
+                  Reduce Output Operator
+                    key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+                    sort order: +++
+                    Statistics: Num rows: 6 Data size: 213 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string), _col8 (type: string)
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: string), VALUE._col5 (type: string)


[08/15] hive git commit: HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)

Posted by se...@apache.org.
HIVE-14558: Add support for listing views similar to "show tables" (Naveen Gangam, reviewed by Aihua Xu)


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

Branch: refs/heads/hive-14535
Commit: 21a0142f333fba231f2648db53a48dc41384ad72
Parents: e19f0e3
Author: Aihua Xu <ai...@apache.org>
Authored: Tue Oct 4 09:53:12 2016 -0400
Committer: Aihua Xu <ai...@apache.org>
Committed: Tue Oct 4 09:53:12 2016 -0400

----------------------------------------------------------------------
 metastore/if/hive_metastore.thrift              |    1 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2163 +++++++-----
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  148 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../hive/metastore/api/ThriftHiveMetastore.java | 3184 ++++++++++++------
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1341 +++++---
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       |  938 ++++--
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   65 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   23 +
 .../hive/metastore/HiveMetaStoreClient.java     |   12 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |   15 +
 .../hadoop/hive/metastore/ObjectStore.java      |    8 +
 .../apache/hadoop/hive/metastore/RawStore.java  |    4 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |    6 +
 .../DummyRawStoreControlledCommit.java          |    6 +
 .../DummyRawStoreForJdoConnection.java          |    6 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   45 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   46 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   43 +
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |    1 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |    2 +
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |    2 +
 .../org/apache/hadoop/hive/ql/plan/DDLWork.java |    2 +-
 .../hadoop/hive/ql/plan/HiveOperation.java      |    1 +
 .../hadoop/hive/ql/plan/ShowTablesDesc.java     |   29 +
 .../authorization/plugin/HiveOperationType.java |    1 +
 .../plugin/sqlstd/Operation2Privilege.java      |    2 +
 ql/src/test/queries/clientpositive/show_views.q |   56 +
 .../results/clientpositive/show_views.q.out     |  320 ++
 30 files changed, 5774 insertions(+), 2708 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/21a0142f/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index 872c0f3..c5ba309 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -1030,6 +1030,7 @@ service ThriftHiveMetastore extends fb303.FacebookService
       4:EnvironmentContext environment_context)
                        throws(1:NoSuchObjectException o1, 2:MetaException o3)
   list<string> get_tables(1: string db_name, 2: string pattern) throws (1: MetaException o1)
+  list<string> get_tables_by_type(1: string db_name, 2: string pattern, 3: string tableType) throws (1: MetaException o1)
   list<TableMeta> get_table_meta(1: string db_patterns, 2: string tbl_patterns, 3: list<string> tbl_types)
                        throws (1: MetaException o1)
   list<string> get_all_tables(1: string db_name) throws (1: MetaException o1)