You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vg...@apache.org on 2018/03/04 07:25:55 UTC

[01/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Repository: hive
Updated Branches:
  refs/heads/master 05d4719ee -> 4047befe4


http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index 167bda2..b0e84d9 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -318,13 +318,13 @@ module ThriftHiveMetastore
       return
     end
 
-    def create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints)
-      send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints)
+    def create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints)
+      send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints)
       recv_create_table_with_constraints()
     end
 
-    def send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints)
-      send_message('create_table_with_constraints', Create_table_with_constraints_args, :tbl => tbl, :primaryKeys => primaryKeys, :foreignKeys => foreignKeys, :uniqueConstraints => uniqueConstraints, :notNullConstraints => notNullConstraints)
+    def send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints)
+      send_message('create_table_with_constraints', Create_table_with_constraints_args, :tbl => tbl, :primaryKeys => primaryKeys, :foreignKeys => foreignKeys, :uniqueConstraints => uniqueConstraints, :notNullConstraints => notNullConstraints, :defaultConstraints => defaultConstraints)
     end
 
     def recv_create_table_with_constraints()
@@ -416,6 +416,22 @@ module ThriftHiveMetastore
       return
     end
 
+    def add_default_constraint(req)
+      send_add_default_constraint(req)
+      recv_add_default_constraint()
+    end
+
+    def send_add_default_constraint(req)
+      send_message('add_default_constraint', Add_default_constraint_args, :req => req)
+    end
+
+    def recv_add_default_constraint()
+      result = receive_message(Add_default_constraint_result)
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      return
+    end
+
     def drop_table(dbname, name, deleteData)
       send_drop_table(dbname, name, deleteData)
       recv_drop_table()
@@ -1622,6 +1638,23 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_not_null_constraints failed: unknown result')
     end
 
+    def get_default_constraints(request)
+      send_get_default_constraints(request)
+      return recv_get_default_constraints()
+    end
+
+    def send_get_default_constraints(request)
+      send_message('get_default_constraints', Get_default_constraints_args, :request => request)
+    end
+
+    def recv_get_default_constraints()
+      result = receive_message(Get_default_constraints_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_default_constraints failed: unknown result')
+    end
+
     def update_table_column_statistics(stats_obj)
       send_update_table_column_statistics(stats_obj)
       return recv_update_table_column_statistics()
@@ -3338,7 +3371,7 @@ module ThriftHiveMetastore
       args = read_args(iprot, Create_table_with_constraints_args)
       result = Create_table_with_constraints_result.new()
       begin
-        @handler.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys, args.uniqueConstraints, args.notNullConstraints)
+        @handler.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys, args.uniqueConstraints, args.notNullConstraints, args.defaultConstraints)
       rescue ::AlreadyExistsException => o1
         result.o1 = o1
       rescue ::InvalidObjectException => o2
@@ -3416,6 +3449,19 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'add_not_null_constraint', seqid)
     end
 
+    def process_add_default_constraint(seqid, iprot, oprot)
+      args = read_args(iprot, Add_default_constraint_args)
+      result = Add_default_constraint_result.new()
+      begin
+        @handler.add_default_constraint(args.req)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'add_default_constraint', seqid)
+    end
+
     def process_drop_table(seqid, iprot, oprot)
       args = read_args(iprot, Drop_table_args)
       result = Drop_table_result.new()
@@ -4363,6 +4409,19 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'get_not_null_constraints', seqid)
     end
 
+    def process_get_default_constraints(seqid, iprot, oprot)
+      args = read_args(iprot, Get_default_constraints_args)
+      result = Get_default_constraints_result.new()
+      begin
+        result.success = @handler.get_default_constraints(args.request)
+      rescue ::MetaException => o1
+        result.o1 = o1
+      rescue ::NoSuchObjectException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'get_default_constraints', seqid)
+    end
+
     def process_update_table_column_statistics(seqid, iprot, oprot)
       args = read_args(iprot, Update_table_column_statistics_args)
       result = Update_table_column_statistics_result.new()
@@ -6074,13 +6133,15 @@ module ThriftHiveMetastore
     FOREIGNKEYS = 3
     UNIQUECONSTRAINTS = 4
     NOTNULLCONSTRAINTS = 5
+    DEFAULTCONSTRAINTS = 6
 
     FIELDS = {
       TBL => {:type => ::Thrift::Types::STRUCT, :name => 'tbl', :class => ::Table},
       PRIMARYKEYS => {:type => ::Thrift::Types::LIST, :name => 'primaryKeys', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLPrimaryKey}},
       FOREIGNKEYS => {:type => ::Thrift::Types::LIST, :name => 'foreignKeys', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLForeignKey}},
       UNIQUECONSTRAINTS => {:type => ::Thrift::Types::LIST, :name => 'uniqueConstraints', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLUniqueConstraint}},
-      NOTNULLCONSTRAINTS => {:type => ::Thrift::Types::LIST, :name => 'notNullConstraints', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLNotNullConstraint}}
+      NOTNULLCONSTRAINTS => {:type => ::Thrift::Types::LIST, :name => 'notNullConstraints', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLNotNullConstraint}},
+      DEFAULTCONSTRAINTS => {:type => ::Thrift::Types::LIST, :name => 'defaultConstraints', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLDefaultConstraint}}
     }
 
     def struct_fields; FIELDS; end
@@ -6283,6 +6344,40 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Add_default_constraint_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQ = 1
+
+    FIELDS = {
+      REQ => {:type => ::Thrift::Types::STRUCT, :name => 'req', :class => ::AddDefaultConstraintRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Add_default_constraint_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchObjectException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Drop_table_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     DBNAME = 1
@@ -9109,6 +9204,42 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Get_default_constraints_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQUEST = 1
+
+    FIELDS = {
+      REQUEST => {:type => ::Thrift::Types::STRUCT, :name => 'request', :class => ::DefaultConstraintsRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_default_constraints_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::DefaultConstraintsResponse},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::NoSuchObjectException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Update_table_column_statistics_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     STATS_OBJ = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index ac71d08..1c422ca 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -1409,13 +1409,13 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         final EnvironmentContext envContext)
             throws AlreadyExistsException, MetaException,
             InvalidObjectException, NoSuchObjectException {
-      create_table_core(ms, tbl, envContext, null, null, null, null);
+      create_table_core(ms, tbl, envContext, null, null, null, null, null);
     }
 
     private void create_table_core(final RawStore ms, final Table tbl,
         final EnvironmentContext envContext, List<SQLPrimaryKey> primaryKeys,
         List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints,
-        List<SQLNotNullConstraint> notNullConstraints)
+        List<SQLNotNullConstraint> notNullConstraints, List<SQLDefaultConstraint> defaultConstraints)
         throws AlreadyExistsException, MetaException,
         InvalidObjectException, NoSuchObjectException {
       if (!MetaStoreUtils.validateName(tbl.getTableName(), conf)) {
@@ -1502,12 +1502,12 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
 
         if (primaryKeys == null && foreignKeys == null
-                && uniqueConstraints == null && notNullConstraints == null) {
+                && uniqueConstraints == null && notNullConstraints == null && defaultConstraints == null) {
           ms.createTable(tbl);
         } else {
           // Set constraint name if null before sending to listener
           List<String> constraintNames = ms.createTableWithConstraints(tbl, primaryKeys, foreignKeys,
-              uniqueConstraints, notNullConstraints);
+              uniqueConstraints, notNullConstraints, defaultConstraints);
           int primaryKeySize = 0;
           if (primaryKeys != null) {
             primaryKeySize = primaryKeys.size();
@@ -1535,6 +1535,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               }
             }
           }
+          int notNullConstraintSize =  notNullConstraints.size();
           if (notNullConstraints != null) {
             for (int i = 0; i < notNullConstraints.size(); i++) {
               if (notNullConstraints.get(i).getNn_name() == null) {
@@ -1542,6 +1543,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               }
             }
           }
+          if (defaultConstraints!= null) {
+            for (int i = 0; i < defaultConstraints.size(); i++) {
+              if (defaultConstraints.get(i).getDc_name() == null) {
+                defaultConstraints.get(i).setDc_name(constraintNames.get(primaryKeySize + foreignKeySize
+                    + uniqueConstraintSize + notNullConstraintSize + i));
+              }
+            }
+          }
         }
 
         if (!transactionalListeners.isEmpty()) {
@@ -1636,14 +1645,15 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     public void create_table_with_constraints(final Table tbl,
         final List<SQLPrimaryKey> primaryKeys, final List<SQLForeignKey> foreignKeys,
         List<SQLUniqueConstraint> uniqueConstraints,
-        List<SQLNotNullConstraint> notNullConstraints)
+        List<SQLNotNullConstraint> notNullConstraints,
+        List<SQLDefaultConstraint> defaultConstraints)
         throws AlreadyExistsException, MetaException, InvalidObjectException {
       startFunction("create_table", ": " + tbl.toString());
       boolean success = false;
       Exception ex = null;
       try {
         create_table_core(getMS(), tbl, null, primaryKeys, foreignKeys,
-            uniqueConstraints, notNullConstraints);
+            uniqueConstraints, notNullConstraints, defaultConstraints);
         success = true;
       } catch (NoSuchObjectException e) {
         ex = e;
@@ -1917,6 +1927,59 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
+    @Override
+    public void add_default_constraint(AddDefaultConstraintRequest req)
+        throws MetaException, InvalidObjectException {
+      List<SQLDefaultConstraint> defaultConstraintCols= req.getDefaultConstraintCols();
+      String constraintName = (defaultConstraintCols != null && defaultConstraintCols.size() > 0) ?
+          defaultConstraintCols.get(0).getDc_name() : "null";
+      startFunction("add_default_constraint", ": " + constraintName);
+      boolean success = false;
+      Exception ex = null;
+      RawStore ms = getMS();
+      try {
+        ms.openTransaction();
+        List<String> constraintNames = ms.addDefaultConstraints(defaultConstraintCols);
+        // Set not null constraint name if null before sending to listener
+        if (defaultConstraintCols != null) {
+          for (int i = 0; i < defaultConstraintCols.size(); i++) {
+            if (defaultConstraintCols.get(i).getDc_name() == null) {
+              defaultConstraintCols.get(i).setDc_name(constraintNames.get(i));
+            }
+          }
+        }
+        if (transactionalListeners.size() > 0) {
+          if (defaultConstraintCols != null && defaultConstraintCols.size() > 0) {
+            //TODO: Even listener for default
+            //AddDefaultConstraintEvent addDefaultConstraintEvent = new AddDefaultConstraintEvent(defaultConstraintCols, true, this);
+            //for (MetaStoreEventListener transactionalListener : transactionalListeners) {
+             // transactionalListener.onAddNotNullConstraint(addDefaultConstraintEvent);
+            //}
+          }
+        }
+        success = ms.commitTransaction();
+      } catch (Exception e) {
+        ex = e;
+        if (e instanceof MetaException) {
+          throw (MetaException) e;
+        } else if (e instanceof InvalidObjectException) {
+          throw (InvalidObjectException) e;
+        } else {
+          throw newMetaException(e);
+        }
+      } finally {
+        if (!success) {
+          ms.rollbackTransaction();
+        } else if (defaultConstraintCols != null && defaultConstraintCols.size() > 0) {
+          for (MetaStoreEventListener listener : listeners) {
+            //AddNotNullConstraintEvent addDefaultConstraintEvent = new AddNotNullConstraintEvent(defaultConstraintCols, true, this);
+            //listener.onAddDefaultConstraint(addDefaultConstraintEvent);
+          }
+        }
+        endFunction("add_default_constraint", success, ex, constraintName);
+      }
+    }
+
     private boolean is_table_exists(RawStore ms, String dbname, String name)
         throws MetaException {
       return (ms.getTable(dbname, name) != null);
@@ -7328,6 +7391,28 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     @Override
+    public DefaultConstraintsResponse get_default_constraints(DefaultConstraintsRequest request)
+        throws TException {
+      String db_name = request.getDb_name();
+      String tbl_name = request.getTbl_name();
+      startTableFunction("get_default_constraints", db_name, tbl_name);
+      List<SQLDefaultConstraint> ret = null;
+      Exception ex = null;
+      try {
+        ret = getMS().getDefaultConstraints(db_name, tbl_name);
+      } catch (Exception e) {
+        ex = e;
+        if (e instanceof MetaException) {
+          throw (MetaException) e;
+        } else {
+          throw newMetaException(e);
+        }
+      } finally {
+        endFunction("get_default_constraints", ret != null, ex, tbl_name);
+      }
+      return new DefaultConstraintsResponse(ret);
+    }
+    @Override
     public String get_metastore_db_uuid() throws TException {
       try {
         return getMS().getMetastoreDbUuid();

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 59c0cff..da0d329 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -801,7 +801,8 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   public void createTableWithConstraints(Table tbl,
     List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
     List<SQLUniqueConstraint> uniqueConstraints,
-    List<SQLNotNullConstraint> notNullConstraints)
+    List<SQLNotNullConstraint> notNullConstraints,
+    List<SQLDefaultConstraint> defaultConstraints)
         throws AlreadyExistsException, InvalidObjectException,
         MetaException, NoSuchObjectException, TException {
     HiveMetaHook hook = getHook(tbl);
@@ -812,7 +813,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     try {
       // Subclasses can override this step (for example, for temporary tables)
       client.create_table_with_constraints(tbl, primaryKeys, foreignKeys,
-          uniqueConstraints, notNullConstraints);
+          uniqueConstraints, notNullConstraints, defaultConstraints);
       if (hook != null) {
         hook.commitCreateTable(tbl);
       }
@@ -854,6 +855,12 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     client.add_not_null_constraint(new AddNotNullConstraintRequest(notNullConstraintCols));
   }
 
+  @Override
+  public void addDefaultConstraint(List<SQLDefaultConstraint> defaultConstraints) throws
+      NoSuchObjectException, MetaException, TException {
+    client.add_default_constraint(new AddDefaultConstraintRequest(defaultConstraints));
+  }
+
   /**
    * @param type
    * @return true or false
@@ -1763,6 +1770,11 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     return client.get_not_null_constraints(req).getNotNullConstraints();
   }
 
+  @Override
+  public List<SQLDefaultConstraint> getDefaultConstraints(DefaultConstraintsRequest req)
+      throws MetaException, NoSuchObjectException, TException {
+    return client.get_default_constraints(req).getDefaultConstraints();
+  }
 
   /** {@inheritDoc} */
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 43aeeb3..037331f 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.DefaultConstraintsRequest;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -90,6 +91,7 @@ import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -1811,11 +1813,15 @@ public interface IMetaStoreClient {
   List<SQLNotNullConstraint> getNotNullConstraints(NotNullConstraintsRequest request) throws MetaException,
     NoSuchObjectException, TException;
 
+  List<SQLDefaultConstraint> getDefaultConstraints(DefaultConstraintsRequest request) throws MetaException,
+      NoSuchObjectException, TException;
+
   void createTableWithConstraints(
     org.apache.hadoop.hive.metastore.api.Table tTbl,
     List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
     List<SQLUniqueConstraint> uniqueConstraints,
-    List<SQLNotNullConstraint> notNullConstraints)
+    List<SQLNotNullConstraint> notNullConstraints,
+    List<SQLDefaultConstraint> defaultConstraints)
     throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, TException;
 
   void dropConstraint(String dbName, String tableName, String constraintName) throws
@@ -1833,6 +1839,9 @@ public interface IMetaStoreClient {
   void addNotNullConstraint(List<SQLNotNullConstraint> notNullConstraintCols) throws
   MetaException, NoSuchObjectException, TException;
 
+  void addDefaultConstraint(List<SQLDefaultConstraint> defaultConstraints) throws
+      MetaException, NoSuchObjectException, TException;
+
   /**
    * Gets the unique id of the backing database instance used for storing metadata
    * @return unique id of the backing database instance

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
index 46412b0..6ead20a 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -2282,4 +2283,64 @@ class MetaStoreDirectSql {
     return ret;
   }
 
+  public List<SQLDefaultConstraint> getDefaultConstraints(String db_name, String tbl_name)
+      throws MetaException {
+    List<SQLDefaultConstraint> ret = new ArrayList<SQLDefaultConstraint>();
+    String queryText =
+        "SELECT " + DBS + ".\"NAME\", " + TBLS + ".\"TBL_NAME\","
+            + "CASE WHEN " + COLUMNS_V2 + ".\"COLUMN_NAME\" IS NOT NULL THEN " + COLUMNS_V2 + ".\"COLUMN_NAME\" "
+            + "ELSE " + PARTITION_KEYS + ".\"PKEY_NAME\" END, "
+            + "" + KEY_CONSTRAINTS + ".\"CONSTRAINT_NAME\", " + KEY_CONSTRAINTS + ".\"ENABLE_VALIDATE_RELY\", "
+            + "" + KEY_CONSTRAINTS + ".\"DEFAULT_VALUE\" "
+            + " from " + TBLS + " "
+            + " INNER JOIN " + KEY_CONSTRAINTS + " ON " + TBLS + ".\"TBL_ID\" = " + KEY_CONSTRAINTS + ".\"PARENT_TBL_ID\" "
+            + " INNER JOIN " + DBS + " ON " + TBLS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\" "
+            + " LEFT OUTER JOIN " + COLUMNS_V2 + " ON " + COLUMNS_V2 + ".\"CD_ID\" = " + KEY_CONSTRAINTS + ".\"PARENT_CD_ID\" AND "
+            + " " + COLUMNS_V2 + ".\"INTEGER_IDX\" = " + KEY_CONSTRAINTS + ".\"PARENT_INTEGER_IDX\" "
+            + " LEFT OUTER JOIN " + PARTITION_KEYS + " ON " + TBLS + ".\"TBL_ID\" = " + PARTITION_KEYS + ".\"TBL_ID\" AND "
+            + " " + PARTITION_KEYS + ".\"INTEGER_IDX\" = " + KEY_CONSTRAINTS + ".\"PARENT_INTEGER_IDX\" "
+            + " WHERE " + KEY_CONSTRAINTS + ".\"CONSTRAINT_TYPE\" = "+ MConstraint.DEFAULT_CONSTRAINT+ " AND"
+            + (db_name == null ? "" : " " + DBS + ".\"NAME\" = ? AND")
+            + (tbl_name == null ? "" : " " + TBLS + ".\"TBL_NAME\" = ? ") ;
+
+    queryText = queryText.trim();
+    if (queryText.endsWith("AND")) {
+      queryText = queryText.substring(0, queryText.length()-3);
+    }
+    if (LOG.isDebugEnabled()){
+      LOG.debug("getDefaultConstraints: directsql : " + queryText);
+    }
+    List<String> pms = new ArrayList<String>();
+    if (db_name != null) {
+      pms.add(db_name);
+    }
+    if (tbl_name != null) {
+      pms.add(tbl_name);
+    }
+
+    Query queryParams = pm.newQuery("javax.jdo.query.SQL", queryText);
+    List<Object[]> sqlResult = ensureList(executeWithArray(
+        queryParams, pms.toArray(), queryText));
+
+    if (!sqlResult.isEmpty()) {
+      for (Object[] line : sqlResult) {
+        int enableValidateRely = extractSqlInt(line[4]);
+        boolean enable = (enableValidateRely & 4) != 0;
+        boolean validate = (enableValidateRely & 2) != 0;
+        boolean rely = (enableValidateRely & 1) != 0;
+        SQLDefaultConstraint currConstraint = new SQLDefaultConstraint(
+            extractSqlString(line[0]),
+            extractSqlString(line[1]),
+            extractSqlString(line[2]),
+            extractSqlString(line[5]),
+            extractSqlString(line[3]),
+            enable,
+            validate,
+            rely);
+        ret.add(currConstraint);
+      }
+    }
+    return ret;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 7b44df4..11667b7 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -117,6 +117,7 @@ import org.apache.hadoop.hive.metastore.api.ResourceType;
 import org.apache.hadoop.hive.metastore.api.ResourceUri;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -1092,8 +1093,8 @@ public class ObjectStore implements RawStore, Configurable {
   @Override
   public List<String> createTableWithConstraints(Table tbl,
     List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
-    List<SQLUniqueConstraint> uniqueConstraints,
-    List<SQLNotNullConstraint> notNullConstraints)
+    List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints,
+    List<SQLDefaultConstraint> defaultConstraints)
     throws InvalidObjectException, MetaException {
     boolean success = false;
     try {
@@ -1106,6 +1107,7 @@ public class ObjectStore implements RawStore, Configurable {
       constraintNames.addAll(addPrimaryKeys(primaryKeys, false));
       constraintNames.addAll(addUniqueConstraints(uniqueConstraints, false));
       constraintNames.addAll(addNotNullConstraints(notNullConstraints, false));
+      constraintNames.addAll(addDefaultConstraints(defaultConstraints, false));
       success = commitTransaction();
       return constraintNames;
     } finally {
@@ -4245,6 +4247,9 @@ public class ObjectStore implements RawStore, Configurable {
             }
           } else {
             currentConstraintName = normalizeIdentifier(foreignKey.getFk_name());
+            if(constraintNameAlreadyExists(currentConstraintName)) {
+              throw new InvalidObjectException("Constraint name already exists: " + currentConstraintName);
+            }
           }
           fkNames.add(currentConstraintName);
           Integer updateRule = foreignKey.getUpdate_rule();
@@ -4396,6 +4401,9 @@ public class ObjectStore implements RawStore, Configurable {
         }
       } else {
         constraintName = normalizeIdentifier(pks.get(i).getPk_name());
+        if(constraintNameAlreadyExists(constraintName)) {
+          throw new InvalidObjectException("Constraint name already exists: " + constraintName);
+        }
       }
       pkNames.add(constraintName);
       int enableValidateRely = (pks.get(i).isEnable_cstr() ? 4 : 0) +
@@ -4461,6 +4469,9 @@ public class ObjectStore implements RawStore, Configurable {
         }
       } else {
         constraintName = normalizeIdentifier(uks.get(i).getUk_name());
+        if(constraintNameAlreadyExists(constraintName)) {
+          throw new InvalidObjectException("Constraint name already exists: " + constraintName);
+        }
       }
       ukNames.add(constraintName);
 
@@ -4491,6 +4502,75 @@ public class ObjectStore implements RawStore, Configurable {
     return addNotNullConstraints(nns, true);
   }
 
+  @Override
+  public List<String> addDefaultConstraints(List<SQLDefaultConstraint> nns)
+      throws InvalidObjectException, MetaException {
+    return addDefaultConstraints(nns, true);
+  }
+
+  private List<String> addDefaultConstraints(List<SQLDefaultConstraint> nns, boolean retrieveCD)
+      throws InvalidObjectException, MetaException {
+    List<String> nnNames = new ArrayList<>();
+    List<MConstraint> cstrs = new ArrayList<>();
+    String constraintName = null;
+
+    for (int i = 0; i < nns.size(); i++) {
+      final String tableDB = normalizeIdentifier(nns.get(i).getTable_db());
+      final String tableName = normalizeIdentifier(nns.get(i).getTable_name());
+      final String columnName = normalizeIdentifier(nns.get(i).getColumn_name());
+
+      // If retrieveCD is false, we do not need to do a deep retrieval of the Table Column Descriptor.
+      // For instance, this is the case when we are creating the table.
+      AttachedMTableInfo nParentTable = getMTable(tableDB, tableName, retrieveCD);
+      MTable parentTable = nParentTable.mtbl;
+      if (parentTable == null) {
+        throw new InvalidObjectException("Parent table not found: " + tableName);
+      }
+
+      MColumnDescriptor parentCD = retrieveCD ? nParentTable.mcd : parentTable.getSd().getCD();
+      int parentIntegerIndex = getColumnIndexFromTableColumns(parentCD == null ? null : parentCD.getCols(), columnName);
+      if (parentIntegerIndex == -1) {
+        if (parentTable.getPartitionKeys() != null) {
+          parentCD = null;
+          parentIntegerIndex = getColumnIndexFromTableColumns(parentTable.getPartitionKeys(), columnName);
+        }
+        if (parentIntegerIndex == -1) {
+          throw new InvalidObjectException("Parent column not found: " + columnName);
+        }
+      }
+      if (nns.get(i).getDc_name() == null) {
+        constraintName = generateConstraintName(tableDB, tableName, columnName, "dc");
+      } else {
+        constraintName = normalizeIdentifier(nns.get(i).getDc_name());
+        if(constraintNameAlreadyExists(constraintName)) {
+          throw new InvalidObjectException("Constraint name already exists: " + constraintName);
+        }
+      }
+      nnNames.add(constraintName);
+
+      int enableValidateRely = (nns.get(i).isEnable_cstr() ? 4 : 0) +
+          (nns.get(i).isValidate_cstr() ? 2 : 0) + (nns.get(i).isRely_cstr() ? 1 : 0);
+      String defaultValue = nns.get(i).getDefault_value();
+      MConstraint muk = new MConstraint(
+          constraintName,
+          MConstraint.DEFAULT_CONSTRAINT,
+          1, // Not null constraint should reference a single column
+          null,
+          null,
+          enableValidateRely,
+          parentTable,
+          null,
+          parentCD,
+          null,
+          null,
+          parentIntegerIndex,
+          defaultValue);
+      cstrs.add(muk);
+    }
+    pm.makePersistentAll(cstrs);
+    return nnNames;
+  }
+
   private List<String> addNotNullConstraints(List<SQLNotNullConstraint> nns, boolean retrieveCD)
           throws InvalidObjectException, MetaException {
     List<String> nnNames = new ArrayList<>();
@@ -4525,6 +4605,9 @@ public class ObjectStore implements RawStore, Configurable {
         constraintName = generateConstraintName(tableDB, tableName, columnName, "nn");
       } else {
         constraintName = normalizeIdentifier(nns.get(i).getNn_name());
+        if(constraintNameAlreadyExists(constraintName)) {
+          throw new InvalidObjectException("Constraint name already exists: " + constraintName);
+        }
       }
       nnNames.add(constraintName);
 
@@ -9555,37 +9638,107 @@ public class ObjectStore implements RawStore, Configurable {
     }
   }
 
-  protected List<SQLNotNullConstraint> getNotNullConstraintsInternal(final String db_name_input,
+  @Override
+  public List<SQLDefaultConstraint> getDefaultConstraints(String db_name, String tbl_name)
+      throws MetaException {
+    try {
+      return getDefaultConstraintsInternal(db_name, tbl_name, true, true);
+    } catch (NoSuchObjectException e) {
+      throw new MetaException(ExceptionUtils.getStackTrace(e));
+    }
+  }
+
+  protected List<SQLDefaultConstraint> getDefaultConstraintsInternal(final String db_name_input,
       final String tbl_name_input, boolean allowSql, boolean allowJdo)
           throws MetaException, NoSuchObjectException {
     final String db_name = normalizeIdentifier(db_name_input);
     final String tbl_name = normalizeIdentifier(tbl_name_input);
+    return new GetListHelper<SQLDefaultConstraint>(db_name, tbl_name, allowSql, allowJdo) {
+
+      @Override
+      protected List<SQLDefaultConstraint> getSqlResult(GetHelper<List<SQLDefaultConstraint>> ctx)
+              throws MetaException {
+        return directSql.getDefaultConstraints(db_name, tbl_name);
+      }
+
+      @Override
+      protected List<SQLDefaultConstraint> getJdoResult(GetHelper<List<SQLDefaultConstraint>> ctx)
+              throws MetaException, NoSuchObjectException {
+        return getDefaultConstraintsViaJdo(db_name, tbl_name);
+      }
+    }.run(false);
+  }
+
+  private List<SQLDefaultConstraint> getDefaultConstraintsViaJdo(String db_name, String tbl_name)
+          throws MetaException {
+    boolean commited = false;
+    List<SQLDefaultConstraint> defaultConstraints= null;
+    Query query = null;
+    try {
+      openTransaction();
+      query = pm.newQuery(MConstraint.class,
+        "parentTable.tableName == tbl_name && parentTable.database.name == db_name &&"
+        + " constraintType == MConstraint.DEFAULT_CONSTRAINT");
+      query.declareParameters("java.lang.String tbl_name, java.lang.String db_name");
+      Collection<?> constraints = (Collection<?>) query.execute(tbl_name, db_name);
+      pm.retrieveAll(constraints);
+      defaultConstraints = new ArrayList<>();
+      for (Iterator<?> i = constraints.iterator(); i.hasNext();) {
+        MConstraint currConstraint = (MConstraint) i.next();
+        List<MFieldSchema> cols = currConstraint.getParentColumn() != null ?
+            currConstraint.getParentColumn().getCols() : currConstraint.getParentTable().getPartitionKeys();
+        int enableValidateRely = currConstraint.getEnableValidateRely();
+        boolean enable = (enableValidateRely & 4) != 0;
+        boolean validate = (enableValidateRely & 2) != 0;
+        boolean rely = (enableValidateRely & 1) != 0;
+        defaultConstraints.add(new SQLDefaultConstraint(db_name,
+         tbl_name,
+         cols.get(currConstraint.getParentIntegerIndex()).getName(),
+         currConstraint.getDefaultValue(), currConstraint.getConstraintName(), enable, validate, rely));
+      }
+      commited = commitTransaction();
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+      if (query != null) {
+        query.closeAll();
+      }
+    }
+    return defaultConstraints;
+  }
+
+  protected List<SQLNotNullConstraint> getNotNullConstraintsInternal(final String db_name_input,
+                                                                     final String tbl_name_input, boolean allowSql, boolean allowJdo)
+      throws MetaException, NoSuchObjectException {
+    final String db_name = normalizeIdentifier(db_name_input);
+    final String tbl_name = normalizeIdentifier(tbl_name_input);
     return new GetListHelper<SQLNotNullConstraint>(db_name, tbl_name, allowSql, allowJdo) {
 
       @Override
       protected List<SQLNotNullConstraint> getSqlResult(GetHelper<List<SQLNotNullConstraint>> ctx)
-              throws MetaException {
+          throws MetaException {
         return directSql.getNotNullConstraints(db_name, tbl_name);
       }
 
       @Override
       protected List<SQLNotNullConstraint> getJdoResult(GetHelper<List<SQLNotNullConstraint>> ctx)
-              throws MetaException, NoSuchObjectException {
+          throws MetaException, NoSuchObjectException {
         return getNotNullConstraintsViaJdo(db_name, tbl_name);
       }
     }.run(false);
   }
 
   private List<SQLNotNullConstraint> getNotNullConstraintsViaJdo(String db_name, String tbl_name)
-          throws MetaException {
+      throws MetaException {
     boolean commited = false;
     List<SQLNotNullConstraint> notNullConstraints = null;
     Query query = null;
     try {
       openTransaction();
       query = pm.newQuery(MConstraint.class,
-        "parentTable.tableName == tbl_name && parentTable.database.name == db_name &&"
-        + " constraintType == MConstraint.NOT_NULL_CONSTRAINT");
+          "parentTable.tableName == tbl_name && parentTable.database.name == db_name &&"
+              + " constraintType == MConstraint.NOT_NULL_CONSTRAINT");
       query.declareParameters("java.lang.String tbl_name, java.lang.String db_name");
       Collection<?> constraints = (Collection<?>) query.execute(tbl_name, db_name);
       pm.retrieveAll(constraints);
@@ -9599,9 +9752,9 @@ public class ObjectStore implements RawStore, Configurable {
         boolean validate = (enableValidateRely & 2) != 0;
         boolean rely = (enableValidateRely & 1) != 0;
         notNullConstraints.add(new SQLNotNullConstraint(db_name,
-         tbl_name,
-         cols.get(currConstraint.getParentIntegerIndex()).getName(),
-         currConstraint.getConstraintName(), enable, validate, rely));
+            tbl_name,
+            cols.get(currConstraint.getParentIntegerIndex()).getName(),
+            currConstraint.getConstraintName(), enable, validate, rely));
       }
       commited = commitTransaction();
     } finally {

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
index f500d63..6a8036f 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -65,6 +65,7 @@ import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -740,9 +741,13 @@ public interface RawStore extends Configurable {
   List<SQLNotNullConstraint> getNotNullConstraints(String db_name,
     String tbl_name) throws MetaException;
 
+  List<SQLDefaultConstraint> getDefaultConstraints(String db_name,
+                                                   String tbl_name) throws MetaException;
+
   List<String> createTableWithConstraints(Table tbl, List<SQLPrimaryKey> primaryKeys,
     List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints,
-    List<SQLNotNullConstraint> notNullConstraints) throws InvalidObjectException, MetaException;
+    List<SQLNotNullConstraint> notNullConstraints,
+    List<SQLDefaultConstraint> defaultConstraints) throws InvalidObjectException, MetaException;
 
   void dropConstraint(String dbName, String tableName, String constraintName) throws NoSuchObjectException;
 
@@ -754,6 +759,8 @@ public interface RawStore extends Configurable {
 
   List<String> addNotNullConstraints(List<SQLNotNullConstraint> nns) throws InvalidObjectException, MetaException;
 
+  List<String> addDefaultConstraints(List<SQLDefaultConstraint> nns) throws InvalidObjectException, MetaException;
+
   /**
    * Gets the unique id of the backing datastore for the metadata
    * @return

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
index 0d132f2..2ee5433 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
@@ -87,6 +87,7 @@ import org.apache.hadoop.hive.metastore.columnstats.aggr.ColumnStatsAggregator;
 import org.apache.hadoop.hive.metastore.columnstats.aggr.ColumnStatsAggregatorFactory;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -2424,12 +2425,20 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
+  public List<SQLDefaultConstraint> getDefaultConstraints(String db_name, String tbl_name)
+      throws MetaException {
+    // TODO constraintCache
+    return rawStore.getDefaultConstraints(db_name, tbl_name);
+  }
+
+  @Override
   public List<String> createTableWithConstraints(Table tbl, List<SQLPrimaryKey> primaryKeys,
       List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints,
-      List<SQLNotNullConstraint> notNullConstraints) throws InvalidObjectException, MetaException {
+      List<SQLNotNullConstraint> notNullConstraints,
+      List<SQLDefaultConstraint> defaultConstraints) throws InvalidObjectException, MetaException {
     // TODO constraintCache
     List<String> constraintNames = rawStore.createTableWithConstraints(tbl, primaryKeys,
-        foreignKeys, uniqueConstraints, notNullConstraints);
+        foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints);
     String dbName = StringUtils.normalizeIdentifier(tbl.getDbName());
     String tblName = StringUtils.normalizeIdentifier(tbl.getTableName());
     if (!shouldCacheTable(dbName, tblName)) {
@@ -2478,6 +2487,13 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
+  public List<String> addDefaultConstraints(List<SQLDefaultConstraint> nns)
+      throws InvalidObjectException, MetaException {
+    // TODO constraintCache
+    return rawStore.addDefaultConstraints(nns);
+  }
+
+  @Override
   public List<ColStatsObjWithSourceInfo> getPartitionColStatsForDatabase(String dbName)
       throws MetaException, NoSuchObjectException {
     return rawStore.getPartitionColStatsForDatabase(dbName);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MConstraint.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MConstraint.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MConstraint.java
index 57fb98b..8c7f57f 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MConstraint.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MConstraint.java
@@ -33,6 +33,7 @@ public class MConstraint
   Integer childIntegerIndex;
   Integer parentIntegerIndex;
   int enableValidateRely;
+  String defaultValue;
 
   // 0 - Primary Key
   // 1 - PK-FK relationship
@@ -42,6 +43,7 @@ public class MConstraint
   public final static int FOREIGN_KEY_CONSTRAINT = 1;
   public final static int UNIQUE_CONSTRAINT = 2;
   public final static int NOT_NULL_CONSTRAINT = 3;
+  public final static int DEFAULT_CONSTRAINT = 4;
 
   @SuppressWarnings("serial")
   public static class PK implements Serializable {
@@ -90,6 +92,29 @@ public class MConstraint
    this.parentIntegerIndex = parentIntegerIndex;
   }
 
+  public MConstraint(String constraintName, int constraintType, int position, Integer deleteRule, Integer updateRule,
+          int enableRelyValidate, MTable parentTable, MTable childTable, MColumnDescriptor parentColumn,
+          MColumnDescriptor childColumn, Integer childIntegerIndex, Integer parentIntegerIndex, String defaultValue) {
+    this.constraintName = constraintName;
+    this.constraintType = constraintType;
+    this.parentTable = parentTable;
+    this.childTable = childTable;
+    this.parentColumn = parentColumn;
+    this.childColumn = childColumn;
+    this.position = position;
+    this.deleteRule = deleteRule;
+    this.updateRule = updateRule;
+    this.enableValidateRely = enableRelyValidate;
+    this.childIntegerIndex = childIntegerIndex;
+    this.parentIntegerIndex = parentIntegerIndex;
+    this.defaultValue = defaultValue;
+  }
+
+  public String getDefaultValue() { return defaultValue; }
+
+  public void setDefaultValue(String defaultValue) {
+    this.defaultValue = defaultValue;
+  }
   public String getConstraintName() {
     return constraintName;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql b/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
index 9d8a703..86ef6b1 100644
--- a/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
@@ -108,7 +108,7 @@ CREATE TABLE "APP"."NOTIFICATION_LOG" ("NL_ID" BIGINT NOT NULL, "DB_NAME" VARCHA
 
 CREATE TABLE "APP"."NOTIFICATION_SEQUENCE" ("NNI_ID" BIGINT NOT NULL, "NEXT_EVENT_ID" BIGINT NOT NULL);
 
-CREATE TABLE "APP"."KEY_CONSTRAINTS" ("CHILD_CD_ID" BIGINT, "CHILD_INTEGER_IDX" INTEGER, "CHILD_TBL_ID" BIGINT, "PARENT_CD_ID" BIGINT NOT NULL, "PARENT_INTEGER_IDX" INTEGER, "PARENT_TBL_ID" BIGINT NOT NULL,  "POSITION" BIGINT NOT NULL, "CONSTRAINT_NAME" VARCHAR(400) NOT NULL, "CONSTRAINT_TYPE" SMALLINT NOT NULL, "UPDATE_RULE" SMALLINT, "DELETE_RULE" SMALLINT, "ENABLE_VALIDATE_RELY" SMALLINT NOT NULL);
+CREATE TABLE "APP"."KEY_CONSTRAINTS" ("CHILD_CD_ID" BIGINT, "CHILD_INTEGER_IDX" INTEGER, "CHILD_TBL_ID" BIGINT, "PARENT_CD_ID" BIGINT NOT NULL, "PARENT_INTEGER_IDX" INTEGER, "PARENT_TBL_ID" BIGINT NOT NULL,  "POSITION" BIGINT NOT NULL, "CONSTRAINT_NAME" VARCHAR(400) NOT NULL, "CONSTRAINT_TYPE" SMALLINT NOT NULL, "UPDATE_RULE" SMALLINT, "DELETE_RULE" SMALLINT, "ENABLE_VALIDATE_RELY" SMALLINT NOT NULL, "DEFAULT_VALUE" VARCHAR(400));
 
 CREATE TABLE "APP"."METASTORE_DB_PROPERTIES" ("PROPERTY_KEY" VARCHAR(255) NOT NULL, "PROPERTY_VALUE" VARCHAR(1000) NOT NULL, "DESCRIPTION" VARCHAR(1000));
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql b/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
index a50c45d..1fc34bc 100644
--- a/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
@@ -120,3 +120,7 @@ RENAME COLUMN COMPLETED_COMPACTIONS.CC_HIGHEST_TXN_ID TO CC_HIGHEST_WRITE_ID;
 -- Modify txn_components/completed_txn_components tables to add write id.
 ALTER TABLE TXN_COMPONENTS ADD TC_WRITEID bigint;
 ALTER TABLE COMPLETED_TXN_COMPONENTS ADD CTC_WRITEID bigint;
+
+-- HIVE-18726
+-- add a new column to support default value for DEFAULT constraint
+ALTER TABLE "APP"."KEY_CONSTRAINTS" ADD COLUMN "DEFAULT_VALUE" VARCHAR(400);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
index 1b7d0da..e0bb25b 100644
--- a/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
@@ -1105,7 +1105,8 @@ CREATE TABLE KEY_CONSTRAINTS
   CONSTRAINT_TYPE SMALLINT NOT NULL,
   UPDATE_RULE SMALLINT,
   DELETE_RULE SMALLINT,
-  ENABLE_VALIDATE_RELY SMALLINT NOT NULL
+  ENABLE_VALIDATE_RELY SMALLINT NOT NULL,
+  DEFAULT_VALUE VARCHAR(400)
 ) ;
 
 ALTER TABLE KEY_CONSTRAINTS ADD CONSTRAINT CONSTRAINTS_PK PRIMARY KEY (CONSTRAINT_NAME, POSITION);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
index 8ab466d..52107b3 100644
--- a/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
@@ -174,3 +174,7 @@ EXEC SP_RENAME 'COMPLETED_COMPACTIONS.CC_HIGHEST_TXN_ID', 'CC_HIGHEST_WRITE_ID',
 -- Modify txn_components/completed_txn_components tables to add write id.
 ALTER TABLE TXN_COMPONENTS ADD TC_WRITEID bigint;
 ALTER TABLE COMPLETED_TXN_COMPONENTS ADD CTC_WRITEID bigint;
+  
+-- HIVE-18726
+  -- add a new column to support default value for DEFAULT constraint
+ ALTER TABLE KEY_CONSTRAINTS ADD DEFAULT_VALUE VARCHAR(400);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
index 886c932..566badf 100644
--- a/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
@@ -859,6 +859,7 @@ CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS`
   `UPDATE_RULE` SMALLINT(6),
   `DELETE_RULE` SMALLINT(6),
   `ENABLE_VALIDATE_RELY` SMALLINT(6) NOT NULL,
+  `DEFAULT_VALUE` VARCHAR(400),
   PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
index a537734..35f08dc 100644
--- a/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
@@ -159,3 +159,7 @@ ALTER TABLE COMPLETED_COMPACTIONS CHANGE `CC_HIGHEST_TXN_ID` `CC_HIGHEST_WRITE_I
 -- Modify txn_components/completed_txn_components tables to add write id.
 ALTER TABLE TXN_COMPONENTS ADD TC_WRITEID bigint;
 ALTER TABLE COMPLETED_TXN_COMPONENTS ADD CTC_WRITEID bigint;
+ 
+-- HIVE-18726
+-- add a new column to support default value for DEFAULT constraint
+ALTER TABLE `KEY_CONSTRAINTS` ADD COLUMN `DEFAULT_VALUE` VARCHAR(400);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
index 366b2d9..82811ee 100644
--- a/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
@@ -869,7 +869,8 @@ CREATE TABLE KEY_CONSTRAINTS
   CONSTRAINT_TYPE NUMBER NOT NULL,
   UPDATE_RULE NUMBER,
   DELETE_RULE NUMBER,
-  ENABLE_VALIDATE_RELY NUMBER NOT NULL
+  ENABLE_VALIDATE_RELY NUMBER NOT NULL,
+  DEFAULT_VALUE VARCHAR(400)
 ) ;
 
 ALTER TABLE KEY_CONSTRAINTS ADD CONSTRAINT CONSTRAINTS_PK PRIMARY KEY (CONSTRAINT_NAME, POSITION);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
index bd786fb..edf14d9 100644
--- a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
@@ -157,6 +157,7 @@ UPDATE DBS
   SET DB_LOCATION_URI = 's3a' || SUBSTR(DB_LOCATION_URI, 4)
   WHERE DB_LOCATION_URI LIKE 's3n://%' ;
 
+
 -- HIVE-18192
 CREATE TABLE TXN_TO_WRITE_ID (
   T2W_TXNID number(19) NOT NULL,
@@ -182,3 +183,7 @@ ALTER TABLE COMPLETED_COMPACTIONS RENAME COLUMN CC_HIGHEST_TXN_ID TO CC_HIGHEST_
 -- Modify txn_components/completed_txn_components tables to add write id.
 ALTER TABLE TXN_COMPONENTS ADD TC_WRITEID number(19);
 ALTER TABLE COMPLETED_TXN_COMPONENTS ADD CTC_WRITEID number(19);
+
+-- HIVE-18726
+-- add a new column to support default value for DEFAULT constraint
+ALTER TABLE KEY_CONSTRAINTS ADD DEFAULT_VALUE VARCHAR(400);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
index 4abf24c..543a358 100644
--- a/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
@@ -624,6 +624,7 @@ CREATE TABLE "KEY_CONSTRAINTS"
   "UPDATE_RULE" SMALLINT,
   "DELETE_RULE"	SMALLINT,
   "ENABLE_VALIDATE_RELY" SMALLINT NOT NULL,
+  "DEFAULT_VALUE" VARCHAR(400),
   PRIMARY KEY ("CONSTRAINT_NAME", "POSITION")
 ) ;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
index 34ed974..ed4ce22 100644
--- a/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
@@ -198,3 +198,7 @@ ALTER TABLE COMPLETED_COMPACTIONS RENAME CC_HIGHEST_TXN_ID TO CC_HIGHEST_WRITE_I
 -- Modify txn_components/completed_txn_components tables to add write id.
 ALTER TABLE TXN_COMPONENTS ADD TC_WRITEID bigint;
 ALTER TABLE COMPLETED_TXN_COMPONENTS ADD CTC_WRITEID bigint;
+ 
+-- HIVE-18726
+-- add a new column to support default value for DEFAULT constraint
+ALTER TABLE "KEY_CONSTRAINTS" ADD COLUMN "DEFAULT_VALUE" VARCHAR(400);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/thrift/hive_metastore.thrift b/standalone-metastore/src/main/thrift/hive_metastore.thrift
index b11ee38..e95014a 100644
--- a/standalone-metastore/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/src/main/thrift/hive_metastore.thrift
@@ -90,6 +90,17 @@ struct SQLNotNullConstraint {
   7: bool rely_cstr      // Rely/No Rely
 }
 
+struct SQLDefaultConstraint {
+  1: string table_db,    // table schema
+  2: string table_name,  // table name
+  3: string column_name, // column name
+  4: string default_value,// default value
+  5: string dc_name,     // default name
+  6: bool enable_cstr,   // Enable/Disable
+  7: bool validate_cstr, // Validate/No validate
+  8: bool rely_cstr      // Rely/No Rely
+}
+
 struct Type {
   1: string          name,             // one of the types in PrimitiveTypes or CollectionTypes or User defined types
   2: optional string type1,            // object type if the name is 'list' (LIST_TYPE), key type if the name is 'map' (MAP_TYPE)
@@ -537,6 +548,16 @@ struct NotNullConstraintsResponse {
   1: required list<SQLNotNullConstraint> notNullConstraints
 }
 
+struct DefaultConstraintsRequest {
+  1: required string db_name,
+  2: required string tbl_name
+}
+
+struct DefaultConstraintsResponse {
+  1: required list<SQLDefaultConstraint> defaultConstraints
+}
+
+
 struct DropConstraintRequest {
   1: required string dbname, 
   2: required string tablename,
@@ -559,6 +580,10 @@ struct AddNotNullConstraintRequest {
   1: required list<SQLNotNullConstraint> notNullConstraintCols
 }
 
+struct AddDefaultConstraintRequest {
+  1: required list<SQLDefaultConstraint> defaultConstraintCols
+}
+
 // Return type for get_partitions_by_expr
 struct PartitionsByExprResult {
   1: required list<Partition> partitions,
@@ -1419,7 +1444,8 @@ service ThriftHiveMetastore extends fb303.FacebookService
               2:InvalidObjectException o2, 3:MetaException o3,
               4:NoSuchObjectException o4)
   void create_table_with_constraints(1:Table tbl, 2: list<SQLPrimaryKey> primaryKeys, 3: list<SQLForeignKey> foreignKeys,
-  4: list<SQLUniqueConstraint> uniqueConstraints, 5: list<SQLNotNullConstraint> notNullConstraints)
+  4: list<SQLUniqueConstraint> uniqueConstraints, 5: list<SQLNotNullConstraint> notNullConstraints,
+  6: list<SQLDefaultConstraint> defaultConstraints)
       throws (1:AlreadyExistsException o1,
               2:InvalidObjectException o2, 3:MetaException o3,
               4:NoSuchObjectException o4)
@@ -1433,6 +1459,8 @@ service ThriftHiveMetastore extends fb303.FacebookService
       throws(1:NoSuchObjectException o1, 2:MetaException o2)
   void add_not_null_constraint(1:AddNotNullConstraintRequest req)
       throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  void add_default_constraint(1:AddDefaultConstraintRequest req)
+      throws(1:NoSuchObjectException o1, 2:MetaException o2)
 
   // drops the table and all the partitions associated with it if the table has partitions
   // delete data (including partitions) if deleteData is set to true
@@ -1696,6 +1724,8 @@ service ThriftHiveMetastore extends fb303.FacebookService
                        throws(1:MetaException o1, 2:NoSuchObjectException o2)
   NotNullConstraintsResponse get_not_null_constraints(1:NotNullConstraintsRequest request)
                        throws(1:MetaException o1, 2:NoSuchObjectException o2)
+  DefaultConstraintsResponse get_default_constraints(1:DefaultConstraintsRequest request)
+                       throws(1:MetaException o1, 2:NoSuchObjectException o2)
 
   // column statistics interfaces
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index 75ea8c4..8c0cc26 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -63,6 +63,7 @@ import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -905,10 +906,18 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
+  public List<SQLDefaultConstraint> getDefaultConstraints(String db_name, String tbl_name)
+      throws MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
   public List<String> createTableWithConstraints(Table tbl,
     List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
     List<SQLUniqueConstraint> uniqueConstraints,
-    List<SQLNotNullConstraint> notNullConstraints)
+    List<SQLNotNullConstraint> notNullConstraints,
+    List<SQLDefaultConstraint> defaultConstraints)
     throws InvalidObjectException, MetaException {
     // TODO Auto-generated method stub
     return null;
@@ -947,6 +956,13 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
+  public List<String> addDefaultConstraints(List<SQLDefaultConstraint> nns)
+      throws InvalidObjectException, MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
   public String getMetastoreDbUuid() throws MetaException {
     throw new MetaException("Get metastore uuid is not implemented");
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index 207d842..a0b2f98 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -915,10 +916,18 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
+  public List<SQLDefaultConstraint> getDefaultConstraints(String db_name, String tbl_name)
+      throws MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
   public List<String> createTableWithConstraints(Table tbl,
     List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
     List<SQLUniqueConstraint> uniqueConstraints,
-    List<SQLNotNullConstraint> notNullConstraints)
+    List<SQLNotNullConstraint> notNullConstraints,
+    List<SQLDefaultConstraint> defaultConstraints)
     throws InvalidObjectException, MetaException {
     // TODO Auto-generated method stub
     return null;
@@ -958,6 +967,13 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
+  public List<String> addDefaultConstraints(List<SQLDefaultConstraint> nns)
+      throws InvalidObjectException, MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
   public String getMetastoreDbUuid() throws MetaException {
     throw new MetaException("Get metastore uuid is not implemented");
   }


[12/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index 835cbb3..7c38d5b 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -198,6 +198,8 @@ class SQLUniqueConstraint;
 
 class SQLNotNullConstraint;
 
+class SQLDefaultConstraint;
+
 class Type;
 
 class HiveObjectRef;
@@ -304,6 +306,10 @@ class NotNullConstraintsRequest;
 
 class NotNullConstraintsResponse;
 
+class DefaultConstraintsRequest;
+
+class DefaultConstraintsResponse;
+
 class DropConstraintRequest;
 
 class AddPrimaryKeyRequest;
@@ -314,6 +320,8 @@ class AddUniqueConstraintRequest;
 
 class AddNotNullConstraintRequest;
 
+class AddDefaultConstraintRequest;
+
 class PartitionsByExprResult;
 
 class PartitionsByExprRequest;
@@ -1084,6 +1092,94 @@ inline std::ostream& operator<<(std::ostream& out, const SQLNotNullConstraint& o
   return out;
 }
 
+typedef struct _SQLDefaultConstraint__isset {
+  _SQLDefaultConstraint__isset() : table_db(false), table_name(false), column_name(false), default_value(false), dc_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  bool table_db :1;
+  bool table_name :1;
+  bool column_name :1;
+  bool default_value :1;
+  bool dc_name :1;
+  bool enable_cstr :1;
+  bool validate_cstr :1;
+  bool rely_cstr :1;
+} _SQLDefaultConstraint__isset;
+
+class SQLDefaultConstraint {
+ public:
+
+  SQLDefaultConstraint(const SQLDefaultConstraint&);
+  SQLDefaultConstraint& operator=(const SQLDefaultConstraint&);
+  SQLDefaultConstraint() : table_db(), table_name(), column_name(), default_value(), dc_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
+  }
+
+  virtual ~SQLDefaultConstraint() throw();
+  std::string table_db;
+  std::string table_name;
+  std::string column_name;
+  std::string default_value;
+  std::string dc_name;
+  bool enable_cstr;
+  bool validate_cstr;
+  bool rely_cstr;
+
+  _SQLDefaultConstraint__isset __isset;
+
+  void __set_table_db(const std::string& val);
+
+  void __set_table_name(const std::string& val);
+
+  void __set_column_name(const std::string& val);
+
+  void __set_default_value(const std::string& val);
+
+  void __set_dc_name(const std::string& val);
+
+  void __set_enable_cstr(const bool val);
+
+  void __set_validate_cstr(const bool val);
+
+  void __set_rely_cstr(const bool val);
+
+  bool operator == (const SQLDefaultConstraint & rhs) const
+  {
+    if (!(table_db == rhs.table_db))
+      return false;
+    if (!(table_name == rhs.table_name))
+      return false;
+    if (!(column_name == rhs.column_name))
+      return false;
+    if (!(default_value == rhs.default_value))
+      return false;
+    if (!(dc_name == rhs.dc_name))
+      return false;
+    if (!(enable_cstr == rhs.enable_cstr))
+      return false;
+    if (!(validate_cstr == rhs.validate_cstr))
+      return false;
+    if (!(rely_cstr == rhs.rely_cstr))
+      return false;
+    return true;
+  }
+  bool operator != (const SQLDefaultConstraint &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const SQLDefaultConstraint & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(SQLDefaultConstraint &a, SQLDefaultConstraint &b);
+
+inline std::ostream& operator<<(std::ostream& out, const SQLDefaultConstraint& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _Type__isset {
   _Type__isset() : name(false), type1(false), type2(false), fields(false) {}
   bool name :1;
@@ -4346,6 +4442,91 @@ inline std::ostream& operator<<(std::ostream& out, const NotNullConstraintsRespo
 }
 
 
+class DefaultConstraintsRequest {
+ public:
+
+  DefaultConstraintsRequest(const DefaultConstraintsRequest&);
+  DefaultConstraintsRequest& operator=(const DefaultConstraintsRequest&);
+  DefaultConstraintsRequest() : db_name(), tbl_name() {
+  }
+
+  virtual ~DefaultConstraintsRequest() throw();
+  std::string db_name;
+  std::string tbl_name;
+
+  void __set_db_name(const std::string& val);
+
+  void __set_tbl_name(const std::string& val);
+
+  bool operator == (const DefaultConstraintsRequest & rhs) const
+  {
+    if (!(db_name == rhs.db_name))
+      return false;
+    if (!(tbl_name == rhs.tbl_name))
+      return false;
+    return true;
+  }
+  bool operator != (const DefaultConstraintsRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const DefaultConstraintsRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(DefaultConstraintsRequest &a, DefaultConstraintsRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const DefaultConstraintsRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
+class DefaultConstraintsResponse {
+ public:
+
+  DefaultConstraintsResponse(const DefaultConstraintsResponse&);
+  DefaultConstraintsResponse& operator=(const DefaultConstraintsResponse&);
+  DefaultConstraintsResponse() {
+  }
+
+  virtual ~DefaultConstraintsResponse() throw();
+  std::vector<SQLDefaultConstraint>  defaultConstraints;
+
+  void __set_defaultConstraints(const std::vector<SQLDefaultConstraint> & val);
+
+  bool operator == (const DefaultConstraintsResponse & rhs) const
+  {
+    if (!(defaultConstraints == rhs.defaultConstraints))
+      return false;
+    return true;
+  }
+  bool operator != (const DefaultConstraintsResponse &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const DefaultConstraintsResponse & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(DefaultConstraintsResponse &a, DefaultConstraintsResponse &b);
+
+inline std::ostream& operator<<(std::ostream& out, const DefaultConstraintsResponse& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
 class DropConstraintRequest {
  public:
 
@@ -4556,6 +4737,46 @@ inline std::ostream& operator<<(std::ostream& out, const AddNotNullConstraintReq
 }
 
 
+class AddDefaultConstraintRequest {
+ public:
+
+  AddDefaultConstraintRequest(const AddDefaultConstraintRequest&);
+  AddDefaultConstraintRequest& operator=(const AddDefaultConstraintRequest&);
+  AddDefaultConstraintRequest() {
+  }
+
+  virtual ~AddDefaultConstraintRequest() throw();
+  std::vector<SQLDefaultConstraint>  defaultConstraintCols;
+
+  void __set_defaultConstraintCols(const std::vector<SQLDefaultConstraint> & val);
+
+  bool operator == (const AddDefaultConstraintRequest & rhs) const
+  {
+    if (!(defaultConstraintCols == rhs.defaultConstraintCols))
+      return false;
+    return true;
+  }
+  bool operator != (const AddDefaultConstraintRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const AddDefaultConstraintRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(AddDefaultConstraintRequest &a, AddDefaultConstraintRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const AddDefaultConstraintRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
 class PartitionsByExprResult {
  public:
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
index 0e5dbf7..29e4e69 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list548 = iprot.readListBegin();
-                struct.txn_ids = new ArrayList<Long>(_list548.size);
-                long _elem549;
-                for (int _i550 = 0; _i550 < _list548.size; ++_i550)
+                org.apache.thrift.protocol.TList _list564 = iprot.readListBegin();
+                struct.txn_ids = new ArrayList<Long>(_list564.size);
+                long _elem565;
+                for (int _i566 = 0; _i566 < _list564.size; ++_i566)
                 {
-                  _elem549 = iprot.readI64();
-                  struct.txn_ids.add(_elem549);
+                  _elem565 = iprot.readI64();
+                  struct.txn_ids.add(_elem565);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txn_ids.size()));
-          for (long _iter551 : struct.txn_ids)
+          for (long _iter567 : struct.txn_ids)
           {
-            oprot.writeI64(_iter551);
+            oprot.writeI64(_iter567);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txn_ids.size());
-        for (long _iter552 : struct.txn_ids)
+        for (long _iter568 : struct.txn_ids)
         {
-          oprot.writeI64(_iter552);
+          oprot.writeI64(_iter568);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AbortTxnsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list553 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.txn_ids = new ArrayList<Long>(_list553.size);
-        long _elem554;
-        for (int _i555 = 0; _i555 < _list553.size; ++_i555)
+        org.apache.thrift.protocol.TList _list569 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.txn_ids = new ArrayList<Long>(_list569.size);
+        long _elem570;
+        for (int _i571 = 0; _i571 < _list569.size; ++_i571)
         {
-          _elem554 = iprot.readI64();
-          struct.txn_ids.add(_elem554);
+          _elem570 = iprot.readI64();
+          struct.txn_ids.add(_elem570);
         }
       }
       struct.setTxn_idsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDefaultConstraintRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDefaultConstraintRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDefaultConstraintRequest.java
new file mode 100644
index 0000000..8703b61
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDefaultConstraintRequest.java
@@ -0,0 +1,443 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class AddDefaultConstraintRequest implements org.apache.thrift.TBase<AddDefaultConstraintRequest, AddDefaultConstraintRequest._Fields>, java.io.Serializable, Cloneable, Comparable<AddDefaultConstraintRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AddDefaultConstraintRequest");
+
+  private static final org.apache.thrift.protocol.TField DEFAULT_CONSTRAINT_COLS_FIELD_DESC = new org.apache.thrift.protocol.TField("defaultConstraintCols", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new AddDefaultConstraintRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new AddDefaultConstraintRequestTupleSchemeFactory());
+  }
+
+  private List<SQLDefaultConstraint> defaultConstraintCols; // 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 {
+    DEFAULT_CONSTRAINT_COLS((short)1, "defaultConstraintCols");
+
+    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: // DEFAULT_CONSTRAINT_COLS
+          return DEFAULT_CONSTRAINT_COLS;
+        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.DEFAULT_CONSTRAINT_COLS, new org.apache.thrift.meta_data.FieldMetaData("defaultConstraintCols", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SQLDefaultConstraint.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AddDefaultConstraintRequest.class, metaDataMap);
+  }
+
+  public AddDefaultConstraintRequest() {
+  }
+
+  public AddDefaultConstraintRequest(
+    List<SQLDefaultConstraint> defaultConstraintCols)
+  {
+    this();
+    this.defaultConstraintCols = defaultConstraintCols;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public AddDefaultConstraintRequest(AddDefaultConstraintRequest other) {
+    if (other.isSetDefaultConstraintCols()) {
+      List<SQLDefaultConstraint> __this__defaultConstraintCols = new ArrayList<SQLDefaultConstraint>(other.defaultConstraintCols.size());
+      for (SQLDefaultConstraint other_element : other.defaultConstraintCols) {
+        __this__defaultConstraintCols.add(new SQLDefaultConstraint(other_element));
+      }
+      this.defaultConstraintCols = __this__defaultConstraintCols;
+    }
+  }
+
+  public AddDefaultConstraintRequest deepCopy() {
+    return new AddDefaultConstraintRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.defaultConstraintCols = null;
+  }
+
+  public int getDefaultConstraintColsSize() {
+    return (this.defaultConstraintCols == null) ? 0 : this.defaultConstraintCols.size();
+  }
+
+  public java.util.Iterator<SQLDefaultConstraint> getDefaultConstraintColsIterator() {
+    return (this.defaultConstraintCols == null) ? null : this.defaultConstraintCols.iterator();
+  }
+
+  public void addToDefaultConstraintCols(SQLDefaultConstraint elem) {
+    if (this.defaultConstraintCols == null) {
+      this.defaultConstraintCols = new ArrayList<SQLDefaultConstraint>();
+    }
+    this.defaultConstraintCols.add(elem);
+  }
+
+  public List<SQLDefaultConstraint> getDefaultConstraintCols() {
+    return this.defaultConstraintCols;
+  }
+
+  public void setDefaultConstraintCols(List<SQLDefaultConstraint> defaultConstraintCols) {
+    this.defaultConstraintCols = defaultConstraintCols;
+  }
+
+  public void unsetDefaultConstraintCols() {
+    this.defaultConstraintCols = null;
+  }
+
+  /** Returns true if field defaultConstraintCols is set (has been assigned a value) and false otherwise */
+  public boolean isSetDefaultConstraintCols() {
+    return this.defaultConstraintCols != null;
+  }
+
+  public void setDefaultConstraintColsIsSet(boolean value) {
+    if (!value) {
+      this.defaultConstraintCols = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case DEFAULT_CONSTRAINT_COLS:
+      if (value == null) {
+        unsetDefaultConstraintCols();
+      } else {
+        setDefaultConstraintCols((List<SQLDefaultConstraint>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case DEFAULT_CONSTRAINT_COLS:
+      return getDefaultConstraintCols();
+
+    }
+    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 DEFAULT_CONSTRAINT_COLS:
+      return isSetDefaultConstraintCols();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof AddDefaultConstraintRequest)
+      return this.equals((AddDefaultConstraintRequest)that);
+    return false;
+  }
+
+  public boolean equals(AddDefaultConstraintRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_defaultConstraintCols = true && this.isSetDefaultConstraintCols();
+    boolean that_present_defaultConstraintCols = true && that.isSetDefaultConstraintCols();
+    if (this_present_defaultConstraintCols || that_present_defaultConstraintCols) {
+      if (!(this_present_defaultConstraintCols && that_present_defaultConstraintCols))
+        return false;
+      if (!this.defaultConstraintCols.equals(that.defaultConstraintCols))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_defaultConstraintCols = true && (isSetDefaultConstraintCols());
+    list.add(present_defaultConstraintCols);
+    if (present_defaultConstraintCols)
+      list.add(defaultConstraintCols);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(AddDefaultConstraintRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetDefaultConstraintCols()).compareTo(other.isSetDefaultConstraintCols());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDefaultConstraintCols()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.defaultConstraintCols, other.defaultConstraintCols);
+      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("AddDefaultConstraintRequest(");
+    boolean first = true;
+
+    sb.append("defaultConstraintCols:");
+    if (this.defaultConstraintCols == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.defaultConstraintCols);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetDefaultConstraintCols()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'defaultConstraintCols' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      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 AddDefaultConstraintRequestStandardSchemeFactory implements SchemeFactory {
+    public AddDefaultConstraintRequestStandardScheme getScheme() {
+      return new AddDefaultConstraintRequestStandardScheme();
+    }
+  }
+
+  private static class AddDefaultConstraintRequestStandardScheme extends StandardScheme<AddDefaultConstraintRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, AddDefaultConstraintRequest 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: // DEFAULT_CONSTRAINT_COLS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list394 = iprot.readListBegin();
+                struct.defaultConstraintCols = new ArrayList<SQLDefaultConstraint>(_list394.size);
+                SQLDefaultConstraint _elem395;
+                for (int _i396 = 0; _i396 < _list394.size; ++_i396)
+                {
+                  _elem395 = new SQLDefaultConstraint();
+                  _elem395.read(iprot);
+                  struct.defaultConstraintCols.add(_elem395);
+                }
+                iprot.readListEnd();
+              }
+              struct.setDefaultConstraintColsIsSet(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, AddDefaultConstraintRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.defaultConstraintCols != null) {
+        oprot.writeFieldBegin(DEFAULT_CONSTRAINT_COLS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraintCols.size()));
+          for (SQLDefaultConstraint _iter397 : struct.defaultConstraintCols)
+          {
+            _iter397.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class AddDefaultConstraintRequestTupleSchemeFactory implements SchemeFactory {
+    public AddDefaultConstraintRequestTupleScheme getScheme() {
+      return new AddDefaultConstraintRequestTupleScheme();
+    }
+  }
+
+  private static class AddDefaultConstraintRequestTupleScheme extends TupleScheme<AddDefaultConstraintRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, AddDefaultConstraintRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.defaultConstraintCols.size());
+        for (SQLDefaultConstraint _iter398 : struct.defaultConstraintCols)
+        {
+          _iter398.write(oprot);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, AddDefaultConstraintRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list399 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.defaultConstraintCols = new ArrayList<SQLDefaultConstraint>(_list399.size);
+        SQLDefaultConstraint _elem400;
+        for (int _i401 = 0; _i401 < _list399.size; ++_i401)
+        {
+          _elem400 = new SQLDefaultConstraint();
+          _elem400.read(iprot);
+          struct.defaultConstraintCols.add(_elem400);
+        }
+      }
+      struct.setDefaultConstraintColsIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
index a01dc24..99c0741 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
@@ -816,13 +816,13 @@ import org.slf4j.LoggerFactory;
           case 5: // PARTITIONNAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list646 = iprot.readListBegin();
-                struct.partitionnames = new ArrayList<String>(_list646.size);
-                String _elem647;
-                for (int _i648 = 0; _i648 < _list646.size; ++_i648)
+                org.apache.thrift.protocol.TList _list662 = iprot.readListBegin();
+                struct.partitionnames = new ArrayList<String>(_list662.size);
+                String _elem663;
+                for (int _i664 = 0; _i664 < _list662.size; ++_i664)
                 {
-                  _elem647 = iprot.readString();
-                  struct.partitionnames.add(_elem647);
+                  _elem663 = iprot.readString();
+                  struct.partitionnames.add(_elem663);
                 }
                 iprot.readListEnd();
               }
@@ -872,9 +872,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITIONNAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionnames.size()));
-          for (String _iter649 : struct.partitionnames)
+          for (String _iter665 : struct.partitionnames)
           {
-            oprot.writeString(_iter649);
+            oprot.writeString(_iter665);
           }
           oprot.writeListEnd();
         }
@@ -910,9 +910,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tablename);
       {
         oprot.writeI32(struct.partitionnames.size());
-        for (String _iter650 : struct.partitionnames)
+        for (String _iter666 : struct.partitionnames)
         {
-          oprot.writeString(_iter650);
+          oprot.writeString(_iter666);
         }
       }
       BitSet optionals = new BitSet();
@@ -937,13 +937,13 @@ import org.slf4j.LoggerFactory;
       struct.tablename = iprot.readString();
       struct.setTablenameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list651 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partitionnames = new ArrayList<String>(_list651.size);
-        String _elem652;
-        for (int _i653 = 0; _i653 < _list651.size; ++_i653)
+        org.apache.thrift.protocol.TList _list667 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partitionnames = new ArrayList<String>(_list667.size);
+        String _elem668;
+        for (int _i669 = 0; _i669 < _list667.size; ++_i669)
         {
-          _elem652 = iprot.readString();
-          struct.partitionnames.add(_elem652);
+          _elem668 = iprot.readString();
+          struct.partitionnames.add(_elem668);
         }
       }
       struct.setPartitionnamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
index c1c0dbf..2bcdea1 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // FOREIGN_KEY_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list362 = iprot.readListBegin();
-                struct.foreignKeyCols = new ArrayList<SQLForeignKey>(_list362.size);
-                SQLForeignKey _elem363;
-                for (int _i364 = 0; _i364 < _list362.size; ++_i364)
+                org.apache.thrift.protocol.TList _list370 = iprot.readListBegin();
+                struct.foreignKeyCols = new ArrayList<SQLForeignKey>(_list370.size);
+                SQLForeignKey _elem371;
+                for (int _i372 = 0; _i372 < _list370.size; ++_i372)
                 {
-                  _elem363 = new SQLForeignKey();
-                  _elem363.read(iprot);
-                  struct.foreignKeyCols.add(_elem363);
+                  _elem371 = new SQLForeignKey();
+                  _elem371.read(iprot);
+                  struct.foreignKeyCols.add(_elem371);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FOREIGN_KEY_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeyCols.size()));
-          for (SQLForeignKey _iter365 : struct.foreignKeyCols)
+          for (SQLForeignKey _iter373 : struct.foreignKeyCols)
           {
-            _iter365.write(oprot);
+            _iter373.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.foreignKeyCols.size());
-        for (SQLForeignKey _iter366 : struct.foreignKeyCols)
+        for (SQLForeignKey _iter374 : struct.foreignKeyCols)
         {
-          _iter366.write(oprot);
+          _iter374.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AddForeignKeyRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list367 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.foreignKeyCols = new ArrayList<SQLForeignKey>(_list367.size);
-        SQLForeignKey _elem368;
-        for (int _i369 = 0; _i369 < _list367.size; ++_i369)
+        org.apache.thrift.protocol.TList _list375 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.foreignKeyCols = new ArrayList<SQLForeignKey>(_list375.size);
+        SQLForeignKey _elem376;
+        for (int _i377 = 0; _i377 < _list375.size; ++_i377)
         {
-          _elem368 = new SQLForeignKey();
-          _elem368.read(iprot);
-          struct.foreignKeyCols.add(_elem368);
+          _elem376 = new SQLForeignKey();
+          _elem376.read(iprot);
+          struct.foreignKeyCols.add(_elem376);
         }
       }
       struct.setForeignKeyColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java
index 0bd85f3..f944e46 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // NOT_NULL_CONSTRAINT_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list378 = iprot.readListBegin();
-                struct.notNullConstraintCols = new ArrayList<SQLNotNullConstraint>(_list378.size);
-                SQLNotNullConstraint _elem379;
-                for (int _i380 = 0; _i380 < _list378.size; ++_i380)
+                org.apache.thrift.protocol.TList _list386 = iprot.readListBegin();
+                struct.notNullConstraintCols = new ArrayList<SQLNotNullConstraint>(_list386.size);
+                SQLNotNullConstraint _elem387;
+                for (int _i388 = 0; _i388 < _list386.size; ++_i388)
                 {
-                  _elem379 = new SQLNotNullConstraint();
-                  _elem379.read(iprot);
-                  struct.notNullConstraintCols.add(_elem379);
+                  _elem387 = new SQLNotNullConstraint();
+                  _elem387.read(iprot);
+                  struct.notNullConstraintCols.add(_elem387);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(NOT_NULL_CONSTRAINT_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraintCols.size()));
-          for (SQLNotNullConstraint _iter381 : struct.notNullConstraintCols)
+          for (SQLNotNullConstraint _iter389 : struct.notNullConstraintCols)
           {
-            _iter381.write(oprot);
+            _iter389.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.notNullConstraintCols.size());
-        for (SQLNotNullConstraint _iter382 : struct.notNullConstraintCols)
+        for (SQLNotNullConstraint _iter390 : struct.notNullConstraintCols)
         {
-          _iter382.write(oprot);
+          _iter390.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AddNotNullConstraintRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list383 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.notNullConstraintCols = new ArrayList<SQLNotNullConstraint>(_list383.size);
-        SQLNotNullConstraint _elem384;
-        for (int _i385 = 0; _i385 < _list383.size; ++_i385)
+        org.apache.thrift.protocol.TList _list391 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.notNullConstraintCols = new ArrayList<SQLNotNullConstraint>(_list391.size);
+        SQLNotNullConstraint _elem392;
+        for (int _i393 = 0; _i393 < _list391.size; ++_i393)
         {
-          _elem384 = new SQLNotNullConstraint();
-          _elem384.read(iprot);
-          struct.notNullConstraintCols.add(_elem384);
+          _elem392 = new SQLNotNullConstraint();
+          _elem392.read(iprot);
+          struct.notNullConstraintCols.add(_elem392);
         }
       }
       struct.setNotNullConstraintColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
index 9119336..d351678 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
@@ -704,14 +704,14 @@ import org.slf4j.LoggerFactory;
           case 3: // PARTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list452 = iprot.readListBegin();
-                struct.parts = new ArrayList<Partition>(_list452.size);
-                Partition _elem453;
-                for (int _i454 = 0; _i454 < _list452.size; ++_i454)
+                org.apache.thrift.protocol.TList _list468 = iprot.readListBegin();
+                struct.parts = new ArrayList<Partition>(_list468.size);
+                Partition _elem469;
+                for (int _i470 = 0; _i470 < _list468.size; ++_i470)
                 {
-                  _elem453 = new Partition();
-                  _elem453.read(iprot);
-                  struct.parts.add(_elem453);
+                  _elem469 = new Partition();
+                  _elem469.read(iprot);
+                  struct.parts.add(_elem469);
                 }
                 iprot.readListEnd();
               }
@@ -763,9 +763,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.parts.size()));
-          for (Partition _iter455 : struct.parts)
+          for (Partition _iter471 : struct.parts)
           {
-            _iter455.write(oprot);
+            _iter471.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -800,9 +800,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.parts.size());
-        for (Partition _iter456 : struct.parts)
+        for (Partition _iter472 : struct.parts)
         {
-          _iter456.write(oprot);
+          _iter472.write(oprot);
         }
       }
       oprot.writeBool(struct.ifNotExists);
@@ -824,14 +824,14 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list457 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.parts = new ArrayList<Partition>(_list457.size);
-        Partition _elem458;
-        for (int _i459 = 0; _i459 < _list457.size; ++_i459)
+        org.apache.thrift.protocol.TList _list473 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.parts = new ArrayList<Partition>(_list473.size);
+        Partition _elem474;
+        for (int _i475 = 0; _i475 < _list473.size; ++_i475)
         {
-          _elem458 = new Partition();
-          _elem458.read(iprot);
-          struct.parts.add(_elem458);
+          _elem474 = new Partition();
+          _elem474.read(iprot);
+          struct.parts.add(_elem474);
         }
       }
       struct.setPartsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
index 57d4953..23525e2 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list444 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list444.size);
-                Partition _elem445;
-                for (int _i446 = 0; _i446 < _list444.size; ++_i446)
+                org.apache.thrift.protocol.TList _list460 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list460.size);
+                Partition _elem461;
+                for (int _i462 = 0; _i462 < _list460.size; ++_i462)
                 {
-                  _elem445 = new Partition();
-                  _elem445.read(iprot);
-                  struct.partitions.add(_elem445);
+                  _elem461 = new Partition();
+                  _elem461.read(iprot);
+                  struct.partitions.add(_elem461);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-            for (Partition _iter447 : struct.partitions)
+            for (Partition _iter463 : struct.partitions)
             {
-              _iter447.write(oprot);
+              _iter463.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitions()) {
         {
           oprot.writeI32(struct.partitions.size());
-          for (Partition _iter448 : struct.partitions)
+          for (Partition _iter464 : struct.partitions)
           {
-            _iter448.write(oprot);
+            _iter464.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list449 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitions = new ArrayList<Partition>(_list449.size);
-          Partition _elem450;
-          for (int _i451 = 0; _i451 < _list449.size; ++_i451)
+          org.apache.thrift.protocol.TList _list465 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitions = new ArrayList<Partition>(_list465.size);
+          Partition _elem466;
+          for (int _i467 = 0; _i467 < _list465.size; ++_i467)
           {
-            _elem450 = new Partition();
-            _elem450.read(iprot);
-            struct.partitions.add(_elem450);
+            _elem466 = new Partition();
+            _elem466.read(iprot);
+            struct.partitions.add(_elem466);
           }
         }
         struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
index 900985b..9efcfe0 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PRIMARY_KEY_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list354 = iprot.readListBegin();
-                struct.primaryKeyCols = new ArrayList<SQLPrimaryKey>(_list354.size);
-                SQLPrimaryKey _elem355;
-                for (int _i356 = 0; _i356 < _list354.size; ++_i356)
+                org.apache.thrift.protocol.TList _list362 = iprot.readListBegin();
+                struct.primaryKeyCols = new ArrayList<SQLPrimaryKey>(_list362.size);
+                SQLPrimaryKey _elem363;
+                for (int _i364 = 0; _i364 < _list362.size; ++_i364)
                 {
-                  _elem355 = new SQLPrimaryKey();
-                  _elem355.read(iprot);
-                  struct.primaryKeyCols.add(_elem355);
+                  _elem363 = new SQLPrimaryKey();
+                  _elem363.read(iprot);
+                  struct.primaryKeyCols.add(_elem363);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PRIMARY_KEY_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeyCols.size()));
-          for (SQLPrimaryKey _iter357 : struct.primaryKeyCols)
+          for (SQLPrimaryKey _iter365 : struct.primaryKeyCols)
           {
-            _iter357.write(oprot);
+            _iter365.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.primaryKeyCols.size());
-        for (SQLPrimaryKey _iter358 : struct.primaryKeyCols)
+        for (SQLPrimaryKey _iter366 : struct.primaryKeyCols)
         {
-          _iter358.write(oprot);
+          _iter366.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AddPrimaryKeyRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list359 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.primaryKeyCols = new ArrayList<SQLPrimaryKey>(_list359.size);
-        SQLPrimaryKey _elem360;
-        for (int _i361 = 0; _i361 < _list359.size; ++_i361)
+        org.apache.thrift.protocol.TList _list367 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.primaryKeyCols = new ArrayList<SQLPrimaryKey>(_list367.size);
+        SQLPrimaryKey _elem368;
+        for (int _i369 = 0; _i369 < _list367.size; ++_i369)
         {
-          _elem360 = new SQLPrimaryKey();
-          _elem360.read(iprot);
-          struct.primaryKeyCols.add(_elem360);
+          _elem368 = new SQLPrimaryKey();
+          _elem368.read(iprot);
+          struct.primaryKeyCols.add(_elem368);
         }
       }
       struct.setPrimaryKeyColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java
index df4f544..0429fd5 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // UNIQUE_CONSTRAINT_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list370 = iprot.readListBegin();
-                struct.uniqueConstraintCols = new ArrayList<SQLUniqueConstraint>(_list370.size);
-                SQLUniqueConstraint _elem371;
-                for (int _i372 = 0; _i372 < _list370.size; ++_i372)
+                org.apache.thrift.protocol.TList _list378 = iprot.readListBegin();
+                struct.uniqueConstraintCols = new ArrayList<SQLUniqueConstraint>(_list378.size);
+                SQLUniqueConstraint _elem379;
+                for (int _i380 = 0; _i380 < _list378.size; ++_i380)
                 {
-                  _elem371 = new SQLUniqueConstraint();
-                  _elem371.read(iprot);
-                  struct.uniqueConstraintCols.add(_elem371);
+                  _elem379 = new SQLUniqueConstraint();
+                  _elem379.read(iprot);
+                  struct.uniqueConstraintCols.add(_elem379);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(UNIQUE_CONSTRAINT_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraintCols.size()));
-          for (SQLUniqueConstraint _iter373 : struct.uniqueConstraintCols)
+          for (SQLUniqueConstraint _iter381 : struct.uniqueConstraintCols)
           {
-            _iter373.write(oprot);
+            _iter381.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.uniqueConstraintCols.size());
-        for (SQLUniqueConstraint _iter374 : struct.uniqueConstraintCols)
+        for (SQLUniqueConstraint _iter382 : struct.uniqueConstraintCols)
         {
-          _iter374.write(oprot);
+          _iter382.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AddUniqueConstraintRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list375 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.uniqueConstraintCols = new ArrayList<SQLUniqueConstraint>(_list375.size);
-        SQLUniqueConstraint _elem376;
-        for (int _i377 = 0; _i377 < _list375.size; ++_i377)
+        org.apache.thrift.protocol.TList _list383 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.uniqueConstraintCols = new ArrayList<SQLUniqueConstraint>(_list383.size);
+        SQLUniqueConstraint _elem384;
+        for (int _i385 = 0; _i385 < _list383.size; ++_i385)
         {
-          _elem376 = new SQLUniqueConstraint();
-          _elem376.read(iprot);
-          struct.uniqueConstraintCols.add(_elem376);
+          _elem384 = new SQLUniqueConstraint();
+          _elem384.read(iprot);
+          struct.uniqueConstraintCols.add(_elem384);
         }
       }
       struct.setUniqueConstraintColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
index 1aec53b..60a32ad 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
@@ -521,13 +521,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list580 = iprot.readListBegin();
-                struct.txnIds = new ArrayList<Long>(_list580.size);
-                long _elem581;
-                for (int _i582 = 0; _i582 < _list580.size; ++_i582)
+                org.apache.thrift.protocol.TList _list596 = iprot.readListBegin();
+                struct.txnIds = new ArrayList<Long>(_list596.size);
+                long _elem597;
+                for (int _i598 = 0; _i598 < _list596.size; ++_i598)
                 {
-                  _elem581 = iprot.readI64();
-                  struct.txnIds.add(_elem581);
+                  _elem597 = iprot.readI64();
+                  struct.txnIds.add(_elem597);
                 }
                 iprot.readListEnd();
               }
@@ -569,9 +569,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txnIds.size()));
-          for (long _iter583 : struct.txnIds)
+          for (long _iter599 : struct.txnIds)
           {
-            oprot.writeI64(_iter583);
+            oprot.writeI64(_iter599);
           }
           oprot.writeListEnd();
         }
@@ -606,9 +606,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txnIds.size());
-        for (long _iter584 : struct.txnIds)
+        for (long _iter600 : struct.txnIds)
         {
-          oprot.writeI64(_iter584);
+          oprot.writeI64(_iter600);
         }
       }
       oprot.writeString(struct.dbName);
@@ -619,13 +619,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AllocateTableWriteIdsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list585 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.txnIds = new ArrayList<Long>(_list585.size);
-        long _elem586;
-        for (int _i587 = 0; _i587 < _list585.size; ++_i587)
+        org.apache.thrift.protocol.TList _list601 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.txnIds = new ArrayList<Long>(_list601.size);
+        long _elem602;
+        for (int _i603 = 0; _i603 < _list601.size; ++_i603)
         {
-          _elem586 = iprot.readI64();
-          struct.txnIds.add(_elem586);
+          _elem602 = iprot.readI64();
+          struct.txnIds.add(_elem602);
         }
       }
       struct.setTxnIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
index e29e1db..9bf2f7f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_TO_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list588 = iprot.readListBegin();
-                struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list588.size);
-                TxnToWriteId _elem589;
-                for (int _i590 = 0; _i590 < _list588.size; ++_i590)
+                org.apache.thrift.protocol.TList _list604 = iprot.readListBegin();
+                struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list604.size);
+                TxnToWriteId _elem605;
+                for (int _i606 = 0; _i606 < _list604.size; ++_i606)
                 {
-                  _elem589 = new TxnToWriteId();
-                  _elem589.read(iprot);
-                  struct.txnToWriteIds.add(_elem589);
+                  _elem605 = new TxnToWriteId();
+                  _elem605.read(iprot);
+                  struct.txnToWriteIds.add(_elem605);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_TO_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.txnToWriteIds.size()));
-          for (TxnToWriteId _iter591 : struct.txnToWriteIds)
+          for (TxnToWriteId _iter607 : struct.txnToWriteIds)
           {
-            _iter591.write(oprot);
+            _iter607.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txnToWriteIds.size());
-        for (TxnToWriteId _iter592 : struct.txnToWriteIds)
+        for (TxnToWriteId _iter608 : struct.txnToWriteIds)
         {
-          _iter592.write(oprot);
+          _iter608.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AllocateTableWriteIdsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list593 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list593.size);
-        TxnToWriteId _elem594;
-        for (int _i595 = 0; _i595 < _list593.size; ++_i595)
+        org.apache.thrift.protocol.TList _list609 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list609.size);
+        TxnToWriteId _elem610;
+        for (int _i611 = 0; _i611 < _list609.size; ++_i611)
         {
-          _elem594 = new TxnToWriteId();
-          _elem594.read(iprot);
-          struct.txnToWriteIds.add(_elem594);
+          _elem610 = new TxnToWriteId();
+          _elem610.read(iprot);
+          struct.txnToWriteIds.add(_elem610);
         }
       }
       struct.setTxnToWriteIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
index ee9841f..9feadc4 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list746 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list746.size);
-                long _elem747;
-                for (int _i748 = 0; _i748 < _list746.size; ++_i748)
+                org.apache.thrift.protocol.TList _list762 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list762.size);
+                long _elem763;
+                for (int _i764 = 0; _i764 < _list762.size; ++_i764)
                 {
-                  _elem747 = iprot.readI64();
-                  struct.fileIds.add(_elem747);
+                  _elem763 = iprot.readI64();
+                  struct.fileIds.add(_elem763);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter749 : struct.fileIds)
+          for (long _iter765 : struct.fileIds)
           {
-            oprot.writeI64(_iter749);
+            oprot.writeI64(_iter765);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter750 : struct.fileIds)
+        for (long _iter766 : struct.fileIds)
         {
-          oprot.writeI64(_iter750);
+          oprot.writeI64(_iter766);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ClearFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list751 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list751.size);
-        long _elem752;
-        for (int _i753 = 0; _i753 < _list751.size; ++_i753)
+        org.apache.thrift.protocol.TList _list767 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list767.size);
+        long _elem768;
+        for (int _i769 = 0; _i769 < _list767.size; ++_i769)
         {
-          _elem752 = iprot.readI64();
-          struct.fileIds.add(_elem752);
+          _elem768 = iprot.readI64();
+          struct.fileIds.add(_elem768);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
index 8dbe4c1..0a1f76f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
@@ -354,13 +354,13 @@ import org.slf4j.LoggerFactory;
           case 1: // VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list762 = iprot.readListBegin();
-                struct.values = new ArrayList<ClientCapability>(_list762.size);
-                ClientCapability _elem763;
-                for (int _i764 = 0; _i764 < _list762.size; ++_i764)
+                org.apache.thrift.protocol.TList _list778 = iprot.readListBegin();
+                struct.values = new ArrayList<ClientCapability>(_list778.size);
+                ClientCapability _elem779;
+                for (int _i780 = 0; _i780 < _list778.size; ++_i780)
                 {
-                  _elem763 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-                  struct.values.add(_elem763);
+                  _elem779 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+                  struct.values.add(_elem779);
                 }
                 iprot.readListEnd();
               }
@@ -386,9 +386,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.values.size()));
-          for (ClientCapability _iter765 : struct.values)
+          for (ClientCapability _iter781 : struct.values)
           {
-            oprot.writeI32(_iter765.getValue());
+            oprot.writeI32(_iter781.getValue());
           }
           oprot.writeListEnd();
         }
@@ -413,9 +413,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.values.size());
-        for (ClientCapability _iter766 : struct.values)
+        for (ClientCapability _iter782 : struct.values)
         {
-          oprot.writeI32(_iter766.getValue());
+          oprot.writeI32(_iter782.getValue());
         }
       }
     }
@@ -424,13 +424,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ClientCapabilities struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list767 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
-        struct.values = new ArrayList<ClientCapability>(_list767.size);
-        ClientCapability _elem768;
-        for (int _i769 = 0; _i769 < _list767.size; ++_i769)
+        org.apache.thrift.protocol.TList _list783 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
+        struct.values = new ArrayList<ClientCapability>(_list783.size);
+        ClientCapability _elem784;
+        for (int _i785 = 0; _i785 < _list783.size; ++_i785)
         {
-          _elem768 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-          struct.values.add(_elem768);
+          _elem784 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+          struct.values.add(_elem784);
         }
       }
       struct.setValuesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
index 1853720..af84a9c 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
@@ -814,15 +814,15 @@ import org.slf4j.LoggerFactory;
           case 6: // PROPERTIES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map628 = iprot.readMapBegin();
-                struct.properties = new HashMap<String,String>(2*_map628.size);
-                String _key629;
-                String _val630;
-                for (int _i631 = 0; _i631 < _map628.size; ++_i631)
+                org.apache.thrift.protocol.TMap _map644 = iprot.readMapBegin();
+                struct.properties = new HashMap<String,String>(2*_map644.size);
+                String _key645;
+                String _val646;
+                for (int _i647 = 0; _i647 < _map644.size; ++_i647)
                 {
-                  _key629 = iprot.readString();
-                  _val630 = iprot.readString();
-                  struct.properties.put(_key629, _val630);
+                  _key645 = iprot.readString();
+                  _val646 = iprot.readString();
+                  struct.properties.put(_key645, _val646);
                 }
                 iprot.readMapEnd();
               }
@@ -878,10 +878,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PROPERTIES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.properties.size()));
-            for (Map.Entry<String, String> _iter632 : struct.properties.entrySet())
+            for (Map.Entry<String, String> _iter648 : struct.properties.entrySet())
             {
-              oprot.writeString(_iter632.getKey());
-              oprot.writeString(_iter632.getValue());
+              oprot.writeString(_iter648.getKey());
+              oprot.writeString(_iter648.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -928,10 +928,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetProperties()) {
         {
           oprot.writeI32(struct.properties.size());
-          for (Map.Entry<String, String> _iter633 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter649 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter633.getKey());
-            oprot.writeString(_iter633.getValue());
+            oprot.writeString(_iter649.getKey());
+            oprot.writeString(_iter649.getValue());
           }
         }
       }
@@ -957,15 +957,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map634 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.properties = new HashMap<String,String>(2*_map634.size);
-          String _key635;
-          String _val636;
-          for (int _i637 = 0; _i637 < _map634.size; ++_i637)
+          org.apache.thrift.protocol.TMap _map650 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.properties = new HashMap<String,String>(2*_map650.size);
+          String _key651;
+          String _val652;
+          for (int _i653 = 0; _i653 < _map650.size; ++_i653)
           {
-            _key635 = iprot.readString();
-            _val636 = iprot.readString();
-            struct.properties.put(_key635, _val636);
+            _key651 = iprot.readString();
+            _val652 = iprot.readString();
+            struct.properties.put(_key651, _val652);
           }
         }
         struct.setPropertiesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
index 717840f..3da8ed5 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
@@ -619,13 +619,13 @@ import org.slf4j.LoggerFactory;
           case 3: // TABLES_USED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set654 = iprot.readSetBegin();
-                struct.tablesUsed = new HashSet<String>(2*_set654.size);
-                String _elem655;
-                for (int _i656 = 0; _i656 < _set654.size; ++_i656)
+                org.apache.thrift.protocol.TSet _set670 = iprot.readSetBegin();
+                struct.tablesUsed = new HashSet<String>(2*_set670.size);
+                String _elem671;
+                for (int _i672 = 0; _i672 < _set670.size; ++_i672)
                 {
-                  _elem655 = iprot.readString();
-                  struct.tablesUsed.add(_elem655);
+                  _elem671 = iprot.readString();
+                  struct.tablesUsed.add(_elem671);
                 }
                 iprot.readSetEnd();
               }
@@ -669,9 +669,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_USED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.tablesUsed.size()));
-          for (String _iter657 : struct.tablesUsed)
+          for (String _iter673 : struct.tablesUsed)
           {
-            oprot.writeString(_iter657);
+            oprot.writeString(_iter673);
           }
           oprot.writeSetEnd();
         }
@@ -705,9 +705,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.tablesUsed.size());
-        for (String _iter658 : struct.tablesUsed)
+        for (String _iter674 : struct.tablesUsed)
         {
-          oprot.writeString(_iter658);
+          oprot.writeString(_iter674);
         }
       }
       BitSet optionals = new BitSet();
@@ -728,13 +728,13 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set659 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.tablesUsed = new HashSet<String>(2*_set659.size);
-        String _elem660;
-        for (int _i661 = 0; _i661 < _set659.size; ++_i661)
+        org.apache.thrift.protocol.TSet _set675 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.tablesUsed = new HashSet<String>(2*_set675.size);
+        String _elem676;
+        for (int _i677 = 0; _i677 < _set675.size; ++_i677)
         {
-          _elem660 = iprot.readString();
-          struct.tablesUsed.add(_elem660);
+          _elem676 = iprot.readString();
+          struct.tablesUsed.add(_elem676);
         }
       }
       struct.setTablesUsedIsSet(true);


[11/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsRequest.java
new file mode 100644
index 0000000..5f4954d
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsRequest.java
@@ -0,0 +1,490 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class DefaultConstraintsRequest implements org.apache.thrift.TBase<DefaultConstraintsRequest, DefaultConstraintsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<DefaultConstraintsRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DefaultConstraintsRequest");
+
+  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 TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new DefaultConstraintsRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new DefaultConstraintsRequestTupleSchemeFactory());
+  }
+
+  private String db_name; // required
+  private String tbl_name; // 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"),
+    TBL_NAME((short)2, "tbl_name");
+
+    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: // TBL_NAME
+          return TBL_NAME;
+        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.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tbl_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DefaultConstraintsRequest.class, metaDataMap);
+  }
+
+  public DefaultConstraintsRequest() {
+  }
+
+  public DefaultConstraintsRequest(
+    String db_name,
+    String tbl_name)
+  {
+    this();
+    this.db_name = db_name;
+    this.tbl_name = tbl_name;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public DefaultConstraintsRequest(DefaultConstraintsRequest other) {
+    if (other.isSetDb_name()) {
+      this.db_name = other.db_name;
+    }
+    if (other.isSetTbl_name()) {
+      this.tbl_name = other.tbl_name;
+    }
+  }
+
+  public DefaultConstraintsRequest deepCopy() {
+    return new DefaultConstraintsRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.db_name = null;
+    this.tbl_name = 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 getTbl_name() {
+    return this.tbl_name;
+  }
+
+  public void setTbl_name(String tbl_name) {
+    this.tbl_name = tbl_name;
+  }
+
+  public void unsetTbl_name() {
+    this.tbl_name = null;
+  }
+
+  /** Returns true if field tbl_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetTbl_name() {
+    return this.tbl_name != null;
+  }
+
+  public void setTbl_nameIsSet(boolean value) {
+    if (!value) {
+      this.tbl_name = 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 TBL_NAME:
+      if (value == null) {
+        unsetTbl_name();
+      } else {
+        setTbl_name((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case DB_NAME:
+      return getDb_name();
+
+    case TBL_NAME:
+      return getTbl_name();
+
+    }
+    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 TBL_NAME:
+      return isSetTbl_name();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof DefaultConstraintsRequest)
+      return this.equals((DefaultConstraintsRequest)that);
+    return false;
+  }
+
+  public boolean equals(DefaultConstraintsRequest 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_tbl_name = true && this.isSetTbl_name();
+    boolean that_present_tbl_name = true && that.isSetTbl_name();
+    if (this_present_tbl_name || that_present_tbl_name) {
+      if (!(this_present_tbl_name && that_present_tbl_name))
+        return false;
+      if (!this.tbl_name.equals(that.tbl_name))
+        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_tbl_name = true && (isSetTbl_name());
+    list.add(present_tbl_name);
+    if (present_tbl_name)
+      list.add(tbl_name);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(DefaultConstraintsRequest 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(isSetTbl_name()).compareTo(other.isSetTbl_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTbl_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tbl_name, other.tbl_name);
+      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("DefaultConstraintsRequest(");
+    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("tbl_name:");
+    if (this.tbl_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tbl_name);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetDb_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'db_name' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTbl_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tbl_name' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      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 DefaultConstraintsRequestStandardSchemeFactory implements SchemeFactory {
+    public DefaultConstraintsRequestStandardScheme getScheme() {
+      return new DefaultConstraintsRequestStandardScheme();
+    }
+  }
+
+  private static class DefaultConstraintsRequestStandardScheme extends StandardScheme<DefaultConstraintsRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, DefaultConstraintsRequest 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: // TBL_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tbl_name = iprot.readString();
+              struct.setTbl_nameIsSet(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, DefaultConstraintsRequest 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.tbl_name != null) {
+        oprot.writeFieldBegin(TBL_NAME_FIELD_DESC);
+        oprot.writeString(struct.tbl_name);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class DefaultConstraintsRequestTupleSchemeFactory implements SchemeFactory {
+    public DefaultConstraintsRequestTupleScheme getScheme() {
+      return new DefaultConstraintsRequestTupleScheme();
+    }
+  }
+
+  private static class DefaultConstraintsRequestTupleScheme extends TupleScheme<DefaultConstraintsRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, DefaultConstraintsRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.db_name);
+      oprot.writeString(struct.tbl_name);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, DefaultConstraintsRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.db_name = iprot.readString();
+      struct.setDb_nameIsSet(true);
+      struct.tbl_name = iprot.readString();
+      struct.setTbl_nameIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsResponse.java
new file mode 100644
index 0000000..22514de
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsResponse.java
@@ -0,0 +1,443 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class DefaultConstraintsResponse implements org.apache.thrift.TBase<DefaultConstraintsResponse, DefaultConstraintsResponse._Fields>, java.io.Serializable, Cloneable, Comparable<DefaultConstraintsResponse> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DefaultConstraintsResponse");
+
+  private static final org.apache.thrift.protocol.TField DEFAULT_CONSTRAINTS_FIELD_DESC = new org.apache.thrift.protocol.TField("defaultConstraints", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new DefaultConstraintsResponseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new DefaultConstraintsResponseTupleSchemeFactory());
+  }
+
+  private List<SQLDefaultConstraint> defaultConstraints; // 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 {
+    DEFAULT_CONSTRAINTS((short)1, "defaultConstraints");
+
+    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: // DEFAULT_CONSTRAINTS
+          return DEFAULT_CONSTRAINTS;
+        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.DEFAULT_CONSTRAINTS, new org.apache.thrift.meta_data.FieldMetaData("defaultConstraints", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SQLDefaultConstraint.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DefaultConstraintsResponse.class, metaDataMap);
+  }
+
+  public DefaultConstraintsResponse() {
+  }
+
+  public DefaultConstraintsResponse(
+    List<SQLDefaultConstraint> defaultConstraints)
+  {
+    this();
+    this.defaultConstraints = defaultConstraints;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public DefaultConstraintsResponse(DefaultConstraintsResponse other) {
+    if (other.isSetDefaultConstraints()) {
+      List<SQLDefaultConstraint> __this__defaultConstraints = new ArrayList<SQLDefaultConstraint>(other.defaultConstraints.size());
+      for (SQLDefaultConstraint other_element : other.defaultConstraints) {
+        __this__defaultConstraints.add(new SQLDefaultConstraint(other_element));
+      }
+      this.defaultConstraints = __this__defaultConstraints;
+    }
+  }
+
+  public DefaultConstraintsResponse deepCopy() {
+    return new DefaultConstraintsResponse(this);
+  }
+
+  @Override
+  public void clear() {
+    this.defaultConstraints = null;
+  }
+
+  public int getDefaultConstraintsSize() {
+    return (this.defaultConstraints == null) ? 0 : this.defaultConstraints.size();
+  }
+
+  public java.util.Iterator<SQLDefaultConstraint> getDefaultConstraintsIterator() {
+    return (this.defaultConstraints == null) ? null : this.defaultConstraints.iterator();
+  }
+
+  public void addToDefaultConstraints(SQLDefaultConstraint elem) {
+    if (this.defaultConstraints == null) {
+      this.defaultConstraints = new ArrayList<SQLDefaultConstraint>();
+    }
+    this.defaultConstraints.add(elem);
+  }
+
+  public List<SQLDefaultConstraint> getDefaultConstraints() {
+    return this.defaultConstraints;
+  }
+
+  public void setDefaultConstraints(List<SQLDefaultConstraint> defaultConstraints) {
+    this.defaultConstraints = defaultConstraints;
+  }
+
+  public void unsetDefaultConstraints() {
+    this.defaultConstraints = null;
+  }
+
+  /** Returns true if field defaultConstraints is set (has been assigned a value) and false otherwise */
+  public boolean isSetDefaultConstraints() {
+    return this.defaultConstraints != null;
+  }
+
+  public void setDefaultConstraintsIsSet(boolean value) {
+    if (!value) {
+      this.defaultConstraints = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case DEFAULT_CONSTRAINTS:
+      if (value == null) {
+        unsetDefaultConstraints();
+      } else {
+        setDefaultConstraints((List<SQLDefaultConstraint>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case DEFAULT_CONSTRAINTS:
+      return getDefaultConstraints();
+
+    }
+    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 DEFAULT_CONSTRAINTS:
+      return isSetDefaultConstraints();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof DefaultConstraintsResponse)
+      return this.equals((DefaultConstraintsResponse)that);
+    return false;
+  }
+
+  public boolean equals(DefaultConstraintsResponse that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_defaultConstraints = true && this.isSetDefaultConstraints();
+    boolean that_present_defaultConstraints = true && that.isSetDefaultConstraints();
+    if (this_present_defaultConstraints || that_present_defaultConstraints) {
+      if (!(this_present_defaultConstraints && that_present_defaultConstraints))
+        return false;
+      if (!this.defaultConstraints.equals(that.defaultConstraints))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_defaultConstraints = true && (isSetDefaultConstraints());
+    list.add(present_defaultConstraints);
+    if (present_defaultConstraints)
+      list.add(defaultConstraints);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(DefaultConstraintsResponse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetDefaultConstraints()).compareTo(other.isSetDefaultConstraints());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDefaultConstraints()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.defaultConstraints, other.defaultConstraints);
+      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("DefaultConstraintsResponse(");
+    boolean first = true;
+
+    sb.append("defaultConstraints:");
+    if (this.defaultConstraints == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.defaultConstraints);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetDefaultConstraints()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'defaultConstraints' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      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 DefaultConstraintsResponseStandardSchemeFactory implements SchemeFactory {
+    public DefaultConstraintsResponseStandardScheme getScheme() {
+      return new DefaultConstraintsResponseStandardScheme();
+    }
+  }
+
+  private static class DefaultConstraintsResponseStandardScheme extends StandardScheme<DefaultConstraintsResponse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, DefaultConstraintsResponse 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: // DEFAULT_CONSTRAINTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list354 = iprot.readListBegin();
+                struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list354.size);
+                SQLDefaultConstraint _elem355;
+                for (int _i356 = 0; _i356 < _list354.size; ++_i356)
+                {
+                  _elem355 = new SQLDefaultConstraint();
+                  _elem355.read(iprot);
+                  struct.defaultConstraints.add(_elem355);
+                }
+                iprot.readListEnd();
+              }
+              struct.setDefaultConstraintsIsSet(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, DefaultConstraintsResponse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.defaultConstraints != null) {
+        oprot.writeFieldBegin(DEFAULT_CONSTRAINTS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraints.size()));
+          for (SQLDefaultConstraint _iter357 : struct.defaultConstraints)
+          {
+            _iter357.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class DefaultConstraintsResponseTupleSchemeFactory implements SchemeFactory {
+    public DefaultConstraintsResponseTupleScheme getScheme() {
+      return new DefaultConstraintsResponseTupleScheme();
+    }
+  }
+
+  private static class DefaultConstraintsResponseTupleScheme extends TupleScheme<DefaultConstraintsResponse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, DefaultConstraintsResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.defaultConstraints.size());
+        for (SQLDefaultConstraint _iter358 : struct.defaultConstraints)
+        {
+          _iter358.write(oprot);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, DefaultConstraintsResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list359 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list359.size);
+        SQLDefaultConstraint _elem360;
+        for (int _i361 = 0; _i361 < _list359.size; ++_i361)
+        {
+          _elem360 = new SQLDefaultConstraint();
+          _elem360.read(iprot);
+          struct.defaultConstraints.add(_elem360);
+        }
+      }
+      struct.setDefaultConstraintsIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
index b9dc04a..daf87c5 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list460 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list460.size);
-                Partition _elem461;
-                for (int _i462 = 0; _i462 < _list460.size; ++_i462)
+                org.apache.thrift.protocol.TList _list476 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list476.size);
+                Partition _elem477;
+                for (int _i478 = 0; _i478 < _list476.size; ++_i478)
                 {
-                  _elem461 = new Partition();
-                  _elem461.read(iprot);
-                  struct.partitions.add(_elem461);
+                  _elem477 = new Partition();
+                  _elem477.read(iprot);
+                  struct.partitions.add(_elem477);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-            for (Partition _iter463 : struct.partitions)
+            for (Partition _iter479 : struct.partitions)
             {
-              _iter463.write(oprot);
+              _iter479.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitions()) {
         {
           oprot.writeI32(struct.partitions.size());
-          for (Partition _iter464 : struct.partitions)
+          for (Partition _iter480 : struct.partitions)
           {
-            _iter464.write(oprot);
+            _iter480.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list465 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitions = new ArrayList<Partition>(_list465.size);
-          Partition _elem466;
-          for (int _i467 = 0; _i467 < _list465.size; ++_i467)
+          org.apache.thrift.protocol.TList _list481 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitions = new ArrayList<Partition>(_list481.size);
+          Partition _elem482;
+          for (int _i483 = 0; _i483 < _list481.size; ++_i483)
           {
-            _elem466 = new Partition();
-            _elem466.read(iprot);
-            struct.partitions.add(_elem466);
+            _elem482 = new Partition();
+            _elem482.read(iprot);
+            struct.partitions.add(_elem482);
           }
         }
         struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
index 8936410..ca357ed 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
@@ -713,13 +713,13 @@ import org.slf4j.LoggerFactory;
           case 5: // PARTITION_VALS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list686 = iprot.readListBegin();
-                struct.partitionVals = new ArrayList<String>(_list686.size);
-                String _elem687;
-                for (int _i688 = 0; _i688 < _list686.size; ++_i688)
+                org.apache.thrift.protocol.TList _list702 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list702.size);
+                String _elem703;
+                for (int _i704 = 0; _i704 < _list702.size; ++_i704)
                 {
-                  _elem687 = iprot.readString();
-                  struct.partitionVals.add(_elem687);
+                  _elem703 = iprot.readString();
+                  struct.partitionVals.add(_elem703);
                 }
                 iprot.readListEnd();
               }
@@ -768,9 +768,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITION_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionVals.size()));
-            for (String _iter689 : struct.partitionVals)
+            for (String _iter705 : struct.partitionVals)
             {
-              oprot.writeString(_iter689);
+              oprot.writeString(_iter705);
             }
             oprot.writeListEnd();
           }
@@ -816,9 +816,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionVals()) {
         {
           oprot.writeI32(struct.partitionVals.size());
-          for (String _iter690 : struct.partitionVals)
+          for (String _iter706 : struct.partitionVals)
           {
-            oprot.writeString(_iter690);
+            oprot.writeString(_iter706);
           }
         }
       }
@@ -843,13 +843,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list691 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partitionVals = new ArrayList<String>(_list691.size);
-          String _elem692;
-          for (int _i693 = 0; _i693 < _list691.size; ++_i693)
+          org.apache.thrift.protocol.TList _list707 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list707.size);
+          String _elem708;
+          for (int _i709 = 0; _i709 < _list707.size; ++_i709)
           {
-            _elem692 = iprot.readString();
-            struct.partitionVals.add(_elem692);
+            _elem708 = iprot.readString();
+            struct.partitionVals.add(_elem708);
           }
         }
         struct.setPartitionValsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
index ca62b88..32e94cc 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
@@ -997,14 +997,14 @@ import org.slf4j.LoggerFactory;
           case 8: // RESOURCE_URIS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list516 = iprot.readListBegin();
-                struct.resourceUris = new ArrayList<ResourceUri>(_list516.size);
-                ResourceUri _elem517;
-                for (int _i518 = 0; _i518 < _list516.size; ++_i518)
+                org.apache.thrift.protocol.TList _list532 = iprot.readListBegin();
+                struct.resourceUris = new ArrayList<ResourceUri>(_list532.size);
+                ResourceUri _elem533;
+                for (int _i534 = 0; _i534 < _list532.size; ++_i534)
                 {
-                  _elem517 = new ResourceUri();
-                  _elem517.read(iprot);
-                  struct.resourceUris.add(_elem517);
+                  _elem533 = new ResourceUri();
+                  _elem533.read(iprot);
+                  struct.resourceUris.add(_elem533);
                 }
                 iprot.readListEnd();
               }
@@ -1063,9 +1063,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(RESOURCE_URIS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.resourceUris.size()));
-          for (ResourceUri _iter519 : struct.resourceUris)
+          for (ResourceUri _iter535 : struct.resourceUris)
           {
-            _iter519.write(oprot);
+            _iter535.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1138,9 +1138,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetResourceUris()) {
         {
           oprot.writeI32(struct.resourceUris.size());
-          for (ResourceUri _iter520 : struct.resourceUris)
+          for (ResourceUri _iter536 : struct.resourceUris)
           {
-            _iter520.write(oprot);
+            _iter536.write(oprot);
           }
         }
       }
@@ -1180,14 +1180,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(7)) {
         {
-          org.apache.thrift.protocol.TList _list521 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.resourceUris = new ArrayList<ResourceUri>(_list521.size);
-          ResourceUri _elem522;
-          for (int _i523 = 0; _i523 < _list521.size; ++_i523)
+          org.apache.thrift.protocol.TList _list537 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.resourceUris = new ArrayList<ResourceUri>(_list537.size);
+          ResourceUri _elem538;
+          for (int _i539 = 0; _i539 < _list537.size; ++_i539)
           {
-            _elem522 = new ResourceUri();
-            _elem522.read(iprot);
-            struct.resourceUris.add(_elem522);
+            _elem538 = new ResourceUri();
+            _elem538.read(iprot);
+            struct.resourceUris.add(_elem538);
           }
         }
         struct.setResourceUrisIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
index ba29e90..dd5ea74 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // FUNCTIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list754 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list754.size);
-                Function _elem755;
-                for (int _i756 = 0; _i756 < _list754.size; ++_i756)
+                org.apache.thrift.protocol.TList _list770 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list770.size);
+                Function _elem771;
+                for (int _i772 = 0; _i772 < _list770.size; ++_i772)
                 {
-                  _elem755 = new Function();
-                  _elem755.read(iprot);
-                  struct.functions.add(_elem755);
+                  _elem771 = new Function();
+                  _elem771.read(iprot);
+                  struct.functions.add(_elem771);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FUNCTIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.functions.size()));
-            for (Function _iter757 : struct.functions)
+            for (Function _iter773 : struct.functions)
             {
-              _iter757.write(oprot);
+              _iter773.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFunctions()) {
         {
           oprot.writeI32(struct.functions.size());
-          for (Function _iter758 : struct.functions)
+          for (Function _iter774 : struct.functions)
           {
-            _iter758.write(oprot);
+            _iter774.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list759 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.functions = new ArrayList<Function>(_list759.size);
-          Function _elem760;
-          for (int _i761 = 0; _i761 < _list759.size; ++_i761)
+          org.apache.thrift.protocol.TList _list775 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list775.size);
+          Function _elem776;
+          for (int _i777 = 0; _i777 < _list775.size; ++_i777)
           {
-            _elem760 = new Function();
-            _elem760.read(iprot);
-            struct.functions.add(_elem760);
+            _elem776 = new Function();
+            _elem776.read(iprot);
+            struct.functions.add(_elem776);
           }
         }
         struct.setFunctionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
index 62b0768..a3a5779 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
@@ -619,13 +619,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list704 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list704.size);
-                long _elem705;
-                for (int _i706 = 0; _i706 < _list704.size; ++_i706)
+                org.apache.thrift.protocol.TList _list720 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list720.size);
+                long _elem721;
+                for (int _i722 = 0; _i722 < _list720.size; ++_i722)
                 {
-                  _elem705 = iprot.readI64();
-                  struct.fileIds.add(_elem705);
+                  _elem721 = iprot.readI64();
+                  struct.fileIds.add(_elem721);
                 }
                 iprot.readListEnd();
               }
@@ -675,9 +675,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter707 : struct.fileIds)
+          for (long _iter723 : struct.fileIds)
           {
-            oprot.writeI64(_iter707);
+            oprot.writeI64(_iter723);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter708 : struct.fileIds)
+        for (long _iter724 : struct.fileIds)
         {
-          oprot.writeI64(_iter708);
+          oprot.writeI64(_iter724);
         }
       }
       oprot.writeBinary(struct.expr);
@@ -745,13 +745,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list709 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list709.size);
-        long _elem710;
-        for (int _i711 = 0; _i711 < _list709.size; ++_i711)
+        org.apache.thrift.protocol.TList _list725 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list725.size);
+        long _elem726;
+        for (int _i727 = 0; _i727 < _list725.size; ++_i727)
         {
-          _elem710 = iprot.readI64();
-          struct.fileIds.add(_elem710);
+          _elem726 = iprot.readI64();
+          struct.fileIds.add(_elem726);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
index 881803f..7604f19 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
@@ -444,16 +444,16 @@ import org.slf4j.LoggerFactory;
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map694 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map694.size);
-                long _key695;
-                MetadataPpdResult _val696;
-                for (int _i697 = 0; _i697 < _map694.size; ++_i697)
+                org.apache.thrift.protocol.TMap _map710 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map710.size);
+                long _key711;
+                MetadataPpdResult _val712;
+                for (int _i713 = 0; _i713 < _map710.size; ++_i713)
                 {
-                  _key695 = iprot.readI64();
-                  _val696 = new MetadataPpdResult();
-                  _val696.read(iprot);
-                  struct.metadata.put(_key695, _val696);
+                  _key711 = iprot.readI64();
+                  _val712 = new MetadataPpdResult();
+                  _val712.read(iprot);
+                  struct.metadata.put(_key711, _val712);
                 }
                 iprot.readMapEnd();
               }
@@ -487,10 +487,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, struct.metadata.size()));
-          for (Map.Entry<Long, MetadataPpdResult> _iter698 : struct.metadata.entrySet())
+          for (Map.Entry<Long, MetadataPpdResult> _iter714 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter698.getKey());
-            _iter698.getValue().write(oprot);
+            oprot.writeI64(_iter714.getKey());
+            _iter714.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -518,10 +518,10 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, MetadataPpdResult> _iter699 : struct.metadata.entrySet())
+        for (Map.Entry<Long, MetadataPpdResult> _iter715 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter699.getKey());
-          _iter699.getValue().write(oprot);
+          oprot.writeI64(_iter715.getKey());
+          _iter715.getValue().write(oprot);
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -531,16 +531,16 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map700 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map700.size);
-        long _key701;
-        MetadataPpdResult _val702;
-        for (int _i703 = 0; _i703 < _map700.size; ++_i703)
+        org.apache.thrift.protocol.TMap _map716 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map716.size);
+        long _key717;
+        MetadataPpdResult _val718;
+        for (int _i719 = 0; _i719 < _map716.size; ++_i719)
         {
-          _key701 = iprot.readI64();
-          _val702 = new MetadataPpdResult();
-          _val702.read(iprot);
-          struct.metadata.put(_key701, _val702);
+          _key717 = iprot.readI64();
+          _val718 = new MetadataPpdResult();
+          _val718.read(iprot);
+          struct.metadata.put(_key717, _val718);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
index a051fb0..ce73688 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list722 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list722.size);
-                long _elem723;
-                for (int _i724 = 0; _i724 < _list722.size; ++_i724)
+                org.apache.thrift.protocol.TList _list738 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list738.size);
+                long _elem739;
+                for (int _i740 = 0; _i740 < _list738.size; ++_i740)
                 {
-                  _elem723 = iprot.readI64();
-                  struct.fileIds.add(_elem723);
+                  _elem739 = iprot.readI64();
+                  struct.fileIds.add(_elem739);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter725 : struct.fileIds)
+          for (long _iter741 : struct.fileIds)
           {
-            oprot.writeI64(_iter725);
+            oprot.writeI64(_iter741);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter726 : struct.fileIds)
+        for (long _iter742 : struct.fileIds)
         {
-          oprot.writeI64(_iter726);
+          oprot.writeI64(_iter742);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list727 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list727.size);
-        long _elem728;
-        for (int _i729 = 0; _i729 < _list727.size; ++_i729)
+        org.apache.thrift.protocol.TList _list743 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list743.size);
+        long _elem744;
+        for (int _i745 = 0; _i745 < _list743.size; ++_i745)
         {
-          _elem728 = iprot.readI64();
-          struct.fileIds.add(_elem728);
+          _elem744 = iprot.readI64();
+          struct.fileIds.add(_elem744);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
index 74ca66a..f797ce8 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
@@ -433,15 +433,15 @@ import org.slf4j.LoggerFactory;
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map712 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map712.size);
-                long _key713;
-                ByteBuffer _val714;
-                for (int _i715 = 0; _i715 < _map712.size; ++_i715)
+                org.apache.thrift.protocol.TMap _map728 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map728.size);
+                long _key729;
+                ByteBuffer _val730;
+                for (int _i731 = 0; _i731 < _map728.size; ++_i731)
                 {
-                  _key713 = iprot.readI64();
-                  _val714 = iprot.readBinary();
-                  struct.metadata.put(_key713, _val714);
+                  _key729 = iprot.readI64();
+                  _val730 = iprot.readBinary();
+                  struct.metadata.put(_key729, _val730);
                 }
                 iprot.readMapEnd();
               }
@@ -475,10 +475,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (Map.Entry<Long, ByteBuffer> _iter716 : struct.metadata.entrySet())
+          for (Map.Entry<Long, ByteBuffer> _iter732 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter716.getKey());
-            oprot.writeBinary(_iter716.getValue());
+            oprot.writeI64(_iter732.getKey());
+            oprot.writeBinary(_iter732.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -506,10 +506,10 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, ByteBuffer> _iter717 : struct.metadata.entrySet())
+        for (Map.Entry<Long, ByteBuffer> _iter733 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter717.getKey());
-          oprot.writeBinary(_iter717.getValue());
+          oprot.writeI64(_iter733.getKey());
+          oprot.writeBinary(_iter733.getValue());
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -519,15 +519,15 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map718 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new HashMap<Long,ByteBuffer>(2*_map718.size);
-        long _key719;
-        ByteBuffer _val720;
-        for (int _i721 = 0; _i721 < _map718.size; ++_i721)
+        org.apache.thrift.protocol.TMap _map734 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new HashMap<Long,ByteBuffer>(2*_map734.size);
+        long _key735;
+        ByteBuffer _val736;
+        for (int _i737 = 0; _i737 < _map734.size; ++_i737)
         {
-          _key719 = iprot.readI64();
-          _val720 = iprot.readBinary();
-          struct.metadata.put(_key719, _val720);
+          _key735 = iprot.readI64();
+          _val736 = iprot.readBinary();
+          struct.metadata.put(_key735, _val736);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
index ae644df..ede4b7e 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
@@ -447,14 +447,14 @@ import org.slf4j.LoggerFactory;
           case 2: // OPEN_TXNS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list524 = iprot.readListBegin();
-                struct.open_txns = new ArrayList<TxnInfo>(_list524.size);
-                TxnInfo _elem525;
-                for (int _i526 = 0; _i526 < _list524.size; ++_i526)
+                org.apache.thrift.protocol.TList _list540 = iprot.readListBegin();
+                struct.open_txns = new ArrayList<TxnInfo>(_list540.size);
+                TxnInfo _elem541;
+                for (int _i542 = 0; _i542 < _list540.size; ++_i542)
                 {
-                  _elem525 = new TxnInfo();
-                  _elem525.read(iprot);
-                  struct.open_txns.add(_elem525);
+                  _elem541 = new TxnInfo();
+                  _elem541.read(iprot);
+                  struct.open_txns.add(_elem541);
                 }
                 iprot.readListEnd();
               }
@@ -483,9 +483,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(OPEN_TXNS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.open_txns.size()));
-          for (TxnInfo _iter527 : struct.open_txns)
+          for (TxnInfo _iter543 : struct.open_txns)
           {
-            _iter527.write(oprot);
+            _iter543.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -511,9 +511,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeI64(struct.txn_high_water_mark);
       {
         oprot.writeI32(struct.open_txns.size());
-        for (TxnInfo _iter528 : struct.open_txns)
+        for (TxnInfo _iter544 : struct.open_txns)
         {
-          _iter528.write(oprot);
+          _iter544.write(oprot);
         }
       }
     }
@@ -524,14 +524,14 @@ import org.slf4j.LoggerFactory;
       struct.txn_high_water_mark = iprot.readI64();
       struct.setTxn_high_water_markIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list529 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.open_txns = new ArrayList<TxnInfo>(_list529.size);
-        TxnInfo _elem530;
-        for (int _i531 = 0; _i531 < _list529.size; ++_i531)
+        org.apache.thrift.protocol.TList _list545 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.open_txns = new ArrayList<TxnInfo>(_list545.size);
+        TxnInfo _elem546;
+        for (int _i547 = 0; _i547 < _list545.size; ++_i547)
         {
-          _elem530 = new TxnInfo();
-          _elem530.read(iprot);
-          struct.open_txns.add(_elem530);
+          _elem546 = new TxnInfo();
+          _elem546.read(iprot);
+          struct.open_txns.add(_elem546);
         }
       }
       struct.setOpen_txnsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
index 662c093..9c6bcb7 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
@@ -615,13 +615,13 @@ import org.slf4j.LoggerFactory;
           case 2: // OPEN_TXNS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list532 = iprot.readListBegin();
-                struct.open_txns = new ArrayList<Long>(_list532.size);
-                long _elem533;
-                for (int _i534 = 0; _i534 < _list532.size; ++_i534)
+                org.apache.thrift.protocol.TList _list548 = iprot.readListBegin();
+                struct.open_txns = new ArrayList<Long>(_list548.size);
+                long _elem549;
+                for (int _i550 = 0; _i550 < _list548.size; ++_i550)
                 {
-                  _elem533 = iprot.readI64();
-                  struct.open_txns.add(_elem533);
+                  _elem549 = iprot.readI64();
+                  struct.open_txns.add(_elem549);
                 }
                 iprot.readListEnd();
               }
@@ -666,9 +666,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(OPEN_TXNS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.open_txns.size()));
-          for (long _iter535 : struct.open_txns)
+          for (long _iter551 : struct.open_txns)
           {
-            oprot.writeI64(_iter535);
+            oprot.writeI64(_iter551);
           }
           oprot.writeListEnd();
         }
@@ -704,9 +704,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeI64(struct.txn_high_water_mark);
       {
         oprot.writeI32(struct.open_txns.size());
-        for (long _iter536 : struct.open_txns)
+        for (long _iter552 : struct.open_txns)
         {
-          oprot.writeI64(_iter536);
+          oprot.writeI64(_iter552);
         }
       }
       oprot.writeBinary(struct.abortedBits);
@@ -726,13 +726,13 @@ import org.slf4j.LoggerFactory;
       struct.txn_high_water_mark = iprot.readI64();
       struct.setTxn_high_water_markIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list537 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.open_txns = new ArrayList<Long>(_list537.size);
-        long _elem538;
-        for (int _i539 = 0; _i539 < _list537.size; ++_i539)
+        org.apache.thrift.protocol.TList _list553 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.open_txns = new ArrayList<Long>(_list553.size);
+        long _elem554;
+        for (int _i555 = 0; _i555 < _list553.size; ++_i555)
         {
-          _elem538 = iprot.readI64();
-          struct.open_txns.add(_elem538);
+          _elem554 = iprot.readI64();
+          struct.open_txns.add(_elem554);
         }
       }
       struct.setOpen_txnsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
index 84af22f..f495dd8 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
@@ -525,13 +525,13 @@ import org.slf4j.LoggerFactory;
           case 2: // TBL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list770 = iprot.readListBegin();
-                struct.tblNames = new ArrayList<String>(_list770.size);
-                String _elem771;
-                for (int _i772 = 0; _i772 < _list770.size; ++_i772)
+                org.apache.thrift.protocol.TList _list786 = iprot.readListBegin();
+                struct.tblNames = new ArrayList<String>(_list786.size);
+                String _elem787;
+                for (int _i788 = 0; _i788 < _list786.size; ++_i788)
                 {
-                  _elem771 = iprot.readString();
-                  struct.tblNames.add(_elem771);
+                  _elem787 = iprot.readString();
+                  struct.tblNames.add(_elem787);
                 }
                 iprot.readListEnd();
               }
@@ -572,9 +572,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tblNames.size()));
-            for (String _iter773 : struct.tblNames)
+            for (String _iter789 : struct.tblNames)
             {
-              oprot.writeString(_iter773);
+              oprot.writeString(_iter789);
             }
             oprot.writeListEnd();
           }
@@ -617,9 +617,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTblNames()) {
         {
           oprot.writeI32(struct.tblNames.size());
-          for (String _iter774 : struct.tblNames)
+          for (String _iter790 : struct.tblNames)
           {
-            oprot.writeString(_iter774);
+            oprot.writeString(_iter790);
           }
         }
       }
@@ -636,13 +636,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list775 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.tblNames = new ArrayList<String>(_list775.size);
-          String _elem776;
-          for (int _i777 = 0; _i777 < _list775.size; ++_i777)
+          org.apache.thrift.protocol.TList _list791 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.tblNames = new ArrayList<String>(_list791.size);
+          String _elem792;
+          for (int _i793 = 0; _i793 < _list791.size; ++_i793)
           {
-            _elem776 = iprot.readString();
-            struct.tblNames.add(_elem776);
+            _elem792 = iprot.readString();
+            struct.tblNames.add(_elem792);
           }
         }
         struct.setTblNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
index 4aba1d2..08755d7 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TABLES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list778 = iprot.readListBegin();
-                struct.tables = new ArrayList<Table>(_list778.size);
-                Table _elem779;
-                for (int _i780 = 0; _i780 < _list778.size; ++_i780)
+                org.apache.thrift.protocol.TList _list794 = iprot.readListBegin();
+                struct.tables = new ArrayList<Table>(_list794.size);
+                Table _elem795;
+                for (int _i796 = 0; _i796 < _list794.size; ++_i796)
                 {
-                  _elem779 = new Table();
-                  _elem779.read(iprot);
-                  struct.tables.add(_elem779);
+                  _elem795 = new Table();
+                  _elem795.read(iprot);
+                  struct.tables.add(_elem795);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tables.size()));
-          for (Table _iter781 : struct.tables)
+          for (Table _iter797 : struct.tables)
           {
-            _iter781.write(oprot);
+            _iter797.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tables.size());
-        for (Table _iter782 : struct.tables)
+        for (Table _iter798 : struct.tables)
         {
-          _iter782.write(oprot);
+          _iter798.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetTablesResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list783 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tables = new ArrayList<Table>(_list783.size);
-        Table _elem784;
-        for (int _i785 = 0; _i785 < _list783.size; ++_i785)
+        org.apache.thrift.protocol.TList _list799 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tables = new ArrayList<Table>(_list799.size);
+        Table _elem800;
+        for (int _i801 = 0; _i801 < _list799.size; ++_i801)
         {
-          _elem784 = new Table();
-          _elem784.read(iprot);
-          struct.tables.add(_elem784);
+          _elem800 = new Table();
+          _elem800.read(iprot);
+          struct.tables.add(_elem800);
         }
       }
       struct.setTablesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
index ec738b0..1eead3e 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
@@ -436,13 +436,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FULL_TABLE_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list556 = iprot.readListBegin();
-                struct.fullTableNames = new ArrayList<String>(_list556.size);
-                String _elem557;
-                for (int _i558 = 0; _i558 < _list556.size; ++_i558)
+                org.apache.thrift.protocol.TList _list572 = iprot.readListBegin();
+                struct.fullTableNames = new ArrayList<String>(_list572.size);
+                String _elem573;
+                for (int _i574 = 0; _i574 < _list572.size; ++_i574)
                 {
-                  _elem557 = iprot.readString();
-                  struct.fullTableNames.add(_elem557);
+                  _elem573 = iprot.readString();
+                  struct.fullTableNames.add(_elem573);
                 }
                 iprot.readListEnd();
               }
@@ -476,9 +476,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FULL_TABLE_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.fullTableNames.size()));
-          for (String _iter559 : struct.fullTableNames)
+          for (String _iter575 : struct.fullTableNames)
           {
-            oprot.writeString(_iter559);
+            oprot.writeString(_iter575);
           }
           oprot.writeListEnd();
         }
@@ -508,9 +508,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fullTableNames.size());
-        for (String _iter560 : struct.fullTableNames)
+        for (String _iter576 : struct.fullTableNames)
         {
-          oprot.writeString(_iter560);
+          oprot.writeString(_iter576);
         }
       }
       oprot.writeString(struct.validTxnList);
@@ -520,13 +520,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetValidWriteIdsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list561 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.fullTableNames = new ArrayList<String>(_list561.size);
-        String _elem562;
-        for (int _i563 = 0; _i563 < _list561.size; ++_i563)
+        org.apache.thrift.protocol.TList _list577 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.fullTableNames = new ArrayList<String>(_list577.size);
+        String _elem578;
+        for (int _i579 = 0; _i579 < _list577.size; ++_i579)
         {
-          _elem562 = iprot.readString();
-          struct.fullTableNames.add(_elem562);
+          _elem578 = iprot.readString();
+          struct.fullTableNames.add(_elem578);
         }
       }
       struct.setFullTableNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
index 50eba33..f421613 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TBL_VALID_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list572 = iprot.readListBegin();
-                struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list572.size);
-                TableValidWriteIds _elem573;
-                for (int _i574 = 0; _i574 < _list572.size; ++_i574)
+                org.apache.thrift.protocol.TList _list588 = iprot.readListBegin();
+                struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list588.size);
+                TableValidWriteIds _elem589;
+                for (int _i590 = 0; _i590 < _list588.size; ++_i590)
                 {
-                  _elem573 = new TableValidWriteIds();
-                  _elem573.read(iprot);
-                  struct.tblValidWriteIds.add(_elem573);
+                  _elem589 = new TableValidWriteIds();
+                  _elem589.read(iprot);
+                  struct.tblValidWriteIds.add(_elem589);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TBL_VALID_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tblValidWriteIds.size()));
-          for (TableValidWriteIds _iter575 : struct.tblValidWriteIds)
+          for (TableValidWriteIds _iter591 : struct.tblValidWriteIds)
           {
-            _iter575.write(oprot);
+            _iter591.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tblValidWriteIds.size());
-        for (TableValidWriteIds _iter576 : struct.tblValidWriteIds)
+        for (TableValidWriteIds _iter592 : struct.tblValidWriteIds)
         {
-          _iter576.write(oprot);
+          _iter592.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetValidWriteIdsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list577 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list577.size);
-        TableValidWriteIds _elem578;
-        for (int _i579 = 0; _i579 < _list577.size; ++_i579)
+        org.apache.thrift.protocol.TList _list593 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list593.size);
+        TableValidWriteIds _elem594;
+        for (int _i595 = 0; _i595 < _list593.size; ++_i595)
         {
-          _elem578 = new TableValidWriteIds();
-          _elem578.read(iprot);
-          struct.tblValidWriteIds.add(_elem578);
+          _elem594 = new TableValidWriteIds();
+          _elem594.read(iprot);
+          struct.tblValidWriteIds.add(_elem594);
         }
       }
       struct.setTblValidWriteIdsIsSet(true);


[04/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index b3a0524..72c0397 100755
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -42,12 +42,13 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('   get_schema_with_environment_context(string db_name, string table_name, EnvironmentContext environment_context)')
   print('  void create_table(Table tbl)')
   print('  void create_table_with_environment_context(Table tbl, EnvironmentContext environment_context)')
-  print('  void create_table_with_constraints(Table tbl,  primaryKeys,  foreignKeys,  uniqueConstraints,  notNullConstraints)')
+  print('  void create_table_with_constraints(Table tbl,  primaryKeys,  foreignKeys,  uniqueConstraints,  notNullConstraints,  defaultConstraints)')
   print('  void drop_constraint(DropConstraintRequest req)')
   print('  void add_primary_key(AddPrimaryKeyRequest req)')
   print('  void add_foreign_key(AddForeignKeyRequest req)')
   print('  void add_unique_constraint(AddUniqueConstraintRequest req)')
   print('  void add_not_null_constraint(AddNotNullConstraintRequest req)')
+  print('  void add_default_constraint(AddDefaultConstraintRequest req)')
   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('  void truncate_table(string dbName, string tableName,  partNames)')
@@ -119,6 +120,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  ForeignKeysResponse get_foreign_keys(ForeignKeysRequest request)')
   print('  UniqueConstraintsResponse get_unique_constraints(UniqueConstraintsRequest request)')
   print('  NotNullConstraintsResponse get_not_null_constraints(NotNullConstraintsRequest request)')
+  print('  DefaultConstraintsResponse get_default_constraints(DefaultConstraintsRequest request)')
   print('  bool update_table_column_statistics(ColumnStatistics stats_obj)')
   print('  bool update_partition_column_statistics(ColumnStatistics stats_obj)')
   print('  ColumnStatistics get_table_column_statistics(string db_name, string tbl_name, string col_name)')
@@ -386,10 +388,10 @@ elif cmd == 'create_table_with_environment_context':
   pp.pprint(client.create_table_with_environment_context(eval(args[0]),eval(args[1]),))
 
 elif cmd == 'create_table_with_constraints':
-  if len(args) != 5:
-    print('create_table_with_constraints requires 5 args')
+  if len(args) != 6:
+    print('create_table_with_constraints requires 6 args')
     sys.exit(1)
-  pp.pprint(client.create_table_with_constraints(eval(args[0]),eval(args[1]),eval(args[2]),eval(args[3]),eval(args[4]),))
+  pp.pprint(client.create_table_with_constraints(eval(args[0]),eval(args[1]),eval(args[2]),eval(args[3]),eval(args[4]),eval(args[5]),))
 
 elif cmd == 'drop_constraint':
   if len(args) != 1:
@@ -421,6 +423,12 @@ elif cmd == 'add_not_null_constraint':
     sys.exit(1)
   pp.pprint(client.add_not_null_constraint(eval(args[0]),))
 
+elif cmd == 'add_default_constraint':
+  if len(args) != 1:
+    print('add_default_constraint requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.add_default_constraint(eval(args[0]),))
+
 elif cmd == 'drop_table':
   if len(args) != 3:
     print('drop_table requires 3 args')
@@ -847,6 +855,12 @@ elif cmd == 'get_not_null_constraints':
     sys.exit(1)
   pp.pprint(client.get_not_null_constraints(eval(args[0]),))
 
+elif cmd == 'get_default_constraints':
+  if len(args) != 1:
+    print('get_default_constraints requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_default_constraints(eval(args[0]),))
+
 elif cmd == 'update_table_column_statistics':
   if len(args) != 1:
     print('update_table_column_statistics requires 1 args')


[13/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index ef138e0..cdcde51 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -1446,6 +1446,232 @@ void SQLNotNullConstraint::printTo(std::ostream& out) const {
 }
 
 
+SQLDefaultConstraint::~SQLDefaultConstraint() throw() {
+}
+
+
+void SQLDefaultConstraint::__set_table_db(const std::string& val) {
+  this->table_db = val;
+}
+
+void SQLDefaultConstraint::__set_table_name(const std::string& val) {
+  this->table_name = val;
+}
+
+void SQLDefaultConstraint::__set_column_name(const std::string& val) {
+  this->column_name = val;
+}
+
+void SQLDefaultConstraint::__set_default_value(const std::string& val) {
+  this->default_value = val;
+}
+
+void SQLDefaultConstraint::__set_dc_name(const std::string& val) {
+  this->dc_name = val;
+}
+
+void SQLDefaultConstraint::__set_enable_cstr(const bool val) {
+  this->enable_cstr = val;
+}
+
+void SQLDefaultConstraint::__set_validate_cstr(const bool val) {
+  this->validate_cstr = val;
+}
+
+void SQLDefaultConstraint::__set_rely_cstr(const bool val) {
+  this->rely_cstr = val;
+}
+
+uint32_t SQLDefaultConstraint::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->table_db);
+          this->__isset.table_db = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->table_name);
+          this->__isset.table_name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->column_name);
+          this->__isset.column_name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->default_value);
+          this->__isset.default_value = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->dc_name);
+          this->__isset.dc_name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->enable_cstr);
+          this->__isset.enable_cstr = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 7:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->validate_cstr);
+          this->__isset.validate_cstr = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 8:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->rely_cstr);
+          this->__isset.rely_cstr = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t SQLDefaultConstraint::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("SQLDefaultConstraint");
+
+  xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->table_db);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->table_name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->column_name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("default_value", ::apache::thrift::protocol::T_STRING, 4);
+  xfer += oprot->writeString(this->default_value);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("dc_name", ::apache::thrift::protocol::T_STRING, 5);
+  xfer += oprot->writeString(this->dc_name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 6);
+  xfer += oprot->writeBool(this->enable_cstr);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 7);
+  xfer += oprot->writeBool(this->validate_cstr);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 8);
+  xfer += oprot->writeBool(this->rely_cstr);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(SQLDefaultConstraint &a, SQLDefaultConstraint &b) {
+  using ::std::swap;
+  swap(a.table_db, b.table_db);
+  swap(a.table_name, b.table_name);
+  swap(a.column_name, b.column_name);
+  swap(a.default_value, b.default_value);
+  swap(a.dc_name, b.dc_name);
+  swap(a.enable_cstr, b.enable_cstr);
+  swap(a.validate_cstr, b.validate_cstr);
+  swap(a.rely_cstr, b.rely_cstr);
+  swap(a.__isset, b.__isset);
+}
+
+SQLDefaultConstraint::SQLDefaultConstraint(const SQLDefaultConstraint& other12) {
+  table_db = other12.table_db;
+  table_name = other12.table_name;
+  column_name = other12.column_name;
+  default_value = other12.default_value;
+  dc_name = other12.dc_name;
+  enable_cstr = other12.enable_cstr;
+  validate_cstr = other12.validate_cstr;
+  rely_cstr = other12.rely_cstr;
+  __isset = other12.__isset;
+}
+SQLDefaultConstraint& SQLDefaultConstraint::operator=(const SQLDefaultConstraint& other13) {
+  table_db = other13.table_db;
+  table_name = other13.table_name;
+  column_name = other13.column_name;
+  default_value = other13.default_value;
+  dc_name = other13.dc_name;
+  enable_cstr = other13.enable_cstr;
+  validate_cstr = other13.validate_cstr;
+  rely_cstr = other13.rely_cstr;
+  __isset = other13.__isset;
+  return *this;
+}
+void SQLDefaultConstraint::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "SQLDefaultConstraint(";
+  out << "table_db=" << to_string(table_db);
+  out << ", " << "table_name=" << to_string(table_name);
+  out << ", " << "column_name=" << to_string(column_name);
+  out << ", " << "default_value=" << to_string(default_value);
+  out << ", " << "dc_name=" << to_string(dc_name);
+  out << ", " << "enable_cstr=" << to_string(enable_cstr);
+  out << ", " << "validate_cstr=" << to_string(validate_cstr);
+  out << ", " << "rely_cstr=" << to_string(rely_cstr);
+  out << ")";
+}
+
+
 Type::~Type() throw() {
 }
 
@@ -1518,14 +1744,14 @@ uint32_t Type::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fields.clear();
-            uint32_t _size12;
-            ::apache::thrift::protocol::TType _etype15;
-            xfer += iprot->readListBegin(_etype15, _size12);
-            this->fields.resize(_size12);
-            uint32_t _i16;
-            for (_i16 = 0; _i16 < _size12; ++_i16)
+            uint32_t _size14;
+            ::apache::thrift::protocol::TType _etype17;
+            xfer += iprot->readListBegin(_etype17, _size14);
+            this->fields.resize(_size14);
+            uint32_t _i18;
+            for (_i18 = 0; _i18 < _size14; ++_i18)
             {
-              xfer += this->fields[_i16].read(iprot);
+              xfer += this->fields[_i18].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -1569,10 +1795,10 @@ uint32_t Type::write(::apache::thrift::protocol::TProtocol* oprot) const {
     xfer += oprot->writeFieldBegin("fields", ::apache::thrift::protocol::T_LIST, 4);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->fields.size()));
-      std::vector<FieldSchema> ::const_iterator _iter17;
-      for (_iter17 = this->fields.begin(); _iter17 != this->fields.end(); ++_iter17)
+      std::vector<FieldSchema> ::const_iterator _iter19;
+      for (_iter19 = this->fields.begin(); _iter19 != this->fields.end(); ++_iter19)
       {
-        xfer += (*_iter17).write(oprot);
+        xfer += (*_iter19).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -1592,19 +1818,19 @@ void swap(Type &a, Type &b) {
   swap(a.__isset, b.__isset);
 }
 
-Type::Type(const Type& other18) {
-  name = other18.name;
-  type1 = other18.type1;
-  type2 = other18.type2;
-  fields = other18.fields;
-  __isset = other18.__isset;
+Type::Type(const Type& other20) {
+  name = other20.name;
+  type1 = other20.type1;
+  type2 = other20.type2;
+  fields = other20.fields;
+  __isset = other20.__isset;
 }
-Type& Type::operator=(const Type& other19) {
-  name = other19.name;
-  type1 = other19.type1;
-  type2 = other19.type2;
-  fields = other19.fields;
-  __isset = other19.__isset;
+Type& Type::operator=(const Type& other21) {
+  name = other21.name;
+  type1 = other21.type1;
+  type2 = other21.type2;
+  fields = other21.fields;
+  __isset = other21.__isset;
   return *this;
 }
 void Type::printTo(std::ostream& out) const {
@@ -1665,9 +1891,9 @@ uint32_t HiveObjectRef::read(::apache::thrift::protocol::TProtocol* iprot) {
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast20;
-          xfer += iprot->readI32(ecast20);
-          this->objectType = (HiveObjectType::type)ecast20;
+          int32_t ecast22;
+          xfer += iprot->readI32(ecast22);
+          this->objectType = (HiveObjectType::type)ecast22;
           this->__isset.objectType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -1693,14 +1919,14 @@ uint32_t HiveObjectRef::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partValues.clear();
-            uint32_t _size21;
-            ::apache::thrift::protocol::TType _etype24;
-            xfer += iprot->readListBegin(_etype24, _size21);
-            this->partValues.resize(_size21);
-            uint32_t _i25;
-            for (_i25 = 0; _i25 < _size21; ++_i25)
+            uint32_t _size23;
+            ::apache::thrift::protocol::TType _etype26;
+            xfer += iprot->readListBegin(_etype26, _size23);
+            this->partValues.resize(_size23);
+            uint32_t _i27;
+            for (_i27 = 0; _i27 < _size23; ++_i27)
             {
-              xfer += iprot->readString(this->partValues[_i25]);
+              xfer += iprot->readString(this->partValues[_i27]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1749,10 +1975,10 @@ uint32_t HiveObjectRef::write(::apache::thrift::protocol::TProtocol* oprot) cons
   xfer += oprot->writeFieldBegin("partValues", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partValues.size()));
-    std::vector<std::string> ::const_iterator _iter26;
-    for (_iter26 = this->partValues.begin(); _iter26 != this->partValues.end(); ++_iter26)
+    std::vector<std::string> ::const_iterator _iter28;
+    for (_iter28 = this->partValues.begin(); _iter28 != this->partValues.end(); ++_iter28)
     {
-      xfer += oprot->writeString((*_iter26));
+      xfer += oprot->writeString((*_iter28));
     }
     xfer += oprot->writeListEnd();
   }
@@ -1777,21 +2003,21 @@ void swap(HiveObjectRef &a, HiveObjectRef &b) {
   swap(a.__isset, b.__isset);
 }
 
-HiveObjectRef::HiveObjectRef(const HiveObjectRef& other27) {
-  objectType = other27.objectType;
-  dbName = other27.dbName;
-  objectName = other27.objectName;
-  partValues = other27.partValues;
-  columnName = other27.columnName;
-  __isset = other27.__isset;
-}
-HiveObjectRef& HiveObjectRef::operator=(const HiveObjectRef& other28) {
-  objectType = other28.objectType;
-  dbName = other28.dbName;
-  objectName = other28.objectName;
-  partValues = other28.partValues;
-  columnName = other28.columnName;
-  __isset = other28.__isset;
+HiveObjectRef::HiveObjectRef(const HiveObjectRef& other29) {
+  objectType = other29.objectType;
+  dbName = other29.dbName;
+  objectName = other29.objectName;
+  partValues = other29.partValues;
+  columnName = other29.columnName;
+  __isset = other29.__isset;
+}
+HiveObjectRef& HiveObjectRef::operator=(const HiveObjectRef& other30) {
+  objectType = other30.objectType;
+  dbName = other30.dbName;
+  objectName = other30.objectName;
+  partValues = other30.partValues;
+  columnName = other30.columnName;
+  __isset = other30.__isset;
   return *this;
 }
 void HiveObjectRef::printTo(std::ostream& out) const {
@@ -1877,9 +2103,9 @@ uint32_t PrivilegeGrantInfo::read(::apache::thrift::protocol::TProtocol* iprot)
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast29;
-          xfer += iprot->readI32(ecast29);
-          this->grantorType = (PrincipalType::type)ecast29;
+          int32_t ecast31;
+          xfer += iprot->readI32(ecast31);
+          this->grantorType = (PrincipalType::type)ecast31;
           this->__isset.grantorType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -1945,21 +2171,21 @@ void swap(PrivilegeGrantInfo &a, PrivilegeGrantInfo &b) {
   swap(a.__isset, b.__isset);
 }
 
-PrivilegeGrantInfo::PrivilegeGrantInfo(const PrivilegeGrantInfo& other30) {
-  privilege = other30.privilege;
-  createTime = other30.createTime;
-  grantor = other30.grantor;
-  grantorType = other30.grantorType;
-  grantOption = other30.grantOption;
-  __isset = other30.__isset;
-}
-PrivilegeGrantInfo& PrivilegeGrantInfo::operator=(const PrivilegeGrantInfo& other31) {
-  privilege = other31.privilege;
-  createTime = other31.createTime;
-  grantor = other31.grantor;
-  grantorType = other31.grantorType;
-  grantOption = other31.grantOption;
-  __isset = other31.__isset;
+PrivilegeGrantInfo::PrivilegeGrantInfo(const PrivilegeGrantInfo& other32) {
+  privilege = other32.privilege;
+  createTime = other32.createTime;
+  grantor = other32.grantor;
+  grantorType = other32.grantorType;
+  grantOption = other32.grantOption;
+  __isset = other32.__isset;
+}
+PrivilegeGrantInfo& PrivilegeGrantInfo::operator=(const PrivilegeGrantInfo& other33) {
+  privilege = other33.privilege;
+  createTime = other33.createTime;
+  grantor = other33.grantor;
+  grantorType = other33.grantorType;
+  grantOption = other33.grantOption;
+  __isset = other33.__isset;
   return *this;
 }
 void PrivilegeGrantInfo::printTo(std::ostream& out) const {
@@ -2033,9 +2259,9 @@ uint32_t HiveObjectPrivilege::read(::apache::thrift::protocol::TProtocol* iprot)
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast32;
-          xfer += iprot->readI32(ecast32);
-          this->principalType = (PrincipalType::type)ecast32;
+          int32_t ecast34;
+          xfer += iprot->readI32(ecast34);
+          this->principalType = (PrincipalType::type)ecast34;
           this->__isset.principalType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2096,19 +2322,19 @@ void swap(HiveObjectPrivilege &a, HiveObjectPrivilege &b) {
   swap(a.__isset, b.__isset);
 }
 
-HiveObjectPrivilege::HiveObjectPrivilege(const HiveObjectPrivilege& other33) {
-  hiveObject = other33.hiveObject;
-  principalName = other33.principalName;
-  principalType = other33.principalType;
-  grantInfo = other33.grantInfo;
-  __isset = other33.__isset;
+HiveObjectPrivilege::HiveObjectPrivilege(const HiveObjectPrivilege& other35) {
+  hiveObject = other35.hiveObject;
+  principalName = other35.principalName;
+  principalType = other35.principalType;
+  grantInfo = other35.grantInfo;
+  __isset = other35.__isset;
 }
-HiveObjectPrivilege& HiveObjectPrivilege::operator=(const HiveObjectPrivilege& other34) {
-  hiveObject = other34.hiveObject;
-  principalName = other34.principalName;
-  principalType = other34.principalType;
-  grantInfo = other34.grantInfo;
-  __isset = other34.__isset;
+HiveObjectPrivilege& HiveObjectPrivilege::operator=(const HiveObjectPrivilege& other36) {
+  hiveObject = other36.hiveObject;
+  principalName = other36.principalName;
+  principalType = other36.principalType;
+  grantInfo = other36.grantInfo;
+  __isset = other36.__isset;
   return *this;
 }
 void HiveObjectPrivilege::printTo(std::ostream& out) const {
@@ -2155,14 +2381,14 @@ uint32_t PrivilegeBag::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->privileges.clear();
-            uint32_t _size35;
-            ::apache::thrift::protocol::TType _etype38;
-            xfer += iprot->readListBegin(_etype38, _size35);
-            this->privileges.resize(_size35);
-            uint32_t _i39;
-            for (_i39 = 0; _i39 < _size35; ++_i39)
+            uint32_t _size37;
+            ::apache::thrift::protocol::TType _etype40;
+            xfer += iprot->readListBegin(_etype40, _size37);
+            this->privileges.resize(_size37);
+            uint32_t _i41;
+            for (_i41 = 0; _i41 < _size37; ++_i41)
             {
-              xfer += this->privileges[_i39].read(iprot);
+              xfer += this->privileges[_i41].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2191,10 +2417,10 @@ uint32_t PrivilegeBag::write(::apache::thrift::protocol::TProtocol* oprot) const
   xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->privileges.size()));
-    std::vector<HiveObjectPrivilege> ::const_iterator _iter40;
-    for (_iter40 = this->privileges.begin(); _iter40 != this->privileges.end(); ++_iter40)
+    std::vector<HiveObjectPrivilege> ::const_iterator _iter42;
+    for (_iter42 = this->privileges.begin(); _iter42 != this->privileges.end(); ++_iter42)
     {
-      xfer += (*_iter40).write(oprot);
+      xfer += (*_iter42).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -2211,13 +2437,13 @@ void swap(PrivilegeBag &a, PrivilegeBag &b) {
   swap(a.__isset, b.__isset);
 }
 
-PrivilegeBag::PrivilegeBag(const PrivilegeBag& other41) {
-  privileges = other41.privileges;
-  __isset = other41.__isset;
+PrivilegeBag::PrivilegeBag(const PrivilegeBag& other43) {
+  privileges = other43.privileges;
+  __isset = other43.__isset;
 }
-PrivilegeBag& PrivilegeBag::operator=(const PrivilegeBag& other42) {
-  privileges = other42.privileges;
-  __isset = other42.__isset;
+PrivilegeBag& PrivilegeBag::operator=(const PrivilegeBag& other44) {
+  privileges = other44.privileges;
+  __isset = other44.__isset;
   return *this;
 }
 void PrivilegeBag::printTo(std::ostream& out) const {
@@ -2269,26 +2495,26 @@ uint32_t PrincipalPrivilegeSet::read(::apache::thrift::protocol::TProtocol* ipro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->userPrivileges.clear();
-            uint32_t _size43;
-            ::apache::thrift::protocol::TType _ktype44;
-            ::apache::thrift::protocol::TType _vtype45;
-            xfer += iprot->readMapBegin(_ktype44, _vtype45, _size43);
-            uint32_t _i47;
-            for (_i47 = 0; _i47 < _size43; ++_i47)
+            uint32_t _size45;
+            ::apache::thrift::protocol::TType _ktype46;
+            ::apache::thrift::protocol::TType _vtype47;
+            xfer += iprot->readMapBegin(_ktype46, _vtype47, _size45);
+            uint32_t _i49;
+            for (_i49 = 0; _i49 < _size45; ++_i49)
             {
-              std::string _key48;
-              xfer += iprot->readString(_key48);
-              std::vector<PrivilegeGrantInfo> & _val49 = this->userPrivileges[_key48];
+              std::string _key50;
+              xfer += iprot->readString(_key50);
+              std::vector<PrivilegeGrantInfo> & _val51 = this->userPrivileges[_key50];
               {
-                _val49.clear();
-                uint32_t _size50;
-                ::apache::thrift::protocol::TType _etype53;
-                xfer += iprot->readListBegin(_etype53, _size50);
-                _val49.resize(_size50);
-                uint32_t _i54;
-                for (_i54 = 0; _i54 < _size50; ++_i54)
+                _val51.clear();
+                uint32_t _size52;
+                ::apache::thrift::protocol::TType _etype55;
+                xfer += iprot->readListBegin(_etype55, _size52);
+                _val51.resize(_size52);
+                uint32_t _i56;
+                for (_i56 = 0; _i56 < _size52; ++_i56)
                 {
-                  xfer += _val49[_i54].read(iprot);
+                  xfer += _val51[_i56].read(iprot);
                 }
                 xfer += iprot->readListEnd();
               }
@@ -2304,26 +2530,26 @@ uint32_t PrincipalPrivilegeSet::read(::apache::thrift::protocol::TProtocol* ipro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->groupPrivileges.clear();
-            uint32_t _size55;
-            ::apache::thrift::protocol::TType _ktype56;
-            ::apache::thrift::protocol::TType _vtype57;
-            xfer += iprot->readMapBegin(_ktype56, _vtype57, _size55);
-            uint32_t _i59;
-            for (_i59 = 0; _i59 < _size55; ++_i59)
+            uint32_t _size57;
+            ::apache::thrift::protocol::TType _ktype58;
+            ::apache::thrift::protocol::TType _vtype59;
+            xfer += iprot->readMapBegin(_ktype58, _vtype59, _size57);
+            uint32_t _i61;
+            for (_i61 = 0; _i61 < _size57; ++_i61)
             {
-              std::string _key60;
-              xfer += iprot->readString(_key60);
-              std::vector<PrivilegeGrantInfo> & _val61 = this->groupPrivileges[_key60];
+              std::string _key62;
+              xfer += iprot->readString(_key62);
+              std::vector<PrivilegeGrantInfo> & _val63 = this->groupPrivileges[_key62];
               {
-                _val61.clear();
-                uint32_t _size62;
-                ::apache::thrift::protocol::TType _etype65;
-                xfer += iprot->readListBegin(_etype65, _size62);
-                _val61.resize(_size62);
-                uint32_t _i66;
-                for (_i66 = 0; _i66 < _size62; ++_i66)
+                _val63.clear();
+                uint32_t _size64;
+                ::apache::thrift::protocol::TType _etype67;
+                xfer += iprot->readListBegin(_etype67, _size64);
+                _val63.resize(_size64);
+                uint32_t _i68;
+                for (_i68 = 0; _i68 < _size64; ++_i68)
                 {
-                  xfer += _val61[_i66].read(iprot);
+                  xfer += _val63[_i68].read(iprot);
                 }
                 xfer += iprot->readListEnd();
               }
@@ -2339,26 +2565,26 @@ uint32_t PrincipalPrivilegeSet::read(::apache::thrift::protocol::TProtocol* ipro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->rolePrivileges.clear();
-            uint32_t _size67;
-            ::apache::thrift::protocol::TType _ktype68;
-            ::apache::thrift::protocol::TType _vtype69;
-            xfer += iprot->readMapBegin(_ktype68, _vtype69, _size67);
-            uint32_t _i71;
-            for (_i71 = 0; _i71 < _size67; ++_i71)
+            uint32_t _size69;
+            ::apache::thrift::protocol::TType _ktype70;
+            ::apache::thrift::protocol::TType _vtype71;
+            xfer += iprot->readMapBegin(_ktype70, _vtype71, _size69);
+            uint32_t _i73;
+            for (_i73 = 0; _i73 < _size69; ++_i73)
             {
-              std::string _key72;
-              xfer += iprot->readString(_key72);
-              std::vector<PrivilegeGrantInfo> & _val73 = this->rolePrivileges[_key72];
+              std::string _key74;
+              xfer += iprot->readString(_key74);
+              std::vector<PrivilegeGrantInfo> & _val75 = this->rolePrivileges[_key74];
               {
-                _val73.clear();
-                uint32_t _size74;
-                ::apache::thrift::protocol::TType _etype77;
-                xfer += iprot->readListBegin(_etype77, _size74);
-                _val73.resize(_size74);
-                uint32_t _i78;
-                for (_i78 = 0; _i78 < _size74; ++_i78)
+                _val75.clear();
+                uint32_t _size76;
+                ::apache::thrift::protocol::TType _etype79;
+                xfer += iprot->readListBegin(_etype79, _size76);
+                _val75.resize(_size76);
+                uint32_t _i80;
+                for (_i80 = 0; _i80 < _size76; ++_i80)
                 {
-                  xfer += _val73[_i78].read(iprot);
+                  xfer += _val75[_i80].read(iprot);
                 }
                 xfer += iprot->readListEnd();
               }
@@ -2390,16 +2616,16 @@ uint32_t PrincipalPrivilegeSet::write(::apache::thrift::protocol::TProtocol* opr
   xfer += oprot->writeFieldBegin("userPrivileges", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->userPrivileges.size()));
-    std::map<std::string, std::vector<PrivilegeGrantInfo> > ::const_iterator _iter79;
-    for (_iter79 = this->userPrivileges.begin(); _iter79 != this->userPrivileges.end(); ++_iter79)
+    std::map<std::string, std::vector<PrivilegeGrantInfo> > ::const_iterator _iter81;
+    for (_iter81 = this->userPrivileges.begin(); _iter81 != this->userPrivileges.end(); ++_iter81)
     {
-      xfer += oprot->writeString(_iter79->first);
+      xfer += oprot->writeString(_iter81->first);
       {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter79->second.size()));
-        std::vector<PrivilegeGrantInfo> ::const_iterator _iter80;
-        for (_iter80 = _iter79->second.begin(); _iter80 != _iter79->second.end(); ++_iter80)
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter81->second.size()));
+        std::vector<PrivilegeGrantInfo> ::const_iterator _iter82;
+        for (_iter82 = _iter81->second.begin(); _iter82 != _iter81->second.end(); ++_iter82)
         {
-          xfer += (*_iter80).write(oprot);
+          xfer += (*_iter82).write(oprot);
         }
         xfer += oprot->writeListEnd();
       }
@@ -2411,16 +2637,16 @@ uint32_t PrincipalPrivilegeSet::write(::apache::thrift::protocol::TProtocol* opr
   xfer += oprot->writeFieldBegin("groupPrivileges", ::apache::thrift::protocol::T_MAP, 2);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->groupPrivileges.size()));
-    std::map<std::string, std::vector<PrivilegeGrantInfo> > ::const_iterator _iter81;
-    for (_iter81 = this->groupPrivileges.begin(); _iter81 != this->groupPrivileges.end(); ++_iter81)
+    std::map<std::string, std::vector<PrivilegeGrantInfo> > ::const_iterator _iter83;
+    for (_iter83 = this->groupPrivileges.begin(); _iter83 != this->groupPrivileges.end(); ++_iter83)
     {
-      xfer += oprot->writeString(_iter81->first);
+      xfer += oprot->writeString(_iter83->first);
       {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter81->second.size()));
-        std::vector<PrivilegeGrantInfo> ::const_iterator _iter82;
-        for (_iter82 = _iter81->second.begin(); _iter82 != _iter81->second.end(); ++_iter82)
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter83->second.size()));
+        std::vector<PrivilegeGrantInfo> ::const_iterator _iter84;
+        for (_iter84 = _iter83->second.begin(); _iter84 != _iter83->second.end(); ++_iter84)
         {
-          xfer += (*_iter82).write(oprot);
+          xfer += (*_iter84).write(oprot);
         }
         xfer += oprot->writeListEnd();
       }
@@ -2432,16 +2658,16 @@ uint32_t PrincipalPrivilegeSet::write(::apache::thrift::protocol::TProtocol* opr
   xfer += oprot->writeFieldBegin("rolePrivileges", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->rolePrivileges.size()));
-    std::map<std::string, std::vector<PrivilegeGrantInfo> > ::const_iterator _iter83;
-    for (_iter83 = this->rolePrivileges.begin(); _iter83 != this->rolePrivileges.end(); ++_iter83)
+    std::map<std::string, std::vector<PrivilegeGrantInfo> > ::const_iterator _iter85;
+    for (_iter85 = this->rolePrivileges.begin(); _iter85 != this->rolePrivileges.end(); ++_iter85)
     {
-      xfer += oprot->writeString(_iter83->first);
+      xfer += oprot->writeString(_iter85->first);
       {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter83->second.size()));
-        std::vector<PrivilegeGrantInfo> ::const_iterator _iter84;
-        for (_iter84 = _iter83->second.begin(); _iter84 != _iter83->second.end(); ++_iter84)
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter85->second.size()));
+        std::vector<PrivilegeGrantInfo> ::const_iterator _iter86;
+        for (_iter86 = _iter85->second.begin(); _iter86 != _iter85->second.end(); ++_iter86)
         {
-          xfer += (*_iter84).write(oprot);
+          xfer += (*_iter86).write(oprot);
         }
         xfer += oprot->writeListEnd();
       }
@@ -2463,17 +2689,17 @@ void swap(PrincipalPrivilegeSet &a, PrincipalPrivilegeSet &b) {
   swap(a.__isset, b.__isset);
 }
 
-PrincipalPrivilegeSet::PrincipalPrivilegeSet(const PrincipalPrivilegeSet& other85) {
-  userPrivileges = other85.userPrivileges;
-  groupPrivileges = other85.groupPrivileges;
-  rolePrivileges = other85.rolePrivileges;
-  __isset = other85.__isset;
+PrincipalPrivilegeSet::PrincipalPrivilegeSet(const PrincipalPrivilegeSet& other87) {
+  userPrivileges = other87.userPrivileges;
+  groupPrivileges = other87.groupPrivileges;
+  rolePrivileges = other87.rolePrivileges;
+  __isset = other87.__isset;
 }
-PrincipalPrivilegeSet& PrincipalPrivilegeSet::operator=(const PrincipalPrivilegeSet& other86) {
-  userPrivileges = other86.userPrivileges;
-  groupPrivileges = other86.groupPrivileges;
-  rolePrivileges = other86.rolePrivileges;
-  __isset = other86.__isset;
+PrincipalPrivilegeSet& PrincipalPrivilegeSet::operator=(const PrincipalPrivilegeSet& other88) {
+  userPrivileges = other88.userPrivileges;
+  groupPrivileges = other88.groupPrivileges;
+  rolePrivileges = other88.rolePrivileges;
+  __isset = other88.__isset;
   return *this;
 }
 void PrincipalPrivilegeSet::printTo(std::ostream& out) const {
@@ -2526,9 +2752,9 @@ uint32_t GrantRevokePrivilegeRequest::read(::apache::thrift::protocol::TProtocol
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast87;
-          xfer += iprot->readI32(ecast87);
-          this->requestType = (GrantRevokeType::type)ecast87;
+          int32_t ecast89;
+          xfer += iprot->readI32(ecast89);
+          this->requestType = (GrantRevokeType::type)ecast89;
           this->__isset.requestType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2593,17 +2819,17 @@ void swap(GrantRevokePrivilegeRequest &a, GrantRevokePrivilegeRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GrantRevokePrivilegeRequest::GrantRevokePrivilegeRequest(const GrantRevokePrivilegeRequest& other88) {
-  requestType = other88.requestType;
-  privileges = other88.privileges;
-  revokeGrantOption = other88.revokeGrantOption;
-  __isset = other88.__isset;
+GrantRevokePrivilegeRequest::GrantRevokePrivilegeRequest(const GrantRevokePrivilegeRequest& other90) {
+  requestType = other90.requestType;
+  privileges = other90.privileges;
+  revokeGrantOption = other90.revokeGrantOption;
+  __isset = other90.__isset;
 }
-GrantRevokePrivilegeRequest& GrantRevokePrivilegeRequest::operator=(const GrantRevokePrivilegeRequest& other89) {
-  requestType = other89.requestType;
-  privileges = other89.privileges;
-  revokeGrantOption = other89.revokeGrantOption;
-  __isset = other89.__isset;
+GrantRevokePrivilegeRequest& GrantRevokePrivilegeRequest::operator=(const GrantRevokePrivilegeRequest& other91) {
+  requestType = other91.requestType;
+  privileges = other91.privileges;
+  revokeGrantOption = other91.revokeGrantOption;
+  __isset = other91.__isset;
   return *this;
 }
 void GrantRevokePrivilegeRequest::printTo(std::ostream& out) const {
@@ -2687,13 +2913,13 @@ void swap(GrantRevokePrivilegeResponse &a, GrantRevokePrivilegeResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-GrantRevokePrivilegeResponse::GrantRevokePrivilegeResponse(const GrantRevokePrivilegeResponse& other90) {
-  success = other90.success;
-  __isset = other90.__isset;
+GrantRevokePrivilegeResponse::GrantRevokePrivilegeResponse(const GrantRevokePrivilegeResponse& other92) {
+  success = other92.success;
+  __isset = other92.__isset;
 }
-GrantRevokePrivilegeResponse& GrantRevokePrivilegeResponse::operator=(const GrantRevokePrivilegeResponse& other91) {
-  success = other91.success;
-  __isset = other91.__isset;
+GrantRevokePrivilegeResponse& GrantRevokePrivilegeResponse::operator=(const GrantRevokePrivilegeResponse& other93) {
+  success = other93.success;
+  __isset = other93.__isset;
   return *this;
 }
 void GrantRevokePrivilegeResponse::printTo(std::ostream& out) const {
@@ -2807,17 +3033,17 @@ void swap(Role &a, Role &b) {
   swap(a.__isset, b.__isset);
 }
 
-Role::Role(const Role& other92) {
-  roleName = other92.roleName;
-  createTime = other92.createTime;
-  ownerName = other92.ownerName;
-  __isset = other92.__isset;
+Role::Role(const Role& other94) {
+  roleName = other94.roleName;
+  createTime = other94.createTime;
+  ownerName = other94.ownerName;
+  __isset = other94.__isset;
 }
-Role& Role::operator=(const Role& other93) {
-  roleName = other93.roleName;
-  createTime = other93.createTime;
-  ownerName = other93.ownerName;
-  __isset = other93.__isset;
+Role& Role::operator=(const Role& other95) {
+  roleName = other95.roleName;
+  createTime = other95.createTime;
+  ownerName = other95.ownerName;
+  __isset = other95.__isset;
   return *this;
 }
 void Role::printTo(std::ostream& out) const {
@@ -2901,9 +3127,9 @@ uint32_t RolePrincipalGrant::read(::apache::thrift::protocol::TProtocol* iprot)
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast94;
-          xfer += iprot->readI32(ecast94);
-          this->principalType = (PrincipalType::type)ecast94;
+          int32_t ecast96;
+          xfer += iprot->readI32(ecast96);
+          this->principalType = (PrincipalType::type)ecast96;
           this->__isset.principalType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2935,9 +3161,9 @@ uint32_t RolePrincipalGrant::read(::apache::thrift::protocol::TProtocol* iprot)
         break;
       case 7:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast95;
-          xfer += iprot->readI32(ecast95);
-          this->grantorPrincipalType = (PrincipalType::type)ecast95;
+          int32_t ecast97;
+          xfer += iprot->readI32(ecast97);
+          this->grantorPrincipalType = (PrincipalType::type)ecast97;
           this->__isset.grantorPrincipalType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -3005,25 +3231,25 @@ void swap(RolePrincipalGrant &a, RolePrincipalGrant &b) {
   swap(a.__isset, b.__isset);
 }
 
-RolePrincipalGrant::RolePrincipalGrant(const RolePrincipalGrant& other96) {
-  roleName = other96.roleName;
-  principalName = other96.principalName;
-  principalType = other96.principalType;
-  grantOption = other96.grantOption;
-  grantTime = other96.grantTime;
-  grantorName = other96.grantorName;
-  grantorPrincipalType = other96.grantorPrincipalType;
-  __isset = other96.__isset;
-}
-RolePrincipalGrant& RolePrincipalGrant::operator=(const RolePrincipalGrant& other97) {
-  roleName = other97.roleName;
-  principalName = other97.principalName;
-  principalType = other97.principalType;
-  grantOption = other97.grantOption;
-  grantTime = other97.grantTime;
-  grantorName = other97.grantorName;
-  grantorPrincipalType = other97.grantorPrincipalType;
-  __isset = other97.__isset;
+RolePrincipalGrant::RolePrincipalGrant(const RolePrincipalGrant& other98) {
+  roleName = other98.roleName;
+  principalName = other98.principalName;
+  principalType = other98.principalType;
+  grantOption = other98.grantOption;
+  grantTime = other98.grantTime;
+  grantorName = other98.grantorName;
+  grantorPrincipalType = other98.grantorPrincipalType;
+  __isset = other98.__isset;
+}
+RolePrincipalGrant& RolePrincipalGrant::operator=(const RolePrincipalGrant& other99) {
+  roleName = other99.roleName;
+  principalName = other99.principalName;
+  principalType = other99.principalType;
+  grantOption = other99.grantOption;
+  grantTime = other99.grantTime;
+  grantorName = other99.grantorName;
+  grantorPrincipalType = other99.grantorPrincipalType;
+  __isset = other99.__isset;
   return *this;
 }
 void RolePrincipalGrant::printTo(std::ostream& out) const {
@@ -3085,9 +3311,9 @@ uint32_t GetRoleGrantsForPrincipalRequest::read(::apache::thrift::protocol::TPro
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast98;
-          xfer += iprot->readI32(ecast98);
-          this->principal_type = (PrincipalType::type)ecast98;
+          int32_t ecast100;
+          xfer += iprot->readI32(ecast100);
+          this->principal_type = (PrincipalType::type)ecast100;
           isset_principal_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -3133,13 +3359,13 @@ void swap(GetRoleGrantsForPrincipalRequest &a, GetRoleGrantsForPrincipalRequest
   swap(a.principal_type, b.principal_type);
 }
 
-GetRoleGrantsForPrincipalRequest::GetRoleGrantsForPrincipalRequest(const GetRoleGrantsForPrincipalRequest& other99) {
-  principal_name = other99.principal_name;
-  principal_type = other99.principal_type;
+GetRoleGrantsForPrincipalRequest::GetRoleGrantsForPrincipalRequest(const GetRoleGrantsForPrincipalRequest& other101) {
+  principal_name = other101.principal_name;
+  principal_type = other101.principal_type;
 }
-GetRoleGrantsForPrincipalRequest& GetRoleGrantsForPrincipalRequest::operator=(const GetRoleGrantsForPrincipalRequest& other100) {
-  principal_name = other100.principal_name;
-  principal_type = other100.principal_type;
+GetRoleGrantsForPrincipalRequest& GetRoleGrantsForPrincipalRequest::operator=(const GetRoleGrantsForPrincipalRequest& other102) {
+  principal_name = other102.principal_name;
+  principal_type = other102.principal_type;
   return *this;
 }
 void GetRoleGrantsForPrincipalRequest::printTo(std::ostream& out) const {
@@ -3185,14 +3411,14 @@ uint32_t GetRoleGrantsForPrincipalResponse::read(::apache::thrift::protocol::TPr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->principalGrants.clear();
-            uint32_t _size101;
-            ::apache::thrift::protocol::TType _etype104;
-            xfer += iprot->readListBegin(_etype104, _size101);
-            this->principalGrants.resize(_size101);
-            uint32_t _i105;
-            for (_i105 = 0; _i105 < _size101; ++_i105)
+            uint32_t _size103;
+            ::apache::thrift::protocol::TType _etype106;
+            xfer += iprot->readListBegin(_etype106, _size103);
+            this->principalGrants.resize(_size103);
+            uint32_t _i107;
+            for (_i107 = 0; _i107 < _size103; ++_i107)
             {
-              xfer += this->principalGrants[_i105].read(iprot);
+              xfer += this->principalGrants[_i107].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3223,10 +3449,10 @@ uint32_t GetRoleGrantsForPrincipalResponse::write(::apache::thrift::protocol::TP
   xfer += oprot->writeFieldBegin("principalGrants", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->principalGrants.size()));
-    std::vector<RolePrincipalGrant> ::const_iterator _iter106;
-    for (_iter106 = this->principalGrants.begin(); _iter106 != this->principalGrants.end(); ++_iter106)
+    std::vector<RolePrincipalGrant> ::const_iterator _iter108;
+    for (_iter108 = this->principalGrants.begin(); _iter108 != this->principalGrants.end(); ++_iter108)
     {
-      xfer += (*_iter106).write(oprot);
+      xfer += (*_iter108).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -3242,11 +3468,11 @@ void swap(GetRoleGrantsForPrincipalResponse &a, GetRoleGrantsForPrincipalRespons
   swap(a.principalGrants, b.principalGrants);
 }
 
-GetRoleGrantsForPrincipalResponse::GetRoleGrantsForPrincipalResponse(const GetRoleGrantsForPrincipalResponse& other107) {
-  principalGrants = other107.principalGrants;
+GetRoleGrantsForPrincipalResponse::GetRoleGrantsForPrincipalResponse(const GetRoleGrantsForPrincipalResponse& other109) {
+  principalGrants = other109.principalGrants;
 }
-GetRoleGrantsForPrincipalResponse& GetRoleGrantsForPrincipalResponse::operator=(const GetRoleGrantsForPrincipalResponse& other108) {
-  principalGrants = other108.principalGrants;
+GetRoleGrantsForPrincipalResponse& GetRoleGrantsForPrincipalResponse::operator=(const GetRoleGrantsForPrincipalResponse& other110) {
+  principalGrants = other110.principalGrants;
   return *this;
 }
 void GetRoleGrantsForPrincipalResponse::printTo(std::ostream& out) const {
@@ -3328,11 +3554,11 @@ void swap(GetPrincipalsInRoleRequest &a, GetPrincipalsInRoleRequest &b) {
   swap(a.roleName, b.roleName);
 }
 
-GetPrincipalsInRoleRequest::GetPrincipalsInRoleRequest(const GetPrincipalsInRoleRequest& other109) {
-  roleName = other109.roleName;
+GetPrincipalsInRoleRequest::GetPrincipalsInRoleRequest(const GetPrincipalsInRoleRequest& other111) {
+  roleName = other111.roleName;
 }
-GetPrincipalsInRoleRequest& GetPrincipalsInRoleRequest::operator=(const GetPrincipalsInRoleRequest& other110) {
-  roleName = other110.roleName;
+GetPrincipalsInRoleRequest& GetPrincipalsInRoleRequest::operator=(const GetPrincipalsInRoleRequest& other112) {
+  roleName = other112.roleName;
   return *this;
 }
 void GetPrincipalsInRoleRequest::printTo(std::ostream& out) const {
@@ -3377,14 +3603,14 @@ uint32_t GetPrincipalsInRoleResponse::read(::apache::thrift::protocol::TProtocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->principalGrants.clear();
-            uint32_t _size111;
-            ::apache::thrift::protocol::TType _etype114;
-            xfer += iprot->readListBegin(_etype114, _size111);
-            this->principalGrants.resize(_size111);
-            uint32_t _i115;
-            for (_i115 = 0; _i115 < _size111; ++_i115)
+            uint32_t _size113;
+            ::apache::thrift::protocol::TType _etype116;
+            xfer += iprot->readListBegin(_etype116, _size113);
+            this->principalGrants.resize(_size113);
+            uint32_t _i117;
+            for (_i117 = 0; _i117 < _size113; ++_i117)
             {
-              xfer += this->principalGrants[_i115].read(iprot);
+              xfer += this->principalGrants[_i117].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3415,10 +3641,10 @@ uint32_t GetPrincipalsInRoleResponse::write(::apache::thrift::protocol::TProtoco
   xfer += oprot->writeFieldBegin("principalGrants", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->principalGrants.size()));
-    std::vector<RolePrincipalGrant> ::const_iterator _iter116;
-    for (_iter116 = this->principalGrants.begin(); _iter116 != this->principalGrants.end(); ++_iter116)
+    std::vector<RolePrincipalGrant> ::const_iterator _iter118;
+    for (_iter118 = this->principalGrants.begin(); _iter118 != this->principalGrants.end(); ++_iter118)
     {
-      xfer += (*_iter116).write(oprot);
+      xfer += (*_iter118).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -3434,11 +3660,11 @@ void swap(GetPrincipalsInRoleResponse &a, GetPrincipalsInRoleResponse &b) {
   swap(a.principalGrants, b.principalGrants);
 }
 
-GetPrincipalsInRoleResponse::GetPrincipalsInRoleResponse(const GetPrincipalsInRoleResponse& other117) {
-  principalGrants = other117.principalGrants;
+GetPrincipalsInRoleResponse::GetPrincipalsInRoleResponse(const GetPrincipalsInRoleResponse& other119) {
+  principalGrants = other119.principalGrants;
 }
-GetPrincipalsInRoleResponse& GetPrincipalsInRoleResponse::operator=(const GetPrincipalsInRoleResponse& other118) {
-  principalGrants = other118.principalGrants;
+GetPrincipalsInRoleResponse& GetPrincipalsInRoleResponse::operator=(const GetPrincipalsInRoleResponse& other120) {
+  principalGrants = other120.principalGrants;
   return *this;
 }
 void GetPrincipalsInRoleResponse::printTo(std::ostream& out) const {
@@ -3507,9 +3733,9 @@ uint32_t GrantRevokeRoleRequest::read(::apache::thrift::protocol::TProtocol* ipr
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast119;
-          xfer += iprot->readI32(ecast119);
-          this->requestType = (GrantRevokeType::type)ecast119;
+          int32_t ecast121;
+          xfer += iprot->readI32(ecast121);
+          this->requestType = (GrantRevokeType::type)ecast121;
           this->__isset.requestType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -3533,9 +3759,9 @@ uint32_t GrantRevokeRoleRequest::read(::apache::thrift::protocol::TProtocol* ipr
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast120;
-          xfer += iprot->readI32(ecast120);
-          this->principalType = (PrincipalType::type)ecast120;
+          int32_t ecast122;
+          xfer += iprot->readI32(ecast122);
+          this->principalType = (PrincipalType::type)ecast122;
           this->__isset.principalType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -3551,9 +3777,9 @@ uint32_t GrantRevokeRoleRequest::read(::apache::thrift::protocol::TProtocol* ipr
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast121;
-          xfer += iprot->readI32(ecast121);
-          this->grantorType = (PrincipalType::type)ecast121;
+          int32_t ecast123;
+          xfer += iprot->readI32(ecast123);
+          this->grantorType = (PrincipalType::type)ecast123;
           this->__isset.grantorType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -3632,25 +3858,25 @@ void swap(GrantRevokeRoleRequest &a, GrantRevokeRoleRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GrantRevokeRoleRequest::GrantRevokeRoleRequest(const GrantRevokeRoleRequest& other122) {
-  requestType = other122.requestType;
-  roleName = other122.roleName;
-  principalName = other122.principalName;
-  principalType = other122.principalType;
-  grantor = other122.grantor;
-  grantorType = other122.grantorType;
-  grantOption = other122.grantOption;
-  __isset = other122.__isset;
-}
-GrantRevokeRoleRequest& GrantRevokeRoleRequest::operator=(const GrantRevokeRoleRequest& other123) {
-  requestType = other123.requestType;
-  roleName = other123.roleName;
-  principalName = other123.principalName;
-  principalType = other123.principalType;
-  grantor = other123.grantor;
-  grantorType = other123.grantorType;
-  grantOption = other123.grantOption;
-  __isset = other123.__isset;
+GrantRevokeRoleRequest::GrantRevokeRoleRequest(const GrantRevokeRoleRequest& other124) {
+  requestType = other124.requestType;
+  roleName = other124.roleName;
+  principalName = other124.principalName;
+  principalType = other124.principalType;
+  grantor = other124.grantor;
+  grantorType = other124.grantorType;
+  grantOption = other124.grantOption;
+  __isset = other124.__isset;
+}
+GrantRevokeRoleRequest& GrantRevokeRoleRequest::operator=(const GrantRevokeRoleRequest& other125) {
+  requestType = other125.requestType;
+  roleName = other125.roleName;
+  principalName = other125.principalName;
+  principalType = other125.principalType;
+  grantor = other125.grantor;
+  grantorType = other125.grantorType;
+  grantOption = other125.grantOption;
+  __isset = other125.__isset;
   return *this;
 }
 void GrantRevokeRoleRequest::printTo(std::ostream& out) const {
@@ -3738,13 +3964,13 @@ void swap(GrantRevokeRoleResponse &a, GrantRevokeRoleResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-GrantRevokeRoleResponse::GrantRevokeRoleResponse(const GrantRevokeRoleResponse& other124) {
-  success = other124.success;
-  __isset = other124.__isset;
+GrantRevokeRoleResponse::GrantRevokeRoleResponse(const GrantRevokeRoleResponse& other126) {
+  success = other126.success;
+  __isset = other126.__isset;
 }
-GrantRevokeRoleResponse& GrantRevokeRoleResponse::operator=(const GrantRevokeRoleResponse& other125) {
-  success = other125.success;
-  __isset = other125.__isset;
+GrantRevokeRoleResponse& GrantRevokeRoleResponse::operator=(const GrantRevokeRoleResponse& other127) {
+  success = other127.success;
+  __isset = other127.__isset;
   return *this;
 }
 void GrantRevokeRoleResponse::printTo(std::ostream& out) const {
@@ -3839,17 +4065,17 @@ uint32_t Database::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size126;
-            ::apache::thrift::protocol::TType _ktype127;
-            ::apache::thrift::protocol::TType _vtype128;
-            xfer += iprot->readMapBegin(_ktype127, _vtype128, _size126);
-            uint32_t _i130;
-            for (_i130 = 0; _i130 < _size126; ++_i130)
+            uint32_t _size128;
+            ::apache::thrift::protocol::TType _ktype129;
+            ::apache::thrift::protocol::TType _vtype130;
+            xfer += iprot->readMapBegin(_ktype129, _vtype130, _size128);
+            uint32_t _i132;
+            for (_i132 = 0; _i132 < _size128; ++_i132)
             {
-              std::string _key131;
-              xfer += iprot->readString(_key131);
-              std::string& _val132 = this->parameters[_key131];
-              xfer += iprot->readString(_val132);
+              std::string _key133;
+              xfer += iprot->readString(_key133);
+              std::string& _val134 = this->parameters[_key133];
+              xfer += iprot->readString(_val134);
             }
             xfer += iprot->readMapEnd();
           }
@@ -3876,9 +4102,9 @@ uint32_t Database::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 7:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast133;
-          xfer += iprot->readI32(ecast133);
-          this->ownerType = (PrincipalType::type)ecast133;
+          int32_t ecast135;
+          xfer += iprot->readI32(ecast135);
+          this->ownerType = (PrincipalType::type)ecast135;
           this->__isset.ownerType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -3916,11 +4142,11 @@ uint32_t Database::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 4);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter134;
-    for (_iter134 = this->parameters.begin(); _iter134 != this->parameters.end(); ++_iter134)
+    std::map<std::string, std::string> ::const_iterator _iter136;
+    for (_iter136 = this->parameters.begin(); _iter136 != this->parameters.end(); ++_iter136)
     {
-      xfer += oprot->writeString(_iter134->first);
-      xfer += oprot->writeString(_iter134->second);
+      xfer += oprot->writeString(_iter136->first);
+      xfer += oprot->writeString(_iter136->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -3958,25 +4184,25 @@ void swap(Database &a, Database &b) {
   swap(a.__isset, b.__isset);
 }
 
-Database::Database(const Database& other135) {
-  name = other135.name;
-  description = other135.description;
-  locationUri = other135.locationUri;
-  parameters = other135.parameters;
-  privileges = other135.privileges;
-  ownerName = other135.ownerName;
-  ownerType = other135.ownerType;
-  __isset = other135.__isset;
-}
-Database& Database::operator=(const Database& other136) {
-  name = other136.name;
-  description = other136.description;
-  locationUri = other136.locationUri;
-  parameters = other136.parameters;
-  privileges = other136.privileges;
-  ownerName = other136.ownerName;
-  ownerType = other136.ownerType;
-  __isset = other136.__isset;
+Database::Database(const Database& other137) {
+  name = other137.name;
+  description = other137.description;
+  locationUri = other137.locationUri;
+  parameters = other137.parameters;
+  privileges = other137.privileges;
+  ownerName = other137.ownerName;
+  ownerType = other137.ownerType;
+  __isset = other137.__isset;
+}
+Database& Database::operator=(const Database& other138) {
+  name = other138.name;
+  description = other138.description;
+  locationUri = other138.locationUri;
+  parameters = other138.parameters;
+  privileges = other138.privileges;
+  ownerName = other138.ownerName;
+  ownerType = other138.ownerType;
+  __isset = other138.__isset;
   return *this;
 }
 void Database::printTo(std::ostream& out) const {
@@ -4050,17 +4276,17 @@ uint32_t SerDeInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size137;
-            ::apache::thrift::protocol::TType _ktype138;
-            ::apache::thrift::protocol::TType _vtype139;
-            xfer += iprot->readMapBegin(_ktype138, _vtype139, _size137);
-            uint32_t _i141;
-            for (_i141 = 0; _i141 < _size137; ++_i141)
+            uint32_t _size139;
+            ::apache::thrift::protocol::TType _ktype140;
+            ::apache::thrift::protocol::TType _vtype141;
+            xfer += iprot->readMapBegin(_ktype140, _vtype141, _size139);
+            uint32_t _i143;
+            for (_i143 = 0; _i143 < _size139; ++_i143)
             {
-              std::string _key142;
-              xfer += iprot->readString(_key142);
-              std::string& _val143 = this->parameters[_key142];
-              xfer += iprot->readString(_val143);
+              std::string _key144;
+              xfer += iprot->readString(_key144);
+              std::string& _val145 = this->parameters[_key144];
+              xfer += iprot->readString(_val145);
             }
             xfer += iprot->readMapEnd();
           }
@@ -4097,11 +4323,11 @@ uint32_t SerDeInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter144;
-    for (_iter144 = this->parameters.begin(); _iter144 != this->parameters.end(); ++_iter144)
+    std::map<std::string, std::string> ::const_iterator _iter146;
+    for (_iter146 = this->parameters.begin(); _iter146 != this->parameters.end(); ++_iter146)
     {
-      xfer += oprot->writeString(_iter144->first);
-      xfer += oprot->writeString(_iter144->second);
+      xfer += oprot->writeString(_iter146->first);
+      xfer += oprot->writeString(_iter146->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -4120,17 +4346,17 @@ void swap(SerDeInfo &a, SerDeInfo &b) {
   swap(a.__isset, b.__isset);
 }
 
-SerDeInfo::SerDeInfo(const SerDeInfo& other145) {
-  name = other145.name;
-  serializationLib = other145.serializationLib;
-  parameters = other145.parameters;
-  __isset = other145.__isset;
+SerDeInfo::SerDeInfo(const SerDeInfo& other147) {
+  name = other147.name;
+  serializationLib = other147.serializationLib;
+  parameters = other147.parameters;
+  __isset = other147.__isset;
 }
-SerDeInfo& SerDeInfo::operator=(const SerDeInfo& other146) {
-  name = other146.name;
-  serializationLib = other146.serializationLib;
-  parameters = other146.parameters;
-  __isset = other146.__isset;
+SerDeInfo& SerDeInfo::operator=(const SerDeInfo& other148) {
+  name = other148.name;
+  serializationLib = other148.serializationLib;
+  parameters = other148.parameters;
+  __isset = other148.__isset;
   return *this;
 }
 void SerDeInfo::printTo(std::ostream& out) const {
@@ -4229,15 +4455,15 @@ void swap(Order &a, Order &b) {
   swap(a.__isset, b.__isset);
 }
 
-Order::Order(const Order& other147) {
-  col = other147.col;
-  order = other147.order;
-  __isset = other147.__isset;
+Order::Order(const Order& other149) {
+  col = other149.col;
+  order = other149.order;
+  __isset = other149.__isset;
 }
-Order& Order::operator=(const Order& other148) {
-  col = other148.col;
-  order = other148.order;
-  __isset = other148.__isset;
+Order& Order::operator=(const Order& other150) {
+  col = other150.col;
+  order = other150.order;
+  __isset = other150.__isset;
   return *this;
 }
 void Order::printTo(std::ostream& out) const {
@@ -4290,14 +4516,14 @@ uint32_t SkewedInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->skewedColNames.clear();
-            uint32_t _size149;
-            ::apache::thrift::protocol::TType _etype152;
-            xfer += iprot->readListBegin(_etype152, _size149);
-            this->skewedColNames.resize(_size149);
-            uint32_t _i153;
-            for (_i153 = 0; _i153 < _size149; ++_i153)
+            uint32_t _size151;
+            ::apache::thrift::protocol::TType _etype154;
+            xfer += iprot->readListBegin(_etype154, _size151);
+            this->skewedColNames.resize(_size151);
+            uint32_t _i155;
+            for (_i155 = 0; _i155 < _size151; ++_i155)
             {
-              xfer += iprot->readString(this->skewedColNames[_i153]);
+              xfer += iprot->readString(this->skewedColNames[_i155]);
             }
             xfer += iprot->readListEnd();
           }
@@ -4310,23 +4536,23 @@ uint32_t SkewedInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->skewedColValues.clear();
-            uint32_t _size154;
-            ::apache::thrift::protocol::TType _etype157;
-            xfer += iprot->readListBegin(_etype157, _size154);
-            this->skewedColValues.resize(_size154);
-            uint32_t _i158;
-            for (_i158 = 0; _i158 < _size154; ++_i158)
+            uint32_t _size156;
+            ::apache::thrift::protocol::TType _etype159;
+            xfer += iprot->readListBegin(_etype159, _size156);
+            this->skewedColValues.resize(_size156);
+            uint32_t _i160;
+            for (_i160 = 0; _i160 < _size156; ++_i160)
             {
               {
-                this->skewedColValues[_i158].clear();
-                uint32_t _size159;
-                ::apache::thrift::protocol::TType _etype162;
-                xfer += iprot->readListBegin(_etype162, _size159);
-                this->skewedColValues[_i158].resize(_size159);
-                uint32_t _i163;
-                for (_i163 = 0; _i163 < _size159; ++_i163)
+                this->skewedColValues[_i160].clear();
+                uint32_t _size161;
+                ::apache::thrift::protocol::TType _etype164;
+                xfer += iprot->readListBegin(_etype164, _size161);
+                this->skewedColValues[_i160].resize(_size161);
+                uint32_t _i165;
+                for (_i165 = 0; _i165 < _size161; ++_i165)
                 {
-                  xfer += iprot->readString(this->skewedColValues[_i158][_i163]);
+                  xfer += iprot->readString(this->skewedColValues[_i160][_i165]);
                 }
                 xfer += iprot->readListEnd();
               }
@@ -4342,29 +4568,29 @@ uint32_t SkewedInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->skewedColValueLocationMaps.clear();
-            uint32_t _size164;
-            ::apache::thrift::protocol::TType _ktype165;
-            ::apache::thrift::protocol::TType _vtype166;
-            xfer += iprot->readMapBegin(_ktype165, _vtype166, _size164);
-            uint32_t _i168;
-            for (_i168 = 0; _i168 < _size164; ++_i168)
+            uint32_t _size166;
+            ::apache::thrift::protocol::TType _ktype167;
+            ::apache::thrift::protocol::TType _vtype168;
+            xfer += iprot->readMapBegin(_ktype167, _vtype168, _size166);
+            uint32_t _i170;
+            for (_i170 = 0; _i170 < _size166; ++_i170)
             {
-              std::vector<std::string>  _key169;
+              std::vector<std::string>  _key171;
               {
-                _key169.clear();
-                uint32_t _size171;
-                ::apache::thrift::protocol::TType _etype174;
-                xfer += iprot->readListBegin(_etype174, _size171);
-                _key169.resize(_size171);
-                uint32_t _i175;
-                for (_i175 = 0; _i175 < _size171; ++_i175)
+                _key171.clear();
+                uint32_t _size173;
+                ::apache::thrift::protocol::TType _etype176;
+                xfer += iprot->readListBegin(_etype176, _size173);
+                _key171.resize(_size173);
+                uint32_t _i177;
+                for (_i177 = 0; _i177 < _size173; ++_i177)
                 {
-                  xfer += iprot->readString(_key169[_i175]);
+                  xfer += iprot->readString(_key171[_i177]);
                 }
                 xfer += iprot->readListEnd();
               }
-              std::string& _val170 = this->skewedColValueLocationMaps[_key169];
-              xfer += iprot->readString(_val170);
+              std::string& _val172 = this->skewedColValueLocationMaps[_key171];
+              xfer += iprot->readString(_val172);
             }
             xfer += iprot->readMapEnd();
           }
@@ -4393,10 +4619,10 @@ uint32_t SkewedInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("skewedColNames", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->skewedColNames.size()));
-    std::vector<std::string> ::const_iterator _iter176;
-    for (_iter176 = this->skewedColNames.begin(); _iter176 != this->skewedColNames.end(); ++_iter176)
+    std::vector<std::string> ::const_iterator _iter178;
+    for (_iter178 = this->skewedColNames.begin(); _iter178 != this->skewedColNames.end(); ++_iter178)
     {
-      xfer += oprot->writeString((*_iter176));
+      xfer += oprot->writeString((*_iter178));
     }
     xfer += oprot->writeListEnd();
   }
@@ -4405,15 +4631,15 @@ uint32_t SkewedInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("skewedColValues", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->skewedColValues.size()));
-    std::vector<std::vector<std::string> > ::const_iterator _iter177;
-    for (_iter177 = this->skewedColValues.begin(); _iter177 != this->skewedColValues.end(); ++_iter177)
+    std::vector<std::vector<std::string> > ::const_iterator _iter179;
+    for (_iter179 = this->skewedColValues.begin(); _iter179 != this->skewedColValues.end(); ++_iter179)
     {
       {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*_iter177).size()));
-        std::vector<std::string> ::const_iterator _iter178;
-        for (_iter178 = (*_iter177).begin(); _iter178 != (*_iter177).end(); ++_iter178)
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*_iter179).size()));
+        std::vector<std::string> ::const_iterator _iter180;
+        for (_iter180 = (*_iter179).begin(); _iter180 != (*_iter179).end(); ++_iter180)
         {
-          xfer += oprot->writeString((*_iter178));
+          xfer += oprot->writeString((*_iter180));
         }
         xfer += oprot->writeListEnd();
       }
@@ -4425,19 +4651,19 @@ uint32_t SkewedInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("skewedColValueLocationMaps", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_LIST, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->skewedColValueLocationMaps.size()));
-    std::map<std::vector<std::string> , std::string> ::const_iterator _iter179;
-    for (_iter179 = this->skewedColValueLocationMaps.begin(); _iter179 != this->skewedColValueLocationMaps.end(); ++_iter179)
+    std::map<std::vector<std::string> , std::string> ::const_iterator _iter181;
+    for (_iter181 = this->skewedColValueLocationMaps.begin(); _iter181 != this->skewedColValueLocationMaps.end(); ++_iter181)
     {
       {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(_iter179->first.size()));
-        std::vector<std::string> ::const_iterator _iter180;
-        for (_iter180 = _iter179->first.begin(); _iter180 != _iter179->first.end(); ++_iter180)
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(_iter181->first.size()));
+        std::vector<std::string> ::const_iterator _iter182;
+        for (_iter182 = _iter181->first.begin(); _iter182 != _iter181->first.end(); ++_iter182)
         {
-          xfer += oprot->writeString((*_iter180));
+          xfer += oprot->writeString((*_iter182));
         }
         xfer += oprot->writeListEnd();
       }
-      xfer += oprot->writeString(_iter179->second);
+      xfer += oprot->writeString(_iter181->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -4456,17 +4682,17 @@ void swap(SkewedInfo &a, SkewedInfo &b) {
   swap(a.__isset, b.__isset);
 }
 
-SkewedInfo::SkewedInfo(const SkewedInfo& other181) {
-  skewedColNames = other181.skewedColNames;
-  skewedColValues = other181.skewedColValues;
-  skewedColValueLocationMaps = other181.skewedColValueLocationMaps;
-  __isset = other181.__isset;
+SkewedInfo::SkewedInfo(const SkewedInfo& other183) {
+  skewedColNames = other183.skewedColNames;
+  skewedColValues = other183.skewedColValues;
+  skewedColValueLocationMaps = other183.skewedColValueLocationMaps;
+  __isset = other183.__isset;
 }
-SkewedInfo& SkewedInfo::operator=(const SkewedInfo& other182) {
-  skewedColNames = other182.skewedColNames;
-  skewedColValues = other182.skewedColValues;
-  skewedColValueLocationMaps = other182.skewedColValueLocationMaps;
-  __isset = other182.__isset;
+SkewedInfo& SkewedInfo::operator=(const SkewedInfo& other184) {
+  skewedColNames = other184.skewedColNames;
+  skewedColValues = other184.skewedColValues;
+  skewedColValueLocationMaps = other184.skewedColValueLocationMaps;
+  __isset = other184.__isset;
   return *this;
 }
 void SkewedInfo::printTo(std::ostream& out) const {
@@ -4558,14 +4784,14 @@ uint32_t StorageDescriptor::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->cols.clear();
-            uint32_t _size183;
-            ::apache::thrift::protocol::TType _etype186;
-            xfer += iprot->readListBegin(_etype186, _size183);
-            this->cols.resize(_size183);
-            uint32_t _i187;
-            for (_i187 = 0; _i187 < _size183; ++_i187)
+            uint32_t _size185;
+            ::apache::thrift::protocol::TType _etype188;
+            xfer += iprot->readListBegin(_etype188, _size185);
+            this->cols.resize(_size185);
+            uint32_t _i189;
+            for (_i189 = 0; _i189 < _size185; ++_i189)
             {
-              xfer += this->cols[_i187].read(iprot);
+              xfer += this->cols[_i189].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4626,14 +4852,14 @@ uint32_t StorageDescriptor::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->bucketCols.clear();
-            uint32_t _size188;
-            ::apache::thrift::protocol::TType _etype191;
-            xfer += iprot->readListBegin(_etype191, _size188);
-            this->bucketCols.resize(_size188);
-            uint32_t _i192;
-            for (_i192 = 0; _i192 < _size188; ++_i192)
+            uint32_t _size190;
+            ::apache::thrift::protocol::TType _etype193;
+            xfer += iprot->readListBegin(_etype193, _size190);
+            this->bucketCols.resize(_size190);
+            uint32_t _i194;
+            for (_i194 = 0; _i194 < _size190; ++_i194)
             {
-              xfer += iprot->readString(this->bucketCols[_i192]);
+              xfer += iprot->readString(this->bucketCols[_i194]);
             }
             xfer += iprot->readListEnd();
           }
@@ -4646,14 +4872,14 @@ uint32_t StorageDescriptor::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->sortCols.clear();
-            uint32_t _size193;
-            ::apache::thrift::protocol::TType _etype196;
-            xfer += iprot->readListBegin(_etype196, _size193);
-            this->sortCols.resize(_size193);
-            uint32_t _i197;
-            for (_i197 = 0; _i197 < _size193; ++_i197)
+            uint32_t _size195;
+            ::apache::thrift::protocol::TType _etype198;
+            xfer += iprot->readListBegin(_etype198, _size195);
+            this->sortCols.resize(_size195);
+            uint32_t _i199;
+            for (_i199 = 0; _i199 < _size195; ++_i199)
             {
-              xfer += this->sortCols[_i197].read(iprot);
+              xfer += this->sortCols[_i199].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4666,17 +4892,17 @@ uint32_t StorageDescriptor::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size198;
-            ::apache::thrift::protocol::TType _ktype199;
-            ::apache::thrift::protocol::TType _vtype200;
-            xfer += iprot->readMapBegin(_ktype199, _vtype200, _size198);
-            uint32_t _i202;
-            for (_i202 = 0; _i202 < _size198; ++_i202)
+            uint32_t _size200;
+            ::apache::thrift::protocol::TType _ktype201;
+            ::apache::thrift::protocol::TType _vtype202;
+            xfer += iprot->readMapBegin(_ktype201, _vtype202, _size200);
+            uint32_t _i204;
+            for (_i204 = 0; _i204 < _size200; ++_i204)
             {
-              std::string _key203;
-              xfer += iprot->readString(_key203);
-              std::string& _val204 = this->parameters[_key203];
-              xfer += iprot->readString(_val204);
+              std::string _key205;
+              xfer += iprot->readString(_key205);
+              std::string& _val206 = this->parameters[_key205];
+              xfer += iprot->readString(_val206);
             }
             xfer += iprot->readMapEnd();
           }
@@ -4721,10 +4947,10 @@ uint32_t StorageDescriptor::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("cols", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->cols.size()));
-    std::vector<FieldSchema> ::const_iterator _iter205;
-    for (_iter205 = this->cols.begin(); _iter205 != this->cols.end(); ++_iter205)
+    std::vector<FieldSchema> ::const_iterator _iter207;
+    for (_iter207 = this->cols.begin(); _iter207 != this->cols.end(); ++_iter207)
     {
-      xfer += (*_iter205).write(oprot);
+      xfer += (*_iter207).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4757,10 +4983,10 @@ uint32_t StorageDescriptor::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("bucketCols", ::apache::thrift::protocol::T_LIST, 8);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->bucketCols.size()));
-    std::vector<std::string> ::const_iterator _iter206;
-    for (_iter206 = this->bucketCols.begin(); _iter206 != this->bucketCols.end(); ++_iter206)
+    std::vector<std::string> ::const_iterator _iter208;
+    for (_iter208 = this->bucketCols.begin(); _iter208 != this->bucketCols.end(); ++_iter208)
     {
-      xfer += oprot->writeString((*_iter206));
+      xfer += oprot->writeString((*_iter208));
     }
     xfer += oprot->writeListEnd();
   }
@@ -4769,10 +4995,10 @@ uint32_t StorageDescriptor::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("sortCols", ::apache::thrift::protocol::T_LIST, 9);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->sortCols.size()));
-    std::vector<Order> ::const_iterator _iter207;
-    for (_iter207 = this->sortCols.begin(); _iter207 != this->sortCols.end(); ++_iter207)
+    std::vector<Order> ::const_iterator _iter209;
+    for (_iter209 = this->sortCols.begin(); _iter209 != this->sortCols.end(); ++_iter209)
     {
-      xfer += (*_iter207).write(oprot);
+      xfer += (*_iter209).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4781,11 +5007,11 @@ uint32_t StorageDescriptor::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 10);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter208;
-    for (_iter208 = this->parameters.begin(); _iter208 != this->parameters.end(); ++_iter208)
+    std::map<std::string, std::string> ::const_iterator _iter210;
+    for (_iter210 = this->parameters.begin(); _iter210 != this->parameters.end(); ++_iter210)
     {
-      xfer += oprot->writeString(_iter208->first);
-      xfer += oprot->writeString(_iter208->second);
+      xfer += oprot->writeString(_iter210->first);
+      xfer += oprot->writeString(_iter210->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -4823,35 +5049,35 @@ void swap(StorageDescriptor &a, StorageDescriptor &b) {
   swap(a.__isset, b.__isset);
 }
 
-StorageDescriptor::StorageDescriptor(const StorageDescriptor& other209) {
-  cols = other209.cols;
-  location = other209.location;
-  inputFormat = other209.inputFormat;
-  outputFormat = other209.outputFormat;
-  compressed = other209.compressed;
-  numBuckets = other209.numBuckets;
-  serdeInfo = other209.serdeInfo;
-  bucketCols = other209.bucketCols;
-  sortCols = other209.sortCols;
-  parameters = other209.parameters;
-  skewedInfo = other209.skewedInfo;
-  storedAsSubDirectories = other209.storedAsSubDirectories;
-  __isset = other209.__isset;
-}
-StorageDescriptor& StorageDescriptor::operator=(const StorageDescriptor& other210) {
-  cols = other210.cols;
-  location = other210.location;
-  inputFormat = other210.inputFormat;
-  outputFormat = other210.outputFormat;
-  compressed = other210.compressed;
-  numBuckets = other210.numBuckets;
-  serdeInfo = other210.serdeInfo;
-  bucketCols = other210.bucketCols;
-  sortCols = other210.sortCols;
-  parameters = other210.parameters;
-  skewedInfo = other210.skewedInfo;
-  storedAsSubDirectories = other210.storedAsSubDirectories;
-  __isset = other210.__isset;
+StorageDescriptor::StorageDescriptor(const StorageDescriptor& other211) {
+  cols = other211.cols;
+  location = other211.location;
+  inputFormat = other211.inputFormat;
+  outputFormat = other211.outputFormat;
+  compressed = other211.compressed;
+  numBuckets = other211.numBuckets;
+  serdeInfo = other211.serdeInfo;
+  bucketCols = other211.bucketCols;
+  sortCols = other211.sortCols;
+  parameters = other211.parameters;
+  skewedInfo = other211.skewedInfo;
+  storedAsSubDirectories = other211.storedAsSubDirectories;
+  __isset = other211.__isset;
+}
+StorageDescriptor& StorageDescriptor::operator=(const StorageDescriptor& other212) {
+  cols = other212.cols;
+  location = other212.location;
+  inputFormat = other212.inputFormat;
+  outputFormat = other212.outputFormat;
+  compressed = other212.compressed;
+  numBuckets = other212.numBuckets;
+  serdeInfo = other212.serdeInfo;
+  bucketCols = other212.bucketCols;
+  sortCols = other212.sortCols;
+  parameters = other212.parameters;
+  skewedInfo = other212.skewedInfo;
+  storedAsSubDirectories = other212.storedAsSubDirectories;
+  __isset = other212.__isset;
   return *this;
 }
 void StorageDescriptor::printTo(std::ostream& out) const {
@@ -5026,14 +5252,14 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionKeys.clear();
-            uint32_t _size211;
-            ::apache::thrift::protocol::TType _etype214;
-            xfer += iprot->readListBegin(_etype214, _size211);
-            this->partitionKeys.resize(_size211);
-            uint32_t _i215;
-            for (_i215 = 0; _i215 < _size211; ++_i215)
+            uint32_t _size213;
+            ::apache::thrift::protocol::TType _etype216;
+            xfer += iprot->readListBegin(_etype216, _size213);
+            this->partitionKeys.resize(_size213);
+            uint32_t _i217;
+            for (_i217 = 0; _i217 < _size213; ++_i217)
             {
-              xfer += this->partitionKeys[_i215].read(iprot);
+              xfer += this->partitionKeys[_i217].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5046,17 +5272,17 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size216;
-            ::apache::thrift::protocol::TType _ktype217;
-            ::apache::thrift::protocol::TType _vtype218;
-            xfer += iprot->readMapBegin(_ktype217, _vtype218, _size216);
-            uint32_t _i220;
-            for (_i220 = 0; _i220 < _size216; ++_i220)
+            uint32_t _size218;
+            ::apache::thrift::protocol::TType _ktype219;
+            ::apache::thrift::protocol::TType _vtype220;
+            xfer += iprot->readMapBegin(_ktype219, _vtype220, _size218);
+            uint32_t _i222;
+            for (_i222 = 0; _i222 < _size218; ++_i222)
             {
-              std::string _key221;
-              xfer += iprot->readString(_key221);
-              std::string& _val222 = this->parameters[_key221];
-              xfer += iprot->readString(_val222);
+              std::string _key223;
+              xfer += iprot->readString(_key223);
+              std::string& _val224 = this->parameters[_key223];
+              xfer += iprot->readString(_val224);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5169,10 +5395,10 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("partitionKeys", ::apache::thrift::protocol::T_LIST, 8);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitionKeys.size()));
-    std::vector<FieldSchema> ::const_iterator _iter223;
-    for (_iter223 = this->partitionKeys.begin(); _iter223 != this->partitionKeys.end(); ++_iter223)
+    std::vector<FieldSchema> ::const_iterator _iter225;
+    for (_iter225 = this->partitionKeys.begin(); _iter225 != this->partitionKeys.end(); ++_iter225)
     {
-      xfer += (*_iter223).write(oprot);
+      xfer += (*_iter225).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5181,11 +5407,11 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 9);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter224;
-    for (_iter224 = this->parameters.begin(); _iter224 != this->parameters.end(); ++_iter224)
+    std::map<std::string, std::string> ::const_iterator _iter226;
+    for (_iter226 = this->parameters.begin(); _iter226 != this->parameters.end(); ++_iter226)
     {
-      xfer += oprot->writeString(_iter224->first);
-      xfer += oprot->writeString(_iter224->second);
+      xfer += oprot->writeString(_iter226->first);
+      xfer += oprot->writeString(_iter226->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -5249,43 +5475,43 @@ void swap(Table &a, Table &b) {
   swap(a.__isset, b.__isset);
 }
 
-Table::Table(const Table& other225) {
-  tableName = other225.tableName;
-  dbName = other225.dbName;
-  owner = other225.owner;
-  createTime = other225.createTime;
-  lastAccessTime = other225.lastAccessTime;
-  retention = other225.retention;
-  sd = other225.sd;
-  partitionKeys = other225.partitionKeys;
-  parameters = other225.parameters;
-  viewOriginalText = other225.viewOriginalText;
-  viewExpandedText = other225.viewExpandedText;
-  tableType = other225.tableType;
-  privileges = other225.privileges;
-  temporary = other225.temporary;
-  rewriteEnabled = other225.rewriteEnabled;
-  creationMetadata = other225.creationMetadata;
-  __isset = other225.__isset;
-}
-Table& Table::operator=(const Table& other226) {
-  tableName = other226.tableName;
-  dbName = other226.dbName;
-  owner = other226.owner;
-  createTime = other226.createTime;
-  lastAccessTime = other226.lastAccessTime;
-  retention = other226.retention;
-  sd = other226.sd;
-  partitionKeys = other226.partitionKeys;
-  parameters = other226.parameters;
-  viewOriginalText = other226.viewOriginalText;
-  viewExpandedText = other226.viewExpandedText;
-  tableType = other226.tableType;
-  privileges = other226.privileges;
-  temporary = other226.temporary;
-  rewriteEnabled = other226.rewriteEnabled;
-  creationMetadata = other226.creationMetadata;
-  __isset = other226.__isset;
+Table::Table(const Table& other227) {
+  tableName = other227.tableName;
+  dbName = other227.dbName;
+  owner = other227.owner;
+  createTime = other227.createTime;
+  lastAccessTime = other227.lastAccessTime;
+  retention = other227.retention;
+  sd = other227.sd;
+  partitionKeys = other227.partitionKeys;
+  parameters = other227.parameters;
+  viewOriginalText = other227.viewOriginalText;
+  viewExpandedText = other227.viewExpandedText;
+  tableType = other227.tableType;
+  privileges = other227.privileges;
+  temporary = other227.temporary;
+  rewriteEnabled = other227.rewriteEnabled;
+  creationMetadata = other227.creationMetadata;
+  __isset = other227.__isset;
+}
+Table& Table::operator=(const Table& other228) {
+  tableName = other228.tableName;
+  dbName = other228.dbName;
+  owner = other228.owner;
+  createTime = other228.createTime;
+  lastAccessTime = other228.lastAccessTime;
+  retention = other228.retention;
+  sd = other228.sd;
+  partitionKeys = other228.partitionKeys;
+  parameters = other228.parameters;
+  viewOriginalText = other228.viewOriginalText;
+  viewExpandedText = other228.viewExpandedText;
+  tableType = other228.tableType;
+  privileges = other228.privileges;
+  temporary = other228.temporary;
+  rewriteEnabled = other228.rewriteEnabled;
+  creationMetadata = other228.creationMetadata;
+  __isset = other228.__isset;
   return *this;
 }
 void Table::printTo(std::ostream& out) const {
@@ -5373,14 +5599,14 @@ uint32_t Partition::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->values.clear();
-            uint32_t _size227;
-            ::apache::thrift::protocol::TType _etype230;
-            xfer += iprot->readListBegin(_etype230, _size227);
-            this->values.resize(_size227);
-            uint32_t _i231;
-            for (_i231 = 0; _i231 < _size227; ++_i231)
+            uint32_t _size229;
+            ::apache::thrift::protocol::TType _etype232;
+            xfer += iprot->readListBegin(_etype232, _size229);
+            this->values.resize(_size229);
+            uint32_t _i233;
+            for (_i233 = 0; _i233 < _size229; ++_i233)
             {
-              xfer += iprot->readString(this->values[_i231]);
+              xfer += iprot->readString(this->values[_i233]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5433,17 +5659,17 @@ uint32_t Partition::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size232;
-            ::apache::thrift::protocol::TType _ktype233;
-            ::apache::thrift::protocol::TType _vtype234;
-            xfer += iprot->readMapBegin(_ktype233, _vtype234, _size232);
-            uint32_t _i236;
-            for (_i236 = 0; _i236 < _size232; ++_i236)
+            uint32_t _size234;
+            ::apache::thrift::protocol::TType _ktype235;
+            ::apache::thrift::protocol::TType _vtype236;
+            xfer += iprot->readMapBegin(_ktype235, _vtype236, _size234);
+            uint32_t _i238;
+            for (_i238 = 0; _i238 < _size234; ++_i238)
             {
-              std::string _key237;
-              xfer += iprot->readString(_key237);
-              std::string& _val238 = this->parameters[_key237];
-              xfer += iprot->readString(_val238);
+              std::string _key239;
+              xfer += iprot->readString(_key239);
+              std::string& _val240 = this->parameters[_key239];
+              xfer += iprot->readString(_val240);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5480,10 +5706,10 @@ uint32_t Partition::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->values.size()));
-    std::vector<std::string> ::const_iterator _iter239;
-    for (_iter239 = this->values.begin(); _iter239 != this->values.end(); ++_iter239)
+    std::vector<std::string> ::const_iterator _iter241;
+    for (_iter241 = this->values.begin(); _iter241 != this->values.end(); ++_iter241)
     {
-      xfer += oprot->writeString((*_iter239));
+      xfer += oprot->writeString((*_iter241));
    

<TRUNCATED>

[02/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 0c60aff..883ecfe 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -1154,6 +1154,162 @@ class SQLNotNullConstraint:
   def __ne__(self, other):
     return not (self == other)
 
+class SQLDefaultConstraint:
+  """
+  Attributes:
+   - table_db
+   - table_name
+   - column_name
+   - default_value
+   - dc_name
+   - enable_cstr
+   - validate_cstr
+   - rely_cstr
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'table_db', None, None, ), # 1
+    (2, TType.STRING, 'table_name', None, None, ), # 2
+    (3, TType.STRING, 'column_name', None, None, ), # 3
+    (4, TType.STRING, 'default_value', None, None, ), # 4
+    (5, TType.STRING, 'dc_name', None, None, ), # 5
+    (6, TType.BOOL, 'enable_cstr', None, None, ), # 6
+    (7, TType.BOOL, 'validate_cstr', None, None, ), # 7
+    (8, TType.BOOL, 'rely_cstr', None, None, ), # 8
+  )
+
+  def __init__(self, table_db=None, table_name=None, column_name=None, default_value=None, dc_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None,):
+    self.table_db = table_db
+    self.table_name = table_name
+    self.column_name = column_name
+    self.default_value = default_value
+    self.dc_name = dc_name
+    self.enable_cstr = enable_cstr
+    self.validate_cstr = validate_cstr
+    self.rely_cstr = rely_cstr
+
+  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.table_db = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.table_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.column_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.default_value = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.dc_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.BOOL:
+          self.enable_cstr = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.BOOL:
+          self.validate_cstr = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.BOOL:
+          self.rely_cstr = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('SQLDefaultConstraint')
+    if self.table_db is not None:
+      oprot.writeFieldBegin('table_db', TType.STRING, 1)
+      oprot.writeString(self.table_db)
+      oprot.writeFieldEnd()
+    if self.table_name is not None:
+      oprot.writeFieldBegin('table_name', TType.STRING, 2)
+      oprot.writeString(self.table_name)
+      oprot.writeFieldEnd()
+    if self.column_name is not None:
+      oprot.writeFieldBegin('column_name', TType.STRING, 3)
+      oprot.writeString(self.column_name)
+      oprot.writeFieldEnd()
+    if self.default_value is not None:
+      oprot.writeFieldBegin('default_value', TType.STRING, 4)
+      oprot.writeString(self.default_value)
+      oprot.writeFieldEnd()
+    if self.dc_name is not None:
+      oprot.writeFieldBegin('dc_name', TType.STRING, 5)
+      oprot.writeString(self.dc_name)
+      oprot.writeFieldEnd()
+    if self.enable_cstr is not None:
+      oprot.writeFieldBegin('enable_cstr', TType.BOOL, 6)
+      oprot.writeBool(self.enable_cstr)
+      oprot.writeFieldEnd()
+    if self.validate_cstr is not None:
+      oprot.writeFieldBegin('validate_cstr', TType.BOOL, 7)
+      oprot.writeBool(self.validate_cstr)
+      oprot.writeFieldEnd()
+    if self.rely_cstr is not None:
+      oprot.writeFieldBegin('rely_cstr', TType.BOOL, 8)
+      oprot.writeBool(self.rely_cstr)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.table_db)
+    value = (value * 31) ^ hash(self.table_name)
+    value = (value * 31) ^ hash(self.column_name)
+    value = (value * 31) ^ hash(self.default_value)
+    value = (value * 31) ^ hash(self.dc_name)
+    value = (value * 31) ^ hash(self.enable_cstr)
+    value = (value * 31) ^ hash(self.validate_cstr)
+    value = (value * 31) ^ hash(self.rely_cstr)
+    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 Type:
   """
   Attributes:
@@ -6982,6 +7138,164 @@ class NotNullConstraintsResponse:
   def __ne__(self, other):
     return not (self == other)
 
+class DefaultConstraintsRequest:
+  """
+  Attributes:
+   - db_name
+   - tbl_name
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'db_name', None, None, ), # 1
+    (2, TType.STRING, 'tbl_name', None, None, ), # 2
+  )
+
+  def __init__(self, db_name=None, tbl_name=None,):
+    self.db_name = db_name
+    self.tbl_name = tbl_name
+
+  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.tbl_name = 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('DefaultConstraintsRequest')
+    if self.db_name is not None:
+      oprot.writeFieldBegin('db_name', TType.STRING, 1)
+      oprot.writeString(self.db_name)
+      oprot.writeFieldEnd()
+    if self.tbl_name is not None:
+      oprot.writeFieldBegin('tbl_name', TType.STRING, 2)
+      oprot.writeString(self.tbl_name)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.db_name is None:
+      raise TProtocol.TProtocolException(message='Required field db_name is unset!')
+    if self.tbl_name is None:
+      raise TProtocol.TProtocolException(message='Required field tbl_name is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.db_name)
+    value = (value * 31) ^ hash(self.tbl_name)
+    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 DefaultConstraintsResponse:
+  """
+  Attributes:
+   - defaultConstraints
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'defaultConstraints', (TType.STRUCT,(SQLDefaultConstraint, SQLDefaultConstraint.thrift_spec)), None, ), # 1
+  )
+
+  def __init__(self, defaultConstraints=None,):
+    self.defaultConstraints = defaultConstraints
+
+  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.LIST:
+          self.defaultConstraints = []
+          (_etype316, _size313) = iprot.readListBegin()
+          for _i317 in xrange(_size313):
+            _elem318 = SQLDefaultConstraint()
+            _elem318.read(iprot)
+            self.defaultConstraints.append(_elem318)
+          iprot.readListEnd()
+        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('DefaultConstraintsResponse')
+    if self.defaultConstraints is not None:
+      oprot.writeFieldBegin('defaultConstraints', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraints))
+      for iter319 in self.defaultConstraints:
+        iter319.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.defaultConstraints is None:
+      raise TProtocol.TProtocolException(message='Required field defaultConstraints is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.defaultConstraints)
+    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 DropConstraintRequest:
   """
   Attributes:
@@ -7105,11 +7419,11 @@ class AddPrimaryKeyRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.primaryKeyCols = []
-          (_etype316, _size313) = iprot.readListBegin()
-          for _i317 in xrange(_size313):
-            _elem318 = SQLPrimaryKey()
-            _elem318.read(iprot)
-            self.primaryKeyCols.append(_elem318)
+          (_etype323, _size320) = iprot.readListBegin()
+          for _i324 in xrange(_size320):
+            _elem325 = SQLPrimaryKey()
+            _elem325.read(iprot)
+            self.primaryKeyCols.append(_elem325)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7126,8 +7440,8 @@ class AddPrimaryKeyRequest:
     if self.primaryKeyCols is not None:
       oprot.writeFieldBegin('primaryKeyCols', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeyCols))
-      for iter319 in self.primaryKeyCols:
-        iter319.write(oprot)
+      for iter326 in self.primaryKeyCols:
+        iter326.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7181,11 +7495,87 @@ class AddForeignKeyRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.foreignKeyCols = []
-          (_etype323, _size320) = iprot.readListBegin()
-          for _i324 in xrange(_size320):
-            _elem325 = SQLForeignKey()
-            _elem325.read(iprot)
-            self.foreignKeyCols.append(_elem325)
+          (_etype330, _size327) = iprot.readListBegin()
+          for _i331 in xrange(_size327):
+            _elem332 = SQLForeignKey()
+            _elem332.read(iprot)
+            self.foreignKeyCols.append(_elem332)
+          iprot.readListEnd()
+        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('AddForeignKeyRequest')
+    if self.foreignKeyCols is not None:
+      oprot.writeFieldBegin('foreignKeyCols', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.foreignKeyCols))
+      for iter333 in self.foreignKeyCols:
+        iter333.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.foreignKeyCols is None:
+      raise TProtocol.TProtocolException(message='Required field foreignKeyCols is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.foreignKeyCols)
+    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 AddUniqueConstraintRequest:
+  """
+  Attributes:
+   - uniqueConstraintCols
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'uniqueConstraintCols', (TType.STRUCT,(SQLUniqueConstraint, SQLUniqueConstraint.thrift_spec)), None, ), # 1
+  )
+
+  def __init__(self, uniqueConstraintCols=None,):
+    self.uniqueConstraintCols = uniqueConstraintCols
+
+  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.LIST:
+          self.uniqueConstraintCols = []
+          (_etype337, _size334) = iprot.readListBegin()
+          for _i338 in xrange(_size334):
+            _elem339 = SQLUniqueConstraint()
+            _elem339.read(iprot)
+            self.uniqueConstraintCols.append(_elem339)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7198,26 +7588,26 @@ class AddForeignKeyRequest:
     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('AddForeignKeyRequest')
-    if self.foreignKeyCols is not None:
-      oprot.writeFieldBegin('foreignKeyCols', TType.LIST, 1)
-      oprot.writeListBegin(TType.STRUCT, len(self.foreignKeyCols))
-      for iter326 in self.foreignKeyCols:
-        iter326.write(oprot)
+    oprot.writeStructBegin('AddUniqueConstraintRequest')
+    if self.uniqueConstraintCols is not None:
+      oprot.writeFieldBegin('uniqueConstraintCols', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraintCols))
+      for iter340 in self.uniqueConstraintCols:
+        iter340.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.foreignKeyCols is None:
-      raise TProtocol.TProtocolException(message='Required field foreignKeyCols is unset!')
+    if self.uniqueConstraintCols is None:
+      raise TProtocol.TProtocolException(message='Required field uniqueConstraintCols is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.foreignKeyCols)
+    value = (value * 31) ^ hash(self.uniqueConstraintCols)
     return value
 
   def __repr__(self):
@@ -7231,19 +7621,19 @@ class AddForeignKeyRequest:
   def __ne__(self, other):
     return not (self == other)
 
-class AddUniqueConstraintRequest:
+class AddNotNullConstraintRequest:
   """
   Attributes:
-   - uniqueConstraintCols
+   - notNullConstraintCols
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.LIST, 'uniqueConstraintCols', (TType.STRUCT,(SQLUniqueConstraint, SQLUniqueConstraint.thrift_spec)), None, ), # 1
+    (1, TType.LIST, 'notNullConstraintCols', (TType.STRUCT,(SQLNotNullConstraint, SQLNotNullConstraint.thrift_spec)), None, ), # 1
   )
 
-  def __init__(self, uniqueConstraintCols=None,):
-    self.uniqueConstraintCols = uniqueConstraintCols
+  def __init__(self, notNullConstraintCols=None,):
+    self.notNullConstraintCols = notNullConstraintCols
 
   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:
@@ -7256,12 +7646,12 @@ class AddUniqueConstraintRequest:
         break
       if fid == 1:
         if ftype == TType.LIST:
-          self.uniqueConstraintCols = []
-          (_etype330, _size327) = iprot.readListBegin()
-          for _i331 in xrange(_size327):
-            _elem332 = SQLUniqueConstraint()
-            _elem332.read(iprot)
-            self.uniqueConstraintCols.append(_elem332)
+          self.notNullConstraintCols = []
+          (_etype344, _size341) = iprot.readListBegin()
+          for _i345 in xrange(_size341):
+            _elem346 = SQLNotNullConstraint()
+            _elem346.read(iprot)
+            self.notNullConstraintCols.append(_elem346)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7274,26 +7664,26 @@ class AddUniqueConstraintRequest:
     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('AddUniqueConstraintRequest')
-    if self.uniqueConstraintCols is not None:
-      oprot.writeFieldBegin('uniqueConstraintCols', TType.LIST, 1)
-      oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraintCols))
-      for iter333 in self.uniqueConstraintCols:
-        iter333.write(oprot)
+    oprot.writeStructBegin('AddNotNullConstraintRequest')
+    if self.notNullConstraintCols is not None:
+      oprot.writeFieldBegin('notNullConstraintCols', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraintCols))
+      for iter347 in self.notNullConstraintCols:
+        iter347.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.uniqueConstraintCols is None:
-      raise TProtocol.TProtocolException(message='Required field uniqueConstraintCols is unset!')
+    if self.notNullConstraintCols is None:
+      raise TProtocol.TProtocolException(message='Required field notNullConstraintCols is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.uniqueConstraintCols)
+    value = (value * 31) ^ hash(self.notNullConstraintCols)
     return value
 
   def __repr__(self):
@@ -7307,19 +7697,19 @@ class AddUniqueConstraintRequest:
   def __ne__(self, other):
     return not (self == other)
 
-class AddNotNullConstraintRequest:
+class AddDefaultConstraintRequest:
   """
   Attributes:
-   - notNullConstraintCols
+   - defaultConstraintCols
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.LIST, 'notNullConstraintCols', (TType.STRUCT,(SQLNotNullConstraint, SQLNotNullConstraint.thrift_spec)), None, ), # 1
+    (1, TType.LIST, 'defaultConstraintCols', (TType.STRUCT,(SQLDefaultConstraint, SQLDefaultConstraint.thrift_spec)), None, ), # 1
   )
 
-  def __init__(self, notNullConstraintCols=None,):
-    self.notNullConstraintCols = notNullConstraintCols
+  def __init__(self, defaultConstraintCols=None,):
+    self.defaultConstraintCols = defaultConstraintCols
 
   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:
@@ -7332,12 +7722,12 @@ class AddNotNullConstraintRequest:
         break
       if fid == 1:
         if ftype == TType.LIST:
-          self.notNullConstraintCols = []
-          (_etype337, _size334) = iprot.readListBegin()
-          for _i338 in xrange(_size334):
-            _elem339 = SQLNotNullConstraint()
-            _elem339.read(iprot)
-            self.notNullConstraintCols.append(_elem339)
+          self.defaultConstraintCols = []
+          (_etype351, _size348) = iprot.readListBegin()
+          for _i352 in xrange(_size348):
+            _elem353 = SQLDefaultConstraint()
+            _elem353.read(iprot)
+            self.defaultConstraintCols.append(_elem353)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7350,26 +7740,26 @@ class AddNotNullConstraintRequest:
     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('AddNotNullConstraintRequest')
-    if self.notNullConstraintCols is not None:
-      oprot.writeFieldBegin('notNullConstraintCols', TType.LIST, 1)
-      oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraintCols))
-      for iter340 in self.notNullConstraintCols:
-        iter340.write(oprot)
+    oprot.writeStructBegin('AddDefaultConstraintRequest')
+    if self.defaultConstraintCols is not None:
+      oprot.writeFieldBegin('defaultConstraintCols', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraintCols))
+      for iter354 in self.defaultConstraintCols:
+        iter354.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.notNullConstraintCols is None:
-      raise TProtocol.TProtocolException(message='Required field notNullConstraintCols is unset!')
+    if self.defaultConstraintCols is None:
+      raise TProtocol.TProtocolException(message='Required field defaultConstraintCols is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.notNullConstraintCols)
+    value = (value * 31) ^ hash(self.defaultConstraintCols)
     return value
 
   def __repr__(self):
@@ -7412,11 +7802,11 @@ class PartitionsByExprResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype344, _size341) = iprot.readListBegin()
-          for _i345 in xrange(_size341):
-            _elem346 = Partition()
-            _elem346.read(iprot)
-            self.partitions.append(_elem346)
+          (_etype358, _size355) = iprot.readListBegin()
+          for _i359 in xrange(_size355):
+            _elem360 = Partition()
+            _elem360.read(iprot)
+            self.partitions.append(_elem360)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7438,8 +7828,8 @@ class PartitionsByExprResult:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter347 in self.partitions:
-        iter347.write(oprot)
+      for iter361 in self.partitions:
+        iter361.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.hasUnknownPartitions is not None:
@@ -7623,11 +8013,11 @@ class TableStatsResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.tableStats = []
-          (_etype351, _size348) = iprot.readListBegin()
-          for _i352 in xrange(_size348):
-            _elem353 = ColumnStatisticsObj()
-            _elem353.read(iprot)
-            self.tableStats.append(_elem353)
+          (_etype365, _size362) = iprot.readListBegin()
+          for _i366 in xrange(_size362):
+            _elem367 = ColumnStatisticsObj()
+            _elem367.read(iprot)
+            self.tableStats.append(_elem367)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7644,8 +8034,8 @@ class TableStatsResult:
     if self.tableStats is not None:
       oprot.writeFieldBegin('tableStats', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tableStats))
-      for iter354 in self.tableStats:
-        iter354.write(oprot)
+      for iter368 in self.tableStats:
+        iter368.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7699,17 +8089,17 @@ class PartitionsStatsResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.partStats = {}
-          (_ktype356, _vtype357, _size355 ) = iprot.readMapBegin()
-          for _i359 in xrange(_size355):
-            _key360 = iprot.readString()
-            _val361 = []
-            (_etype365, _size362) = iprot.readListBegin()
-            for _i366 in xrange(_size362):
-              _elem367 = ColumnStatisticsObj()
-              _elem367.read(iprot)
-              _val361.append(_elem367)
+          (_ktype370, _vtype371, _size369 ) = iprot.readMapBegin()
+          for _i373 in xrange(_size369):
+            _key374 = iprot.readString()
+            _val375 = []
+            (_etype379, _size376) = iprot.readListBegin()
+            for _i380 in xrange(_size376):
+              _elem381 = ColumnStatisticsObj()
+              _elem381.read(iprot)
+              _val375.append(_elem381)
             iprot.readListEnd()
-            self.partStats[_key360] = _val361
+            self.partStats[_key374] = _val375
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7726,11 +8116,11 @@ class PartitionsStatsResult:
     if self.partStats is not None:
       oprot.writeFieldBegin('partStats', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.LIST, len(self.partStats))
-      for kiter368,viter369 in self.partStats.items():
-        oprot.writeString(kiter368)
-        oprot.writeListBegin(TType.STRUCT, len(viter369))
-        for iter370 in viter369:
-          iter370.write(oprot)
+      for kiter382,viter383 in self.partStats.items():
+        oprot.writeString(kiter382)
+        oprot.writeListBegin(TType.STRUCT, len(viter383))
+        for iter384 in viter383:
+          iter384.write(oprot)
         oprot.writeListEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
@@ -7801,10 +8191,10 @@ class TableStatsRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.colNames = []
-          (_etype374, _size371) = iprot.readListBegin()
-          for _i375 in xrange(_size371):
-            _elem376 = iprot.readString()
-            self.colNames.append(_elem376)
+          (_etype388, _size385) = iprot.readListBegin()
+          for _i389 in xrange(_size385):
+            _elem390 = iprot.readString()
+            self.colNames.append(_elem390)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7829,8 +8219,8 @@ class TableStatsRequest:
     if self.colNames is not None:
       oprot.writeFieldBegin('colNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.colNames))
-      for iter377 in self.colNames:
-        oprot.writeString(iter377)
+      for iter391 in self.colNames:
+        oprot.writeString(iter391)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7909,20 +8299,20 @@ class PartitionsStatsRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.colNames = []
-          (_etype381, _size378) = iprot.readListBegin()
-          for _i382 in xrange(_size378):
-            _elem383 = iprot.readString()
-            self.colNames.append(_elem383)
+          (_etype395, _size392) = iprot.readListBegin()
+          for _i396 in xrange(_size392):
+            _elem397 = iprot.readString()
+            self.colNames.append(_elem397)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype387, _size384) = iprot.readListBegin()
-          for _i388 in xrange(_size384):
-            _elem389 = iprot.readString()
-            self.partNames.append(_elem389)
+          (_etype401, _size398) = iprot.readListBegin()
+          for _i402 in xrange(_size398):
+            _elem403 = iprot.readString()
+            self.partNames.append(_elem403)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7947,15 +8337,15 @@ class PartitionsStatsRequest:
     if self.colNames is not None:
       oprot.writeFieldBegin('colNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.colNames))
-      for iter390 in self.colNames:
-        oprot.writeString(iter390)
+      for iter404 in self.colNames:
+        oprot.writeString(iter404)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 4)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter391 in self.partNames:
-        oprot.writeString(iter391)
+      for iter405 in self.partNames:
+        oprot.writeString(iter405)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8018,11 +8408,11 @@ class AddPartitionsResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype395, _size392) = iprot.readListBegin()
-          for _i396 in xrange(_size392):
-            _elem397 = Partition()
-            _elem397.read(iprot)
-            self.partitions.append(_elem397)
+          (_etype409, _size406) = iprot.readListBegin()
+          for _i410 in xrange(_size406):
+            _elem411 = Partition()
+            _elem411.read(iprot)
+            self.partitions.append(_elem411)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8039,8 +8429,8 @@ class AddPartitionsResult:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter398 in self.partitions:
-        iter398.write(oprot)
+      for iter412 in self.partitions:
+        iter412.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8114,11 +8504,11 @@ class AddPartitionsRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.parts = []
-          (_etype402, _size399) = iprot.readListBegin()
-          for _i403 in xrange(_size399):
-            _elem404 = Partition()
-            _elem404.read(iprot)
-            self.parts.append(_elem404)
+          (_etype416, _size413) = iprot.readListBegin()
+          for _i417 in xrange(_size413):
+            _elem418 = Partition()
+            _elem418.read(iprot)
+            self.parts.append(_elem418)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8153,8 +8543,8 @@ class AddPartitionsRequest:
     if self.parts is not None:
       oprot.writeFieldBegin('parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.parts))
-      for iter405 in self.parts:
-        iter405.write(oprot)
+      for iter419 in self.parts:
+        iter419.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.ifNotExists is not None:
@@ -8226,11 +8616,11 @@ class DropPartitionsResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype409, _size406) = iprot.readListBegin()
-          for _i410 in xrange(_size406):
-            _elem411 = Partition()
-            _elem411.read(iprot)
-            self.partitions.append(_elem411)
+          (_etype423, _size420) = iprot.readListBegin()
+          for _i424 in xrange(_size420):
+            _elem425 = Partition()
+            _elem425.read(iprot)
+            self.partitions.append(_elem425)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8247,8 +8637,8 @@ class DropPartitionsResult:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter412 in self.partitions:
-        iter412.write(oprot)
+      for iter426 in self.partitions:
+        iter426.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8383,21 +8773,21 @@ class RequestPartsSpec:
       if fid == 1:
         if ftype == TType.LIST:
           self.names = []
-          (_etype416, _size413) = iprot.readListBegin()
-          for _i417 in xrange(_size413):
-            _elem418 = iprot.readString()
-            self.names.append(_elem418)
+          (_etype430, _size427) = iprot.readListBegin()
+          for _i431 in xrange(_size427):
+            _elem432 = iprot.readString()
+            self.names.append(_elem432)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.exprs = []
-          (_etype422, _size419) = iprot.readListBegin()
-          for _i423 in xrange(_size419):
-            _elem424 = DropPartitionsExpr()
-            _elem424.read(iprot)
-            self.exprs.append(_elem424)
+          (_etype436, _size433) = iprot.readListBegin()
+          for _i437 in xrange(_size433):
+            _elem438 = DropPartitionsExpr()
+            _elem438.read(iprot)
+            self.exprs.append(_elem438)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8414,15 +8804,15 @@ class RequestPartsSpec:
     if self.names is not None:
       oprot.writeFieldBegin('names', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.names))
-      for iter425 in self.names:
-        oprot.writeString(iter425)
+      for iter439 in self.names:
+        oprot.writeString(iter439)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.exprs is not None:
       oprot.writeFieldBegin('exprs', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.exprs))
-      for iter426 in self.exprs:
-        iter426.write(oprot)
+      for iter440 in self.exprs:
+        iter440.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8670,11 +9060,11 @@ class PartitionValuesRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partitionKeys = []
-          (_etype430, _size427) = iprot.readListBegin()
-          for _i431 in xrange(_size427):
-            _elem432 = FieldSchema()
-            _elem432.read(iprot)
-            self.partitionKeys.append(_elem432)
+          (_etype444, _size441) = iprot.readListBegin()
+          for _i445 in xrange(_size441):
+            _elem446 = FieldSchema()
+            _elem446.read(iprot)
+            self.partitionKeys.append(_elem446)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8691,11 +9081,11 @@ class PartitionValuesRequest:
       elif fid == 6:
         if ftype == TType.LIST:
           self.partitionOrder = []
-          (_etype436, _size433) = iprot.readListBegin()
-          for _i437 in xrange(_size433):
-            _elem438 = FieldSchema()
-            _elem438.read(iprot)
-            self.partitionOrder.append(_elem438)
+          (_etype450, _size447) = iprot.readListBegin()
+          for _i451 in xrange(_size447):
+            _elem452 = FieldSchema()
+            _elem452.read(iprot)
+            self.partitionOrder.append(_elem452)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8730,8 +9120,8 @@ class PartitionValuesRequest:
     if self.partitionKeys is not None:
       oprot.writeFieldBegin('partitionKeys', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.partitionKeys))
-      for iter439 in self.partitionKeys:
-        iter439.write(oprot)
+      for iter453 in self.partitionKeys:
+        iter453.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.applyDistinct is not None:
@@ -8745,8 +9135,8 @@ class PartitionValuesRequest:
     if self.partitionOrder is not None:
       oprot.writeFieldBegin('partitionOrder', TType.LIST, 6)
       oprot.writeListBegin(TType.STRUCT, len(self.partitionOrder))
-      for iter440 in self.partitionOrder:
-        iter440.write(oprot)
+      for iter454 in self.partitionOrder:
+        iter454.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.ascending is not None:
@@ -8819,10 +9209,10 @@ class PartitionValuesRow:
       if fid == 1:
         if ftype == TType.LIST:
           self.row = []
-          (_etype444, _size441) = iprot.readListBegin()
-          for _i445 in xrange(_size441):
-            _elem446 = iprot.readString()
-            self.row.append(_elem446)
+          (_etype458, _size455) = iprot.readListBegin()
+          for _i459 in xrange(_size455):
+            _elem460 = iprot.readString()
+            self.row.append(_elem460)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8839,8 +9229,8 @@ class PartitionValuesRow:
     if self.row is not None:
       oprot.writeFieldBegin('row', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.row))
-      for iter447 in self.row:
-        oprot.writeString(iter447)
+      for iter461 in self.row:
+        oprot.writeString(iter461)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8894,11 +9284,11 @@ class PartitionValuesResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitionValues = []
-          (_etype451, _size448) = iprot.readListBegin()
-          for _i452 in xrange(_size448):
-            _elem453 = PartitionValuesRow()
-            _elem453.read(iprot)
-            self.partitionValues.append(_elem453)
+          (_etype465, _size462) = iprot.readListBegin()
+          for _i466 in xrange(_size462):
+            _elem467 = PartitionValuesRow()
+            _elem467.read(iprot)
+            self.partitionValues.append(_elem467)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8915,8 +9305,8 @@ class PartitionValuesResponse:
     if self.partitionValues is not None:
       oprot.writeFieldBegin('partitionValues', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitionValues))
-      for iter454 in self.partitionValues:
-        iter454.write(oprot)
+      for iter468 in self.partitionValues:
+        iter468.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9104,11 +9494,11 @@ class Function:
       elif fid == 8:
         if ftype == TType.LIST:
           self.resourceUris = []
-          (_etype458, _size455) = iprot.readListBegin()
-          for _i459 in xrange(_size455):
-            _elem460 = ResourceUri()
-            _elem460.read(iprot)
-            self.resourceUris.append(_elem460)
+          (_etype472, _size469) = iprot.readListBegin()
+          for _i473 in xrange(_size469):
+            _elem474 = ResourceUri()
+            _elem474.read(iprot)
+            self.resourceUris.append(_elem474)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9153,8 +9543,8 @@ class Function:
     if self.resourceUris is not None:
       oprot.writeFieldBegin('resourceUris', TType.LIST, 8)
       oprot.writeListBegin(TType.STRUCT, len(self.resourceUris))
-      for iter461 in self.resourceUris:
-        iter461.write(oprot)
+      for iter475 in self.resourceUris:
+        iter475.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9398,11 +9788,11 @@ class GetOpenTxnsInfoResponse:
       elif fid == 2:
         if ftype == TType.LIST:
           self.open_txns = []
-          (_etype465, _size462) = iprot.readListBegin()
-          for _i466 in xrange(_size462):
-            _elem467 = TxnInfo()
-            _elem467.read(iprot)
-            self.open_txns.append(_elem467)
+          (_etype479, _size476) = iprot.readListBegin()
+          for _i480 in xrange(_size476):
+            _elem481 = TxnInfo()
+            _elem481.read(iprot)
+            self.open_txns.append(_elem481)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9423,8 +9813,8 @@ class GetOpenTxnsInfoResponse:
     if self.open_txns is not None:
       oprot.writeFieldBegin('open_txns', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.open_txns))
-      for iter468 in self.open_txns:
-        iter468.write(oprot)
+      for iter482 in self.open_txns:
+        iter482.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9495,10 +9885,10 @@ class GetOpenTxnsResponse:
       elif fid == 2:
         if ftype == TType.LIST:
           self.open_txns = []
-          (_etype472, _size469) = iprot.readListBegin()
-          for _i473 in xrange(_size469):
-            _elem474 = iprot.readI64()
-            self.open_txns.append(_elem474)
+          (_etype486, _size483) = iprot.readListBegin()
+          for _i487 in xrange(_size483):
+            _elem488 = iprot.readI64()
+            self.open_txns.append(_elem488)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9529,8 +9919,8 @@ class GetOpenTxnsResponse:
     if self.open_txns is not None:
       oprot.writeFieldBegin('open_txns', TType.LIST, 2)
       oprot.writeListBegin(TType.I64, len(self.open_txns))
-      for iter475 in self.open_txns:
-        oprot.writeI64(iter475)
+      for iter489 in self.open_txns:
+        oprot.writeI64(iter489)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.min_open_txn is not None:
@@ -9709,10 +10099,10 @@ class OpenTxnsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.txn_ids = []
-          (_etype479, _size476) = iprot.readListBegin()
-          for _i480 in xrange(_size476):
-            _elem481 = iprot.readI64()
-            self.txn_ids.append(_elem481)
+          (_etype493, _size490) = iprot.readListBegin()
+          for _i494 in xrange(_size490):
+            _elem495 = iprot.readI64()
+            self.txn_ids.append(_elem495)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9729,8 +10119,8 @@ class OpenTxnsResponse:
     if self.txn_ids is not None:
       oprot.writeFieldBegin('txn_ids', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.txn_ids))
-      for iter482 in self.txn_ids:
-        oprot.writeI64(iter482)
+      for iter496 in self.txn_ids:
+        oprot.writeI64(iter496)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9851,10 +10241,10 @@ class AbortTxnsRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.txn_ids = []
-          (_etype486, _size483) = iprot.readListBegin()
-          for _i487 in xrange(_size483):
-            _elem488 = iprot.readI64()
-            self.txn_ids.append(_elem488)
+          (_etype500, _size497) = iprot.readListBegin()
+          for _i501 in xrange(_size497):
+            _elem502 = iprot.readI64()
+            self.txn_ids.append(_elem502)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9871,8 +10261,8 @@ class AbortTxnsRequest:
     if self.txn_ids is not None:
       oprot.writeFieldBegin('txn_ids', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.txn_ids))
-      for iter489 in self.txn_ids:
-        oprot.writeI64(iter489)
+      for iter503 in self.txn_ids:
+        oprot.writeI64(iter503)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9996,10 +10386,10 @@ class GetValidWriteIdsRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fullTableNames = []
-          (_etype493, _size490) = iprot.readListBegin()
-          for _i494 in xrange(_size490):
-            _elem495 = iprot.readString()
-            self.fullTableNames.append(_elem495)
+          (_etype507, _size504) = iprot.readListBegin()
+          for _i508 in xrange(_size504):
+            _elem509 = iprot.readString()
+            self.fullTableNames.append(_elem509)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10021,8 +10411,8 @@ class GetValidWriteIdsRequest:
     if self.fullTableNames is not None:
       oprot.writeFieldBegin('fullTableNames', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.fullTableNames))
-      for iter496 in self.fullTableNames:
-        oprot.writeString(iter496)
+      for iter510 in self.fullTableNames:
+        oprot.writeString(iter510)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.validTxnList is not None:
@@ -10105,10 +10495,10 @@ class TableValidWriteIds:
       elif fid == 3:
         if ftype == TType.LIST:
           self.invalidWriteIds = []
-          (_etype500, _size497) = iprot.readListBegin()
-          for _i501 in xrange(_size497):
-            _elem502 = iprot.readI64()
-            self.invalidWriteIds.append(_elem502)
+          (_etype514, _size511) = iprot.readListBegin()
+          for _i515 in xrange(_size511):
+            _elem516 = iprot.readI64()
+            self.invalidWriteIds.append(_elem516)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10143,8 +10533,8 @@ class TableValidWriteIds:
     if self.invalidWriteIds is not None:
       oprot.writeFieldBegin('invalidWriteIds', TType.LIST, 3)
       oprot.writeListBegin(TType.I64, len(self.invalidWriteIds))
-      for iter503 in self.invalidWriteIds:
-        oprot.writeI64(iter503)
+      for iter517 in self.invalidWriteIds:
+        oprot.writeI64(iter517)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.minOpenWriteId is not None:
@@ -10216,11 +10606,11 @@ class GetValidWriteIdsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.tblValidWriteIds = []
-          (_etype507, _size504) = iprot.readListBegin()
-          for _i508 in xrange(_size504):
-            _elem509 = TableValidWriteIds()
-            _elem509.read(iprot)
-            self.tblValidWriteIds.append(_elem509)
+          (_etype521, _size518) = iprot.readListBegin()
+          for _i522 in xrange(_size518):
+            _elem523 = TableValidWriteIds()
+            _elem523.read(iprot)
+            self.tblValidWriteIds.append(_elem523)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10237,8 +10627,8 @@ class GetValidWriteIdsResponse:
     if self.tblValidWriteIds is not None:
       oprot.writeFieldBegin('tblValidWriteIds', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tblValidWriteIds))
-      for iter510 in self.tblValidWriteIds:
-        iter510.write(oprot)
+      for iter524 in self.tblValidWriteIds:
+        iter524.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10298,10 +10688,10 @@ class AllocateTableWriteIdsRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.txnIds = []
-          (_etype514, _size511) = iprot.readListBegin()
-          for _i515 in xrange(_size511):
-            _elem516 = iprot.readI64()
-            self.txnIds.append(_elem516)
+          (_etype528, _size525) = iprot.readListBegin()
+          for _i529 in xrange(_size525):
+            _elem530 = iprot.readI64()
+            self.txnIds.append(_elem530)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10328,8 +10718,8 @@ class AllocateTableWriteIdsRequest:
     if self.txnIds is not None:
       oprot.writeFieldBegin('txnIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.txnIds))
-      for iter517 in self.txnIds:
-        oprot.writeI64(iter517)
+      for iter531 in self.txnIds:
+        oprot.writeI64(iter531)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.dbName is not None:
@@ -10479,11 +10869,11 @@ class AllocateTableWriteIdsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.txnToWriteIds = []
-          (_etype521, _size518) = iprot.readListBegin()
-          for _i522 in xrange(_size518):
-            _elem523 = TxnToWriteId()
-            _elem523.read(iprot)
-            self.txnToWriteIds.append(_elem523)
+          (_etype535, _size532) = iprot.readListBegin()
+          for _i536 in xrange(_size532):
+            _elem537 = TxnToWriteId()
+            _elem537.read(iprot)
+            self.txnToWriteIds.append(_elem537)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10500,8 +10890,8 @@ class AllocateTableWriteIdsResponse:
     if self.txnToWriteIds is not None:
       oprot.writeFieldBegin('txnToWriteIds', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.txnToWriteIds))
-      for iter524 in self.txnToWriteIds:
-        iter524.write(oprot)
+      for iter538 in self.txnToWriteIds:
+        iter538.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10729,11 +11119,11 @@ class LockRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.component = []
-          (_etype528, _size525) = iprot.readListBegin()
-          for _i529 in xrange(_size525):
-            _elem530 = LockComponent()
-            _elem530.read(iprot)
-            self.component.append(_elem530)
+          (_etype542, _size539) = iprot.readListBegin()
+          for _i543 in xrange(_size539):
+            _elem544 = LockComponent()
+            _elem544.read(iprot)
+            self.component.append(_elem544)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10770,8 +11160,8 @@ class LockRequest:
     if self.component is not None:
       oprot.writeFieldBegin('component', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.component))
-      for iter531 in self.component:
-        iter531.write(oprot)
+      for iter545 in self.component:
+        iter545.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.txnid is not None:
@@ -11469,11 +11859,11 @@ class ShowLocksResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.locks = []
-          (_etype535, _size532) = iprot.readListBegin()
-          for _i536 in xrange(_size532):
-            _elem537 = ShowLocksResponseElement()
-            _elem537.read(iprot)
-            self.locks.append(_elem537)
+          (_etype549, _size546) = iprot.readListBegin()
+          for _i550 in xrange(_size546):
+            _elem551 = ShowLocksResponseElement()
+            _elem551.read(iprot)
+            self.locks.append(_elem551)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11490,8 +11880,8 @@ class ShowLocksResponse:
     if self.locks is not None:
       oprot.writeFieldBegin('locks', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.locks))
-      for iter538 in self.locks:
-        iter538.write(oprot)
+      for iter552 in self.locks:
+        iter552.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11706,20 +12096,20 @@ class HeartbeatTxnRangeResponse:
       if fid == 1:
         if ftype == TType.SET:
           self.aborted = set()
-          (_etype542, _size539) = iprot.readSetBegin()
-          for _i543 in xrange(_size539):
-            _elem544 = iprot.readI64()
-            self.aborted.add(_elem544)
+          (_etype556, _size553) = iprot.readSetBegin()
+          for _i557 in xrange(_size553):
+            _elem558 = iprot.readI64()
+            self.aborted.add(_elem558)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.SET:
           self.nosuch = set()
-          (_etype548, _size545) = iprot.readSetBegin()
-          for _i549 in xrange(_size545):
-            _elem550 = iprot.readI64()
-            self.nosuch.add(_elem550)
+          (_etype562, _size559) = iprot.readSetBegin()
+          for _i563 in xrange(_size559):
+            _elem564 = iprot.readI64()
+            self.nosuch.add(_elem564)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -11736,15 +12126,15 @@ class HeartbeatTxnRangeResponse:
     if self.aborted is not None:
       oprot.writeFieldBegin('aborted', TType.SET, 1)
       oprot.writeSetBegin(TType.I64, len(self.aborted))
-      for iter551 in self.aborted:
-        oprot.writeI64(iter551)
+      for iter565 in self.aborted:
+        oprot.writeI64(iter565)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.nosuch is not None:
       oprot.writeFieldBegin('nosuch', TType.SET, 2)
       oprot.writeSetBegin(TType.I64, len(self.nosuch))
-      for iter552 in self.nosuch:
-        oprot.writeI64(iter552)
+      for iter566 in self.nosuch:
+        oprot.writeI64(iter566)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11841,11 +12231,11 @@ class CompactionRequest:
       elif fid == 6:
         if ftype == TType.MAP:
           self.properties = {}
-          (_ktype554, _vtype555, _size553 ) = iprot.readMapBegin()
-          for _i557 in xrange(_size553):
-            _key558 = iprot.readString()
-            _val559 = iprot.readString()
-            self.properties[_key558] = _val559
+          (_ktype568, _vtype569, _size567 ) = iprot.readMapBegin()
+          for _i571 in xrange(_size567):
+            _key572 = iprot.readString()
+            _val573 = iprot.readString()
+            self.properties[_key572] = _val573
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -11882,9 +12272,9 @@ class CompactionRequest:
     if self.properties is not None:
       oprot.writeFieldBegin('properties', TType.MAP, 6)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
-      for kiter560,viter561 in self.properties.items():
-        oprot.writeString(kiter560)
-        oprot.writeString(viter561)
+      for kiter574,viter575 in self.properties.items():
+        oprot.writeString(kiter574)
+        oprot.writeString(viter575)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12319,11 +12709,11 @@ class ShowCompactResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.compacts = []
-          (_etype565, _size562) = iprot.readListBegin()
-          for _i566 in xrange(_size562):
-            _elem567 = ShowCompactResponseElement()
-            _elem567.read(iprot)
-            self.compacts.append(_elem567)
+          (_etype579, _size576) = iprot.readListBegin()
+          for _i580 in xrange(_size576):
+            _elem581 = ShowCompactResponseElement()
+            _elem581.read(iprot)
+            self.compacts.append(_elem581)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12340,8 +12730,8 @@ class ShowCompactResponse:
     if self.compacts is not None:
       oprot.writeFieldBegin('compacts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.compacts))
-      for iter568 in self.compacts:
-        iter568.write(oprot)
+      for iter582 in self.compacts:
+        iter582.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12430,10 +12820,10 @@ class AddDynamicPartitions:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionnames = []
-          (_etype572, _size569) = iprot.readListBegin()
-          for _i573 in xrange(_size569):
-            _elem574 = iprot.readString()
-            self.partitionnames.append(_elem574)
+          (_etype586, _size583) = iprot.readListBegin()
+          for _i587 in xrange(_size583):
+            _elem588 = iprot.readString()
+            self.partitionnames.append(_elem588)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12471,8 +12861,8 @@ class AddDynamicPartitions:
     if self.partitionnames is not None:
       oprot.writeFieldBegin('partitionnames', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionnames))
-      for iter575 in self.partitionnames:
-        oprot.writeString(iter575)
+      for iter589 in self.partitionnames:
+        oprot.writeString(iter589)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.operationType is not None:
@@ -12694,10 +13084,10 @@ class CreationMetadata:
       elif fid == 3:
         if ftype == TType.SET:
           self.tablesUsed = set()
-          (_etype579, _size576) = iprot.readSetBegin()
-          for _i580 in xrange(_size576):
-            _elem581 = iprot.readString()
-            self.tablesUsed.add(_elem581)
+          (_etype593, _size590) = iprot.readSetBegin()
+          for _i594 in xrange(_size590):
+            _elem595 = iprot.readString()
+            self.tablesUsed.add(_elem595)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -12727,8 +13117,8 @@ class CreationMetadata:
     if self.tablesUsed is not None:
       oprot.writeFieldBegin('tablesUsed', TType.SET, 3)
       oprot.writeSetBegin(TType.STRING, len(self.tablesUsed))
-      for iter582 in self.tablesUsed:
-        oprot.writeString(iter582)
+      for iter596 in self.tablesUsed:
+        oprot.writeString(iter596)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.validTxnList is not None:
@@ -13024,11 +13414,11 @@ class NotificationEventResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.events = []
-          (_etype586, _size583) = iprot.readListBegin()
-          for _i587 in xrange(_size583):
-            _elem588 = NotificationEvent()
-            _elem588.read(iprot)
-            self.events.append(_elem588)
+          (_etype600, _size597) = iprot.readListBegin()
+          for _i601 in xrange(_size597):
+            _elem602 = NotificationEvent()
+            _elem602.read(iprot)
+            self.events.append(_elem602)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13045,8 +13435,8 @@ class NotificationEventResponse:
     if self.events is not None:
       oprot.writeFieldBegin('events', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.events))
-      for iter589 in self.events:
-        iter589.write(oprot)
+      for iter603 in self.events:
+        iter603.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13327,20 +13717,20 @@ class InsertEventRequestData:
       elif fid == 2:
         if ftype == TType.LIST:
           self.filesAdded = []
-          (_etype593, _size590) = iprot.readListBegin()
-          for _i594 in xrange(_size590):
-            _elem595 = iprot.readString()
-            self.filesAdded.append(_elem595)
+          (_etype607, _size604) = iprot.readListBegin()
+          for _i608 in xrange(_size604):
+            _elem609 = iprot.readString()
+            self.filesAdded.append(_elem609)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.filesAddedChecksum = []
-          (_etype599, _size596) = iprot.readListBegin()
-          for _i600 in xrange(_size596):
-            _elem601 = iprot.readString()
-            self.filesAddedChecksum.append(_elem601)
+          (_etype613, _size610) = iprot.readListBegin()
+          for _i614 in xrange(_size610):
+            _elem615 = iprot.readString()
+            self.filesAddedChecksum.append(_elem615)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13361,15 +13751,15 @@ class InsertEventRequestData:
     if self.filesAdded is not None:
       oprot.writeFieldBegin('filesAdded', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.filesAdded))
-      for iter602 in self.filesAdded:
-        oprot.writeString(iter602)
+      for iter616 in self.filesAdded:
+        oprot.writeString(iter616)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.filesAddedChecksum is not None:
       oprot.writeFieldBegin('filesAddedChecksum', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.filesAddedChecksum))
-      for iter603 in self.filesAddedChecksum:
-        oprot.writeString(iter603)
+      for iter617 in self.filesAddedChecksum:
+        oprot.writeString(iter617)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13524,10 +13914,10 @@ class FireEventRequest:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionVals = []
-          (_etype607, _size604) = iprot.readListBegin()
-          for _i608 in xrange(_size604):
-            _elem609 = iprot.readString()
-            self.partitionVals.append(_elem609)
+          (_etype621, _size618) = iprot.readListBegin()
+          for _i622 in xrange(_size618):
+            _elem623 = iprot.readString()
+            self.partitionVals.append(_elem623)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13560,8 +13950,8 @@ class FireEventRequest:
     if self.partitionVals is not None:
       oprot.writeFieldBegin('partitionVals', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionVals))
-      for iter610 in self.partitionVals:
-        oprot.writeString(iter610)
+      for iter624 in self.partitionVals:
+        oprot.writeString(iter624)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13748,12 +14138,12 @@ class GetFileMetadataByExprResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype612, _vtype613, _size611 ) = iprot.readMapBegin()
-          for _i615 in xrange(_size611):
-            _key616 = iprot.readI64()
-            _val617 = MetadataPpdResult()
-            _val617.read(iprot)
-            self.metadata[_key616] = _val617
+          (_ktype626, _vtype627, _size625 ) = iprot.readMapBegin()
+          for _i629 in xrange(_size625):
+            _key630 = iprot.readI64()
+            _val631 = MetadataPpdResult()
+            _val631.read(iprot)
+            self.metadata[_key630] = _val631
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -13775,9 +14165,9 @@ class GetFileMetadataByExprResult:
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.MAP, 1)
       oprot.writeMapBegin(TType.I64, TType.STRUCT, len(self.metadata))
-      for kiter618,viter619 in self.metadata.items():
-        oprot.writeI64(kiter618)
-        viter619.write(oprot)
+      for kiter632,viter633 in self.metadata.items():
+        oprot.writeI64(kiter632)
+        viter633.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -13847,10 +14237,10 @@ class GetFileMetadataByExprRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype623, _size620) = iprot.readListBegin()
-          for _i624 in xrange(_size620):
-            _elem625 = iprot.readI64()
-            self.fileIds.append(_elem625)
+          (_etype637, _size634) = iprot.readListBegin()
+          for _i638 in xrange(_size634):
+            _elem639 = iprot.readI64()
+            self.fileIds.append(_elem639)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13882,8 +14272,8 @@ class GetFileMetadataByExprRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter626 in self.fileIds:
-        oprot.writeI64(iter626)
+      for iter640 in self.fileIds:
+        oprot.writeI64(iter640)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.expr is not None:
@@ -13957,11 +14347,11 @@ class GetFileMetadataResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype628, _vtype629, _size627 ) = iprot.readMapBegin()
-          for _i631 in xrange(_size627):
-            _key632 = iprot.readI64()
-            _val633 = iprot.readString()
-            self.metadata[_key632] = _val633
+          (_ktype642, _vtype643, _size641 ) = iprot.readMapBegin()
+          for _i645 in xrange(_size641):
+            _key646 = iprot.readI64()
+            _val647 = iprot.readString()
+            self.metadata[_key646] = _val647
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -13983,9 +14373,9 @@ class GetFileMetadataResult:
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.MAP, 1)
       oprot.writeMapBegin(TType.I64, TType.STRING, len(self.metadata))
-      for kiter634,viter635 in self.metadata.items():
-        oprot.writeI64(kiter634)
-        oprot.writeString(viter635)
+      for kiter648,viter649 in self.metadata.items():
+        oprot.writeI64(kiter648)
+        oprot.writeString(viter649)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -14046,10 +14436,10 @@ class GetFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype639, _size636) = iprot.readListBegin()
-          for _i640 in xrange(_size636):
-            _elem641 = iprot.readI64()
-            self.fileIds.append(_elem641)
+          (_etype653, _size650) = iprot.readListBegin()
+          for _i654 in xrange(_size650):
+            _elem655 = iprot.readI64()
+            self.fileIds.append(_elem655)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14066,8 +14456,8 @@ class GetFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter642 in self.fileIds:
-        oprot.writeI64(iter642)
+      for iter656 in self.fileIds:
+        oprot.writeI64(iter656)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14173,20 +14563,20 @@ class PutFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype646, _size643) = iprot.readListBegin()
-          for _i647 in xrange(_size643):
-            _elem648 = iprot.readI64()
-            self.fileIds.append(_elem648)
+          (_etype660, _size657) = iprot.readListBegin()
+          for _i661 in xrange(_size657):
+            _elem662 = iprot.readI64()
+            self.fileIds.append(_elem662)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.metadata = []
-          (_etype652, _size649) = iprot.readListBegin()
-          for _i653 in xrange(_size649):
-            _elem654 = iprot.readString()
-            self.metadata.append(_elem654)
+          (_etype666, _size663) = iprot.readListBegin()
+          for _i667 in xrange(_size663):
+            _elem668 = iprot.readString()
+            self.metadata.append(_elem668)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14208,15 +14598,15 @@ class PutFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter655 in self.fileIds:
-        oprot.writeI64(iter655)
+      for iter669 in self.fileIds:
+        oprot.writeI64(iter669)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.metadata))
-      for iter656 in self.metadata:
-        oprot.writeString(iter656)
+      for iter670 in self.metadata:
+        oprot.writeString(iter670)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.type is not None:
@@ -14324,10 +14714,10 @@ class ClearFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype660, _size657) = iprot.readListBegin()
-          for _i661 in xrange(_size657):
-            _elem662 = iprot.readI64()
-            self.fileIds.append(_elem662)
+          (_etype674, _size671) = iprot.readListBegin()
+          for _i675 in xrange(_size671):
+            _elem676 = iprot.readI64()
+            self.fileIds.append(_elem676)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14344,8 +14734,8 @@ class ClearFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter663 in self.fileIds:
-        oprot.writeI64(iter663)
+      for iter677 in self.fileIds:
+        oprot.writeI64(iter677)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14574,11 +14964,11 @@ class GetAllFunctionsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.functions = []
-          (_etype667, _size664) = iprot.readListBegin()
-          for _i668 in xrange(_size664):
-            _elem669 = Function()
-            _elem669.read(iprot)
-            self.functions.append(_elem669)
+          (_etype681, _size678) = iprot.readListBegin()
+          for _i682 in xrange(_size678):
+            _elem683 = Function()
+            _elem683.read(iprot)
+            self.functions.append(_elem683)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14595,8 +14985,8 @@ class GetAllFunctionsResponse:
     if self.functions is not None:
       oprot.writeFieldBegin('functions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.functions))
-      for iter670 in self.functions:
-        iter670.write(oprot)
+      for iter684 in self.functions:
+        iter684.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14648,10 +15038,10 @@ class ClientCapabilities:
       if fid == 1:
         if ftype == TType.LIST:
           self.values = []
-          (_etype674, _size671) = iprot.readListBegin()
-          for _i675 in xrange(_size671):
-            _elem676 = iprot.readI32()
-            self.values.append(_elem676)
+          (_etype688, _size685) = iprot.readListBegin()
+          for _i689 in xrange(_size685):
+            _elem690 = iprot.readI32()
+            self.values.append(_elem690)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14668,8 +15058,8 @@ class ClientCapabilities:
     if self.values is not None:
       oprot.writeFieldBegin('values', TType.LIST, 1)
       oprot.writeListBegin(TType.I32, len(self.values))
-      for iter677 in self.values:
-        oprot.writeI32(iter677)
+      for iter691 in self.values:
+        oprot.writeI32(iter691)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14898,10 +15288,10 @@ class GetTablesRequest:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tblNames = []
-          (_etype681, _size678) = iprot.readListBegin()
-          for _i682 in xrange(_size678):
-            _elem683 = iprot.readString()
-            self.tblNames.append(_elem683)
+          (_etype695, _size692) = iprot.readListBegin()
+          for _i696 in xrange(_size692):
+            _elem697 = iprot.readString()
+            self.tblNames.append(_elem697)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14928,8 +15318,8 @@ class GetTablesRequest:
     if self.tblNames is not None:
       oprot.writeFieldBegin('tblNames', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tblNames))
-      for iter684 in self.tblNames:
-        oprot.writeString(iter684)
+      for iter698 in self.tblNames:
+        oprot.writeString(iter698)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.capabilities is not None:
@@ -14989,11 +15379,11 @@ class GetTablesResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.tables = []
-          (_etype688, _size685) = iprot.readListBegin()
-          for _i689 in xrange(_size685):
-            _elem690 = Table()
-            _elem690.read(iprot)
-            self.tables.append(_elem690)
+          (_etype702, _size699) = iprot.readListBegin()
+          for _i703 in xrange(_size699):
+            _elem704 = Table()
+            _elem704.read(iprot)
+            self.tables.append(_elem704)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15010,8 +15400,8 @@ class GetTablesResult:
     if self.tables is not None:
       oprot.writeFieldBegin('tables', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tables))
-      for iter691 in self.tables:
-        iter691.write(oprot)
+      for iter705 in self.tables:
+        iter705.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15309,10 +15699,10 @@ class Materialization:
       if fid == 1:
         if ftype == TType.SET:
           self.tablesUsed = set()
-          (_etype695, _size692) = iprot.readSetBegin()
-          for _i696 in xrange(_size692):
-            _elem697 = iprot.readString()
-            self.tablesUsed.add(_elem697)
+          (_etype709, _size706) = iprot.readSetBegin()
+          for _i710 in xrange(_size706):
+            _elem711 = iprot.readString()
+            self.tablesUsed.add(_elem711)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -15339,8 +15729,8 @@ class Materialization:
     if self.tablesUsed is not None:
       oprot.writeFieldBegin('tablesUsed', TType.SET, 1)
       oprot.writeSetBegin(TType.STRING, len(self.tablesUsed))
-      for iter698 in self.tablesUsed:
-        oprot.writeString(iter698)
+      for iter712 in self.tablesUsed:
+        oprot.writeString(iter712)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.validTxnList is not None:
@@ -16244,44 +16634,44 @@ class WMFullResourcePlan:
       elif fid == 2:
         if ftype == TType.LIST:
           self.pools = []
-          (_etype702, _size699) = iprot.readListBegin()
-          for _i703 in xrange(_size699):
-            _elem704 = WMPool()
-            _elem704.read(iprot)
-            self.pools.append(_elem704)
+          (_etype716, _size713) = iprot.readListBegin()
+          for _i717 in xrange(_size713):
+            _elem718 = WMPool()
+            _elem718.read(iprot)
+            self.pools.append(_elem718)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.mappings = []
-          (_etype708, _size705) = iprot.readListBegin()
-          for _i709 in xrange(_size705):
-            _elem710 = WMMapping()
-            _elem710.read(iprot)
-            self.mappings.append(_elem710)
+          (_etype722, _size719) = iprot.readListBegin()
+          for _i723 in xrange(_size719):
+            _elem724 = WMMapping()
+            _elem724.read(iprot)
+            self.mappings.append(_elem724)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype714, _size711) = iprot.readListBegin()
-          for _i715 in xrange(_size711):
-            _elem716 = WMTrigger()
-            _elem716.read(iprot)
-            self.triggers.append(_elem716)
+          (_etype728, _size725) = iprot.readListBegin()
+          for _i729 in xrange(_size725):
+            _elem730 = WMTrigger()
+            _elem730.read(iprot)
+            self.triggers.append(_elem730)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.poolTriggers = []
-          (_etype720, _size717) = iprot.readListBegin()
-          for _i721 in xrange(_size717):
-            _elem722 = WMPoolTrigger()
-            _elem722.read(iprot)
-            self.poolTriggers.append(_elem722)
+          (_etype734, _size731) = iprot.readListBegin()
+          for _i735 in xrange(_size731):
+            _elem736 = WMPoolTrigger()
+            _elem736.read(iprot)
+            self.poolTriggers.append(_elem736)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16302,29 +16692,29 @@ class WMFullResourcePlan:
     if self.pools is not None:
       oprot.writeFieldBegin('pools', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.pools))
-      for iter723 in self.pools:
-        iter723.write(oprot)
+      for iter737 in self.pools:
+        iter737.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.mappings is not None:
       oprot.writeFieldBegin('mappings', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.mappings))
-      for iter724 in self.mappings:
-        iter724.write(oprot)
+      for iter738 in self.mappings:
+        iter738.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.triggers is not None:
       oprot.writeFieldBegin('triggers', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.triggers))
-      for iter725 in self.triggers:
-        iter725.write(oprot)
+      for iter739 in self.triggers:
+        iter739.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.poolTriggers is not None:
       oprot.writeFieldBegin('poolTriggers', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.poolTriggers))
-      for iter726 in self.poolTriggers:
-        iter726.write(oprot)
+      for iter740 in self.poolTriggers:
+        iter740.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16798,11 +17188,11 @@ class WMGetAllResourcePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.resourcePlans = []
-          (_etype730, _size727) = iprot.readListBegin()
-          for _i731 in xrange(_size727):
-            _elem732 = WMResourcePlan()
-            _elem732.read(iprot)
-            self.resourcePlans.append(_elem732)
+          (_etype744, _size741) = iprot.readListBegin()
+          for _i745 in xrange(_size741):
+            _elem746 = WMResourcePlan()
+            _elem746.read(iprot)
+            self.resourcePlans.append(_elem746)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16819,8 +17209,8 @@ class WMGetAllResourcePlanResponse:
     if self.resourcePlans is not None:
       oprot.writeFieldBegin('resourcePlans', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.resourcePlans))
-      for iter733 in self.resourcePlans:
-        iter733.write(oprot)
+      for iter747 in self.resourcePlans:
+        iter747.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17124,20 +17514,20 @@ class WMValidateResourcePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.errors = []
-          (_etype737, _size734) = iprot.readListBegin()
-          for _i738 in xrange(_size734):
-            _elem739 = iprot.readString()
-            self.errors.append(_elem739)
+          (_etype751, _size748) = iprot.readListBegin()
+          for _i752 in xrange(_size748):
+            _elem753 = iprot.readString()
+            self.errors.append(_elem753)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.warnings = []
-          (_etype743, _size740) = iprot.readListBegin()
-          for _i744 in xrange(_size740):
-            _elem745 = iprot.readString()
-            self.warnings.append(_elem745)
+          (_etype757, _size754) = iprot.readListBegin()
+          for _i758 in xrange(_size754):
+            _elem759 = iprot.readString()
+            self.warnings.append(_elem759)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17154,15 +17544,15 @@ class WMValidateResourcePlanResponse:
     if self.errors is not None:
       oprot.writeFieldBegin('errors', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.errors))
-      for iter746 in self.errors:
-        oprot.writeString(iter746)
+      for iter760 in self.errors:
+        oprot.writeString(iter760)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.warnings is not None:
       oprot.writeFieldBegin('warnings', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.warnings))
-      for iter747 in self.warnings:
-        oprot.writeString(iter747)
+      for iter761 in self.warnings:
+        oprot.writeString(iter761)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17739,11 +18129,11 @@ class WMGetTriggersForResourePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype751, _size748) = iprot.readListBegin()
-          for _i752 in xrange(_size748):
-            _elem753 = WMTrigger()
-            _elem753.read(iprot)
-            self.triggers.append(_elem753)
+          (_etype765, _size762) = iprot.readListBegin()
+          for _i766 in xrange(_size762):
+            _elem767 = WMTrigger()
+            _elem767.read(iprot)
+            self.triggers.append(_elem767)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17760,8 +18150,8 @@ class WMGetTriggersForResourePlanResponse:
     if self.triggers is not None:
       oprot.writeFieldBegin('triggers', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.triggers))
-      for iter754 in self.triggers:
-        iter754.write(oprot)
+      for iter768 in self.triggers:
+        iter768.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 625baae..8438728 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -308,6 +308,36 @@ class SQLNotNullConstraint
   ::Thrift::Struct.generate_accessors self
 end
 
+class SQLDefaultConstraint
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  TABLE_DB = 1
+  TABLE_NAME = 2
+  COLUMN_NAME = 3
+  DEFAULT_VALUE = 4
+  DC_NAME = 5
+  ENABLE_CSTR = 6
+  VALIDATE_CSTR = 7
+  RELY_CSTR = 8
+
+  FIELDS = {
+    TABLE_DB => {:type => ::Thrift::Types::STRING, :name => 'table_db'},
+    TABLE_NAME => {:type => ::Thrift::Types::STRING, :name => 'table_name'},
+    COLUMN_NAME => {:type => ::Thrift::Types::STRING, :name => 'column_name'},
+    DEFAULT_VALUE => {:type => ::Thrift::Types::STRING, :name => 'default_value'},
+    DC_NAME => {:type => ::Thrift::Types::STRING, :name => 'dc_name'},
+    ENABLE_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'enable_cstr'},
+    VALIDATE_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'validate_cstr'},
+    RELY_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'rely_cstr'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class Type
   include ::Thrift::Struct, ::Thrift::Struct_Union
   NAME = 1
@@ -1557,6 +1587,43 @@ class NotNullConstraintsResponse
   ::Thrift::Struct.generate_accessors self
 end
 
+class DefaultConstraintsRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  DB_NAME = 1
+  TBL_NAME = 2
+
+  FIELDS = {
+    DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'},
+    TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'tbl_name'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field db_name is unset!') unless @db_name
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tbl_name is unset!') unless @tbl_name
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class DefaultConstraintsResponse
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  DEFAULTCONSTRAINTS = 1
+
+  FIELDS = {
+    DEFAULTCONSTRAINTS => {:type => ::Thrift::Types::LIST, :name => 'defaultConstraints', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLDefaultConstraint}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field defaultConstraints is unset!') unless @defaultConstraints
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class DropConstraintRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
   DBNAME = 1
@@ -1648,6 +1715,23 @@ class AddNotNullConstraintRequest
   ::Thrift::Struct.generate_accessors self
 end
 
+class AddDefaultConstraintRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  DEFAULTCONSTRAINTCOLS = 1
+
+  FIELDS = {
+    DEFAULTCONSTRAINTCOLS => {:type => ::Thrift::Types::LIST, :name => 'defaultConstraintCols', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLDefaultConstraint}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field defaultConstraintCols is unset!') unless @defaultConstraintCols
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class PartitionsByExprResult
   include ::Thrift::Struct, ::Thrift::Struct_Union
   PARTITIONS = 1


[14/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index f4a9c75..353d7d9 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -40,12 +40,13 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void get_schema_with_environment_context(std::vector<FieldSchema> & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) = 0;
   virtual void create_table(const Table& tbl) = 0;
   virtual void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) = 0;
-  virtual void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints) = 0;
+  virtual void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints, const std::vector<SQLDefaultConstraint> & defaultConstraints) = 0;
   virtual void drop_constraint(const DropConstraintRequest& req) = 0;
   virtual void add_primary_key(const AddPrimaryKeyRequest& req) = 0;
   virtual void add_foreign_key(const AddForeignKeyRequest& req) = 0;
   virtual void add_unique_constraint(const AddUniqueConstraintRequest& req) = 0;
   virtual void add_not_null_constraint(const AddNotNullConstraintRequest& req) = 0;
+  virtual void add_default_constraint(const AddDefaultConstraintRequest& req) = 0;
   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 truncate_table(const std::string& dbName, const std::string& tableName, const std::vector<std::string> & partNames) = 0;
@@ -117,6 +118,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request) = 0;
   virtual void get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request) = 0;
   virtual void get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request) = 0;
+  virtual void get_default_constraints(DefaultConstraintsResponse& _return, const DefaultConstraintsRequest& request) = 0;
   virtual bool update_table_column_statistics(const ColumnStatistics& stats_obj) = 0;
   virtual bool update_partition_column_statistics(const ColumnStatistics& stats_obj) = 0;
   virtual void get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name) = 0;
@@ -291,7 +293,7 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void create_table_with_environment_context(const Table& /* tbl */, const EnvironmentContext& /* environment_context */) {
     return;
   }
-  void create_table_with_constraints(const Table& /* tbl */, const std::vector<SQLPrimaryKey> & /* primaryKeys */, const std::vector<SQLForeignKey> & /* foreignKeys */, const std::vector<SQLUniqueConstraint> & /* uniqueConstraints */, const std::vector<SQLNotNullConstraint> & /* notNullConstraints */) {
+  void create_table_with_constraints(const Table& /* tbl */, const std::vector<SQLPrimaryKey> & /* primaryKeys */, const std::vector<SQLForeignKey> & /* foreignKeys */, const std::vector<SQLUniqueConstraint> & /* uniqueConstraints */, const std::vector<SQLNotNullConstraint> & /* notNullConstraints */, const std::vector<SQLDefaultConstraint> & /* defaultConstraints */) {
     return;
   }
   void drop_constraint(const DropConstraintRequest& /* req */) {
@@ -309,6 +311,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void add_not_null_constraint(const AddNotNullConstraintRequest& /* req */) {
     return;
   }
+  void add_default_constraint(const AddDefaultConstraintRequest& /* req */) {
+    return;
+  }
   void drop_table(const std::string& /* dbname */, const std::string& /* name */, const bool /* deleteData */) {
     return;
   }
@@ -532,6 +537,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void get_not_null_constraints(NotNullConstraintsResponse& /* _return */, const NotNullConstraintsRequest& /* request */) {
     return;
   }
+  void get_default_constraints(DefaultConstraintsResponse& /* _return */, const DefaultConstraintsRequest& /* request */) {
+    return;
+  }
   bool update_table_column_statistics(const ColumnStatistics& /* stats_obj */) {
     bool _return = false;
     return _return;
@@ -3043,12 +3051,13 @@ class ThriftHiveMetastore_create_table_with_environment_context_presult {
 };
 
 typedef struct _ThriftHiveMetastore_create_table_with_constraints_args__isset {
-  _ThriftHiveMetastore_create_table_with_constraints_args__isset() : tbl(false), primaryKeys(false), foreignKeys(false), uniqueConstraints(false), notNullConstraints(false) {}
+  _ThriftHiveMetastore_create_table_with_constraints_args__isset() : tbl(false), primaryKeys(false), foreignKeys(false), uniqueConstraints(false), notNullConstraints(false), defaultConstraints(false) {}
   bool tbl :1;
   bool primaryKeys :1;
   bool foreignKeys :1;
   bool uniqueConstraints :1;
   bool notNullConstraints :1;
+  bool defaultConstraints :1;
 } _ThriftHiveMetastore_create_table_with_constraints_args__isset;
 
 class ThriftHiveMetastore_create_table_with_constraints_args {
@@ -3065,6 +3074,7 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
   std::vector<SQLForeignKey>  foreignKeys;
   std::vector<SQLUniqueConstraint>  uniqueConstraints;
   std::vector<SQLNotNullConstraint>  notNullConstraints;
+  std::vector<SQLDefaultConstraint>  defaultConstraints;
 
   _ThriftHiveMetastore_create_table_with_constraints_args__isset __isset;
 
@@ -3078,6 +3088,8 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
 
   void __set_notNullConstraints(const std::vector<SQLNotNullConstraint> & val);
 
+  void __set_defaultConstraints(const std::vector<SQLDefaultConstraint> & val);
+
   bool operator == (const ThriftHiveMetastore_create_table_with_constraints_args & rhs) const
   {
     if (!(tbl == rhs.tbl))
@@ -3090,6 +3102,8 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       return false;
     if (!(notNullConstraints == rhs.notNullConstraints))
       return false;
+    if (!(defaultConstraints == rhs.defaultConstraints))
+      return false;
     return true;
   }
   bool operator != (const ThriftHiveMetastore_create_table_with_constraints_args &rhs) const {
@@ -3114,6 +3128,7 @@ class ThriftHiveMetastore_create_table_with_constraints_pargs {
   const std::vector<SQLForeignKey> * foreignKeys;
   const std::vector<SQLUniqueConstraint> * uniqueConstraints;
   const std::vector<SQLNotNullConstraint> * notNullConstraints;
+  const std::vector<SQLDefaultConstraint> * defaultConstraints;
 
   uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
 
@@ -3758,6 +3773,118 @@ class ThriftHiveMetastore_add_not_null_constraint_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_add_default_constraint_args__isset {
+  _ThriftHiveMetastore_add_default_constraint_args__isset() : req(false) {}
+  bool req :1;
+} _ThriftHiveMetastore_add_default_constraint_args__isset;
+
+class ThriftHiveMetastore_add_default_constraint_args {
+ public:
+
+  ThriftHiveMetastore_add_default_constraint_args(const ThriftHiveMetastore_add_default_constraint_args&);
+  ThriftHiveMetastore_add_default_constraint_args& operator=(const ThriftHiveMetastore_add_default_constraint_args&);
+  ThriftHiveMetastore_add_default_constraint_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_add_default_constraint_args() throw();
+  AddDefaultConstraintRequest req;
+
+  _ThriftHiveMetastore_add_default_constraint_args__isset __isset;
+
+  void __set_req(const AddDefaultConstraintRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_add_default_constraint_args & rhs) const
+  {
+    if (!(req == rhs.req))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_default_constraint_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_default_constraint_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_add_default_constraint_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_default_constraint_pargs() throw();
+  const AddDefaultConstraintRequest* req;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_default_constraint_result__isset {
+  _ThriftHiveMetastore_add_default_constraint_result__isset() : o1(false), o2(false) {}
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_add_default_constraint_result__isset;
+
+class ThriftHiveMetastore_add_default_constraint_result {
+ public:
+
+  ThriftHiveMetastore_add_default_constraint_result(const ThriftHiveMetastore_add_default_constraint_result&);
+  ThriftHiveMetastore_add_default_constraint_result& operator=(const ThriftHiveMetastore_add_default_constraint_result&);
+  ThriftHiveMetastore_add_default_constraint_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_add_default_constraint_result() throw();
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_add_default_constraint_result__isset __isset;
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_add_default_constraint_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_default_constraint_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_default_constraint_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_default_constraint_presult__isset {
+  _ThriftHiveMetastore_add_default_constraint_presult__isset() : o1(false), o2(false) {}
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_add_default_constraint_presult__isset;
+
+class ThriftHiveMetastore_add_default_constraint_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_default_constraint_presult() throw();
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_add_default_constraint_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_drop_table_args__isset {
   _ThriftHiveMetastore_drop_table_args__isset() : dbname(false), name(false), deleteData(false) {}
   bool dbname :1;
@@ -13201,6 +13328,126 @@ class ThriftHiveMetastore_get_not_null_constraints_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_get_default_constraints_args__isset {
+  _ThriftHiveMetastore_get_default_constraints_args__isset() : request(false) {}
+  bool request :1;
+} _ThriftHiveMetastore_get_default_constraints_args__isset;
+
+class ThriftHiveMetastore_get_default_constraints_args {
+ public:
+
+  ThriftHiveMetastore_get_default_constraints_args(const ThriftHiveMetastore_get_default_constraints_args&);
+  ThriftHiveMetastore_get_default_constraints_args& operator=(const ThriftHiveMetastore_get_default_constraints_args&);
+  ThriftHiveMetastore_get_default_constraints_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_default_constraints_args() throw();
+  DefaultConstraintsRequest request;
+
+  _ThriftHiveMetastore_get_default_constraints_args__isset __isset;
+
+  void __set_request(const DefaultConstraintsRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_get_default_constraints_args & rhs) const
+  {
+    if (!(request == rhs.request))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_default_constraints_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_default_constraints_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_default_constraints_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_default_constraints_pargs() throw();
+  const DefaultConstraintsRequest* request;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_default_constraints_result__isset {
+  _ThriftHiveMetastore_get_default_constraints_result__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_default_constraints_result__isset;
+
+class ThriftHiveMetastore_get_default_constraints_result {
+ public:
+
+  ThriftHiveMetastore_get_default_constraints_result(const ThriftHiveMetastore_get_default_constraints_result&);
+  ThriftHiveMetastore_get_default_constraints_result& operator=(const ThriftHiveMetastore_get_default_constraints_result&);
+  ThriftHiveMetastore_get_default_constraints_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_default_constraints_result() throw();
+  DefaultConstraintsResponse success;
+  MetaException o1;
+  NoSuchObjectException o2;
+
+  _ThriftHiveMetastore_get_default_constraints_result__isset __isset;
+
+  void __set_success(const DefaultConstraintsResponse& val);
+
+  void __set_o1(const MetaException& val);
+
+  void __set_o2(const NoSuchObjectException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_default_constraints_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_default_constraints_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_default_constraints_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_default_constraints_presult__isset {
+  _ThriftHiveMetastore_get_default_constraints_presult__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_default_constraints_presult__isset;
+
+class ThriftHiveMetastore_get_default_constraints_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_default_constraints_presult() throw();
+  DefaultConstraintsResponse* success;
+  MetaException o1;
+  NoSuchObjectException o2;
+
+  _ThriftHiveMetastore_get_default_constraints_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_update_table_column_statistics_args__isset {
   _ThriftHiveMetastore_update_table_column_statistics_args__isset() : stats_obj(false) {}
   bool stats_obj :1;
@@ -23619,8 +23866,8 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context);
   void send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context);
   void recv_create_table_with_environment_context();
-  void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints);
-  void send_create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints);
+  void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints, const std::vector<SQLDefaultConstraint> & defaultConstraints);
+  void send_create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints, const std::vector<SQLDefaultConstraint> & defaultConstraints);
   void recv_create_table_with_constraints();
   void drop_constraint(const DropConstraintRequest& req);
   void send_drop_constraint(const DropConstraintRequest& req);
@@ -23637,6 +23884,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void add_not_null_constraint(const AddNotNullConstraintRequest& req);
   void send_add_not_null_constraint(const AddNotNullConstraintRequest& req);
   void recv_add_not_null_constraint();
+  void add_default_constraint(const AddDefaultConstraintRequest& req);
+  void send_add_default_constraint(const AddDefaultConstraintRequest& req);
+  void recv_add_default_constraint();
   void drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
   void send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
   void recv_drop_table();
@@ -23850,6 +24100,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request);
   void send_get_not_null_constraints(const NotNullConstraintsRequest& request);
   void recv_get_not_null_constraints(NotNullConstraintsResponse& _return);
+  void get_default_constraints(DefaultConstraintsResponse& _return, const DefaultConstraintsRequest& request);
+  void send_get_default_constraints(const DefaultConstraintsRequest& request);
+  void recv_get_default_constraints(DefaultConstraintsResponse& _return);
   bool update_table_column_statistics(const ColumnStatistics& stats_obj);
   void send_update_table_column_statistics(const ColumnStatistics& stats_obj);
   bool recv_update_table_column_statistics();
@@ -24151,6 +24404,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_add_foreign_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_add_unique_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_add_not_null_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_add_default_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   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_truncate_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -24222,6 +24476,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_get_foreign_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_unique_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_not_null_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_get_default_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_update_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_update_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -24339,6 +24594,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["add_foreign_key"] = &ThriftHiveMetastoreProcessor::process_add_foreign_key;
     processMap_["add_unique_constraint"] = &ThriftHiveMetastoreProcessor::process_add_unique_constraint;
     processMap_["add_not_null_constraint"] = &ThriftHiveMetastoreProcessor::process_add_not_null_constraint;
+    processMap_["add_default_constraint"] = &ThriftHiveMetastoreProcessor::process_add_default_constraint;
     processMap_["drop_table"] = &ThriftHiveMetastoreProcessor::process_drop_table;
     processMap_["drop_table_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_drop_table_with_environment_context;
     processMap_["truncate_table"] = &ThriftHiveMetastoreProcessor::process_truncate_table;
@@ -24410,6 +24666,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["get_foreign_keys"] = &ThriftHiveMetastoreProcessor::process_get_foreign_keys;
     processMap_["get_unique_constraints"] = &ThriftHiveMetastoreProcessor::process_get_unique_constraints;
     processMap_["get_not_null_constraints"] = &ThriftHiveMetastoreProcessor::process_get_not_null_constraints;
+    processMap_["get_default_constraints"] = &ThriftHiveMetastoreProcessor::process_get_default_constraints;
     processMap_["update_table_column_statistics"] = &ThriftHiveMetastoreProcessor::process_update_table_column_statistics;
     processMap_["update_partition_column_statistics"] = &ThriftHiveMetastoreProcessor::process_update_partition_column_statistics;
     processMap_["get_table_column_statistics"] = &ThriftHiveMetastoreProcessor::process_get_table_column_statistics;
@@ -24704,13 +24961,13 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     ifaces_[i]->create_table_with_environment_context(tbl, environment_context);
   }
 
-  void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints) {
+  void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints, const std::vector<SQLDefaultConstraint> & defaultConstraints) {
     size_t sz = ifaces_.size();
     size_t i = 0;
     for (; i < (sz - 1); ++i) {
-      ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints);
+      ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints);
     }
-    ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints);
+    ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints);
   }
 
   void drop_constraint(const DropConstraintRequest& req) {
@@ -24758,6 +25015,15 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     ifaces_[i]->add_not_null_constraint(req);
   }
 
+  void add_default_constraint(const AddDefaultConstraintRequest& req) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->add_default_constraint(req);
+    }
+    ifaces_[i]->add_default_constraint(req);
+  }
+
   void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -25444,6 +25710,16 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     return;
   }
 
+  void get_default_constraints(DefaultConstraintsResponse& _return, const DefaultConstraintsRequest& request) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_default_constraints(_return, request);
+    }
+    ifaces_[i]->get_default_constraints(_return, request);
+    return;
+  }
+
   bool update_table_column_statistics(const ColumnStatistics& stats_obj) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -26375,8 +26651,8 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context);
   int32_t send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context);
   void recv_create_table_with_environment_context(const int32_t seqid);
-  void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints);
-  int32_t send_create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints);
+  void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints, const std::vector<SQLDefaultConstraint> & defaultConstraints);
+  int32_t send_create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints, const std::vector<SQLDefaultConstraint> & defaultConstraints);
   void recv_create_table_with_constraints(const int32_t seqid);
   void drop_constraint(const DropConstraintRequest& req);
   int32_t send_drop_constraint(const DropConstraintRequest& req);
@@ -26393,6 +26669,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void add_not_null_constraint(const AddNotNullConstraintRequest& req);
   int32_t send_add_not_null_constraint(const AddNotNullConstraintRequest& req);
   void recv_add_not_null_constraint(const int32_t seqid);
+  void add_default_constraint(const AddDefaultConstraintRequest& req);
+  int32_t send_add_default_constraint(const AddDefaultConstraintRequest& req);
+  void recv_add_default_constraint(const int32_t seqid);
   void drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
   int32_t send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
   void recv_drop_table(const int32_t seqid);
@@ -26606,6 +26885,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request);
   int32_t send_get_not_null_constraints(const NotNullConstraintsRequest& request);
   void recv_get_not_null_constraints(NotNullConstraintsResponse& _return, const int32_t seqid);
+  void get_default_constraints(DefaultConstraintsResponse& _return, const DefaultConstraintsRequest& request);
+  int32_t send_get_default_constraints(const DefaultConstraintsRequest& request);
+  void recv_get_default_constraints(DefaultConstraintsResponse& _return, const int32_t seqid);
   bool update_table_column_statistics(const ColumnStatistics& stats_obj);
   int32_t send_update_table_column_statistics(const ColumnStatistics& stats_obj);
   bool recv_update_table_column_statistics(const int32_t seqid);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index b7326f0..16536aa 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -112,7 +112,7 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("create_table_with_environment_context\n");
   }
 
-  void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints) {
+  void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints, const std::vector<SQLDefaultConstraint> & defaultConstraints) {
     // Your implementation goes here
     printf("create_table_with_constraints\n");
   }
@@ -142,6 +142,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("add_not_null_constraint\n");
   }
 
+  void add_default_constraint(const AddDefaultConstraintRequest& req) {
+    // Your implementation goes here
+    printf("add_default_constraint\n");
+  }
+
   void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) {
     // Your implementation goes here
     printf("drop_table\n");
@@ -497,6 +502,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("get_not_null_constraints\n");
   }
 
+  void get_default_constraints(DefaultConstraintsResponse& _return, const DefaultConstraintsRequest& request) {
+    // Your implementation goes here
+    printf("get_default_constraints\n");
+  }
+
   bool update_table_column_statistics(const ColumnStatistics& stats_obj) {
     // Your implementation goes here
     printf("update_table_column_statistics\n");


[15/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 921cba1..1510d4f 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -1240,14 +1240,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1091;
-            ::apache::thrift::protocol::TType _etype1094;
-            xfer += iprot->readListBegin(_etype1094, _size1091);
-            this->success.resize(_size1091);
-            uint32_t _i1095;
-            for (_i1095 = 0; _i1095 < _size1091; ++_i1095)
+            uint32_t _size1111;
+            ::apache::thrift::protocol::TType _etype1114;
+            xfer += iprot->readListBegin(_etype1114, _size1111);
+            this->success.resize(_size1111);
+            uint32_t _i1115;
+            for (_i1115 = 0; _i1115 < _size1111; ++_i1115)
             {
-              xfer += iprot->readString(this->success[_i1095]);
+              xfer += iprot->readString(this->success[_i1115]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1286,10 +1286,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1096;
-      for (_iter1096 = this->success.begin(); _iter1096 != this->success.end(); ++_iter1096)
+      std::vector<std::string> ::const_iterator _iter1116;
+      for (_iter1116 = this->success.begin(); _iter1116 != this->success.end(); ++_iter1116)
       {
-        xfer += oprot->writeString((*_iter1096));
+        xfer += oprot->writeString((*_iter1116));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1334,14 +1334,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1097;
-            ::apache::thrift::protocol::TType _etype1100;
-            xfer += iprot->readListBegin(_etype1100, _size1097);
-            (*(this->success)).resize(_size1097);
-            uint32_t _i1101;
-            for (_i1101 = 0; _i1101 < _size1097; ++_i1101)
+            uint32_t _size1117;
+            ::apache::thrift::protocol::TType _etype1120;
+            xfer += iprot->readListBegin(_etype1120, _size1117);
+            (*(this->success)).resize(_size1117);
+            uint32_t _i1121;
+            for (_i1121 = 0; _i1121 < _size1117; ++_i1121)
             {
-              xfer += iprot->readString((*(this->success))[_i1101]);
+              xfer += iprot->readString((*(this->success))[_i1121]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1458,14 +1458,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1102;
-            ::apache::thrift::protocol::TType _etype1105;
-            xfer += iprot->readListBegin(_etype1105, _size1102);
-            this->success.resize(_size1102);
-            uint32_t _i1106;
-            for (_i1106 = 0; _i1106 < _size1102; ++_i1106)
+            uint32_t _size1122;
+            ::apache::thrift::protocol::TType _etype1125;
+            xfer += iprot->readListBegin(_etype1125, _size1122);
+            this->success.resize(_size1122);
+            uint32_t _i1126;
+            for (_i1126 = 0; _i1126 < _size1122; ++_i1126)
             {
-              xfer += iprot->readString(this->success[_i1106]);
+              xfer += iprot->readString(this->success[_i1126]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1504,10 +1504,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1107;
-      for (_iter1107 = this->success.begin(); _iter1107 != this->success.end(); ++_iter1107)
+      std::vector<std::string> ::const_iterator _iter1127;
+      for (_iter1127 = this->success.begin(); _iter1127 != this->success.end(); ++_iter1127)
       {
-        xfer += oprot->writeString((*_iter1107));
+        xfer += oprot->writeString((*_iter1127));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1552,14 +1552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1108;
-            ::apache::thrift::protocol::TType _etype1111;
-            xfer += iprot->readListBegin(_etype1111, _size1108);
-            (*(this->success)).resize(_size1108);
-            uint32_t _i1112;
-            for (_i1112 = 0; _i1112 < _size1108; ++_i1112)
+            uint32_t _size1128;
+            ::apache::thrift::protocol::TType _etype1131;
+            xfer += iprot->readListBegin(_etype1131, _size1128);
+            (*(this->success)).resize(_size1128);
+            uint32_t _i1132;
+            for (_i1132 = 0; _i1132 < _size1128; ++_i1132)
             {
-              xfer += iprot->readString((*(this->success))[_i1112]);
+              xfer += iprot->readString((*(this->success))[_i1132]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2621,17 +2621,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1113;
-            ::apache::thrift::protocol::TType _ktype1114;
-            ::apache::thrift::protocol::TType _vtype1115;
-            xfer += iprot->readMapBegin(_ktype1114, _vtype1115, _size1113);
-            uint32_t _i1117;
-            for (_i1117 = 0; _i1117 < _size1113; ++_i1117)
+            uint32_t _size1133;
+            ::apache::thrift::protocol::TType _ktype1134;
+            ::apache::thrift::protocol::TType _vtype1135;
+            xfer += iprot->readMapBegin(_ktype1134, _vtype1135, _size1133);
+            uint32_t _i1137;
+            for (_i1137 = 0; _i1137 < _size1133; ++_i1137)
             {
-              std::string _key1118;
-              xfer += iprot->readString(_key1118);
-              Type& _val1119 = this->success[_key1118];
-              xfer += _val1119.read(iprot);
+              std::string _key1138;
+              xfer += iprot->readString(_key1138);
+              Type& _val1139 = this->success[_key1138];
+              xfer += _val1139.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2670,11 +2670,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter1120;
-      for (_iter1120 = this->success.begin(); _iter1120 != this->success.end(); ++_iter1120)
+      std::map<std::string, Type> ::const_iterator _iter1140;
+      for (_iter1140 = this->success.begin(); _iter1140 != this->success.end(); ++_iter1140)
       {
-        xfer += oprot->writeString(_iter1120->first);
-        xfer += _iter1120->second.write(oprot);
+        xfer += oprot->writeString(_iter1140->first);
+        xfer += _iter1140->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -2719,17 +2719,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1121;
-            ::apache::thrift::protocol::TType _ktype1122;
-            ::apache::thrift::protocol::TType _vtype1123;
-            xfer += iprot->readMapBegin(_ktype1122, _vtype1123, _size1121);
-            uint32_t _i1125;
-            for (_i1125 = 0; _i1125 < _size1121; ++_i1125)
+            uint32_t _size1141;
+            ::apache::thrift::protocol::TType _ktype1142;
+            ::apache::thrift::protocol::TType _vtype1143;
+            xfer += iprot->readMapBegin(_ktype1142, _vtype1143, _size1141);
+            uint32_t _i1145;
+            for (_i1145 = 0; _i1145 < _size1141; ++_i1145)
             {
-              std::string _key1126;
-              xfer += iprot->readString(_key1126);
-              Type& _val1127 = (*(this->success))[_key1126];
-              xfer += _val1127.read(iprot);
+              std::string _key1146;
+              xfer += iprot->readString(_key1146);
+              Type& _val1147 = (*(this->success))[_key1146];
+              xfer += _val1147.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2883,14 +2883,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1128;
-            ::apache::thrift::protocol::TType _etype1131;
-            xfer += iprot->readListBegin(_etype1131, _size1128);
-            this->success.resize(_size1128);
-            uint32_t _i1132;
-            for (_i1132 = 0; _i1132 < _size1128; ++_i1132)
+            uint32_t _size1148;
+            ::apache::thrift::protocol::TType _etype1151;
+            xfer += iprot->readListBegin(_etype1151, _size1148);
+            this->success.resize(_size1148);
+            uint32_t _i1152;
+            for (_i1152 = 0; _i1152 < _size1148; ++_i1152)
             {
-              xfer += this->success[_i1132].read(iprot);
+              xfer += this->success[_i1152].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2945,10 +2945,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1133;
-      for (_iter1133 = this->success.begin(); _iter1133 != this->success.end(); ++_iter1133)
+      std::vector<FieldSchema> ::const_iterator _iter1153;
+      for (_iter1153 = this->success.begin(); _iter1153 != this->success.end(); ++_iter1153)
       {
-        xfer += (*_iter1133).write(oprot);
+        xfer += (*_iter1153).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3001,14 +3001,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1134;
-            ::apache::thrift::protocol::TType _etype1137;
-            xfer += iprot->readListBegin(_etype1137, _size1134);
-            (*(this->success)).resize(_size1134);
-            uint32_t _i1138;
-            for (_i1138 = 0; _i1138 < _size1134; ++_i1138)
+            uint32_t _size1154;
+            ::apache::thrift::protocol::TType _etype1157;
+            xfer += iprot->readListBegin(_etype1157, _size1154);
+            (*(this->success)).resize(_size1154);
+            uint32_t _i1158;
+            for (_i1158 = 0; _i1158 < _size1154; ++_i1158)
             {
-              xfer += (*(this->success))[_i1138].read(iprot);
+              xfer += (*(this->success))[_i1158].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3194,14 +3194,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1139;
-            ::apache::thrift::protocol::TType _etype1142;
-            xfer += iprot->readListBegin(_etype1142, _size1139);
-            this->success.resize(_size1139);
-            uint32_t _i1143;
-            for (_i1143 = 0; _i1143 < _size1139; ++_i1143)
+            uint32_t _size1159;
+            ::apache::thrift::protocol::TType _etype1162;
+            xfer += iprot->readListBegin(_etype1162, _size1159);
+            this->success.resize(_size1159);
+            uint32_t _i1163;
+            for (_i1163 = 0; _i1163 < _size1159; ++_i1163)
             {
-              xfer += this->success[_i1143].read(iprot);
+              xfer += this->success[_i1163].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3256,10 +3256,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1144;
-      for (_iter1144 = this->success.begin(); _iter1144 != this->success.end(); ++_iter1144)
+      std::vector<FieldSchema> ::const_iterator _iter1164;
+      for (_iter1164 = this->success.begin(); _iter1164 != this->success.end(); ++_iter1164)
       {
-        xfer += (*_iter1144).write(oprot);
+        xfer += (*_iter1164).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3312,14 +3312,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1145;
-            ::apache::thrift::protocol::TType _etype1148;
-            xfer += iprot->readListBegin(_etype1148, _size1145);
-            (*(this->success)).resize(_size1145);
-            uint32_t _i1149;
-            for (_i1149 = 0; _i1149 < _size1145; ++_i1149)
+            uint32_t _size1165;
+            ::apache::thrift::protocol::TType _etype1168;
+            xfer += iprot->readListBegin(_etype1168, _size1165);
+            (*(this->success)).resize(_size1165);
+            uint32_t _i1169;
+            for (_i1169 = 0; _i1169 < _size1165; ++_i1169)
             {
-              xfer += (*(this->success))[_i1149].read(iprot);
+              xfer += (*(this->success))[_i1169].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3489,14 +3489,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1150;
-            ::apache::thrift::protocol::TType _etype1153;
-            xfer += iprot->readListBegin(_etype1153, _size1150);
-            this->success.resize(_size1150);
-            uint32_t _i1154;
-            for (_i1154 = 0; _i1154 < _size1150; ++_i1154)
+            uint32_t _size1170;
+            ::apache::thrift::protocol::TType _etype1173;
+            xfer += iprot->readListBegin(_etype1173, _size1170);
+            this->success.resize(_size1170);
+            uint32_t _i1174;
+            for (_i1174 = 0; _i1174 < _size1170; ++_i1174)
             {
-              xfer += this->success[_i1154].read(iprot);
+              xfer += this->success[_i1174].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3551,10 +3551,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1155;
-      for (_iter1155 = this->success.begin(); _iter1155 != this->success.end(); ++_iter1155)
+      std::vector<FieldSchema> ::const_iterator _iter1175;
+      for (_iter1175 = this->success.begin(); _iter1175 != this->success.end(); ++_iter1175)
       {
-        xfer += (*_iter1155).write(oprot);
+        xfer += (*_iter1175).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3607,14 +3607,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1156;
-            ::apache::thrift::protocol::TType _etype1159;
-            xfer += iprot->readListBegin(_etype1159, _size1156);
-            (*(this->success)).resize(_size1156);
-            uint32_t _i1160;
-            for (_i1160 = 0; _i1160 < _size1156; ++_i1160)
+            uint32_t _size1176;
+            ::apache::thrift::protocol::TType _etype1179;
+            xfer += iprot->readListBegin(_etype1179, _size1176);
+            (*(this->success)).resize(_size1176);
+            uint32_t _i1180;
+            for (_i1180 = 0; _i1180 < _size1176; ++_i1180)
             {
-              xfer += (*(this->success))[_i1160].read(iprot);
+              xfer += (*(this->success))[_i1180].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3800,14 +3800,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1161;
-            ::apache::thrift::protocol::TType _etype1164;
-            xfer += iprot->readListBegin(_etype1164, _size1161);
-            this->success.resize(_size1161);
-            uint32_t _i1165;
-            for (_i1165 = 0; _i1165 < _size1161; ++_i1165)
+            uint32_t _size1181;
+            ::apache::thrift::protocol::TType _etype1184;
+            xfer += iprot->readListBegin(_etype1184, _size1181);
+            this->success.resize(_size1181);
+            uint32_t _i1185;
+            for (_i1185 = 0; _i1185 < _size1181; ++_i1185)
             {
-              xfer += this->success[_i1165].read(iprot);
+              xfer += this->success[_i1185].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3862,10 +3862,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1166;
-      for (_iter1166 = this->success.begin(); _iter1166 != this->success.end(); ++_iter1166)
+      std::vector<FieldSchema> ::const_iterator _iter1186;
+      for (_iter1186 = this->success.begin(); _iter1186 != this->success.end(); ++_iter1186)
       {
-        xfer += (*_iter1166).write(oprot);
+        xfer += (*_iter1186).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3918,14 +3918,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1167;
-            ::apache::thrift::protocol::TType _etype1170;
-            xfer += iprot->readListBegin(_etype1170, _size1167);
-            (*(this->success)).resize(_size1167);
-            uint32_t _i1171;
-            for (_i1171 = 0; _i1171 < _size1167; ++_i1171)
+            uint32_t _size1187;
+            ::apache::thrift::protocol::TType _etype1190;
+            xfer += iprot->readListBegin(_etype1190, _size1187);
+            (*(this->success)).resize(_size1187);
+            uint32_t _i1191;
+            for (_i1191 = 0; _i1191 < _size1187; ++_i1191)
             {
-              xfer += (*(this->success))[_i1171].read(iprot);
+              xfer += (*(this->success))[_i1191].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4518,14 +4518,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size1172;
-            ::apache::thrift::protocol::TType _etype1175;
-            xfer += iprot->readListBegin(_etype1175, _size1172);
-            this->primaryKeys.resize(_size1172);
-            uint32_t _i1176;
-            for (_i1176 = 0; _i1176 < _size1172; ++_i1176)
+            uint32_t _size1192;
+            ::apache::thrift::protocol::TType _etype1195;
+            xfer += iprot->readListBegin(_etype1195, _size1192);
+            this->primaryKeys.resize(_size1192);
+            uint32_t _i1196;
+            for (_i1196 = 0; _i1196 < _size1192; ++_i1196)
             {
-              xfer += this->primaryKeys[_i1176].read(iprot);
+              xfer += this->primaryKeys[_i1196].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4538,14 +4538,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size1177;
-            ::apache::thrift::protocol::TType _etype1180;
-            xfer += iprot->readListBegin(_etype1180, _size1177);
-            this->foreignKeys.resize(_size1177);
-            uint32_t _i1181;
-            for (_i1181 = 0; _i1181 < _size1177; ++_i1181)
+            uint32_t _size1197;
+            ::apache::thrift::protocol::TType _etype1200;
+            xfer += iprot->readListBegin(_etype1200, _size1197);
+            this->foreignKeys.resize(_size1197);
+            uint32_t _i1201;
+            for (_i1201 = 0; _i1201 < _size1197; ++_i1201)
             {
-              xfer += this->foreignKeys[_i1181].read(iprot);
+              xfer += this->foreignKeys[_i1201].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4558,14 +4558,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size1182;
-            ::apache::thrift::protocol::TType _etype1185;
-            xfer += iprot->readListBegin(_etype1185, _size1182);
-            this->uniqueConstraints.resize(_size1182);
-            uint32_t _i1186;
-            for (_i1186 = 0; _i1186 < _size1182; ++_i1186)
+            uint32_t _size1202;
+            ::apache::thrift::protocol::TType _etype1205;
+            xfer += iprot->readListBegin(_etype1205, _size1202);
+            this->uniqueConstraints.resize(_size1202);
+            uint32_t _i1206;
+            for (_i1206 = 0; _i1206 < _size1202; ++_i1206)
             {
-              xfer += this->uniqueConstraints[_i1186].read(iprot);
+              xfer += this->uniqueConstraints[_i1206].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4578,14 +4578,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraints.clear();
-            uint32_t _size1187;
-            ::apache::thrift::protocol::TType _etype1190;
-            xfer += iprot->readListBegin(_etype1190, _size1187);
-            this->notNullConstraints.resize(_size1187);
-            uint32_t _i1191;
-            for (_i1191 = 0; _i1191 < _size1187; ++_i1191)
+            uint32_t _size1207;
+            ::apache::thrift::protocol::TType _etype1210;
+            xfer += iprot->readListBegin(_etype1210, _size1207);
+            this->notNullConstraints.resize(_size1207);
+            uint32_t _i1211;
+            for (_i1211 = 0; _i1211 < _size1207; ++_i1211)
             {
-              xfer += this->notNullConstraints[_i1191].read(iprot);
+              xfer += this->notNullConstraints[_i1211].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4594,6 +4594,26 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
           xfer += iprot->skip(ftype);
         }
         break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->defaultConstraints.clear();
+            uint32_t _size1212;
+            ::apache::thrift::protocol::TType _etype1215;
+            xfer += iprot->readListBegin(_etype1215, _size1212);
+            this->defaultConstraints.resize(_size1212);
+            uint32_t _i1216;
+            for (_i1216 = 0; _i1216 < _size1212; ++_i1216)
+            {
+              xfer += this->defaultConstraints[_i1216].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.defaultConstraints = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -4618,10 +4638,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter1192;
-    for (_iter1192 = this->primaryKeys.begin(); _iter1192 != this->primaryKeys.end(); ++_iter1192)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1217;
+    for (_iter1217 = this->primaryKeys.begin(); _iter1217 != this->primaryKeys.end(); ++_iter1217)
     {
-      xfer += (*_iter1192).write(oprot);
+      xfer += (*_iter1217).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4630,10 +4650,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter1193;
-    for (_iter1193 = this->foreignKeys.begin(); _iter1193 != this->foreignKeys.end(); ++_iter1193)
+    std::vector<SQLForeignKey> ::const_iterator _iter1218;
+    for (_iter1218 = this->foreignKeys.begin(); _iter1218 != this->foreignKeys.end(); ++_iter1218)
     {
-      xfer += (*_iter1193).write(oprot);
+      xfer += (*_iter1218).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4642,10 +4662,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->uniqueConstraints.size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1194;
-    for (_iter1194 = this->uniqueConstraints.begin(); _iter1194 != this->uniqueConstraints.end(); ++_iter1194)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1219;
+    for (_iter1219 = this->uniqueConstraints.begin(); _iter1219 != this->uniqueConstraints.end(); ++_iter1219)
     {
-      xfer += (*_iter1194).write(oprot);
+      xfer += (*_iter1219).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4654,10 +4674,22 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->notNullConstraints.size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1195;
-    for (_iter1195 = this->notNullConstraints.begin(); _iter1195 != this->notNullConstraints.end(); ++_iter1195)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1220;
+    for (_iter1220 = this->notNullConstraints.begin(); _iter1220 != this->notNullConstraints.end(); ++_iter1220)
+    {
+      xfer += (*_iter1220).write(oprot);
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->defaultConstraints.size()));
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1221;
+    for (_iter1221 = this->defaultConstraints.begin(); _iter1221 != this->defaultConstraints.end(); ++_iter1221)
     {
-      xfer += (*_iter1195).write(oprot);
+      xfer += (*_iter1221).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4685,10 +4717,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->primaryKeys)).size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter1196;
-    for (_iter1196 = (*(this->primaryKeys)).begin(); _iter1196 != (*(this->primaryKeys)).end(); ++_iter1196)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1222;
+    for (_iter1222 = (*(this->primaryKeys)).begin(); _iter1222 != (*(this->primaryKeys)).end(); ++_iter1222)
     {
-      xfer += (*_iter1196).write(oprot);
+      xfer += (*_iter1222).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4697,10 +4729,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->foreignKeys)).size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter1197;
-    for (_iter1197 = (*(this->foreignKeys)).begin(); _iter1197 != (*(this->foreignKeys)).end(); ++_iter1197)
+    std::vector<SQLForeignKey> ::const_iterator _iter1223;
+    for (_iter1223 = (*(this->foreignKeys)).begin(); _iter1223 != (*(this->foreignKeys)).end(); ++_iter1223)
     {
-      xfer += (*_iter1197).write(oprot);
+      xfer += (*_iter1223).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4709,10 +4741,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->uniqueConstraints)).size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1198;
-    for (_iter1198 = (*(this->uniqueConstraints)).begin(); _iter1198 != (*(this->uniqueConstraints)).end(); ++_iter1198)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1224;
+    for (_iter1224 = (*(this->uniqueConstraints)).begin(); _iter1224 != (*(this->uniqueConstraints)).end(); ++_iter1224)
     {
-      xfer += (*_iter1198).write(oprot);
+      xfer += (*_iter1224).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4721,10 +4753,22 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->notNullConstraints)).size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1199;
-    for (_iter1199 = (*(this->notNullConstraints)).begin(); _iter1199 != (*(this->notNullConstraints)).end(); ++_iter1199)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1225;
+    for (_iter1225 = (*(this->notNullConstraints)).begin(); _iter1225 != (*(this->notNullConstraints)).end(); ++_iter1225)
+    {
+      xfer += (*_iter1225).write(oprot);
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->defaultConstraints)).size()));
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1226;
+    for (_iter1226 = (*(this->defaultConstraints)).begin(); _iter1226 != (*(this->defaultConstraints)).end(); ++_iter1226)
     {
-      xfer += (*_iter1199).write(oprot);
+      xfer += (*_iter1226).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5939,6 +5983,213 @@ uint32_t ThriftHiveMetastore_add_not_null_constraint_presult::read(::apache::thr
 }
 
 
+ThriftHiveMetastore_add_default_constraint_args::~ThriftHiveMetastore_add_default_constraint_args() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_add_default_constraint_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_STRUCT) {
+          xfer += this->req.read(iprot);
+          this->__isset.req = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_add_default_constraint_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_default_constraint_args");
+
+  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->req.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_add_default_constraint_pargs::~ThriftHiveMetastore_add_default_constraint_pargs() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_add_default_constraint_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_default_constraint_pargs");
+
+  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->req)).write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_add_default_constraint_result::~ThriftHiveMetastore_add_default_constraint_result() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_add_default_constraint_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 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o2.read(iprot);
+          this->__isset.o2 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_add_default_constraint_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_default_constraint_result");
+
+  if (this->__isset.o1) {
+    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
+    xfer += this->o1.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o2) {
+    xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2);
+    xfer += this->o2.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_add_default_constraint_presult::~ThriftHiveMetastore_add_default_constraint_presult() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_add_default_constraint_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 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o2.read(iprot);
+          this->__isset.o2 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+
 ThriftHiveMetastore_drop_table_args::~ThriftHiveMetastore_drop_table_args() throw() {
 }
 
@@ -6478,14 +6729,14 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size1200;
-            ::apache::thrift::protocol::TType _etype1203;
-            xfer += iprot->readListBegin(_etype1203, _size1200);
-            this->partNames.resize(_size1200);
-            uint32_t _i1204;
-            for (_i1204 = 0; _i1204 < _size1200; ++_i1204)
+            uint32_t _size1227;
+            ::apache::thrift::protocol::TType _etype1230;
+            xfer += iprot->readListBegin(_etype1230, _size1227);
+            this->partNames.resize(_size1227);
+            uint32_t _i1231;
+            for (_i1231 = 0; _i1231 < _size1227; ++_i1231)
             {
-              xfer += iprot->readString(this->partNames[_i1204]);
+              xfer += iprot->readString(this->partNames[_i1231]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6522,10 +6773,10 @@ uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-    std::vector<std::string> ::const_iterator _iter1205;
-    for (_iter1205 = this->partNames.begin(); _iter1205 != this->partNames.end(); ++_iter1205)
+    std::vector<std::string> ::const_iterator _iter1232;
+    for (_iter1232 = this->partNames.begin(); _iter1232 != this->partNames.end(); ++_iter1232)
     {
-      xfer += oprot->writeString((*_iter1205));
+      xfer += oprot->writeString((*_iter1232));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6557,10 +6808,10 @@ uint32_t ThriftHiveMetastore_truncate_table_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partNames)).size()));
-    std::vector<std::string> ::const_iterator _iter1206;
-    for (_iter1206 = (*(this->partNames)).begin(); _iter1206 != (*(this->partNames)).end(); ++_iter1206)
+    std::vector<std::string> ::const_iterator _iter1233;
+    for (_iter1233 = (*(this->partNames)).begin(); _iter1233 != (*(this->partNames)).end(); ++_iter1233)
     {
-      xfer += oprot->writeString((*_iter1206));
+      xfer += oprot->writeString((*_iter1233));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6804,14 +7055,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1207;
-            ::apache::thrift::protocol::TType _etype1210;
-            xfer += iprot->readListBegin(_etype1210, _size1207);
-            this->success.resize(_size1207);
-            uint32_t _i1211;
-            for (_i1211 = 0; _i1211 < _size1207; ++_i1211)
+            uint32_t _size1234;
+            ::apache::thrift::protocol::TType _etype1237;
+            xfer += iprot->readListBegin(_etype1237, _size1234);
+            this->success.resize(_size1234);
+            uint32_t _i1238;
+            for (_i1238 = 0; _i1238 < _size1234; ++_i1238)
             {
-              xfer += iprot->readString(this->success[_i1211]);
+              xfer += iprot->readString(this->success[_i1238]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6850,10 +7101,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1212;
-      for (_iter1212 = this->success.begin(); _iter1212 != this->success.end(); ++_iter1212)
+      std::vector<std::string> ::const_iterator _iter1239;
+      for (_iter1239 = this->success.begin(); _iter1239 != this->success.end(); ++_iter1239)
       {
-        xfer += oprot->writeString((*_iter1212));
+        xfer += oprot->writeString((*_iter1239));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6898,14 +7149,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1213;
-            ::apache::thrift::protocol::TType _etype1216;
-            xfer += iprot->readListBegin(_etype1216, _size1213);
-            (*(this->success)).resize(_size1213);
-            uint32_t _i1217;
-            for (_i1217 = 0; _i1217 < _size1213; ++_i1217)
+            uint32_t _size1240;
+            ::apache::thrift::protocol::TType _etype1243;
+            xfer += iprot->readListBegin(_etype1243, _size1240);
+            (*(this->success)).resize(_size1240);
+            uint32_t _i1244;
+            for (_i1244 = 0; _i1244 < _size1240; ++_i1244)
             {
-              xfer += iprot->readString((*(this->success))[_i1217]);
+              xfer += iprot->readString((*(this->success))[_i1244]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7075,14 +7326,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1218;
-            ::apache::thrift::protocol::TType _etype1221;
-            xfer += iprot->readListBegin(_etype1221, _size1218);
-            this->success.resize(_size1218);
-            uint32_t _i1222;
-            for (_i1222 = 0; _i1222 < _size1218; ++_i1222)
+            uint32_t _size1245;
+            ::apache::thrift::protocol::TType _etype1248;
+            xfer += iprot->readListBegin(_etype1248, _size1245);
+            this->success.resize(_size1245);
+            uint32_t _i1249;
+            for (_i1249 = 0; _i1249 < _size1245; ++_i1249)
             {
-              xfer += iprot->readString(this->success[_i1222]);
+              xfer += iprot->readString(this->success[_i1249]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7121,10 +7372,10 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1223;
-      for (_iter1223 = this->success.begin(); _iter1223 != this->success.end(); ++_iter1223)
+      std::vector<std::string> ::const_iterator _iter1250;
+      for (_iter1250 = this->success.begin(); _iter1250 != this->success.end(); ++_iter1250)
       {
-        xfer += oprot->writeString((*_iter1223));
+        xfer += oprot->writeString((*_iter1250));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7169,14 +7420,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1224;
-            ::apache::thrift::protocol::TType _etype1227;
-            xfer += iprot->readListBegin(_etype1227, _size1224);
-            (*(this->success)).resize(_size1224);
-            uint32_t _i1228;
-            for (_i1228 = 0; _i1228 < _size1224; ++_i1228)
+            uint32_t _size1251;
+            ::apache::thrift::protocol::TType _etype1254;
+            xfer += iprot->readListBegin(_etype1254, _size1251);
+            (*(this->success)).resize(_size1251);
+            uint32_t _i1255;
+            for (_i1255 = 0; _i1255 < _size1251; ++_i1255)
             {
-              xfer += iprot->readString((*(this->success))[_i1228]);
+              xfer += iprot->readString((*(this->success))[_i1255]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7314,14 +7565,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1229;
-            ::apache::thrift::protocol::TType _etype1232;
-            xfer += iprot->readListBegin(_etype1232, _size1229);
-            this->success.resize(_size1229);
-            uint32_t _i1233;
-            for (_i1233 = 0; _i1233 < _size1229; ++_i1233)
+            uint32_t _size1256;
+            ::apache::thrift::protocol::TType _etype1259;
+            xfer += iprot->readListBegin(_etype1259, _size1256);
+            this->success.resize(_size1256);
+            uint32_t _i1260;
+            for (_i1260 = 0; _i1260 < _size1256; ++_i1260)
             {
-              xfer += iprot->readString(this->success[_i1233]);
+              xfer += iprot->readString(this->success[_i1260]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7360,10 +7611,10 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::write(
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1234;
-      for (_iter1234 = this->success.begin(); _iter1234 != this->success.end(); ++_iter1234)
+      std::vector<std::string> ::const_iterator _iter1261;
+      for (_iter1261 = this->success.begin(); _iter1261 != this->success.end(); ++_iter1261)
       {
-        xfer += oprot->writeString((*_iter1234));
+        xfer += oprot->writeString((*_iter1261));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7408,14 +7659,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_presult::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1235;
-            ::apache::thrift::protocol::TType _etype1238;
-            xfer += iprot->readListBegin(_etype1238, _size1235);
-            (*(this->success)).resize(_size1235);
-            uint32_t _i1239;
-            for (_i1239 = 0; _i1239 < _size1235; ++_i1239)
+            uint32_t _size1262;
+            ::apache::thrift::protocol::TType _etype1265;
+            xfer += iprot->readListBegin(_etype1265, _size1262);
+            (*(this->success)).resize(_size1262);
+            uint32_t _i1266;
+            for (_i1266 = 0; _i1266 < _size1262; ++_i1266)
             {
-              xfer += iprot->readString((*(this->success))[_i1239]);
+              xfer += iprot->readString((*(this->success))[_i1266]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7490,14 +7741,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 _size1240;
-            ::apache::thrift::protocol::TType _etype1243;
-            xfer += iprot->readListBegin(_etype1243, _size1240);
-            this->tbl_types.resize(_size1240);
-            uint32_t _i1244;
-            for (_i1244 = 0; _i1244 < _size1240; ++_i1244)
+            uint32_t _size1267;
+            ::apache::thrift::protocol::TType _etype1270;
+            xfer += iprot->readListBegin(_etype1270, _size1267);
+            this->tbl_types.resize(_size1267);
+            uint32_t _i1271;
+            for (_i1271 = 0; _i1271 < _size1267; ++_i1271)
             {
-              xfer += iprot->readString(this->tbl_types[_i1244]);
+              xfer += iprot->readString(this->tbl_types[_i1271]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7534,10 +7785,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 _iter1245;
-    for (_iter1245 = this->tbl_types.begin(); _iter1245 != this->tbl_types.end(); ++_iter1245)
+    std::vector<std::string> ::const_iterator _iter1272;
+    for (_iter1272 = this->tbl_types.begin(); _iter1272 != this->tbl_types.end(); ++_iter1272)
     {
-      xfer += oprot->writeString((*_iter1245));
+      xfer += oprot->writeString((*_iter1272));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7569,10 +7820,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 _iter1246;
-    for (_iter1246 = (*(this->tbl_types)).begin(); _iter1246 != (*(this->tbl_types)).end(); ++_iter1246)
+    std::vector<std::string> ::const_iterator _iter1273;
+    for (_iter1273 = (*(this->tbl_types)).begin(); _iter1273 != (*(this->tbl_types)).end(); ++_iter1273)
     {
-      xfer += oprot->writeString((*_iter1246));
+      xfer += oprot->writeString((*_iter1273));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7613,14 +7864,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1247;
-            ::apache::thrift::protocol::TType _etype1250;
-            xfer += iprot->readListBegin(_etype1250, _size1247);
-            this->success.resize(_size1247);
-            uint32_t _i1251;
-            for (_i1251 = 0; _i1251 < _size1247; ++_i1251)
+            uint32_t _size1274;
+            ::apache::thrift::protocol::TType _etype1277;
+            xfer += iprot->readListBegin(_etype1277, _size1274);
+            this->success.resize(_size1274);
+            uint32_t _i1278;
+            for (_i1278 = 0; _i1278 < _size1274; ++_i1278)
             {
-              xfer += this->success[_i1251].read(iprot);
+              xfer += this->success[_i1278].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7659,10 +7910,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 _iter1252;
-      for (_iter1252 = this->success.begin(); _iter1252 != this->success.end(); ++_iter1252)
+      std::vector<TableMeta> ::const_iterator _iter1279;
+      for (_iter1279 = this->success.begin(); _iter1279 != this->success.end(); ++_iter1279)
       {
-        xfer += (*_iter1252).write(oprot);
+        xfer += (*_iter1279).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -7707,14 +7958,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1253;
-            ::apache::thrift::protocol::TType _etype1256;
-            xfer += iprot->readListBegin(_etype1256, _size1253);
-            (*(this->success)).resize(_size1253);
-            uint32_t _i1257;
-            for (_i1257 = 0; _i1257 < _size1253; ++_i1257)
+            uint32_t _size1280;
+            ::apache::thrift::protocol::TType _etype1283;
+            xfer += iprot->readListBegin(_etype1283, _size1280);
+            (*(this->success)).resize(_size1280);
+            uint32_t _i1284;
+            for (_i1284 = 0; _i1284 < _size1280; ++_i1284)
             {
-              xfer += (*(this->success))[_i1257].read(iprot);
+              xfer += (*(this->success))[_i1284].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7852,14 +8103,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1258;
-            ::apache::thrift::protocol::TType _etype1261;
-            xfer += iprot->readListBegin(_etype1261, _size1258);
-            this->success.resize(_size1258);
-            uint32_t _i1262;
-            for (_i1262 = 0; _i1262 < _size1258; ++_i1262)
+            uint32_t _size1285;
+            ::apache::thrift::protocol::TType _etype1288;
+            xfer += iprot->readListBegin(_etype1288, _size1285);
+            this->success.resize(_size1285);
+            uint32_t _i1289;
+            for (_i1289 = 0; _i1289 < _size1285; ++_i1289)
             {
-              xfer += iprot->readString(this->success[_i1262]);
+              xfer += iprot->readString(this->success[_i1289]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7898,10 +8149,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 _iter1263;
-      for (_iter1263 = this->success.begin(); _iter1263 != this->success.end(); ++_iter1263)
+      std::vector<std::string> ::const_iterator _iter1290;
+      for (_iter1290 = this->success.begin(); _iter1290 != this->success.end(); ++_iter1290)
       {
-        xfer += oprot->writeString((*_iter1263));
+        xfer += oprot->writeString((*_iter1290));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7946,14 +8197,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1264;
-            ::apache::thrift::protocol::TType _etype1267;
-            xfer += iprot->readListBegin(_etype1267, _size1264);
-            (*(this->success)).resize(_size1264);
-            uint32_t _i1268;
-            for (_i1268 = 0; _i1268 < _size1264; ++_i1268)
+            uint32_t _size1291;
+            ::apache::thrift::protocol::TType _etype1294;
+            xfer += iprot->readListBegin(_etype1294, _size1291);
+            (*(this->success)).resize(_size1291);
+            uint32_t _i1295;
+            for (_i1295 = 0; _i1295 < _size1291; ++_i1295)
             {
-              xfer += iprot->readString((*(this->success))[_i1268]);
+              xfer += iprot->readString((*(this->success))[_i1295]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8263,14 +8514,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 _size1269;
-            ::apache::thrift::protocol::TType _etype1272;
-            xfer += iprot->readListBegin(_etype1272, _size1269);
-            this->tbl_names.resize(_size1269);
-            uint32_t _i1273;
-            for (_i1273 = 0; _i1273 < _size1269; ++_i1273)
+            uint32_t _size1296;
+            ::apache::thrift::protocol::TType _etype1299;
+            xfer += iprot->readListBegin(_etype1299, _size1296);
+            this->tbl_names.resize(_size1296);
+            uint32_t _i1300;
+            for (_i1300 = 0; _i1300 < _size1296; ++_i1300)
             {
-              xfer += iprot->readString(this->tbl_names[_i1273]);
+              xfer += iprot->readString(this->tbl_names[_i1300]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8303,10 +8554,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 _iter1274;
-    for (_iter1274 = this->tbl_names.begin(); _iter1274 != this->tbl_names.end(); ++_iter1274)
+    std::vector<std::string> ::const_iterator _iter1301;
+    for (_iter1301 = this->tbl_names.begin(); _iter1301 != this->tbl_names.end(); ++_iter1301)
     {
-      xfer += oprot->writeString((*_iter1274));
+      xfer += oprot->writeString((*_iter1301));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8334,10 +8585,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 _iter1275;
-    for (_iter1275 = (*(this->tbl_names)).begin(); _iter1275 != (*(this->tbl_names)).end(); ++_iter1275)
+    std::vector<std::string> ::const_iterator _iter1302;
+    for (_iter1302 = (*(this->tbl_names)).begin(); _iter1302 != (*(this->tbl_names)).end(); ++_iter1302)
     {
-      xfer += oprot->writeString((*_iter1275));
+      xfer += oprot->writeString((*_iter1302));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8378,14 +8629,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 _size1276;
-            ::apache::thrift::protocol::TType _etype1279;
-            xfer += iprot->readListBegin(_etype1279, _size1276);
-            this->success.resize(_size1276);
-            uint32_t _i1280;
-            for (_i1280 = 0; _i1280 < _size1276; ++_i1280)
+            uint32_t _size1303;
+            ::apache::thrift::protocol::TType _etype1306;
+            xfer += iprot->readListBegin(_etype1306, _size1303);
+            this->success.resize(_size1303);
+            uint32_t _i1307;
+            for (_i1307 = 0; _i1307 < _size1303; ++_i1307)
             {
-              xfer += this->success[_i1280].read(iprot);
+              xfer += this->success[_i1307].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8416,10 +8667,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 _iter1281;
-      for (_iter1281 = this->success.begin(); _iter1281 != this->success.end(); ++_iter1281)
+      std::vector<Table> ::const_iterator _iter1308;
+      for (_iter1308 = this->success.begin(); _iter1308 != this->success.end(); ++_iter1308)
       {
-        xfer += (*_iter1281).write(oprot);
+        xfer += (*_iter1308).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -8460,14 +8711,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 _size1282;
-            ::apache::thrift::protocol::TType _etype1285;
-            xfer += iprot->readListBegin(_etype1285, _size1282);
-            (*(this->success)).resize(_size1282);
-            uint32_t _i1286;
-            for (_i1286 = 0; _i1286 < _size1282; ++_i1286)
+            uint32_t _size1309;
+            ::apache::thrift::protocol::TType _etype1312;
+            xfer += iprot->readListBegin(_etype1312, _size1309);
+            (*(this->success)).resize(_size1309);
+            uint32_t _i1313;
+            for (_i1313 = 0; _i1313 < _size1309; ++_i1313)
             {
-              xfer += (*(this->success))[_i1286].read(iprot);
+              xfer += (*(this->success))[_i1313].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9000,14 +9251,14 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size1287;
-            ::apache::thrift::protocol::TType _etype1290;
-            xfer += iprot->readListBegin(_etype1290, _size1287);
-            this->tbl_names.resize(_size1287);
-            uint32_t _i1291;
-            for (_i1291 = 0; _i1291 < _size1287; ++_i1291)
+            uint32_t _size1314;
+            ::apache::thrift::protocol::TType _etype1317;
+            xfer += iprot->readListBegin(_etype1317, _size1314);
+            this->tbl_names.resize(_size1314);
+            uint32_t _i1318;
+            for (_i1318 = 0; _i1318 < _size1314; ++_i1318)
             {
-              xfer += iprot->readString(this->tbl_names[_i1291]);
+              xfer += iprot->readString(this->tbl_names[_i1318]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9040,10 +9291,10 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::write(:
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter1292;
-    for (_iter1292 = this->tbl_names.begin(); _iter1292 != this->tbl_names.end(); ++_iter1292)
+    std::vector<std::string> ::const_iterator _iter1319;
+    for (_iter1319 = this->tbl_names.begin(); _iter1319 != this->tbl_names.end(); ++_iter1319)
     {
-      xfer += oprot->writeString((*_iter1292));
+      xfer += oprot->writeString((*_iter1319));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9071,10 +9322,10 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_pargs::write(
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1293;
-    for (_iter1293 = (*(this->tbl_names)).begin(); _iter1293 != (*(this->tbl_names)).end(); ++_iter1293)
+    std::vector<std::string> ::const_iterator _iter1320;
+    for (_iter1320 = (*(this->tbl_names)).begin(); _iter1320 != (*(this->tbl_names)).end(); ++_iter1320)
     {
-      xfer += oprot->writeString((*_iter1293));
+      xfer += oprot->writeString((*_iter1320));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9115,17 +9366,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::read(
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1294;
-            ::apache::thrift::protocol::TType _ktype1295;
-            ::apache::thrift::protocol::TType _vtype1296;
-            xfer += iprot->readMapBegin(_ktype1295, _vtype1296, _size1294);
-            uint32_t _i1298;
-            for (_i1298 = 0; _i1298 < _size1294; ++_i1298)
+            uint32_t _size1321;
+            ::apache::thrift::protocol::TType _ktype1322;
+            ::apache::thrift::protocol::TType _vtype1323;
+            xfer += iprot->readMapBegin(_ktype1322, _vtype1323, _size1321);
+            uint32_t _i1325;
+            for (_i1325 = 0; _i1325 < _size1321; ++_i1325)
             {
-              std::string _key1299;
-              xfer += iprot->readString(_key1299);
-              Materialization& _val1300 = this->success[_key1299];
-              xfer += _val1300.read(iprot);
+              std::string _key1326;
+              xfer += iprot->readString(_key1326);
+              Materialization& _val1327 = this->success[_key1326];
+              xfer += _val1327.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -9180,11 +9431,11 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::write
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Materialization> ::const_iterator _iter1301;
-      for (_iter1301 = this->success.begin(); _iter1301 != this->success.end(); ++_iter1301)
+      std::map<std::string, Materialization> ::const_iterator _iter1328;
+      for (_iter1328 = this->success.begin(); _iter1328 != this->success.end(); ++_iter1328)
       {
-        xfer += oprot->writeString(_iter1301->first);
-        xfer += _iter1301->second.write(oprot);
+        xfer += oprot->writeString(_iter1328->first);
+        xfer += _iter1328->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -9237,17 +9488,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_presult::read
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1302;
-            ::apache::thrift::protocol::TType _ktype1303;
-            ::apache::thrift::protocol::TType _vtype1304;
-            xfer += iprot->readMapBegin(_ktype1303, _vtype1304, _size1302);
-            uint32_t _i1306;
-            for (_i1306 = 0; _i1306 < _size1302; ++_i1306)
+            uint32_t _size1329;
+            ::apache::thrift::protocol::TType _ktype1330;
+            ::apache::thrift::protocol::TType _vtype1331;
+            xfer += iprot->readMapBegin(_ktype1330, _vtype1331, _size1329);
+            uint32_t _i1333;
+            for (_i1333 = 0; _i1333 < _size1329; ++_i1333)
             {
-              std::string _key1307;
-              xfer += iprot->readString(_key1307);
-              Materialization& _val1308 = (*(this->success))[_key1307];
-              xfer += _val1308.read(iprot);
+              std::string _key1334;
+              xfer += iprot->readString(_key1334);
+              Materialization& _val1335 = (*(this->success))[_key1334];
+              xfer += _val1335.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -9692,14 +9943,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 _size1309;
-            ::apache::thrift::protocol::TType _etype1312;
-            xfer += iprot->readListBegin(_etype1312, _size1309);
-            this->success.resize(_size1309);
-            uint32_t _i1313;
-            for (_i1313 = 0; _i1313 < _size1309; ++_i1313)
+            uint32_t _size1336;
+            ::apache::thrift::protocol::TType _etype1339;
+            xfer += iprot->readListBegin(_etype1339, _size1336);
+            this->success.resize(_size1336);
+            uint32_t _i1340;
+            for (_i1340 = 0; _i1340 < _size1336; ++_i1340)
             {
-              xfer += iprot->readString(this->success[_i1313]);
+              xfer += iprot->readString(this->success[_i1340]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9754,10 +10005,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 _iter1314;
-      for (_iter1314 = this->success.begin(); _iter1314 != this->success.end(); ++_iter1314)
+      std::vector<std::string> ::const_iterator _iter1341;
+      for (_iter1341 = this->success.begin(); _iter1341 != this->success.end(); ++_iter1341)
       {
-        xfer += oprot->writeString((*_iter1314));
+        xfer += oprot->writeString((*_iter1341));
       }
       xfer += oprot->writeListEnd();
     }
@@ -9810,14 +10061,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 _size1315;
-            ::apache::thrift::protocol::TType _etype1318;
-            xfer += iprot->readListBegin(_etype1318, _size1315);
-            (*(this->success)).resize(_size1315);
-            uint32_t _i1319;
-            for (_i1319 = 0; _i1319 < _size1315; ++_i1319)
+            uint32_t _size1342;
+            ::apache::thrift::protocol::TType _etype1345;
+            xfer += iprot->readListBegin(_etype1345, _size1342);
+            (*(this->success)).resize(_size1342);
+            uint32_t _i1346;
+            for (_i1346 = 0; _i1346 < _size1342; ++_i1346)
             {
-              xfer += iprot->readString((*(this->success))[_i1319]);
+              xfer += iprot->readString((*(this->success))[_i1346]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11151,14 +11402,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size1320;
-            ::apache::thrift::protocol::TType _etype1323;
-            xfer += iprot->readListBegin(_etype1323, _size1320);
-            this->new_parts.resize(_size1320);
-            uint32_t _i1324;
-            for (_i1324 = 0; _i1324 < _size1320; ++_i1324)
+            uint32_t _size1347;
+            ::apache::thrift::protocol::TType _etype1350;
+            xfer += iprot->readListBegin(_etype1350, _size1347);
+            this->new_parts.resize(_size1347);
+            uint32_t _i1351;
+            for (_i1351 = 0; _i1351 < _size1347; ++_i1351)
             {
-              xfer += this->new_parts[_i1324].read(iprot);
+              xfer += this->new_parts[_i1351].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11187,10 +11438,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 _iter1325;
-    for (_iter1325 = this->new_parts.begin(); _iter1325 != this->new_parts.end(); ++_iter1325)
+    std::vector<Partition> ::const_iterator _iter1352;
+    for (_iter1352 = this->new_parts.begin(); _iter1352 != this->new_parts.end(); ++_iter1352)
     {
-      xfer += (*_iter1325).write(oprot);
+      xfer += (*_iter1352).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11214,10 +11465,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 _iter1326;
-    for (_iter1326 = (*(this->new_parts)).begin(); _iter1326 != (*(this->new_parts)).end(); ++_iter1326)
+    std::vector<Partition> ::const_iterator _iter1353;
+    for (_iter1353 = (*(this->new_parts)).begin(); _iter1353 != (*(this->new_parts)).end(); ++_iter1353)
     {
-      xfer += (*_iter1326).write(oprot);
+      xfer += (*_iter1353).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11426,14 +11677,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 _size1327;
-            ::apache::thrift::protocol::TType _etype1330;
-            xfer += iprot->readListBegin(_etype1330, _size1327);
-            this->new_parts.resize(_size1327);
-            uint32_t _i1331;
-            for (_i1331 = 0; _i1331 < _size1327; ++_i1331)
+            uint32_t _size1354;
+            ::apache::thrift::protocol::TType _etype1357;
+            xfer += iprot->readListBegin(_etype1357, _size1354);
+            this->new_parts.resize(_size1354);
+            uint32_t _i1358;
+            for (_i1358 = 0; _i1358 < _size1354; ++_i1358)
             {
-              xfer += this->new_parts[_i1331].read(iprot);
+              xfer += this->new_parts[_i1358].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11462,10 +11713,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 _iter1332;
-    for (_iter1332 = this->new_parts.begin(); _iter1332 != this->new_parts.end(); ++_iter1332)
+    std::vector<PartitionSpec> ::const_iterator _iter1359;
+    for (_iter1359 = this->new_parts.begin(); _iter1359 != this->new_parts.end(); ++_iter1359)
     {
-      xfer += (*_iter1332).write(oprot);
+      xfer += (*_iter1359).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11489,10 +11740,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 _iter1333;
-    for (_iter1333 = (*(this->new_parts)).begin(); _iter1333 != (*(this->new_parts)).end(); ++_iter1333)
+    std::vector<PartitionSpec> ::const_iterator _iter1360;
+    for (_iter1360 = (*(this->new_parts)).begin(); _iter1360 != (*(this->new_parts)).end(); ++_iter1360)
     {
-      xfer += (*_iter1333).write(oprot);
+      xfer += (*_iter1360).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11717,14 +11968,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1334;
-            ::apache::thrift::protocol::TType _etype1337;
-            xfer += iprot->readListBegin(_etype1337, _size1334);
-            this->part_vals.resize(_size1334);
-            uint32_t _i1338;
-            for (_i1338 = 0; _i1338 < _size1334; ++_i1338)
+            uint32_t _size1361;
+            ::apache::thrift::protocol::TType _etype1364;
+            xfer += iprot->readListBegin(_etype1364, _size1361);
+            this->part_vals.resize(_size1361);
+            uint32_t _i1365;
+            for (_i1365 = 0; _i1365 < _size1361; ++_i1365)
             {
-              xfer += iprot->readString(this->part_vals[_i1338]);
+              xfer += iprot->readString(this->part_vals[_i1365]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11761,10 +12012,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 _iter1339;
-    for (_iter1339 = this->part_vals.begin(); _iter1339 != this->part_vals.end(); ++_iter1339)
+    std::vector<std::string> ::const_iterator _iter1366;
+    for (_iter1366 = this->part_vals.begin(); _iter1366 != this->part_vals.end(); ++_iter1366)
     {
-      xfer += oprot->writeString((*_iter1339));
+      xfer += oprot->writeString((*_iter1366));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11796,10 +12047,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 _iter1340;
-    for (_iter1340 = (*(this->part_vals)).begin(); _iter1340 != (*(this->part_vals)).end(); ++_iter1340)
+    std::vector<std::string> ::const_iterator _iter1367;
+    for (_iter1367 = (*(this->part_vals)).begin(); _iter1367 != (*(this->part_vals)).end(); ++_iter1367)
     {
-      xfer += oprot->writeString((*_iter1340));
+      xfer += oprot->writeString((*_iter1367));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12271,14 +12522,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1341;
-            ::apache::thrift::protocol::TType _etype1344;
-            xfer += iprot->readListBegin(_etype1344, _size1341);
-            this->part_vals.resize(_size1341);
-            uint32_t _i1345;
-            for (_i1345 = 0; _i1345 < _size1341; ++_i1345)
+            uint32_t _size1368;
+            ::apache::thrift::protocol::TType _etype1371;
+            xfer += iprot->readListBegin(_etype1371, _size1368);
+            this->part_vals.resize(_size1368);
+            uint32_t _i1372;
+            for (_i1372 = 0; _i1372 < _size1368; ++_i1372)
             {
-              xfer += iprot->readString(this->part_vals[_i1345]);
+              xfer += iprot->readString(this->part_vals[_i1372]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12323,10 +12574,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 _iter1346;
-    for (_iter1346 = this->part_vals.begin(); _iter1346 != this->part_vals.end(); ++_iter1346)
+    std::vector<std::string> ::const_iterator _iter1373;
+    for (_iter1373 = this->part_vals.begin(); _iter1373 != this->part_vals.end(); ++_iter1373)
     {
-      xfer += oprot->writeString((*_iter1346));
+      xfer += oprot->writeString((*_iter1373));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12362,10 +12613,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 _iter1347;
-    for (_iter1347 = (*(this->part_vals)).begin(); _iter1347 != (*(this->part_vals)).end(); ++_iter1347)
+    std::vector<std::string> ::const_iterator _iter1374;
+    for (_iter1374 = (*(this->part_vals)).begin(); _iter1374 != (*(this->part_vals)).end(); ++_iter1374)
     {
-      xfer += oprot->writeString((*_iter1347));
+      xfer += oprot->writeString((*_iter1374));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13168,14 +13419,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1348;
-            ::apache::thrift::protocol::TType _etype1351;
-            xfer += iprot->readListBegin(_etype1351, _size1348);
-            this->part_vals.resize(_size1348);
-            uint32_t _i1352;
-            for (_i1352 = 0; _i1352 < _size1348; ++_i1352)
+            uint32_t _size1375;
+            ::apache::thrift::protocol::TType _etype1378;
+            xfer += iprot->readListBegin(_etype1378, _size1375);
+            this->part_vals.resize(_size1375);
+            uint32_t _i1379;
+            for (_i1379 = 0; _i1379 < _size1375; ++_i1379)
             {
-              xfer += iprot->readString(this->part_vals[_i1352]);
+              xfer += iprot->readString(this->part_vals[_i1379]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13220,10 +13471,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 _iter1353;
-    for (_iter1353 = this->part_vals.begin(); _iter1353 != this->part_vals.end(); ++_iter1353)
+    std::vector<std::string> ::const_iterator _iter1380;
+    for (_iter1380 = this->part_vals.begin(); _iter1380 != this->part_vals.end(); ++_iter1380)
     {
-      xfer += oprot->writeString((*_iter1353));
+      xfer += oprot->writeString((*_iter1380));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13259,10 +13510,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 _iter1354;
-    for (_iter1354 = (*(this->part_vals)).begin(); _iter1354 != (*(this->part_vals)).end(); ++_iter1354)
+    std::vector<std::string> ::const_iterator _iter1381;
+    for (_iter1381 = (*(this->part_vals)).begin(); _iter1381 != (*(this->part_vals)).end(); ++_iter1381)
     {
-      xfer += oprot->writeString((*_iter1354));
+      xfer += oprot->writeString((*_iter1381));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13471,14 +13722,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1355;
-            ::apache::thrift::protocol::TType _etype1358;
-            xfer += iprot->readListBegin(_etype1358, _size1355);
-            this->part_vals.resize(_size1355);
-            uint32_t _i1359;
-            for (_i1359 = 0; _i1359 < _size1355; ++_i1359)
+            uint32_t _size1382;
+            ::apache::thrift::protocol::TType _etype1385;
+            xfer += iprot->readListBegin(_etype1385, _size1382);
+            this->part_vals.resize(_size1382);
+            uint32_t _i1386;
+            for (_i1386 = 0; _i1386 < _size1382; ++_i1386)
             {
-              xfer += iprot->readString(this->part_vals[_i1359]);
+              xfer += iprot->readString(this->part_vals[_i1386]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13531,10 +13782,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 _iter1360;
-    for (_iter1360 = this->part_vals.begin(); _iter1360 != this->part_vals.end(); ++_iter1360)
+    std::vector<std::string> ::const_iterator _iter1387;
+    for (_iter1387 = this->part_vals.begin(); _iter1387 != this->part_vals.end(); ++_iter1387)
     {
-      xfer += oprot->writeString((*_iter1360));
+      xfer += oprot->writeString((*_iter1387));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13574,10 +13825,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 _iter1361;
-    for (_iter1361 = (*(this->part_vals)).begin(); _iter1361 != (*(this->part_vals)).end(); ++_iter1361)
+    std::vector<std::string> ::const_iterator _iter1388;
+    for (_iter1388 = (*(this->part_vals)).begin(); _iter1388 != (*(this->part_vals)).end(); ++_iter1388)
     {
-      xfer += oprot->writeString((*_iter1361));
+      xfer += oprot->writeString((*_iter1388));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14583,14 +14834,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::r

<TRUNCATED>

[09/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLDefaultConstraint.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLDefaultConstraint.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLDefaultConstraint.java
new file mode 100644
index 0000000..185b77e
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLDefaultConstraint.java
@@ -0,0 +1,1109 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class SQLDefaultConstraint implements org.apache.thrift.TBase<SQLDefaultConstraint, SQLDefaultConstraint._Fields>, java.io.Serializable, Cloneable, Comparable<SQLDefaultConstraint> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SQLDefaultConstraint");
+
+  private static final org.apache.thrift.protocol.TField TABLE_DB_FIELD_DESC = new org.apache.thrift.protocol.TField("table_db", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("table_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("column_name", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField DEFAULT_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("default_value", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField DC_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dc_name", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField ENABLE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("enable_cstr", org.apache.thrift.protocol.TType.BOOL, (short)6);
+  private static final org.apache.thrift.protocol.TField VALIDATE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("validate_cstr", org.apache.thrift.protocol.TType.BOOL, (short)7);
+  private static final org.apache.thrift.protocol.TField RELY_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("rely_cstr", org.apache.thrift.protocol.TType.BOOL, (short)8);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SQLDefaultConstraintStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SQLDefaultConstraintTupleSchemeFactory());
+  }
+
+  private String table_db; // required
+  private String table_name; // required
+  private String column_name; // required
+  private String default_value; // required
+  private String dc_name; // required
+  private boolean enable_cstr; // required
+  private boolean validate_cstr; // required
+  private boolean rely_cstr; // 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 {
+    TABLE_DB((short)1, "table_db"),
+    TABLE_NAME((short)2, "table_name"),
+    COLUMN_NAME((short)3, "column_name"),
+    DEFAULT_VALUE((short)4, "default_value"),
+    DC_NAME((short)5, "dc_name"),
+    ENABLE_CSTR((short)6, "enable_cstr"),
+    VALIDATE_CSTR((short)7, "validate_cstr"),
+    RELY_CSTR((short)8, "rely_cstr");
+
+    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: // TABLE_DB
+          return TABLE_DB;
+        case 2: // TABLE_NAME
+          return TABLE_NAME;
+        case 3: // COLUMN_NAME
+          return COLUMN_NAME;
+        case 4: // DEFAULT_VALUE
+          return DEFAULT_VALUE;
+        case 5: // DC_NAME
+          return DC_NAME;
+        case 6: // ENABLE_CSTR
+          return ENABLE_CSTR;
+        case 7: // VALIDATE_CSTR
+          return VALIDATE_CSTR;
+        case 8: // RELY_CSTR
+          return RELY_CSTR;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __ENABLE_CSTR_ISSET_ID = 0;
+  private static final int __VALIDATE_CSTR_ISSET_ID = 1;
+  private static final int __RELY_CSTR_ISSET_ID = 2;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TABLE_DB, new org.apache.thrift.meta_data.FieldMetaData("table_db", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("table_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.COLUMN_NAME, new org.apache.thrift.meta_data.FieldMetaData("column_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.DEFAULT_VALUE, new org.apache.thrift.meta_data.FieldMetaData("default_value", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.DC_NAME, new org.apache.thrift.meta_data.FieldMetaData("dc_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ENABLE_CSTR, new org.apache.thrift.meta_data.FieldMetaData("enable_cstr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.VALIDATE_CSTR, new org.apache.thrift.meta_data.FieldMetaData("validate_cstr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.RELY_CSTR, new org.apache.thrift.meta_data.FieldMetaData("rely_cstr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SQLDefaultConstraint.class, metaDataMap);
+  }
+
+  public SQLDefaultConstraint() {
+  }
+
+  public SQLDefaultConstraint(
+    String table_db,
+    String table_name,
+    String column_name,
+    String default_value,
+    String dc_name,
+    boolean enable_cstr,
+    boolean validate_cstr,
+    boolean rely_cstr)
+  {
+    this();
+    this.table_db = table_db;
+    this.table_name = table_name;
+    this.column_name = column_name;
+    this.default_value = default_value;
+    this.dc_name = dc_name;
+    this.enable_cstr = enable_cstr;
+    setEnable_cstrIsSet(true);
+    this.validate_cstr = validate_cstr;
+    setValidate_cstrIsSet(true);
+    this.rely_cstr = rely_cstr;
+    setRely_cstrIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SQLDefaultConstraint(SQLDefaultConstraint other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetTable_db()) {
+      this.table_db = other.table_db;
+    }
+    if (other.isSetTable_name()) {
+      this.table_name = other.table_name;
+    }
+    if (other.isSetColumn_name()) {
+      this.column_name = other.column_name;
+    }
+    if (other.isSetDefault_value()) {
+      this.default_value = other.default_value;
+    }
+    if (other.isSetDc_name()) {
+      this.dc_name = other.dc_name;
+    }
+    this.enable_cstr = other.enable_cstr;
+    this.validate_cstr = other.validate_cstr;
+    this.rely_cstr = other.rely_cstr;
+  }
+
+  public SQLDefaultConstraint deepCopy() {
+    return new SQLDefaultConstraint(this);
+  }
+
+  @Override
+  public void clear() {
+    this.table_db = null;
+    this.table_name = null;
+    this.column_name = null;
+    this.default_value = null;
+    this.dc_name = null;
+    setEnable_cstrIsSet(false);
+    this.enable_cstr = false;
+    setValidate_cstrIsSet(false);
+    this.validate_cstr = false;
+    setRely_cstrIsSet(false);
+    this.rely_cstr = false;
+  }
+
+  public String getTable_db() {
+    return this.table_db;
+  }
+
+  public void setTable_db(String table_db) {
+    this.table_db = table_db;
+  }
+
+  public void unsetTable_db() {
+    this.table_db = null;
+  }
+
+  /** Returns true if field table_db is set (has been assigned a value) and false otherwise */
+  public boolean isSetTable_db() {
+    return this.table_db != null;
+  }
+
+  public void setTable_dbIsSet(boolean value) {
+    if (!value) {
+      this.table_db = null;
+    }
+  }
+
+  public String getTable_name() {
+    return this.table_name;
+  }
+
+  public void setTable_name(String table_name) {
+    this.table_name = table_name;
+  }
+
+  public void unsetTable_name() {
+    this.table_name = null;
+  }
+
+  /** Returns true if field table_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetTable_name() {
+    return this.table_name != null;
+  }
+
+  public void setTable_nameIsSet(boolean value) {
+    if (!value) {
+      this.table_name = null;
+    }
+  }
+
+  public String getColumn_name() {
+    return this.column_name;
+  }
+
+  public void setColumn_name(String column_name) {
+    this.column_name = column_name;
+  }
+
+  public void unsetColumn_name() {
+    this.column_name = null;
+  }
+
+  /** Returns true if field column_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetColumn_name() {
+    return this.column_name != null;
+  }
+
+  public void setColumn_nameIsSet(boolean value) {
+    if (!value) {
+      this.column_name = null;
+    }
+  }
+
+  public String getDefault_value() {
+    return this.default_value;
+  }
+
+  public void setDefault_value(String default_value) {
+    this.default_value = default_value;
+  }
+
+  public void unsetDefault_value() {
+    this.default_value = null;
+  }
+
+  /** Returns true if field default_value is set (has been assigned a value) and false otherwise */
+  public boolean isSetDefault_value() {
+    return this.default_value != null;
+  }
+
+  public void setDefault_valueIsSet(boolean value) {
+    if (!value) {
+      this.default_value = null;
+    }
+  }
+
+  public String getDc_name() {
+    return this.dc_name;
+  }
+
+  public void setDc_name(String dc_name) {
+    this.dc_name = dc_name;
+  }
+
+  public void unsetDc_name() {
+    this.dc_name = null;
+  }
+
+  /** Returns true if field dc_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetDc_name() {
+    return this.dc_name != null;
+  }
+
+  public void setDc_nameIsSet(boolean value) {
+    if (!value) {
+      this.dc_name = null;
+    }
+  }
+
+  public boolean isEnable_cstr() {
+    return this.enable_cstr;
+  }
+
+  public void setEnable_cstr(boolean enable_cstr) {
+    this.enable_cstr = enable_cstr;
+    setEnable_cstrIsSet(true);
+  }
+
+  public void unsetEnable_cstr() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ENABLE_CSTR_ISSET_ID);
+  }
+
+  /** Returns true if field enable_cstr is set (has been assigned a value) and false otherwise */
+  public boolean isSetEnable_cstr() {
+    return EncodingUtils.testBit(__isset_bitfield, __ENABLE_CSTR_ISSET_ID);
+  }
+
+  public void setEnable_cstrIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ENABLE_CSTR_ISSET_ID, value);
+  }
+
+  public boolean isValidate_cstr() {
+    return this.validate_cstr;
+  }
+
+  public void setValidate_cstr(boolean validate_cstr) {
+    this.validate_cstr = validate_cstr;
+    setValidate_cstrIsSet(true);
+  }
+
+  public void unsetValidate_cstr() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALIDATE_CSTR_ISSET_ID);
+  }
+
+  /** Returns true if field validate_cstr is set (has been assigned a value) and false otherwise */
+  public boolean isSetValidate_cstr() {
+    return EncodingUtils.testBit(__isset_bitfield, __VALIDATE_CSTR_ISSET_ID);
+  }
+
+  public void setValidate_cstrIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALIDATE_CSTR_ISSET_ID, value);
+  }
+
+  public boolean isRely_cstr() {
+    return this.rely_cstr;
+  }
+
+  public void setRely_cstr(boolean rely_cstr) {
+    this.rely_cstr = rely_cstr;
+    setRely_cstrIsSet(true);
+  }
+
+  public void unsetRely_cstr() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RELY_CSTR_ISSET_ID);
+  }
+
+  /** Returns true if field rely_cstr is set (has been assigned a value) and false otherwise */
+  public boolean isSetRely_cstr() {
+    return EncodingUtils.testBit(__isset_bitfield, __RELY_CSTR_ISSET_ID);
+  }
+
+  public void setRely_cstrIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RELY_CSTR_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TABLE_DB:
+      if (value == null) {
+        unsetTable_db();
+      } else {
+        setTable_db((String)value);
+      }
+      break;
+
+    case TABLE_NAME:
+      if (value == null) {
+        unsetTable_name();
+      } else {
+        setTable_name((String)value);
+      }
+      break;
+
+    case COLUMN_NAME:
+      if (value == null) {
+        unsetColumn_name();
+      } else {
+        setColumn_name((String)value);
+      }
+      break;
+
+    case DEFAULT_VALUE:
+      if (value == null) {
+        unsetDefault_value();
+      } else {
+        setDefault_value((String)value);
+      }
+      break;
+
+    case DC_NAME:
+      if (value == null) {
+        unsetDc_name();
+      } else {
+        setDc_name((String)value);
+      }
+      break;
+
+    case ENABLE_CSTR:
+      if (value == null) {
+        unsetEnable_cstr();
+      } else {
+        setEnable_cstr((Boolean)value);
+      }
+      break;
+
+    case VALIDATE_CSTR:
+      if (value == null) {
+        unsetValidate_cstr();
+      } else {
+        setValidate_cstr((Boolean)value);
+      }
+      break;
+
+    case RELY_CSTR:
+      if (value == null) {
+        unsetRely_cstr();
+      } else {
+        setRely_cstr((Boolean)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TABLE_DB:
+      return getTable_db();
+
+    case TABLE_NAME:
+      return getTable_name();
+
+    case COLUMN_NAME:
+      return getColumn_name();
+
+    case DEFAULT_VALUE:
+      return getDefault_value();
+
+    case DC_NAME:
+      return getDc_name();
+
+    case ENABLE_CSTR:
+      return isEnable_cstr();
+
+    case VALIDATE_CSTR:
+      return isValidate_cstr();
+
+    case RELY_CSTR:
+      return isRely_cstr();
+
+    }
+    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 TABLE_DB:
+      return isSetTable_db();
+    case TABLE_NAME:
+      return isSetTable_name();
+    case COLUMN_NAME:
+      return isSetColumn_name();
+    case DEFAULT_VALUE:
+      return isSetDefault_value();
+    case DC_NAME:
+      return isSetDc_name();
+    case ENABLE_CSTR:
+      return isSetEnable_cstr();
+    case VALIDATE_CSTR:
+      return isSetValidate_cstr();
+    case RELY_CSTR:
+      return isSetRely_cstr();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SQLDefaultConstraint)
+      return this.equals((SQLDefaultConstraint)that);
+    return false;
+  }
+
+  public boolean equals(SQLDefaultConstraint that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_table_db = true && this.isSetTable_db();
+    boolean that_present_table_db = true && that.isSetTable_db();
+    if (this_present_table_db || that_present_table_db) {
+      if (!(this_present_table_db && that_present_table_db))
+        return false;
+      if (!this.table_db.equals(that.table_db))
+        return false;
+    }
+
+    boolean this_present_table_name = true && this.isSetTable_name();
+    boolean that_present_table_name = true && that.isSetTable_name();
+    if (this_present_table_name || that_present_table_name) {
+      if (!(this_present_table_name && that_present_table_name))
+        return false;
+      if (!this.table_name.equals(that.table_name))
+        return false;
+    }
+
+    boolean this_present_column_name = true && this.isSetColumn_name();
+    boolean that_present_column_name = true && that.isSetColumn_name();
+    if (this_present_column_name || that_present_column_name) {
+      if (!(this_present_column_name && that_present_column_name))
+        return false;
+      if (!this.column_name.equals(that.column_name))
+        return false;
+    }
+
+    boolean this_present_default_value = true && this.isSetDefault_value();
+    boolean that_present_default_value = true && that.isSetDefault_value();
+    if (this_present_default_value || that_present_default_value) {
+      if (!(this_present_default_value && that_present_default_value))
+        return false;
+      if (!this.default_value.equals(that.default_value))
+        return false;
+    }
+
+    boolean this_present_dc_name = true && this.isSetDc_name();
+    boolean that_present_dc_name = true && that.isSetDc_name();
+    if (this_present_dc_name || that_present_dc_name) {
+      if (!(this_present_dc_name && that_present_dc_name))
+        return false;
+      if (!this.dc_name.equals(that.dc_name))
+        return false;
+    }
+
+    boolean this_present_enable_cstr = true;
+    boolean that_present_enable_cstr = true;
+    if (this_present_enable_cstr || that_present_enable_cstr) {
+      if (!(this_present_enable_cstr && that_present_enable_cstr))
+        return false;
+      if (this.enable_cstr != that.enable_cstr)
+        return false;
+    }
+
+    boolean this_present_validate_cstr = true;
+    boolean that_present_validate_cstr = true;
+    if (this_present_validate_cstr || that_present_validate_cstr) {
+      if (!(this_present_validate_cstr && that_present_validate_cstr))
+        return false;
+      if (this.validate_cstr != that.validate_cstr)
+        return false;
+    }
+
+    boolean this_present_rely_cstr = true;
+    boolean that_present_rely_cstr = true;
+    if (this_present_rely_cstr || that_present_rely_cstr) {
+      if (!(this_present_rely_cstr && that_present_rely_cstr))
+        return false;
+      if (this.rely_cstr != that.rely_cstr)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_table_db = true && (isSetTable_db());
+    list.add(present_table_db);
+    if (present_table_db)
+      list.add(table_db);
+
+    boolean present_table_name = true && (isSetTable_name());
+    list.add(present_table_name);
+    if (present_table_name)
+      list.add(table_name);
+
+    boolean present_column_name = true && (isSetColumn_name());
+    list.add(present_column_name);
+    if (present_column_name)
+      list.add(column_name);
+
+    boolean present_default_value = true && (isSetDefault_value());
+    list.add(present_default_value);
+    if (present_default_value)
+      list.add(default_value);
+
+    boolean present_dc_name = true && (isSetDc_name());
+    list.add(present_dc_name);
+    if (present_dc_name)
+      list.add(dc_name);
+
+    boolean present_enable_cstr = true;
+    list.add(present_enable_cstr);
+    if (present_enable_cstr)
+      list.add(enable_cstr);
+
+    boolean present_validate_cstr = true;
+    list.add(present_validate_cstr);
+    if (present_validate_cstr)
+      list.add(validate_cstr);
+
+    boolean present_rely_cstr = true;
+    list.add(present_rely_cstr);
+    if (present_rely_cstr)
+      list.add(rely_cstr);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SQLDefaultConstraint other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetTable_db()).compareTo(other.isSetTable_db());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTable_db()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table_db, other.table_db);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTable_name()).compareTo(other.isSetTable_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTable_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table_name, other.table_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetColumn_name()).compareTo(other.isSetColumn_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetColumn_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column_name, other.column_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDefault_value()).compareTo(other.isSetDefault_value());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDefault_value()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.default_value, other.default_value);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDc_name()).compareTo(other.isSetDc_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDc_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dc_name, other.dc_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetEnable_cstr()).compareTo(other.isSetEnable_cstr());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetEnable_cstr()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.enable_cstr, other.enable_cstr);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetValidate_cstr()).compareTo(other.isSetValidate_cstr());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetValidate_cstr()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.validate_cstr, other.validate_cstr);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetRely_cstr()).compareTo(other.isSetRely_cstr());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetRely_cstr()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rely_cstr, other.rely_cstr);
+      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("SQLDefaultConstraint(");
+    boolean first = true;
+
+    sb.append("table_db:");
+    if (this.table_db == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.table_db);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("table_name:");
+    if (this.table_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.table_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("column_name:");
+    if (this.column_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.column_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("default_value:");
+    if (this.default_value == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.default_value);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("dc_name:");
+    if (this.dc_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.dc_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("enable_cstr:");
+    sb.append(this.enable_cstr);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("validate_cstr:");
+    sb.append(this.validate_cstr);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("rely_cstr:");
+    sb.append(this.rely_cstr);
+    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 {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class SQLDefaultConstraintStandardSchemeFactory implements SchemeFactory {
+    public SQLDefaultConstraintStandardScheme getScheme() {
+      return new SQLDefaultConstraintStandardScheme();
+    }
+  }
+
+  private static class SQLDefaultConstraintStandardScheme extends StandardScheme<SQLDefaultConstraint> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SQLDefaultConstraint 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: // TABLE_DB
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.table_db = iprot.readString();
+              struct.setTable_dbIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TABLE_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.table_name = iprot.readString();
+              struct.setTable_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // COLUMN_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.column_name = iprot.readString();
+              struct.setColumn_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // DEFAULT_VALUE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.default_value = iprot.readString();
+              struct.setDefault_valueIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // DC_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.dc_name = iprot.readString();
+              struct.setDc_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // ENABLE_CSTR
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.enable_cstr = iprot.readBool();
+              struct.setEnable_cstrIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // VALIDATE_CSTR
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.validate_cstr = iprot.readBool();
+              struct.setValidate_cstrIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // RELY_CSTR
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.rely_cstr = iprot.readBool();
+              struct.setRely_cstrIsSet(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, SQLDefaultConstraint struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.table_db != null) {
+        oprot.writeFieldBegin(TABLE_DB_FIELD_DESC);
+        oprot.writeString(struct.table_db);
+        oprot.writeFieldEnd();
+      }
+      if (struct.table_name != null) {
+        oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
+        oprot.writeString(struct.table_name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.column_name != null) {
+        oprot.writeFieldBegin(COLUMN_NAME_FIELD_DESC);
+        oprot.writeString(struct.column_name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.default_value != null) {
+        oprot.writeFieldBegin(DEFAULT_VALUE_FIELD_DESC);
+        oprot.writeString(struct.default_value);
+        oprot.writeFieldEnd();
+      }
+      if (struct.dc_name != null) {
+        oprot.writeFieldBegin(DC_NAME_FIELD_DESC);
+        oprot.writeString(struct.dc_name);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(ENABLE_CSTR_FIELD_DESC);
+      oprot.writeBool(struct.enable_cstr);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(VALIDATE_CSTR_FIELD_DESC);
+      oprot.writeBool(struct.validate_cstr);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(RELY_CSTR_FIELD_DESC);
+      oprot.writeBool(struct.rely_cstr);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SQLDefaultConstraintTupleSchemeFactory implements SchemeFactory {
+    public SQLDefaultConstraintTupleScheme getScheme() {
+      return new SQLDefaultConstraintTupleScheme();
+    }
+  }
+
+  private static class SQLDefaultConstraintTupleScheme extends TupleScheme<SQLDefaultConstraint> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SQLDefaultConstraint struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetTable_db()) {
+        optionals.set(0);
+      }
+      if (struct.isSetTable_name()) {
+        optionals.set(1);
+      }
+      if (struct.isSetColumn_name()) {
+        optionals.set(2);
+      }
+      if (struct.isSetDefault_value()) {
+        optionals.set(3);
+      }
+      if (struct.isSetDc_name()) {
+        optionals.set(4);
+      }
+      if (struct.isSetEnable_cstr()) {
+        optionals.set(5);
+      }
+      if (struct.isSetValidate_cstr()) {
+        optionals.set(6);
+      }
+      if (struct.isSetRely_cstr()) {
+        optionals.set(7);
+      }
+      oprot.writeBitSet(optionals, 8);
+      if (struct.isSetTable_db()) {
+        oprot.writeString(struct.table_db);
+      }
+      if (struct.isSetTable_name()) {
+        oprot.writeString(struct.table_name);
+      }
+      if (struct.isSetColumn_name()) {
+        oprot.writeString(struct.column_name);
+      }
+      if (struct.isSetDefault_value()) {
+        oprot.writeString(struct.default_value);
+      }
+      if (struct.isSetDc_name()) {
+        oprot.writeString(struct.dc_name);
+      }
+      if (struct.isSetEnable_cstr()) {
+        oprot.writeBool(struct.enable_cstr);
+      }
+      if (struct.isSetValidate_cstr()) {
+        oprot.writeBool(struct.validate_cstr);
+      }
+      if (struct.isSetRely_cstr()) {
+        oprot.writeBool(struct.rely_cstr);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SQLDefaultConstraint struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(8);
+      if (incoming.get(0)) {
+        struct.table_db = iprot.readString();
+        struct.setTable_dbIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.table_name = iprot.readString();
+        struct.setTable_nameIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.column_name = iprot.readString();
+        struct.setColumn_nameIsSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.default_value = iprot.readString();
+        struct.setDefault_valueIsSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.dc_name = iprot.readString();
+        struct.setDc_nameIsSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.enable_cstr = iprot.readBool();
+        struct.setEnable_cstrIsSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.validate_cstr = iprot.readBool();
+        struct.setValidate_cstrIsSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.rely_cstr = iprot.readBool();
+        struct.setRely_cstrIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
index fb7b94e..bd335ef 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COMPACTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list638 = iprot.readListBegin();
-                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list638.size);
-                ShowCompactResponseElement _elem639;
-                for (int _i640 = 0; _i640 < _list638.size; ++_i640)
+                org.apache.thrift.protocol.TList _list654 = iprot.readListBegin();
+                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list654.size);
+                ShowCompactResponseElement _elem655;
+                for (int _i656 = 0; _i656 < _list654.size; ++_i656)
                 {
-                  _elem639 = new ShowCompactResponseElement();
-                  _elem639.read(iprot);
-                  struct.compacts.add(_elem639);
+                  _elem655 = new ShowCompactResponseElement();
+                  _elem655.read(iprot);
+                  struct.compacts.add(_elem655);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COMPACTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.compacts.size()));
-          for (ShowCompactResponseElement _iter641 : struct.compacts)
+          for (ShowCompactResponseElement _iter657 : struct.compacts)
           {
-            _iter641.write(oprot);
+            _iter657.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.compacts.size());
-        for (ShowCompactResponseElement _iter642 : struct.compacts)
+        for (ShowCompactResponseElement _iter658 : struct.compacts)
         {
-          _iter642.write(oprot);
+          _iter658.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ShowCompactResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list643 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list643.size);
-        ShowCompactResponseElement _elem644;
-        for (int _i645 = 0; _i645 < _list643.size; ++_i645)
+        org.apache.thrift.protocol.TList _list659 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list659.size);
+        ShowCompactResponseElement _elem660;
+        for (int _i661 = 0; _i661 < _list659.size; ++_i661)
         {
-          _elem644 = new ShowCompactResponseElement();
-          _elem644.read(iprot);
-          struct.compacts.add(_elem644);
+          _elem660 = new ShowCompactResponseElement();
+          _elem660.read(iprot);
+          struct.compacts.add(_elem660);
         }
       }
       struct.setCompactsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
index 02dd278..eff942b 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // LOCKS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list604 = iprot.readListBegin();
-                struct.locks = new ArrayList<ShowLocksResponseElement>(_list604.size);
-                ShowLocksResponseElement _elem605;
-                for (int _i606 = 0; _i606 < _list604.size; ++_i606)
+                org.apache.thrift.protocol.TList _list620 = iprot.readListBegin();
+                struct.locks = new ArrayList<ShowLocksResponseElement>(_list620.size);
+                ShowLocksResponseElement _elem621;
+                for (int _i622 = 0; _i622 < _list620.size; ++_i622)
                 {
-                  _elem605 = new ShowLocksResponseElement();
-                  _elem605.read(iprot);
-                  struct.locks.add(_elem605);
+                  _elem621 = new ShowLocksResponseElement();
+                  _elem621.read(iprot);
+                  struct.locks.add(_elem621);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(LOCKS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.locks.size()));
-          for (ShowLocksResponseElement _iter607 : struct.locks)
+          for (ShowLocksResponseElement _iter623 : struct.locks)
           {
-            _iter607.write(oprot);
+            _iter623.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetLocks()) {
         {
           oprot.writeI32(struct.locks.size());
-          for (ShowLocksResponseElement _iter608 : struct.locks)
+          for (ShowLocksResponseElement _iter624 : struct.locks)
           {
-            _iter608.write(oprot);
+            _iter624.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list609 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.locks = new ArrayList<ShowLocksResponseElement>(_list609.size);
-          ShowLocksResponseElement _elem610;
-          for (int _i611 = 0; _i611 < _list609.size; ++_i611)
+          org.apache.thrift.protocol.TList _list625 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.locks = new ArrayList<ShowLocksResponseElement>(_list625.size);
+          ShowLocksResponseElement _elem626;
+          for (int _i627 = 0; _i627 < _list625.size; ++_i627)
           {
-            _elem610 = new ShowLocksResponseElement();
-            _elem610.read(iprot);
-            struct.locks.add(_elem610);
+            _elem626 = new ShowLocksResponseElement();
+            _elem626.read(iprot);
+            struct.locks.add(_elem626);
           }
         }
         struct.setLocksIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
index 69be837..ad886b8 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
@@ -537,13 +537,13 @@ import org.slf4j.LoggerFactory;
           case 3: // COL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list420 = iprot.readListBegin();
-                struct.colNames = new ArrayList<String>(_list420.size);
-                String _elem421;
-                for (int _i422 = 0; _i422 < _list420.size; ++_i422)
+                org.apache.thrift.protocol.TList _list436 = iprot.readListBegin();
+                struct.colNames = new ArrayList<String>(_list436.size);
+                String _elem437;
+                for (int _i438 = 0; _i438 < _list436.size; ++_i438)
                 {
-                  _elem421 = iprot.readString();
-                  struct.colNames.add(_elem421);
+                  _elem437 = iprot.readString();
+                  struct.colNames.add(_elem437);
                 }
                 iprot.readListEnd();
               }
@@ -579,9 +579,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COL_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.colNames.size()));
-          for (String _iter423 : struct.colNames)
+          for (String _iter439 : struct.colNames)
           {
-            oprot.writeString(_iter423);
+            oprot.writeString(_iter439);
           }
           oprot.writeListEnd();
         }
@@ -608,9 +608,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.colNames.size());
-        for (String _iter424 : struct.colNames)
+        for (String _iter440 : struct.colNames)
         {
-          oprot.writeString(_iter424);
+          oprot.writeString(_iter440);
         }
       }
     }
@@ -623,13 +623,13 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list425 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.colNames = new ArrayList<String>(_list425.size);
-        String _elem426;
-        for (int _i427 = 0; _i427 < _list425.size; ++_i427)
+        org.apache.thrift.protocol.TList _list441 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.colNames = new ArrayList<String>(_list441.size);
+        String _elem442;
+        for (int _i443 = 0; _i443 < _list441.size; ++_i443)
         {
-          _elem426 = iprot.readString();
-          struct.colNames.add(_elem426);
+          _elem442 = iprot.readString();
+          struct.colNames.add(_elem442);
         }
       }
       struct.setColNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
index e65166e..64af8ba 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TABLE_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list394 = iprot.readListBegin();
-                struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list394.size);
-                ColumnStatisticsObj _elem395;
-                for (int _i396 = 0; _i396 < _list394.size; ++_i396)
+                org.apache.thrift.protocol.TList _list410 = iprot.readListBegin();
+                struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list410.size);
+                ColumnStatisticsObj _elem411;
+                for (int _i412 = 0; _i412 < _list410.size; ++_i412)
                 {
-                  _elem395 = new ColumnStatisticsObj();
-                  _elem395.read(iprot);
-                  struct.tableStats.add(_elem395);
+                  _elem411 = new ColumnStatisticsObj();
+                  _elem411.read(iprot);
+                  struct.tableStats.add(_elem411);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLE_STATS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tableStats.size()));
-          for (ColumnStatisticsObj _iter397 : struct.tableStats)
+          for (ColumnStatisticsObj _iter413 : struct.tableStats)
           {
-            _iter397.write(oprot);
+            _iter413.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tableStats.size());
-        for (ColumnStatisticsObj _iter398 : struct.tableStats)
+        for (ColumnStatisticsObj _iter414 : struct.tableStats)
         {
-          _iter398.write(oprot);
+          _iter414.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, TableStatsResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list399 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list399.size);
-        ColumnStatisticsObj _elem400;
-        for (int _i401 = 0; _i401 < _list399.size; ++_i401)
+        org.apache.thrift.protocol.TList _list415 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list415.size);
+        ColumnStatisticsObj _elem416;
+        for (int _i417 = 0; _i417 < _list415.size; ++_i417)
         {
-          _elem400 = new ColumnStatisticsObj();
-          _elem400.read(iprot);
-          struct.tableStats.add(_elem400);
+          _elem416 = new ColumnStatisticsObj();
+          _elem416.read(iprot);
+          struct.tableStats.add(_elem416);
         }
       }
       struct.setTableStatsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
index 1d43fb8..3f9b0d9 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
@@ -708,13 +708,13 @@ import org.slf4j.LoggerFactory;
           case 3: // INVALID_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list564 = iprot.readListBegin();
-                struct.invalidWriteIds = new ArrayList<Long>(_list564.size);
-                long _elem565;
-                for (int _i566 = 0; _i566 < _list564.size; ++_i566)
+                org.apache.thrift.protocol.TList _list580 = iprot.readListBegin();
+                struct.invalidWriteIds = new ArrayList<Long>(_list580.size);
+                long _elem581;
+                for (int _i582 = 0; _i582 < _list580.size; ++_i582)
                 {
-                  _elem565 = iprot.readI64();
-                  struct.invalidWriteIds.add(_elem565);
+                  _elem581 = iprot.readI64();
+                  struct.invalidWriteIds.add(_elem581);
                 }
                 iprot.readListEnd();
               }
@@ -764,9 +764,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(INVALID_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.invalidWriteIds.size()));
-          for (long _iter567 : struct.invalidWriteIds)
+          for (long _iter583 : struct.invalidWriteIds)
           {
-            oprot.writeI64(_iter567);
+            oprot.writeI64(_iter583);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeI64(struct.writeIdHighWaterMark);
       {
         oprot.writeI32(struct.invalidWriteIds.size());
-        for (long _iter568 : struct.invalidWriteIds)
+        for (long _iter584 : struct.invalidWriteIds)
         {
-          oprot.writeI64(_iter568);
+          oprot.writeI64(_iter584);
         }
       }
       oprot.writeBinary(struct.abortedBits);
@@ -827,13 +827,13 @@ import org.slf4j.LoggerFactory;
       struct.writeIdHighWaterMark = iprot.readI64();
       struct.setWriteIdHighWaterMarkIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list569 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.invalidWriteIds = new ArrayList<Long>(_list569.size);
-        long _elem570;
-        for (int _i571 = 0; _i571 < _list569.size; ++_i571)
+        org.apache.thrift.protocol.TList _list585 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.invalidWriteIds = new ArrayList<Long>(_list585.size);
+        long _elem586;
+        for (int _i587 = 0; _i587 < _list585.size; ++_i587)
         {
-          _elem570 = iprot.readI64();
-          struct.invalidWriteIds.add(_elem570);
+          _elem586 = iprot.readI64();
+          struct.invalidWriteIds.add(_elem586);
         }
       }
       struct.setInvalidWriteIdsIsSet(true);


[19/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)


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

Branch: refs/heads/master
Commit: 4047befe48c8f762c58d8854e058385c1df151c6
Parents: 05d4719
Author: Vineet Garg <vg...@apache.org>
Authored: Sat Mar 3 23:24:16 2018 -0800
Committer: Vineet Garg <vg...@apache.org>
Committed: Sat Mar 3 23:24:16 2018 -0800

----------------------------------------------------------------------
 .../listener/DummyRawStoreFailEvent.java        |    16 +-
 .../test/resources/testconfiguration.properties |     1 +
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |     3 +-
 .../upgrade/derby/hive-schema-3.0.0.derby.sql   |     2 +-
 .../upgrade/hive/hive-schema-3.0.0.hive.sql     |     4 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |    18 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |    12 +
 .../hive/ql/metadata/DefaultConstraint.java     |   121 +
 .../apache/hadoop/hive/ql/metadata/Hive.java    |    85 +-
 .../formatting/JsonMetaDataFormatter.java       |     6 +-
 .../formatting/MetaDataFormatUtils.java         |    43 +-
 .../metadata/formatting/MetaDataFormatter.java  |     3 +-
 .../formatting/TextMetaDataFormatter.java       |     8 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |   158 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |    66 +-
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |    16 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   132 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |     4 +
 .../message/AddNotNullConstraintHandler.java    |     3 +-
 .../hadoop/hive/ql/plan/AlterTableDesc.java     |    17 +-
 .../hadoop/hive/ql/plan/CreateTableDesc.java    |    20 +-
 .../hadoop/hive/ql/plan/ImportTableDesc.java    |     3 +-
 .../hive/ql/udf/generic/GenericUDFToChar.java   |     1 +
 .../alter_external_with_constraint.q            |     3 -
 .../alter_external_with_default_constraint.q    |     3 +
 .../alter_external_with_notnull_constraint.q    |     3 +
 .../alter_tableprops_external_with_constraint.q |     3 -
 ...ableprops_external_with_default_constraint.q |     3 +
 ...ableprops_external_with_notnull_constraint.q |     3 +
 .../clientnegative/constraint_duplicate_name.q  |     2 +
 .../clientnegative/constraint_invalide_name.q   |     3 +
 .../constraint_partition_columns.q              |     2 +
 .../create_external_with_constraint.q           |     1 -
 .../create_external_with_default_constraint.q   |     1 +
 .../create_external_with_notnull_constraint.q   |     1 +
 .../default_constraint_complex_default_value.q  |     2 +
 .../default_constraint_invalid_default_value.q  |     2 +
 .../default_constraint_invalid_default_value2.q |     2 +
 ...lt_constraint_invalid_default_value_length.q |     4 +
 ...ault_constraint_invalid_default_value_type.q |     2 +
 .../default_constraint_invalid_type.q           |     1 +
 .../clientpositive/create_with_constraints.q    |     1 +
 .../queries/clientpositive/default_constraint.q |   180 +
 .../clientpositive/enforce_constraint_notnull.q |     2 +-
 .../alter_external_with_constraint.q.out        |     9 -
 ...alter_external_with_default_constraint.q.out |     9 +
 ...alter_external_with_notnull_constraint.q.out |     9 +
 ...er_tableprops_external_with_constraint.q.out |     9 -
 ...props_external_with_default_constraint.q.out |     9 +
 ...props_external_with_notnull_constraint.q.out |     9 +
 .../constraint_duplicate_name.q.out             |    13 +
 .../constraint_invalide_name.q.out              |     1 +
 .../constraint_partition_columns.q.out          |     1 +
 .../create_external_with_constraint.q.out       |     1 -
 ...reate_external_with_default_constraint.q.out |     1 +
 ...reate_external_with_notnull_constraint.q.out |     1 +
 ...fault_constraint_complex_default_value.q.out |     1 +
 ...fault_constraint_invalid_default_value.q.out |     1 +
 ...ault_constraint_invalid_default_value2.q.out |     1 +
 ...onstraint_invalid_default_value_length.q.out |     1 +
 ..._constraint_invalid_default_value_type.q.out |     1 +
 .../default_constraint_invalid_type.q.out       |     1 +
 .../llap/default_constraint.q.out               |  2793 +++++
 .../llap/enforce_constraint_notnull.q.out       |     4 +-
 .../results/clientpositive/llap/lineage2.q.out  |     2 +-
 .../clientpositive/llap/resourceplan.q.out      |     8 +-
 .../clientpositive/llap/vector_char_4.q.out     |     2 +-
 .../vector_windowing_range_multiorder.q.out     |     8 +-
 .../clientpositive/llap/vectorized_casts.q.out  |     2 +-
 .../clientpositive/nonmr_fetch_threshold.q.out  |     2 +-
 .../results/clientpositive/show_functions.q.out |    10 +
 .../results/clientpositive/spark/union32.q.out  |     4 +-
 .../clientpositive/spark/vector_char_4.q.out    |     2 +-
 .../test/results/clientpositive/union32.q.out   |     4 +-
 .../results/clientpositive/vector_char_4.q.out  |     2 +-
 .../clientpositive/vectorized_casts.q.out       |     2 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  |  4244 ++++---
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |   302 +-
 .../ThriftHiveMetastore_server.skeleton.cpp     |    12 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  6541 +++++-----
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   221 +
 .../hive/metastore/api/AbortTxnsRequest.java    |    32 +-
 .../api/AddDefaultConstraintRequest.java        |   443 +
 .../metastore/api/AddDynamicPartitions.java     |    32 +-
 .../metastore/api/AddForeignKeyRequest.java     |    36 +-
 .../api/AddNotNullConstraintRequest.java        |    36 +-
 .../metastore/api/AddPartitionsRequest.java     |    36 +-
 .../hive/metastore/api/AddPartitionsResult.java |    36 +-
 .../metastore/api/AddPrimaryKeyRequest.java     |    36 +-
 .../api/AddUniqueConstraintRequest.java         |    36 +-
 .../api/AllocateTableWriteIdsRequest.java       |    32 +-
 .../api/AllocateTableWriteIdsResponse.java      |    36 +-
 .../metastore/api/ClearFileMetadataRequest.java |    32 +-
 .../hive/metastore/api/ClientCapabilities.java  |    32 +-
 .../hive/metastore/api/CompactionRequest.java   |    44 +-
 .../hive/metastore/api/CreationMetadata.java    |    32 +-
 .../api/DefaultConstraintsRequest.java          |   490 +
 .../api/DefaultConstraintsResponse.java         |   443 +
 .../metastore/api/DropPartitionsResult.java     |    36 +-
 .../hive/metastore/api/FireEventRequest.java    |    32 +-
 .../hadoop/hive/metastore/api/Function.java     |    36 +-
 .../metastore/api/GetAllFunctionsResponse.java  |    36 +-
 .../api/GetFileMetadataByExprRequest.java       |    32 +-
 .../api/GetFileMetadataByExprResult.java        |    48 +-
 .../metastore/api/GetFileMetadataRequest.java   |    32 +-
 .../metastore/api/GetFileMetadataResult.java    |    44 +-
 .../metastore/api/GetOpenTxnsInfoResponse.java  |    36 +-
 .../hive/metastore/api/GetOpenTxnsResponse.java |    32 +-
 .../hive/metastore/api/GetTablesRequest.java    |    32 +-
 .../hive/metastore/api/GetTablesResult.java     |    36 +-
 .../metastore/api/GetValidWriteIdsRequest.java  |    32 +-
 .../metastore/api/GetValidWriteIdsResponse.java |    36 +-
 .../api/HeartbeatTxnRangeResponse.java          |    64 +-
 .../metastore/api/InsertEventRequestData.java   |    64 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |    36 +-
 .../hive/metastore/api/Materialization.java     |    32 +-
 .../api/NotificationEventResponse.java          |    36 +-
 .../hive/metastore/api/OpenTxnsResponse.java    |    32 +-
 .../metastore/api/PartitionValuesRequest.java   |    72 +-
 .../metastore/api/PartitionValuesResponse.java  |    36 +-
 .../hive/metastore/api/PartitionValuesRow.java  |    32 +-
 .../metastore/api/PartitionsByExprResult.java   |    36 +-
 .../metastore/api/PartitionsStatsRequest.java   |    64 +-
 .../metastore/api/PartitionsStatsResult.java    |    76 +-
 .../metastore/api/PutFileMetadataRequest.java   |    64 +-
 .../hive/metastore/api/RequestPartsSpec.java    |    68 +-
 .../metastore/api/SQLDefaultConstraint.java     |  1109 ++
 .../hive/metastore/api/ShowCompactResponse.java |    36 +-
 .../hive/metastore/api/ShowLocksResponse.java   |    36 +-
 .../hive/metastore/api/TableStatsRequest.java   |    32 +-
 .../hive/metastore/api/TableStatsResult.java    |    36 +-
 .../hive/metastore/api/TableValidWriteIds.java  |    32 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 10893 ++++++++++-------
 .../hive/metastore/api/WMFullResourcePlan.java  |   144 +-
 .../api/WMGetAllResourcePlanResponse.java       |    36 +-
 .../WMGetTriggersForResourePlanResponse.java    |    36 +-
 .../api/WMValidateResourcePlanResponse.java     |    64 +-
 .../gen-php/metastore/ThriftHiveMetastore.php   |  2081 ++--
 .../src/gen/thrift/gen-php/metastore/Types.php  |  1840 ++-
 .../hive_metastore/ThriftHiveMetastore-remote   |    22 +-
 .../hive_metastore/ThriftHiveMetastore.py       |  1400 ++-
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  1278 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |    84 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   143 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |    97 +-
 .../hive/metastore/HiveMetaStoreClient.java     |    16 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |    11 +-
 .../hive/metastore/MetaStoreDirectSql.java      |    61 +
 .../hadoop/hive/metastore/ObjectStore.java      |   175 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |     9 +-
 .../hive/metastore/cache/CachedStore.java       |    20 +-
 .../hive/metastore/model/MConstraint.java       |    25 +
 .../main/sql/derby/hive-schema-3.0.0.derby.sql  |     2 +-
 .../sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql  |     4 +
 .../main/sql/mssql/hive-schema-3.0.0.mssql.sql  |     3 +-
 .../sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql  |     4 +
 .../main/sql/mysql/hive-schema-3.0.0.mysql.sql  |     1 +
 .../sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql  |     4 +
 .../sql/oracle/hive-schema-3.0.0.oracle.sql     |     3 +-
 .../oracle/upgrade-2.3.0-to-3.0.0.oracle.sql    |     5 +
 .../sql/postgres/hive-schema-3.0.0.postgres.sql |     1 +
 .../upgrade-2.3.0-to-3.0.0.postgres.sql         |     4 +
 .../src/main/thrift/hive_metastore.thrift       |    32 +-
 .../DummyRawStoreControlledCommit.java          |    18 +-
 .../DummyRawStoreForJdoConnection.java          |    18 +-
 165 files changed, 25512 insertions(+), 12588 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
index a3725c5..d0ff871 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
@@ -64,6 +64,7 @@ import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -948,10 +949,17 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
+  public List<SQLDefaultConstraint> getDefaultConstraints(String db_name, String tbl_name)
+      throws MetaException {
+    return null;
+  }
+
+  @Override
   public List<String> createTableWithConstraints(Table tbl,
                                          List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
                                          List<SQLUniqueConstraint> uniqueConstraints,
-                                         List<SQLNotNullConstraint> notNullConstraints)
+                                         List<SQLNotNullConstraint> notNullConstraints,
+                                         List<SQLDefaultConstraint> defaultConstraints)
       throws InvalidObjectException, MetaException {
     return null;
   }
@@ -986,6 +994,12 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
+  public List<String> addDefaultConstraints(List<SQLDefaultConstraint> nns)
+      throws InvalidObjectException, MetaException {
+    return null;
+  }
+
+  @Override
   public String getMetastoreDbUuid() throws MetaException {
     throw new MetaException("getMetastoreDbUuid is not implemented");
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 9d80920..e8aa827 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -534,6 +534,7 @@ minillaplocal.query.files=\
   correlationoptimizer2.q,\
   correlationoptimizer4.q,\
   correlationoptimizer6.q,\
+  default_constraint.q,\
   disable_merge_for_bucketing.q,\
   cross_prod_1.q,\
   cross_prod_3.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 255bd5f..07e6eaa 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -1715,7 +1715,8 @@ public class QTestUtil {
       "pk_-?[0-9]*_[0-9]*_[0-9]*",
       "fk_-?[0-9]*_[0-9]*_[0-9]*",
       "uk_-?[0-9]*_[0-9]*_[0-9]*",
-      "nn_-?[0-9]*_[0-9]*_[0-9]*",
+      "nn_-?[0-9]*_[0-9]*_[0-9]*", // not null constraint name
+      "dc_-?[0-9]*_[0-9]*_[0-9]*", // default constraint name
       ".*at com\\.sun\\.proxy.*",
       ".*at com\\.jolbox.*",
       ".*at com\\.zaxxer.*",

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/metastore/scripts/upgrade/derby/hive-schema-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/hive-schema-3.0.0.derby.sql b/metastore/scripts/upgrade/derby/hive-schema-3.0.0.derby.sql
index a8f227b..e335b84 100644
--- a/metastore/scripts/upgrade/derby/hive-schema-3.0.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/hive-schema-3.0.0.derby.sql
@@ -110,7 +110,7 @@ CREATE TABLE "APP"."NOTIFICATION_LOG" ("NL_ID" BIGINT NOT NULL, "DB_NAME" VARCHA
 
 CREATE TABLE "APP"."NOTIFICATION_SEQUENCE" ("NNI_ID" BIGINT NOT NULL, "NEXT_EVENT_ID" BIGINT NOT NULL);
 
-CREATE TABLE "APP"."KEY_CONSTRAINTS" ("CHILD_CD_ID" BIGINT, "CHILD_INTEGER_IDX" INTEGER, "CHILD_TBL_ID" BIGINT, "PARENT_CD_ID" BIGINT NOT NULL, "PARENT_INTEGER_IDX" INTEGER, "PARENT_TBL_ID" BIGINT NOT NULL,  "POSITION" BIGINT NOT NULL, "CONSTRAINT_NAME" VARCHAR(400) NOT NULL, "CONSTRAINT_TYPE" SMALLINT NOT NULL, "UPDATE_RULE" SMALLINT, "DELETE_RULE" SMALLINT, "ENABLE_VALIDATE_RELY" SMALLINT NOT NULL);
+CREATE TABLE "APP"."KEY_CONSTRAINTS" ("CHILD_CD_ID" BIGINT, "CHILD_INTEGER_IDX" INTEGER, "CHILD_TBL_ID" BIGINT, "PARENT_CD_ID" BIGINT NOT NULL, "PARENT_INTEGER_IDX" INTEGER, "PARENT_TBL_ID" BIGINT NOT NULL,  "POSITION" BIGINT NOT NULL, "CONSTRAINT_NAME" VARCHAR(400) NOT NULL, "CONSTRAINT_TYPE" SMALLINT NOT NULL, "UPDATE_RULE" SMALLINT, "DELETE_RULE" SMALLINT, "ENABLE_VALIDATE_RELY" SMALLINT NOT NULL, "DEFAULT_VALUE" VARCHAR(400));
 
 CREATE TABLE "APP"."METASTORE_DB_PROPERTIES" ("PROPERTY_KEY" VARCHAR(255) NOT NULL, "PROPERTY_VALUE" VARCHAR(1000) NOT NULL, "DESCRIPTION" VARCHAR(1000));
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/metastore/scripts/upgrade/hive/hive-schema-3.0.0.hive.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/hive/hive-schema-3.0.0.hive.sql b/metastore/scripts/upgrade/hive/hive-schema-3.0.0.hive.sql
index 84d523e..f4b2a2b 100644
--- a/metastore/scripts/upgrade/hive/hive-schema-3.0.0.hive.sql
+++ b/metastore/scripts/upgrade/hive/hive-schema-3.0.0.hive.sql
@@ -935,6 +935,7 @@ CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS`
   `UPDATE_RULE` string,
   `DELETE_RULE` string,
   `ENABLE_VALIDATE_RELY` int,
+  `DEFAULT_VALUE` string,
   CONSTRAINT `SYS_PK_KEY_CONSTRAINTS` PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
@@ -953,7 +954,8 @@ TBLPROPERTIES (
   \"CONSTRAINT_TYPE\",
   \"UPDATE_RULE\",
   \"DELETE_RULE\",
-  \"ENABLE_VALIDATE_RELY\"
+  \"ENABLE_VALIDATE_RELY\",
+  \"DEFAULT_VALUE\"
 FROM
   \"KEY_CONSTRAINTS\""
 );

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/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 f99178d..4fc0a93 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
@@ -92,6 +92,7 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -143,6 +144,7 @@ import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject;
 import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData;
 import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
 import org.apache.hadoop.hive.ql.metadata.CheckResult;
+import org.apache.hadoop.hive.ql.metadata.DefaultConstraint;
 import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -3617,11 +3619,13 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       ForeignKeyInfo fkInfo = null;
       UniqueConstraint ukInfo = null;
       NotNullConstraint nnInfo = null;
+      DefaultConstraint dInfo = null;
       if (descTbl.isExt() || descTbl.isFormatted()) {
         pkInfo = db.getPrimaryKeys(tbl.getDbName(), tbl.getTableName());
         fkInfo = db.getForeignKeys(tbl.getDbName(), tbl.getTableName());
         ukInfo = db.getUniqueConstraints(tbl.getDbName(), tbl.getTableName());
         nnInfo = db.getNotNullConstraints(tbl.getDbName(), tbl.getTableName());
+        dInfo = db.getDefaultConstraints(tbl.getDbName(), tbl.getTableName());
       }
       fixDecimalColumnTypeName(cols);
       // In case the query is served by HiveServer2, don't pad it with spaces,
@@ -3630,7 +3634,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       formatter.describeTable(outStream, colPath, tableName, tbl, part,
           cols, descTbl.isFormatted(), descTbl.isExt(),
           isOutputPadded, colStats,
-          pkInfo, fkInfo, ukInfo, nnInfo);
+          pkInfo, fkInfo, ukInfo, nnInfo, dInfo);
 
       LOG.debug("DDLTask: written data for {}", tableName);
 
@@ -4424,6 +4428,10 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
               && !alterTbl.getNotNullConstraintCols().isEmpty()) {
         db.addNotNullConstraint(alterTbl.getNotNullConstraintCols());
       }
+      if (alterTbl.getDefaultConstraintCols() != null
+          && !alterTbl.getDefaultConstraintCols().isEmpty()) {
+        db.addDefaultConstraint(alterTbl.getDefaultConstraintCols());
+      }
     } catch (NoSuchObjectException e) {
       throw new HiveException(e);
     }
@@ -4746,6 +4754,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     List<SQLForeignKey> foreignKeys = crtTbl.getForeignKeys();
     List<SQLUniqueConstraint> uniqueConstraints = crtTbl.getUniqueConstraints();
     List<SQLNotNullConstraint> notNullConstraints = crtTbl.getNotNullConstraints();
+    List<SQLDefaultConstraint> defaultConstraints = crtTbl.getDefaultConstraints();
     LOG.debug("creating table {} on {}",tbl.getFullyQualifiedName(),tbl.getDataLocation());
 
     if (crtTbl.getReplicationSpec().isInReplicationScope() && (!crtTbl.getReplaceMode())){
@@ -4771,12 +4780,13 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       // replace-mode creates are really alters using CreateTableDesc.
       db.alterTable(tbl, null);
     } else {
-      if ((foreignKeys != null && foreignKeys.size() > 0 ) ||
+      if ((foreignKeys != null && foreignKeys.size() > 0) ||
           (primaryKeys != null && primaryKeys.size() > 0) ||
           (uniqueConstraints != null && uniqueConstraints.size() > 0) ||
-          (notNullConstraints != null && notNullConstraints.size() > 0)) {
+          (notNullConstraints != null && notNullConstraints.size() > 0) ||
+          defaultConstraints != null && defaultConstraints.size() > 0) {
         db.createTable(tbl, crtTbl.getIfNotExists(), primaryKeys, foreignKeys,
-                uniqueConstraints, notNullConstraints);
+                uniqueConstraints, notNullConstraints, defaultConstraints);
       } else {
         db.createTable(tbl, crtTbl.getIfNotExists());
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/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 32fc257..77e9263 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
@@ -396,6 +396,18 @@ public final class FunctionRegistry {
     system.registerUDF(serdeConstants.FLOAT_TYPE_NAME, UDFToFloat.class, false, UDFToFloat.class.getSimpleName());
     system.registerUDF(serdeConstants.DOUBLE_TYPE_NAME, UDFToDouble.class, false, UDFToDouble.class.getSimpleName());
     system.registerUDF(serdeConstants.STRING_TYPE_NAME, UDFToString.class, false, UDFToString.class.getSimpleName());
+    // following mapping is to enable UDFName to UDF while generating expression for default value (in operator tree)
+    //  e.g. cast(4 as string) is serialized as UDFToString(4) into metastore, to allow us to generate appropriate UDF for
+    //  UDFToString we need the following mappings
+    // Rest of the types e.g. DATE, CHAR, VARCHAR etc are already registered
+    system.registerUDF(UDFToString.class.getSimpleName(), UDFToString.class, false, UDFToString.class.getSimpleName());
+    system.registerUDF(UDFToBoolean.class.getSimpleName(), UDFToBoolean.class, false, UDFToBoolean.class.getSimpleName());
+    system.registerUDF(UDFToDouble.class.getSimpleName(), UDFToDouble.class, false, UDFToDouble.class.getSimpleName());
+    system.registerUDF(UDFToFloat.class.getSimpleName(), UDFToFloat.class, false, UDFToFloat.class.getSimpleName());
+    system.registerUDF(UDFToInteger.class.getSimpleName(), UDFToInteger.class, false, UDFToInteger.class.getSimpleName());
+    system.registerUDF(UDFToLong.class.getSimpleName(), UDFToLong.class, false, UDFToLong.class.getSimpleName());
+    system.registerUDF(UDFToShort.class.getSimpleName(), UDFToShort.class, false, UDFToShort.class.getSimpleName());
+    system.registerUDF(UDFToByte.class.getSimpleName(), UDFToByte.class, false, UDFToByte.class.getSimpleName());
 
     system.registerGenericUDF(serdeConstants.DATE_TYPE_NAME, GenericUDFToDate.class);
     system.registerGenericUDF(serdeConstants.TIMESTAMP_TYPE_NAME, GenericUDFTimestamp.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/java/org/apache/hadoop/hive/ql/metadata/DefaultConstraint.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/DefaultConstraint.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/DefaultConstraint.java
new file mode 100644
index 0000000..59df3da
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/DefaultConstraint.java
@@ -0,0 +1,121 @@
+/*
+ * 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.metadata;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
+
+/**
+ * DefaultConstraintInfo is a metadata structure containing the default constraints
+ * associated with a table.
+ */
+@SuppressWarnings("serial")
+public class DefaultConstraint implements Serializable {
+
+  public class DefaultConstraintCol {
+    public String colName;
+    public String defaultVal;
+
+    public DefaultConstraintCol(String colName, String defaultVal) {
+      this.colName = colName;
+      this.defaultVal = defaultVal;
+    }
+  }
+
+  // Mapping from constraint name to list of default constraints
+  Map<String, List<DefaultConstraintCol>> defaultConstraints;
+
+  // Mapping from column name to default value
+  Map<String, String> colNameToDefaultValueMap;
+  String tableName;
+  String databaseName;
+
+  public DefaultConstraint() {}
+
+  public DefaultConstraint(List<SQLDefaultConstraint> defaultConstraintList, String tableName, String databaseName) {
+    this.tableName = tableName;
+    this.databaseName = databaseName;
+    defaultConstraints = new TreeMap<String, List<DefaultConstraintCol>>();
+    colNameToDefaultValueMap = new TreeMap<String, String>();
+    if (defaultConstraintList == null) {
+      return;
+    }
+    for (SQLDefaultConstraint uk : defaultConstraintList) {
+      if (uk.getTable_db().equalsIgnoreCase(databaseName) &&
+          uk.getTable_name().equalsIgnoreCase(tableName)) {
+        String colName = uk.getColumn_name();
+        String defVal = uk.getDefault_value();
+        colNameToDefaultValueMap.put(colName, defVal);
+        DefaultConstraintCol currCol = new DefaultConstraintCol(
+                colName, defVal);
+        String constraintName = uk.getDc_name();
+        if (defaultConstraints.containsKey(constraintName)) {
+          defaultConstraints.get(constraintName).add(currCol);
+        } else {
+          List<DefaultConstraintCol> currList = new ArrayList<DefaultConstraintCol>();
+          currList.add(currCol);
+          defaultConstraints.put(constraintName, currList);
+        }
+      }
+    }
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public String getDatabaseName() {
+    return databaseName;
+  }
+
+  public Map<String, List<DefaultConstraintCol>> getDefaultConstraints() {
+    return defaultConstraints;
+  }
+  public Map<String, String> getColNameToDefaultValueMap() {
+    return colNameToDefaultValueMap;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Default Constraints for " + databaseName + "." + tableName + ":");
+    sb.append("[");
+    if (defaultConstraints != null && defaultConstraints.size() > 0) {
+      for (Map.Entry<String, List<DefaultConstraintCol>> me : defaultConstraints.entrySet()) {
+        sb.append(" {Constraint Name: " + me.getKey() + ",");
+        List<DefaultConstraintCol> currCol = me.getValue();
+        if (currCol != null && currCol.size() > 0) {
+          for (DefaultConstraintCol ukc : currCol) {
+            sb.append (" (Column Name: " + ukc.colName + ", Default Value: " + ukc.defaultVal + "),");
+          }
+          sb.setLength(sb.length()-1);
+        }
+        sb.append("},");
+      }
+      sb.setLength(sb.length()-1);
+    }
+    sb.append("]");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/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 baa9070..68a87e6 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
@@ -18,6 +18,12 @@
 
 package org.apache.hadoop.hive.ql.metadata;
 
+import com.google.common.base.Splitter;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE;
 import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT;
 import static org.apache.hadoop.hive.serde.serdeConstants.STRING_TYPE_NAME;
@@ -93,6 +99,7 @@ import org.apache.hadoop.hive.metastore.api.CompactionResponse;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DefaultConstraintsRequest;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.FireEventRequest;
@@ -118,6 +125,7 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -126,7 +134,6 @@ import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.UniqueConstraintsRequest;
-import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMNullablePool;
@@ -844,13 +851,20 @@ public class Hive {
    *          primary key columns associated with the table
    * @param foreignKeys
    *          foreign key columns associated with the table
+   * @param uniqueConstraints
+   *          UNIQUE constraints associated with the table
+   * @param notNullConstraints
+   *          NOT NULL constraints associated with the table
+   * @param defaultConstraints
+   *          DEFAULT constraints associated with the table
    * @throws HiveException
    */
   public void createTable(Table tbl, boolean ifNotExists,
     List<SQLPrimaryKey> primaryKeys,
     List<SQLForeignKey> foreignKeys,
     List<SQLUniqueConstraint> uniqueConstraints,
-    List<SQLNotNullConstraint> notNullConstraints)
+    List<SQLNotNullConstraint> notNullConstraints,
+    List<SQLDefaultConstraint> defaultConstraints)
             throws HiveException {
     try {
       if (tbl.getDbName() == null || "".equals(tbl.getDbName().trim())) {
@@ -877,11 +891,11 @@ public class Hive {
         }
       }
       if (primaryKeys == null && foreignKeys == null
-              && uniqueConstraints == null && notNullConstraints == null) {
+              && uniqueConstraints == null && notNullConstraints == null && defaultConstraints == null) {
         getMSC().createTable(tTbl);
       } else {
         getMSC().createTableWithConstraints(tTbl, primaryKeys, foreignKeys,
-            uniqueConstraints, notNullConstraints);
+            uniqueConstraints, notNullConstraints, defaultConstraints);
       }
 
     } catch (AlreadyExistsException e) {
@@ -894,7 +908,7 @@ public class Hive {
   }
 
   public void createTable(Table tbl, boolean ifNotExists) throws HiveException {
-   createTable(tbl, ifNotExists, null, null, null, null);
+   createTable(tbl, ifNotExists, null, null, null, null, null);
  }
 
   public static List<FieldSchema> getFieldsFromDeserializerForMsStorage(
@@ -4451,6 +4465,18 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
   }
 
+  public List<SQLDefaultConstraint> getDefaultConstraintList(String dbName, String tblName) throws HiveException, NoSuchObjectException {
+    try {
+      return getMSC().getDefaultConstraints(new DefaultConstraintsRequest(dbName, tblName));
+    } catch (NoSuchObjectException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
+  /**
+
   /**
    * Get all primary key columns associated with the table.
    *
@@ -4617,6 +4643,30 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
   }
 
+  /**
+   * Get Default constraints associated with the table that are enabled
+   *
+   * @param dbName Database Name
+   * @param tblName Table Name
+   * @return Default constraints associated with the table.
+   * @throws HiveException
+   */
+  public DefaultConstraint getEnabledDefaultConstraints(String dbName, String tblName)
+      throws HiveException {
+    try {
+      List<SQLDefaultConstraint> defaultConstraints = getMSC().getDefaultConstraints(
+          new DefaultConstraintsRequest(dbName, tblName));
+      if (defaultConstraints != null && !defaultConstraints.isEmpty()) {
+        defaultConstraints = defaultConstraints.stream()
+            .filter(nnc -> nnc.isEnable_cstr())
+            .collect(Collectors.toList());
+      }
+      return new DefaultConstraint(defaultConstraints, tblName, dbName);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
   private NotNullConstraint getNotNullConstraints(String dbName, String tblName, boolean onlyReliable)
       throws HiveException {
     try {
@@ -4633,6 +4683,21 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
   }
 
+  public DefaultConstraint getDefaultConstraints(String dbName, String tblName)
+      throws HiveException {
+    try {
+      List<SQLDefaultConstraint> defaultConstraints = getMSC().getDefaultConstraints(
+          new DefaultConstraintsRequest(dbName, tblName));
+      if (defaultConstraints != null && !defaultConstraints.isEmpty()) {
+        defaultConstraints = defaultConstraints.stream()
+            .collect(Collectors.toList());
+      }
+      return new DefaultConstraint(defaultConstraints, tblName, dbName);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
   public void addPrimaryKey(List<SQLPrimaryKey> primaryKeyCols)
     throws HiveException, NoSuchObjectException {
     try {
@@ -4669,6 +4734,16 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
   }
 
+  public void addDefaultConstraint(List<SQLDefaultConstraint> defaultConstraints)
+      throws HiveException, NoSuchObjectException {
+    try {
+      getMSC().addDefaultConstraint(defaultConstraints);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
+
   public void createResourcePlan(WMResourcePlan resourcePlan, String copyFromName)
       throws HiveException {
     try {

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
index 77e5678..da82f68 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
+import org.apache.hadoop.hive.ql.metadata.DefaultConstraint;
 import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -114,7 +115,7 @@ public class JsonMetaDataFormatter implements MetaDataFormatter {
       boolean isFormatted, boolean isExt,
       boolean isOutputPadded, List<ColumnStatisticsObj> colStats,
       PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo,
-      UniqueConstraint ukInfo, NotNullConstraint nnInfo) throws HiveException {
+      UniqueConstraint ukInfo, NotNullConstraint nnInfo, DefaultConstraint dInfo) throws HiveException {
     MapBuilder builder = MapBuilder.create();
     builder.put("columns", makeColsUnformatted(cols));
 
@@ -137,6 +138,9 @@ public class JsonMetaDataFormatter implements MetaDataFormatter {
       if (nnInfo != null && !nnInfo.getNotNullConstraints().isEmpty()) {
         builder.put("notNullConstraintInfo", nnInfo);
       }
+      if (dInfo != null && !dInfo.getDefaultConstraints().isEmpty()) {
+        builder.put("defaultConstraintInfo", dInfo);
+      }
     }
 
     asJson(out, builder.build());

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
index a5b6a4b..bfc7b38 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
@@ -41,15 +41,16 @@ import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.api.WMPoolTrigger;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.ql.metadata.DefaultConstraint;
 import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.NotNullConstraint;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.PrimaryKeyInfo;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.metadata.UniqueConstraint;
 import org.apache.hadoop.hive.ql.metadata.UniqueConstraint.UniqueConstraintCol;
 import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo.ForeignKeyCol;
-import org.apache.hadoop.hive.ql.metadata.NotNullConstraint;
 import org.apache.hadoop.hive.ql.plan.DescTableDesc;
 import org.apache.hadoop.hive.ql.plan.PlanUtils;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
@@ -133,7 +134,7 @@ public final class MetaDataFormatUtils {
   }
 
   public static String getConstraintsInformation(PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo,
-          UniqueConstraint ukInfo, NotNullConstraint nnInfo) {
+          UniqueConstraint ukInfo, NotNullConstraint nnInfo, DefaultConstraint dInfo) {
     StringBuilder constraintsInfo = new StringBuilder(DEFAULT_STRINGBUILDER_SIZE);
 
     constraintsInfo.append(LINE_DELIM).append("# Constraints").append(LINE_DELIM);
@@ -153,6 +154,10 @@ public final class MetaDataFormatUtils {
       constraintsInfo.append(LINE_DELIM).append("# Not Null Constraints").append(LINE_DELIM);
       getNotNullConstraintsInformation(constraintsInfo, nnInfo);
     }
+    if (dInfo != null && !dInfo.getDefaultConstraints().isEmpty()) {
+      constraintsInfo.append(LINE_DELIM).append("# Default Constraints").append(LINE_DELIM);
+      getDefaultConstraintsInformation(constraintsInfo, dInfo);
+    }
     return constraintsInfo.toString();
   }
 
@@ -253,6 +258,40 @@ public final class MetaDataFormatUtils {
     }
   }
 
+  private static void getDefaultConstraintColInformation(StringBuilder constraintsInfo,
+                                                        DefaultConstraint.DefaultConstraintCol ukCol) {
+    String[] fkcFields = new String[2];
+    fkcFields[0] = "Column Name:" + ukCol.colName;
+    fkcFields[1] = "Default Value:" + ukCol.defaultVal;
+    formatOutput(fkcFields, constraintsInfo);
+  }
+  private static void getDefaultConstraintRelInformation(
+      StringBuilder constraintsInfo,
+      String constraintName,
+      List<DefaultConstraint.DefaultConstraintCol> ukRel) {
+    formatOutput("Constraint Name:", constraintName, constraintsInfo);
+    if (ukRel != null && ukRel.size() > 0) {
+      for (DefaultConstraint.DefaultConstraintCol ukc : ukRel) {
+        getDefaultConstraintColInformation(constraintsInfo, ukc);
+      }
+    }
+    constraintsInfo.append(LINE_DELIM);
+  }
+
+  private static void getDefaultConstraintsInformation(StringBuilder constraintsInfo,
+                                                        DefaultConstraint dInfo) {
+    formatOutput("Table:",
+        dInfo.getDatabaseName() + "." + dInfo.getTableName(),
+        constraintsInfo);
+    Map<String, List<DefaultConstraint.DefaultConstraintCol>> defaultConstraints = dInfo.getDefaultConstraints();
+    if (defaultConstraints != null && defaultConstraints.size() > 0) {
+      for (Map.Entry<String, List<DefaultConstraint.DefaultConstraintCol>> me : defaultConstraints.entrySet()) {
+        getDefaultConstraintRelInformation(constraintsInfo, me.getKey(), me.getValue());
+      }
+    }
+  }
+
+
   public static String getPartitionInformation(Partition part) {
     StringBuilder tableInfo = new StringBuilder(DEFAULT_STRINGBUILDER_SIZE);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java
index 88d5554..6309bfd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
+import org.apache.hadoop.hive.ql.metadata.DefaultConstraint;
 import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -89,7 +90,7 @@ public interface MetaDataFormatter {
       boolean isFormatted, boolean isExt,
       boolean isOutputPadded, List<ColumnStatisticsObj> colStats,
       PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo,
-      UniqueConstraint ukInfo, NotNullConstraint nnInfo)
+      UniqueConstraint ukInfo, NotNullConstraint nnInfo, DefaultConstraint dInfo)
           throws HiveException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
index 607e111..0065848 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.metadata.DefaultConstraint;
 import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -130,7 +131,7 @@ class TextMetaDataFormatter implements MetaDataFormatter {
       boolean isFormatted, boolean isExt,
       boolean isOutputPadded, List<ColumnStatisticsObj> colStats,
       PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo,
-      UniqueConstraint ukInfo, NotNullConstraint nnInfo) throws HiveException {
+      UniqueConstraint ukInfo, NotNullConstraint nnInfo, DefaultConstraint dInfo) throws HiveException {
     try {
       List<FieldSchema> partCols = tbl.isPartitioned() ? tbl.getPartCols() : null;
       String output = "";
@@ -187,8 +188,9 @@ class TextMetaDataFormatter implements MetaDataFormatter {
           if ((pkInfo != null && !pkInfo.getColNames().isEmpty()) ||
               (fkInfo != null && !fkInfo.getForeignKeys().isEmpty()) ||
               (ukInfo != null && !ukInfo.getUniqueConstraints().isEmpty()) ||
-              (nnInfo != null && !nnInfo.getNotNullConstraints().isEmpty())) {
-            output = MetaDataFormatUtils.getConstraintsInformation(pkInfo, fkInfo, ukInfo, nnInfo);
+              (nnInfo != null && !nnInfo.getNotNullConstraints().isEmpty()) ||
+              dInfo != null && !dInfo.getDefaultConstraints().isEmpty()) {
+            output = MetaDataFormatUtils.getConstraintsInformation(pkInfo, fkInfo, ukInfo, nnInfo, dInfo);
             outStream.write(output.getBytes("UTF-8"));
           }
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
index 171825e..c3c029e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -58,6 +59,7 @@ import org.apache.hadoop.hive.ql.QueryState;
 import org.apache.hadoop.hive.ql.cache.results.CacheUsage;
 import org.apache.hadoop.hive.ql.cache.results.QueryResultsCache;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -76,6 +78,7 @@ import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.optimizer.listbucketingpruner.ListBucketingPrunerUtils;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.plan.FetchWork;
 import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
 import org.apache.hadoop.hive.ql.plan.ListBucketingCtx;
@@ -83,6 +86,10 @@ import org.apache.hadoop.hive.ql.plan.PlanUtils;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCurrentDate;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCurrentTimestamp;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCurrentUser;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNull;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
@@ -643,7 +650,8 @@ public abstract class BaseSemanticAnalyzer {
    */
   public static List<FieldSchema> getColumns(ASTNode ast, boolean lowerCase) throws SemanticException {
     return getColumns(ast, lowerCase, new ArrayList<SQLPrimaryKey>(), new ArrayList<SQLForeignKey>(),
-            new ArrayList<SQLUniqueConstraint>(), new ArrayList<SQLNotNullConstraint>());
+            new ArrayList<SQLUniqueConstraint>(), new ArrayList<SQLNotNullConstraint>(),
+        new ArrayList<SQLDefaultConstraint>());
   }
 
   private static class ConstraintInfo {
@@ -652,6 +660,7 @@ public abstract class BaseSemanticAnalyzer {
     final boolean enable;
     final boolean validate;
     final boolean rely;
+    final String defaultValue;
 
     ConstraintInfo(String colName, String constraintName,
         boolean enable, boolean validate, boolean rely) {
@@ -660,6 +669,16 @@ public abstract class BaseSemanticAnalyzer {
       this.enable = enable;
       this.validate = validate;
       this.rely = rely;
+      this.defaultValue = null;
+    }
+    ConstraintInfo(String colName, String constraintName,
+                   boolean enable, boolean validate, boolean rely, String defaultValue) {
+      this.colName = colName;
+      this.constraintName = constraintName;
+      this.enable = enable;
+      this.validate = validate;
+      this.rely = rely;
+      this.defaultValue = defaultValue;
     }
   }
 
@@ -677,7 +696,7 @@ public abstract class BaseSemanticAnalyzer {
       ASTNode child, List<String> columnNames, List<SQLPrimaryKey> primaryKeys)
           throws SemanticException {
     List<ConstraintInfo> primaryKeyInfos = new ArrayList<ConstraintInfo>();
-    generateConstraintInfos(child, columnNames, primaryKeyInfos);
+    generateConstraintInfos(child, columnNames, primaryKeyInfos, null);
     constraintInfosToPrimaryKeys(databaseName, tableName, primaryKeyInfos, primaryKeys);
   }
 
@@ -705,7 +724,7 @@ public abstract class BaseSemanticAnalyzer {
       ASTNode child, List<String> columnNames, List<SQLUniqueConstraint> uniqueConstraints)
           throws SemanticException {
     List<ConstraintInfo> uniqueInfos = new ArrayList<ConstraintInfo>();
-    generateConstraintInfos(child, columnNames, uniqueInfos);
+    generateConstraintInfos(child, columnNames, uniqueInfos, null);
     constraintInfosToUniqueConstraints(databaseName, tableName, uniqueInfos, uniqueConstraints);
   }
 
@@ -718,11 +737,28 @@ public abstract class BaseSemanticAnalyzer {
     }
   }
 
+  protected static void processDefaultConstraints(String databaseName, String tableName,
+      ASTNode child, List<String> columnNames, List<SQLDefaultConstraint> defaultConstraints, final ASTNode typeChild)
+      throws SemanticException {
+    List<ConstraintInfo> defaultInfos = new ArrayList<ConstraintInfo>();
+    generateConstraintInfos(child, columnNames, defaultInfos, typeChild);
+    constraintInfosToDefaultConstraints(databaseName, tableName, defaultInfos, defaultConstraints);
+  }
+
+  private static void constraintInfosToDefaultConstraints(String databaseName, String tableName,
+     List<ConstraintInfo> defaultInfos, List<SQLDefaultConstraint> defaultConstraints) {
+    for (ConstraintInfo defaultInfo : defaultInfos) {
+      defaultConstraints.add(new SQLDefaultConstraint(databaseName, tableName, defaultInfo.colName,
+          defaultInfo.defaultValue, defaultInfo.constraintName, defaultInfo.enable,
+          defaultInfo.validate, defaultInfo.rely));
+    }
+  }
+
   protected static void processNotNullConstraints(String databaseName, String tableName,
       ASTNode child, List<String> columnNames, List<SQLNotNullConstraint> notNullConstraints)
           throws SemanticException {
     List<ConstraintInfo> notNullInfos = new ArrayList<ConstraintInfo>();
-    generateConstraintInfos(child, columnNames, notNullInfos);
+    generateConstraintInfos(child, columnNames, notNullInfos, null);
     constraintInfosToNotNullConstraints(databaseName, tableName, notNullInfos, notNullConstraints);
   }
 
@@ -749,19 +785,87 @@ public abstract class BaseSemanticAnalyzer {
       checkColumnName(columnName.getText());
       columnNames.add(unescapeIdentifier(columnName.getText().toLowerCase()));
     }
-    generateConstraintInfos(child, columnNames.build(), cstrInfos);
+    generateConstraintInfos(child, columnNames.build(), cstrInfos, null);
+  }
+
+  private static boolean isDefaultValueAllowed(final ExprNodeDesc defaultValExpr) {
+    if(defaultValExpr instanceof ExprNodeConstantDesc) {
+      return true;
+    }
+    else if(FunctionRegistry.isOpCast(defaultValExpr)) {
+      return isDefaultValueAllowed(defaultValExpr.getChildren().get(0));
+    }
+    else if(defaultValExpr instanceof ExprNodeGenericFuncDesc){
+      ExprNodeGenericFuncDesc defFunc = (ExprNodeGenericFuncDesc)defaultValExpr;
+      if(defFunc.getGenericUDF() instanceof GenericUDFOPNull
+          || defFunc.getGenericUDF() instanceof GenericUDFCurrentTimestamp
+          || defFunc.getGenericUDF() instanceof GenericUDFCurrentDate
+          || defFunc.getGenericUDF() instanceof GenericUDFCurrentUser){
+        return true;
+      }
+    }
+    return false;
   }
 
   /**
+   * Validate and get the default value from the AST
+   * @param defaultValueAST AST node corresponding to default value
+   * @return retrieve the default value and return it as string
+   * @throws SemanticException
+   */
+  private static String getDefaultValue(ASTNode defaultValueAST, ASTNode typeChild) throws SemanticException{
+    // first create expression from defaultValueAST
+    TypeCheckCtx typeCheckCtx = new TypeCheckCtx(null);
+    ExprNodeDesc defaultValExpr = TypeCheckProcFactory
+        .genExprNode(defaultValueAST, typeCheckCtx).get(defaultValueAST);
+
+    if(defaultValExpr == null) {
+      throw new SemanticException(
+          ErrorMsg.INVALID_CSTR_SYNTAX.getMsg("Invalid Default value!"));
+    }
+
+    //get default value to be be stored in metastore
+    String defaultValueText  = defaultValExpr.getExprString();
+    final int DEFAULT_MAX_LEN = 255;
+    if(defaultValueText.length() > DEFAULT_MAX_LEN) {
+      throw new SemanticException(
+          ErrorMsg.INVALID_CSTR_SYNTAX.getMsg("Invalid Default value:  " + defaultValueText +
+                                                  " .Maximum character length allowed is " + DEFAULT_MAX_LEN +" ."));
+    }
+
+    // Make sure the default value expression type is exactly same as column's type.
+    TypeInfo defaultValTypeInfo = defaultValExpr.getTypeInfo();
+    TypeInfo colTypeInfo = TypeInfoUtils.getTypeInfoFromTypeString(getTypeStringFromAST(typeChild));
+    if(!defaultValTypeInfo.equals(colTypeInfo)) {
+      throw new SemanticException(
+          ErrorMsg.INVALID_CSTR_SYNTAX.getMsg("Invalid type: " + defaultValTypeInfo.getTypeName()
+                                                  + " for default value: "
+                                                  + defaultValueText
+                                                  + ". Please make sure that the type is compatible with column type: "
+                                                  + colTypeInfo.getTypeName()));
+    }
+
+    // throw an error if default value isn't what hive allows
+    if(!isDefaultValueAllowed(defaultValExpr)) {
+      throw new SemanticException(
+          ErrorMsg.INVALID_CSTR_SYNTAX.getMsg("Invalid Default value: " + defaultValueText
+                                                  + ". DEFAULT only allows constant or function expressions"));
+    }
+    return defaultValueText;
+  }
+
+
+  /**
    * Get the constraint from the AST and populate the cstrInfos with the required
    * information.
    * @param child  The node with the constraint token
    * @param columnNames The name of the columns for the primary key
    * @param cstrInfos Constraint information
+   * @param typeChildForDefault type of column used for default value type check
    * @throws SemanticException
    */
   private static void generateConstraintInfos(ASTNode child, List<String> columnNames,
-      List<ConstraintInfo> cstrInfos) throws SemanticException {
+      List<ConstraintInfo> cstrInfos, ASTNode typeChildForDefault) throws SemanticException {
     // The ANTLR grammar looks like :
     // 1. KW_CONSTRAINT idfr=identifier KW_PRIMARY KW_KEY pkCols=columnParenthesesList
     //  constraintOptsCreate?
@@ -773,9 +877,12 @@ public abstract class BaseSemanticAnalyzer {
     // when the user does not specify the constraint name.
     // Default values
     String constraintName = null;
+    //by default if user hasn't provided any optional constraint properties
+    // it will be considered ENABLE and NOVALIDATE and RELY=false
     boolean enable = true;
-    boolean validate = true;
+    boolean validate = false;
     boolean rely = false;
+    String defaultValue = null;
     for (int i = 0; i < child.getChildCount(); i++) {
       ASTNode grandChild = (ASTNode) child.getChild(i);
       int type = grandChild.getToken().getType();
@@ -797,12 +904,25 @@ public abstract class BaseSemanticAnalyzer {
         validate = false;
       } else if (type == HiveParser.TOK_RELY) {
         rely = true;
+      } else if( type == HiveParser.TOK_NORELY) {
+        rely = false;
+      } else if( child.getToken().getType() == HiveParser.TOK_DEFAULT_VALUE){
+        // try to get default value only if this is DEFAULT constraint
+        defaultValue = getDefaultValue(grandChild, typeChildForDefault);
       }
     }
 
+    // metastore schema only allows maximum 255 for constraint name column
+    final int CONSTRAINT_NAME_MAX_LENGTH = 255;
+    if(constraintName != null && constraintName.length() > CONSTRAINT_NAME_MAX_LENGTH) {
+      throw new SemanticException(
+          ErrorMsg.INVALID_CSTR_SYNTAX.getMsg("Constraint name: " + constraintName + " exceeded maximum allowed "
+                                              + "length: " + CONSTRAINT_NAME_MAX_LENGTH ));
+    }
+
     // NOT NULL constraint could be enforced/enabled
-    if (child.getToken().getType() != HiveParser.TOK_NOT_NULL
-        && enable) {
+    if (enable && child.getToken().getType() != HiveParser.TOK_NOT_NULL
+        && child.getToken().getType() != HiveParser.TOK_DEFAULT_VALUE) {
       throw new SemanticException(
           ErrorMsg.INVALID_CSTR_SYNTAX.getMsg("ENABLE/ENFORCED feature not supported yet. "
               + "Please use DISABLE/NOT ENFORCED instead."));
@@ -815,13 +935,12 @@ public abstract class BaseSemanticAnalyzer {
 
     for (String columnName : columnNames) {
       cstrInfos.add(new ConstraintInfo(columnName, constraintName,
-          enable, validate, rely));
+          enable, validate, rely, defaultValue));
     }
   }
 
   /**
    * Process the foreign keys from the AST and populate the foreign keys in the SQLForeignKey list
-   * @param parent  Parent of the foreign key token node
    * @param child Foreign Key token node
    * @param foreignKeys SQLForeignKey list
    * @throws SemanticException
@@ -906,7 +1025,8 @@ public abstract class BaseSemanticAnalyzer {
     }
   }
 
-  protected boolean hasEnabledOrValidatedConstraints(List<SQLNotNullConstraint> notNullConstraints){
+  protected boolean hasEnabledOrValidatedConstraints(List<SQLNotNullConstraint> notNullConstraints,
+                                                     List<SQLDefaultConstraint> defaultConstraints){
     if(notNullConstraints != null) {
       for (SQLNotNullConstraint nnC : notNullConstraints) {
         if (nnC.isEnable_cstr() || nnC.isValidate_cstr()) {
@@ -914,6 +1034,13 @@ public abstract class BaseSemanticAnalyzer {
         }
       }
     }
+    if(defaultConstraints!= null) {
+      for (SQLDefaultConstraint nnC : defaultConstraints) {
+        if (nnC.isEnable_cstr() || nnC.isValidate_cstr()) {
+          return true;
+        }
+      }
+    }
     return false;
   }
 
@@ -929,7 +1056,8 @@ public abstract class BaseSemanticAnalyzer {
    */
   public static List<FieldSchema> getColumns(ASTNode ast, boolean lowerCase,
     List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
-    List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints)
+    List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints,
+                                           List<SQLDefaultConstraint> defaultConstraints)
         throws SemanticException {
     List<FieldSchema> colList = new ArrayList<FieldSchema>();
     Tree parent = ast.getParent();
@@ -987,6 +1115,10 @@ public abstract class BaseSemanticAnalyzer {
               String[] qualifiedTabName = getQualifiedTableName((ASTNode) parent.getChild(0));
               // Process column constraint
               switch (constraintChild.getToken().getType()) {
+              case HiveParser.TOK_DEFAULT_VALUE:
+                processDefaultConstraints(qualifiedTabName[0], qualifiedTabName[1], constraintChild,
+                    ImmutableList.of(col.getName()), defaultConstraints, typeChild);
+                break;
                 case HiveParser.TOK_NOT_NULL:
                   processNotNullConstraints(qualifiedTabName[0], qualifiedTabName[1], constraintChild,
                           ImmutableList.of(col.getName()), notNullConstraints);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/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 e393424..1793402 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
@@ -53,6 +53,7 @@ import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -86,6 +87,7 @@ import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
 import org.apache.hadoop.hive.ql.lockmgr.LockException;
 import org.apache.hadoop.hive.ql.lockmgr.TxnManagerFactory;
+import org.apache.hadoop.hive.ql.metadata.DefaultConstraint;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
@@ -1652,10 +1654,13 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
   private boolean hasConstraintsEnabled(final String tblName) throws SemanticException{
 
     NotNullConstraint nnc = null;
+    DefaultConstraint dc = null;
     try {
       // retrieve enabled NOT NULL constraint from metastore
       nnc = Hive.get().getEnabledNotNullConstraints(
           db.getDatabaseCurrent().getName(), tblName);
+      dc = Hive.get().getEnabledDefaultConstraints(
+          db.getDatabaseCurrent().getName(), tblName);
     } catch (Exception e) {
       if (e instanceof SemanticException) {
         throw (SemanticException) e;
@@ -1663,7 +1668,8 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         throw (new RuntimeException(e));
       }
     }
-    if(nnc != null  && !nnc.getNotNullConstraints().isEmpty()) {
+    if((nnc != null  && !nnc.getNotNullConstraints().isEmpty())
+        || (dc != null && !dc.getDefaultConstraints().isEmpty())) {
       return true;
     }
     return false;
@@ -3109,32 +3115,38 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     List<SQLForeignKey> foreignKeys = null;
     List<SQLUniqueConstraint> uniqueConstraints = null;
     List<SQLNotNullConstraint> notNullConstraints = null;
+    List<SQLDefaultConstraint> defaultConstraints= null;
     if (constraintChild != null) {
       // Process column constraint
       switch (constraintChild.getToken().getType()) {
-        case HiveParser.TOK_NOT_NULL:
-          notNullConstraints = new ArrayList<>();
-          processNotNullConstraints(qualified[0], qualified[1], constraintChild,
-                  ImmutableList.of(newColName), notNullConstraints);
-          break;
-        case HiveParser.TOK_UNIQUE:
-          uniqueConstraints = new ArrayList<>();
-          processUniqueConstraints(qualified[0], qualified[1], constraintChild,
-                  ImmutableList.of(newColName), uniqueConstraints);
-          break;
-        case HiveParser.TOK_PRIMARY_KEY:
-          primaryKeys = new ArrayList<>();
-          processPrimaryKeys(qualified[0], qualified[1], constraintChild,
-                  ImmutableList.of(newColName), primaryKeys);
-          break;
-        case HiveParser.TOK_FOREIGN_KEY:
-          foreignKeys = new ArrayList<>();
-          processForeignKeys(qualified[0], qualified[1], constraintChild,
-                  foreignKeys);
-          break;
-        default:
-          throw new SemanticException(ErrorMsg.NOT_RECOGNIZED_CONSTRAINT.getMsg(
-              constraintChild.getToken().getText()));
+      case HiveParser.TOK_DEFAULT_VALUE:
+        defaultConstraints = new ArrayList<>();
+        processDefaultConstraints(qualified[0], qualified[1], constraintChild,
+                                  ImmutableList.of(newColName), defaultConstraints, (ASTNode)ast.getChild(2));
+        break;
+      case HiveParser.TOK_NOT_NULL:
+        notNullConstraints = new ArrayList<>();
+        processNotNullConstraints(qualified[0], qualified[1], constraintChild,
+                                  ImmutableList.of(newColName), notNullConstraints);
+        break;
+      case HiveParser.TOK_UNIQUE:
+        uniqueConstraints = new ArrayList<>();
+        processUniqueConstraints(qualified[0], qualified[1], constraintChild,
+                                 ImmutableList.of(newColName), uniqueConstraints);
+        break;
+      case HiveParser.TOK_PRIMARY_KEY:
+        primaryKeys = new ArrayList<>();
+        processPrimaryKeys(qualified[0], qualified[1], constraintChild,
+                           ImmutableList.of(newColName), primaryKeys);
+        break;
+      case HiveParser.TOK_FOREIGN_KEY:
+        foreignKeys = new ArrayList<>();
+        processForeignKeys(qualified[0], qualified[1], constraintChild,
+                           foreignKeys);
+        break;
+      default:
+        throw new SemanticException(ErrorMsg.NOT_RECOGNIZED_CONSTRAINT.getMsg(
+            constraintChild.getToken().getText()));
       }
     }
 
@@ -3142,7 +3154,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     Table tab = getTable(qualified);
 
     if(tab.getTableType() == TableType.EXTERNAL_TABLE
-        && hasEnabledOrValidatedConstraints(notNullConstraints)){
+        && hasEnabledOrValidatedConstraints(notNullConstraints, defaultConstraints)){
       throw new SemanticException(
           ErrorMsg.INVALID_CSTR_SYNTAX.getMsg("Constraints are disallowed with External tables. "
               + "Only RELY is allowed."));
@@ -3159,7 +3171,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     String tblName = getDotName(qualified);
     AlterTableDesc alterTblDesc;
     if (primaryKeys == null && foreignKeys == null
-            && uniqueConstraints == null && notNullConstraints == null) {
+            && uniqueConstraints == null && notNullConstraints == null && defaultConstraints == null) {
       alterTblDesc = new AlterTableDesc(tblName, partSpec,
           unescapeIdentifier(oldColName), unescapeIdentifier(newColName),
           newType, newComment, first, flagCol, isCascade);
@@ -3167,7 +3179,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       alterTblDesc = new AlterTableDesc(tblName, partSpec,
           unescapeIdentifier(oldColName), unescapeIdentifier(newColName),
           newType, newComment, first, flagCol, isCascade,
-          primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints);
+          primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints);
     }
     addInputsOutputsAlterTable(tblName, partSpec, alterTblDesc);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/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 733ec79..0c6aece 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
@@ -109,6 +109,7 @@ TOK_NOT_NULL;
 TOK_UNIQUE;
 TOK_PRIMARY_KEY;
 TOK_FOREIGN_KEY;
+TOK_DEFAULT_VALUE;
 TOK_VALIDATE;
 TOK_NOVALIDATE;
 TOK_RELY;
@@ -2371,8 +2372,8 @@ colConstraint
 @after { popMsg(state); }
     : (KW_CONSTRAINT constraintName=identifier)? columnConstraintType constraintOptsCreate?
     -> {$constraintName.tree != null}?
-            ^(columnConstraintType ^(TOK_CONSTRAINT_NAME $constraintName) constraintOptsCreate?)
-    -> ^(columnConstraintType constraintOptsCreate?)
+            ^({$columnConstraintType.tree} ^(TOK_CONSTRAINT_NAME $constraintName) constraintOptsCreate?)
+    -> ^({$columnConstraintType.tree} constraintOptsCreate?)
     ;
 
 alterColumnConstraint[CommonTree fkColName]
@@ -2396,15 +2397,22 @@ alterColConstraint
 @after { popMsg(state); }
     : (KW_CONSTRAINT constraintName=identifier)? columnConstraintType constraintOptsAlter?
     -> {$constraintName.tree != null}?
-            ^(columnConstraintType ^(TOK_CONSTRAINT_NAME $constraintName) constraintOptsAlter?)
-    -> ^(columnConstraintType constraintOptsAlter?)
+            ^({$columnConstraintType.tree} ^(TOK_CONSTRAINT_NAME $constraintName) constraintOptsAlter?)
+    -> ^({$columnConstraintType.tree} constraintOptsAlter?)
     ;
 
 columnConstraintType
     : KW_NOT KW_NULL       ->    TOK_NOT_NULL
+    | KW_DEFAULT defaultVal->    ^(TOK_DEFAULT_VALUE defaultVal)
     | tableConstraintType
     ;
 
+defaultVal
+    : constant
+    | function
+    | castExpression
+    ;
+
 tableConstraintType
     : KW_PRIMARY KW_KEY    ->    TOK_PRIMARY_KEY
     | KW_UNIQUE            ->    TOK_UNIQUE

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index eeab111..7f446ca 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -76,6 +76,7 @@ import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -131,6 +132,7 @@ import org.apache.hadoop.hive.ql.lib.Dispatcher;
 import org.apache.hadoop.hive.ql.lib.GraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+import org.apache.hadoop.hive.ql.metadata.DefaultConstraint;
 import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
 import org.apache.hadoop.hive.ql.lockmgr.LockException;
 import org.apache.hadoop.hive.ql.metadata.DummyPartition;
@@ -4354,6 +4356,70 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     return output;
   }
 
+  private RowResolver getColForInsertStmtSpec(Map<String, ExprNodeDesc> targetCol2Projection, final Table target,
+                                              Map<String, ColumnInfo> targetCol2ColumnInfo, int colListPos,
+                                              List<TypeInfo> targetTableColTypes, ArrayList<ExprNodeDesc> new_col_list,
+                                              List<String> targetTableColNames)
+      throws SemanticException {
+    RowResolver newOutputRR = new RowResolver();
+    Map<String, String> colNameToDefaultVal = null;
+
+    // see if we need to fetch default constraints from metastore
+    if(targetCol2Projection.size() < targetTableColNames.size()) {
+      try {
+          DefaultConstraint dc = Hive.get().getEnabledDefaultConstraints(target.getDbName(), target.getTableName());
+          colNameToDefaultVal = dc.getColNameToDefaultValueMap();
+      } catch (Exception e) {
+        if (e instanceof SemanticException) {
+          throw (SemanticException) e;
+        } else {
+          throw (new RuntimeException(e));
+        }
+      }
+
+    }
+    boolean defaultConstraintsFetch = true;
+    for (int i = 0; i < targetTableColNames.size(); i++) {
+      String f = targetTableColNames.get(i);
+      if(targetCol2Projection.containsKey(f)) {
+        //put existing column in new list to make sure it is in the right position
+        new_col_list.add(targetCol2Projection.get(f));
+        ColumnInfo ci = targetCol2ColumnInfo.get(f);
+        ci.setInternalName(getColumnInternalName(colListPos));
+        newOutputRR.put(ci.getTabAlias(), ci.getInternalName(), ci);
+      }
+      else {
+        //add new 'synthetic' columns for projections not provided by Select
+        assert(colNameToDefaultVal != null);
+        ExprNodeDesc exp = null;
+        if(colNameToDefaultVal.containsKey(f)) {
+          // make an expression for default value
+          String defaultValue = colNameToDefaultVal.get(f);
+          ParseDriver parseDriver = new ParseDriver();
+          try {
+            ASTNode defValAst = parseDriver.parseExpression(defaultValue);
+
+            exp = TypeCheckProcFactory.genExprNode(defValAst, new TypeCheckCtx(null)).get(defValAst);
+          } catch(Exception e) {
+            throw new SemanticException("Error while parsing default value: " + defaultValue
+              + ". Error message: " + e.getMessage());
+          }
+          LOG.debug("Added default value from metastore: " + exp);
+        }
+        else {
+          exp = new ExprNodeConstantDesc(targetTableColTypes.get(i), null);
+        }
+        new_col_list.add(exp);
+        final String tableAlias = null;//this column doesn't come from any table
+        ColumnInfo colInfo = new ColumnInfo(getColumnInternalName(colListPos),
+                                            exp.getWritableObjectInspector(), tableAlias, false);
+        newOutputRR.put(colInfo.getTabAlias(), colInfo.getInternalName(), colInfo);
+      }
+      colListPos++;
+    }
+    return newOutputRR;
+  }
+
   /**
    * This modifies the Select projections when the Select is part of an insert statement and
    * the insert statement specifies a column list for the target table, e.g.
@@ -4422,29 +4488,12 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         }
       }
     }
-    RowResolver newOutputRR = new RowResolver();
+
     //now make the select produce <regular columns>,<dynamic partition columns> with
     //where missing columns are NULL-filled
-    for (int i = 0; i < targetTableColNames.size(); i++) {
-      String f = targetTableColNames.get(i);
-      if(targetCol2Projection.containsKey(f)) {
-        //put existing column in new list to make sure it is in the right position
-        new_col_list.add(targetCol2Projection.get(f));
-        ColumnInfo ci = targetCol2ColumnInfo.get(f);//todo: is this OK?
-        ci.setInternalName(getColumnInternalName(colListPos));
-        newOutputRR.put(ci.getTabAlias(), ci.getInternalName(), ci);
-      }
-      else {
-        //add new 'synthetic' columns for projections not provided by Select
-        ExprNodeDesc exp = new ExprNodeConstantDesc(targetTableColTypes.get(i), null);
-        new_col_list.add(exp);
-        final String tableAlias = null;//this column doesn't come from any table
-        ColumnInfo colInfo = new ColumnInfo(getColumnInternalName(colListPos),
-            exp.getWritableObjectInspector(), tableAlias, false);
-        newOutputRR.put(colInfo.getTabAlias(), colInfo.getInternalName(), colInfo);
-      }
-      colListPos++;
-    }
+    Table tbl = target == null? partition.getTable() : target;
+    RowResolver newOutputRR =  getColForInsertStmtSpec(targetCol2Projection, tbl, targetCol2ColumnInfo, colListPos,
+                                                       targetTableColTypes, new_col_list, targetTableColNames);
     col_list.clear();
     col_list.addAll(new_col_list);
     return newOutputRR;
@@ -12420,6 +12469,30 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   }
 
   /**
+   * Checks to see if given partition columns has DEFAULT constraints (whether ENABLED or DISABLED)
+   *  Or has NOT NULL constraints (only ENABLED)
+   * @param partCols partition columns
+   * @param defConstraints default constraints
+   * @param notNullConstraints not null constraints
+   * @return
+   */
+  boolean hasConstraints(final List<FieldSchema> partCols, final List<SQLDefaultConstraint> defConstraints,
+                         final List<SQLNotNullConstraint> notNullConstraints) {
+    for(FieldSchema partFS: partCols) {
+      for(SQLDefaultConstraint dc:defConstraints) {
+        if(dc.getColumn_name().equals(partFS.getName())) {
+          return true;
+        }
+      }
+      for(SQLNotNullConstraint nc:notNullConstraints) {
+        if(nc.getColumn_name().equals(partFS.getName()) && nc.isEnable_cstr()) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+  /**
    * Analyze the create table command. If it is a regular create-table or
    * create-table-like statements, we create a DDLWork and return true. If it is
    * a create-table-as-select, we get the necessary info such as the SerDe and
@@ -12440,6 +12513,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     List<SQLForeignKey> foreignKeys = new ArrayList<SQLForeignKey>();
     List<SQLUniqueConstraint> uniqueConstraints = new ArrayList<>();
     List<SQLNotNullConstraint> notNullConstraints = new ArrayList<>();
+    List<SQLDefaultConstraint> defaultConstraints= new ArrayList<>();
     List<Order> sortCols = new ArrayList<Order>();
     int numBuckets = -1;
     String comment = null;
@@ -12533,14 +12607,20 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         break;
       case HiveParser.TOK_TABCOLLIST:
         cols = getColumns(child, true, primaryKeys, foreignKeys,
-            uniqueConstraints, notNullConstraints);
+            uniqueConstraints, notNullConstraints, defaultConstraints);
         break;
       case HiveParser.TOK_TABLECOMMENT:
         comment = unescapeSQLString(child.getChild(0).getText());
         break;
       case HiveParser.TOK_TABLEPARTCOLS:
         partCols = getColumns(child, false, primaryKeys, foreignKeys,
-            uniqueConstraints, notNullConstraints);
+            uniqueConstraints, notNullConstraints, defaultConstraints);
+        if(hasConstraints(partCols, defaultConstraints, notNullConstraints)) {
+          //TODO: these constraints should be supported for partition columns
+          throw new SemanticException(
+              ErrorMsg.INVALID_CSTR_SYNTAX.getMsg("NOT NULL and Default Constraints are not allowed with " +
+                                                      "partition columns. "));
+        }
         break;
       case HiveParser.TOK_ALTERTABLE_BUCKETS:
         bucketCols = getColumnNames((ASTNode) child.getChild(0));
@@ -12598,7 +12678,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       throw new SemanticException("Unrecognized command.");
     }
 
-    if(isExt && hasEnabledOrValidatedConstraints(notNullConstraints)){
+    if(isExt && hasEnabledOrValidatedConstraints(notNullConstraints, defaultConstraints)){
       throw new SemanticException(
           ErrorMsg.INVALID_CSTR_SYNTAX.getMsg("Constraints are disallowed with External tables. "
               + "Only RELY is allowed."));
@@ -12656,7 +12736,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           comment,
           storageFormat.getInputFormat(), storageFormat.getOutputFormat(), location, storageFormat.getSerde(),
           storageFormat.getStorageHandler(), storageFormat.getSerdeProps(), tblProps, ifNotExists, skewedColNames,
-          skewedValues, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints);
+          skewedValues, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints);
       crtTblDesc.setStoredAsSubDirectories(storedAsDirs);
       crtTblDesc.setNullFormat(rowFormatParams.nullFormat);
 
@@ -12755,7 +12835,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           storageFormat.getOutputFormat(), location, storageFormat.getSerde(),
           storageFormat.getStorageHandler(), storageFormat.getSerdeProps(), tblProps, ifNotExists,
           skewedColNames, skewedValues, true, primaryKeys, foreignKeys,
-          uniqueConstraints, notNullConstraints);
+          uniqueConstraints, notNullConstraints, defaultConstraints);
       tableDesc.setMaterialization(isMaterialization);
       tableDesc.setStoredAsSubDirectories(storedAsDirs);
       tableDesc.setNullFormat(rowFormatParams.nullFormat);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
index 1a7e7e3..3e7b3a1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
@@ -638,6 +638,10 @@ public class TypeCheckProcFactory {
       ASTNode expr = (ASTNode) nd;
       ASTNode parent = stack.size() > 1 ? (ASTNode) stack.get(stack.size() - 2) : null;
       RowResolver input = ctx.getInputRR();
+      if(input == null) {
+        ctx.setError(ErrorMsg.INVALID_COLUMN.getMsg(expr), expr);
+        return null;
+      }
 
       if (expr.getType() != HiveParser.TOK_TABLE_OR_COL) {
         ctx.setError(ErrorMsg.INVALID_COLUMN.getMsg(expr), expr);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddNotNullConstraintHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddNotNullConstraintHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddNotNullConstraintHandler.java
index 9c12e7e..3425858 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddNotNullConstraintHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddNotNullConstraintHandler.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -64,7 +65,7 @@ public class AddNotNullConstraintHandler extends AbstractMessageHandler {
     }
 
     AlterTableDesc addConstraintsDesc = new AlterTableDesc(actualDbName + "." + actualTblName, new ArrayList<SQLPrimaryKey>(), new ArrayList<SQLForeignKey>(),
-        new ArrayList<SQLUniqueConstraint>(), nns, context.eventOnlyReplicationSpec());
+        new ArrayList<SQLUniqueConstraint>(), nns, new ArrayList<SQLDefaultConstraint>(), context.eventOnlyReplicationSpec());
     Task<DDLWork> addConstraintsTask = TaskFactory.get(new DDLWork(readEntitySet, writeEntitySet, addConstraintsDesc), context.hiveConf);
     tasks.add(addConstraintsTask);
     context.log.debug("Added add constrains task : {}:{}", addConstraintsTask.getId(), actualTblName);


[08/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index adddd07..9a97d6b 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -78,7 +78,7 @@ import org.slf4j.LoggerFactory;
 
     public void create_table_with_environment_context(Table tbl, EnvironmentContext environment_context) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException;
 
-    public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException;
+    public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints, List<SQLDefaultConstraint> defaultConstraints) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException;
 
     public void drop_constraint(DropConstraintRequest req) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
 
@@ -90,6 +90,8 @@ import org.slf4j.LoggerFactory;
 
     public void add_not_null_constraint(AddNotNullConstraintRequest req) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
 
+    public void add_default_constraint(AddDefaultConstraintRequest req) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
     public void drop_table(String dbname, String name, boolean deleteData) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
 
     public void drop_table_with_environment_context(String dbname, String name, boolean deleteData, EnvironmentContext environment_context) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
@@ -232,6 +234,8 @@ import org.slf4j.LoggerFactory;
 
     public NotNullConstraintsResponse get_not_null_constraints(NotNullConstraintsRequest request) throws MetaException, NoSuchObjectException, org.apache.thrift.TException;
 
+    public DefaultConstraintsResponse get_default_constraints(DefaultConstraintsRequest request) throws MetaException, NoSuchObjectException, org.apache.thrift.TException;
+
     public boolean update_table_column_statistics(ColumnStatistics stats_obj) throws NoSuchObjectException, InvalidObjectException, MetaException, InvalidInputException, org.apache.thrift.TException;
 
     public boolean update_partition_column_statistics(ColumnStatistics stats_obj) throws NoSuchObjectException, InvalidObjectException, MetaException, InvalidInputException, org.apache.thrift.TException;
@@ -450,7 +454,7 @@ import org.slf4j.LoggerFactory;
 
     public void create_table_with_environment_context(Table tbl, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
-    public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+    public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints, List<SQLDefaultConstraint> defaultConstraints, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void drop_constraint(DropConstraintRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
@@ -462,6 +466,8 @@ import org.slf4j.LoggerFactory;
 
     public void add_not_null_constraint(AddNotNullConstraintRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void add_default_constraint(AddDefaultConstraintRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void drop_table(String dbname, String name, boolean deleteData, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void drop_table_with_environment_context(String dbname, String name, boolean deleteData, EnvironmentContext environment_context, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -604,6 +610,8 @@ import org.slf4j.LoggerFactory;
 
     public void get_not_null_constraints(NotNullConstraintsRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void get_default_constraints(DefaultConstraintsRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void update_table_column_statistics(ColumnStatistics stats_obj, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void update_partition_column_statistics(ColumnStatistics stats_obj, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -1336,13 +1344,13 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
-    public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException
+    public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints, List<SQLDefaultConstraint> defaultConstraints) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException
     {
-      send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints);
+      send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints);
       recv_create_table_with_constraints();
     }
 
-    public void send_create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints) throws org.apache.thrift.TException
+    public void send_create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints, List<SQLDefaultConstraint> defaultConstraints) throws org.apache.thrift.TException
     {
       create_table_with_constraints_args args = new create_table_with_constraints_args();
       args.setTbl(tbl);
@@ -1350,6 +1358,7 @@ import org.slf4j.LoggerFactory;
       args.setForeignKeys(foreignKeys);
       args.setUniqueConstraints(uniqueConstraints);
       args.setNotNullConstraints(notNullConstraints);
+      args.setDefaultConstraints(defaultConstraints);
       sendBase("create_table_with_constraints", args);
     }
 
@@ -1502,6 +1511,32 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
+    public void add_default_constraint(AddDefaultConstraintRequest req) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_add_default_constraint(req);
+      recv_add_default_constraint();
+    }
+
+    public void send_add_default_constraint(AddDefaultConstraintRequest req) throws org.apache.thrift.TException
+    {
+      add_default_constraint_args args = new add_default_constraint_args();
+      args.setReq(req);
+      sendBase("add_default_constraint", args);
+    }
+
+    public void recv_add_default_constraint() throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      add_default_constraint_result result = new add_default_constraint_result();
+      receiveBase(result, "add_default_constraint");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      return;
+    }
+
     public void drop_table(String dbname, String name, boolean deleteData) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
     {
       send_drop_table(dbname, name, deleteData);
@@ -3691,6 +3726,35 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_not_null_constraints failed: unknown result");
     }
 
+    public DefaultConstraintsResponse get_default_constraints(DefaultConstraintsRequest request) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
+    {
+      send_get_default_constraints(request);
+      return recv_get_default_constraints();
+    }
+
+    public void send_get_default_constraints(DefaultConstraintsRequest request) throws org.apache.thrift.TException
+    {
+      get_default_constraints_args args = new get_default_constraints_args();
+      args.setRequest(request);
+      sendBase("get_default_constraints", args);
+    }
+
+    public DefaultConstraintsResponse recv_get_default_constraints() throws MetaException, NoSuchObjectException, org.apache.thrift.TException
+    {
+      get_default_constraints_result result = new get_default_constraints_result();
+      receiveBase(result, "get_default_constraints");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_default_constraints failed: unknown result");
+    }
+
     public boolean update_table_column_statistics(ColumnStatistics stats_obj) throws NoSuchObjectException, InvalidObjectException, MetaException, InvalidInputException, org.apache.thrift.TException
     {
       send_update_table_column_statistics(stats_obj);
@@ -6762,9 +6826,9 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints, List<SQLDefaultConstraint> defaultConstraints, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      create_table_with_constraints_call method_call = new create_table_with_constraints_call(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, resultHandler, this, ___protocolFactory, ___transport);
+      create_table_with_constraints_call method_call = new create_table_with_constraints_call(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
@@ -6775,13 +6839,15 @@ import org.slf4j.LoggerFactory;
       private List<SQLForeignKey> foreignKeys;
       private List<SQLUniqueConstraint> uniqueConstraints;
       private List<SQLNotNullConstraint> notNullConstraints;
-      public create_table_with_constraints_call(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private List<SQLDefaultConstraint> defaultConstraints;
+      public create_table_with_constraints_call(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints, List<SQLDefaultConstraint> defaultConstraints, 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.tbl = tbl;
         this.primaryKeys = primaryKeys;
         this.foreignKeys = foreignKeys;
         this.uniqueConstraints = uniqueConstraints;
         this.notNullConstraints = notNullConstraints;
+        this.defaultConstraints = defaultConstraints;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
@@ -6792,6 +6858,7 @@ import org.slf4j.LoggerFactory;
         args.setForeignKeys(foreignKeys);
         args.setUniqueConstraints(uniqueConstraints);
         args.setNotNullConstraints(notNullConstraints);
+        args.setDefaultConstraints(defaultConstraints);
         args.write(prot);
         prot.writeMessageEnd();
       }
@@ -6966,6 +7033,38 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public void add_default_constraint(AddDefaultConstraintRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      add_default_constraint_call method_call = new add_default_constraint_call(req, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_default_constraint_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private AddDefaultConstraintRequest req;
+      public add_default_constraint_call(AddDefaultConstraintRequest req, 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.req = req;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("add_default_constraint", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        add_default_constraint_args args = new add_default_constraint_args();
+        args.setReq(req);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NoSuchObjectException, 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);
+        (new Client(prot)).recv_add_default_constraint();
+      }
+    }
+
     public void drop_table(String dbname, String name, boolean deleteData, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       drop_table_call method_call = new drop_table_call(dbname, name, deleteData, resultHandler, this, ___protocolFactory, ___transport);
@@ -9637,6 +9736,38 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public void get_default_constraints(DefaultConstraintsRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_default_constraints_call method_call = new get_default_constraints_call(request, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_default_constraints_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private DefaultConstraintsRequest request;
+      public get_default_constraints_call(DefaultConstraintsRequest request, 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.request = request;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_default_constraints", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_default_constraints_args args = new get_default_constraints_args();
+        args.setRequest(request);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public DefaultConstraintsResponse getResult() throws MetaException, NoSuchObjectException, 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_default_constraints();
+      }
+    }
+
     public void update_table_column_statistics(ColumnStatistics stats_obj, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       update_table_column_statistics_call method_call = new update_table_column_statistics_call(stats_obj, resultHandler, this, ___protocolFactory, ___transport);
@@ -12588,6 +12719,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("add_foreign_key", new add_foreign_key());
       processMap.put("add_unique_constraint", new add_unique_constraint());
       processMap.put("add_not_null_constraint", new add_not_null_constraint());
+      processMap.put("add_default_constraint", new add_default_constraint());
       processMap.put("drop_table", new drop_table());
       processMap.put("drop_table_with_environment_context", new drop_table_with_environment_context());
       processMap.put("truncate_table", new truncate_table());
@@ -12659,6 +12791,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("get_foreign_keys", new get_foreign_keys());
       processMap.put("get_unique_constraints", new get_unique_constraints());
       processMap.put("get_not_null_constraints", new get_not_null_constraints());
+      processMap.put("get_default_constraints", new get_default_constraints());
       processMap.put("update_table_column_statistics", new update_table_column_statistics());
       processMap.put("update_partition_column_statistics", new update_partition_column_statistics());
       processMap.put("get_table_column_statistics", new get_table_column_statistics());
@@ -13249,7 +13382,7 @@ import org.slf4j.LoggerFactory;
       public create_table_with_constraints_result getResult(I iface, create_table_with_constraints_args args) throws org.apache.thrift.TException {
         create_table_with_constraints_result result = new create_table_with_constraints_result();
         try {
-          iface.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys, args.uniqueConstraints, args.notNullConstraints);
+          iface.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys, args.uniqueConstraints, args.notNullConstraints, args.defaultConstraints);
         } catch (AlreadyExistsException o1) {
           result.o1 = o1;
         } catch (InvalidObjectException o2) {
@@ -13393,6 +13526,32 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_default_constraint<I extends Iface> extends org.apache.thrift.ProcessFunction<I, add_default_constraint_args> {
+      public add_default_constraint() {
+        super("add_default_constraint");
+      }
+
+      public add_default_constraint_args getEmptyArgsInstance() {
+        return new add_default_constraint_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public add_default_constraint_result getResult(I iface, add_default_constraint_args args) throws org.apache.thrift.TException {
+        add_default_constraint_result result = new add_default_constraint_result();
+        try {
+          iface.add_default_constraint(args.req);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (MetaException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_table<I extends Iface> extends org.apache.thrift.ProcessFunction<I, drop_table_args> {
       public drop_table() {
         super("drop_table");
@@ -15273,6 +15432,32 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_default_constraints<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_default_constraints_args> {
+      public get_default_constraints() {
+        super("get_default_constraints");
+      }
+
+      public get_default_constraints_args getEmptyArgsInstance() {
+        return new get_default_constraints_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_default_constraints_result getResult(I iface, get_default_constraints_args args) throws org.apache.thrift.TException {
+        get_default_constraints_result result = new get_default_constraints_result();
+        try {
+          result.success = iface.get_default_constraints(args.request);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        } catch (NoSuchObjectException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class update_table_column_statistics<I extends Iface> extends org.apache.thrift.ProcessFunction<I, update_table_column_statistics_args> {
       public update_table_column_statistics() {
         super("update_table_column_statistics");
@@ -17514,6 +17699,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("add_foreign_key", new add_foreign_key());
       processMap.put("add_unique_constraint", new add_unique_constraint());
       processMap.put("add_not_null_constraint", new add_not_null_constraint());
+      processMap.put("add_default_constraint", new add_default_constraint());
       processMap.put("drop_table", new drop_table());
       processMap.put("drop_table_with_environment_context", new drop_table_with_environment_context());
       processMap.put("truncate_table", new truncate_table());
@@ -17585,6 +17771,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("get_foreign_keys", new get_foreign_keys());
       processMap.put("get_unique_constraints", new get_unique_constraints());
       processMap.put("get_not_null_constraints", new get_not_null_constraints());
+      processMap.put("get_default_constraints", new get_default_constraints());
       processMap.put("update_table_column_statistics", new update_table_column_statistics());
       processMap.put("update_partition_column_statistics", new update_partition_column_statistics());
       processMap.put("get_table_column_statistics", new get_table_column_statistics());
@@ -18886,7 +19073,7 @@ import org.slf4j.LoggerFactory;
       }
 
       public void start(I iface, create_table_with_constraints_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys, args.uniqueConstraints, args.notNullConstraints,resultHandler);
+        iface.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys, args.uniqueConstraints, args.notNullConstraints, args.defaultConstraints,resultHandler);
       }
     }
 
@@ -19195,6 +19382,67 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_default_constraint<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_default_constraint_args, Void> {
+      public add_default_constraint() {
+        super("add_default_constraint");
+      }
+
+      public add_default_constraint_args getEmptyArgsInstance() {
+        return new add_default_constraint_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            add_default_constraint_result result = new add_default_constraint_result();
+            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;
+            add_default_constraint_result result = new add_default_constraint_result();
+            if (e instanceof NoSuchObjectException) {
+                        result.o1 = (NoSuchObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o2 = (MetaException) e;
+                        result.setO2IsSet(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, add_default_constraint_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.add_default_constraint(args.req,resultHandler);
+      }
+    }
+
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_table<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_table_args, Void> {
       public drop_table() {
         super("drop_table");
@@ -23657,22 +23905,21 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class update_table_column_statistics<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, update_table_column_statistics_args, Boolean> {
-      public update_table_column_statistics() {
-        super("update_table_column_statistics");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_default_constraints<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_default_constraints_args, DefaultConstraintsResponse> {
+      public get_default_constraints() {
+        super("get_default_constraints");
       }
 
-      public update_table_column_statistics_args getEmptyArgsInstance() {
-        return new update_table_column_statistics_args();
+      public get_default_constraints_args getEmptyArgsInstance() {
+        return new get_default_constraints_args();
       }
 
-      public AsyncMethodCallback<Boolean> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<DefaultConstraintsResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Boolean>() { 
-          public void onComplete(Boolean o) {
-            update_table_column_statistics_result result = new update_table_column_statistics_result();
+        return new AsyncMethodCallback<DefaultConstraintsResponse>() { 
+          public void onComplete(DefaultConstraintsResponse o) {
+            get_default_constraints_result result = new get_default_constraints_result();
             result.success = o;
-            result.setSuccessIsSet(true);
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -23684,27 +23931,17 @@ import org.slf4j.LoggerFactory;
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            update_table_column_statistics_result result = new update_table_column_statistics_result();
-            if (e instanceof NoSuchObjectException) {
-                        result.o1 = (NoSuchObjectException) e;
+            get_default_constraints_result result = new get_default_constraints_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof InvalidObjectException) {
-                        result.o2 = (InvalidObjectException) e;
+            else             if (e instanceof NoSuchObjectException) {
+                        result.o2 = (NoSuchObjectException) e;
                         result.setO2IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof MetaException) {
-                        result.o3 = (MetaException) e;
-                        result.setO3IsSet(true);
-                        msg = result;
-            }
-            else             if (e instanceof InvalidInputException) {
-                        result.o4 = (InvalidInputException) e;
-                        result.setO4IsSet(true);
-                        msg = result;
-            }
              else 
             {
               msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
@@ -23725,25 +23962,25 @@ import org.slf4j.LoggerFactory;
         return false;
       }
 
-      public void start(I iface, update_table_column_statistics_args args, org.apache.thrift.async.AsyncMethodCallback<Boolean> resultHandler) throws TException {
-        iface.update_table_column_statistics(args.stats_obj,resultHandler);
+      public void start(I iface, get_default_constraints_args args, org.apache.thrift.async.AsyncMethodCallback<DefaultConstraintsResponse> resultHandler) throws TException {
+        iface.get_default_constraints(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class update_partition_column_statistics<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, update_partition_column_statistics_args, Boolean> {
-      public update_partition_column_statistics() {
-        super("update_partition_column_statistics");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class update_table_column_statistics<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, update_table_column_statistics_args, Boolean> {
+      public update_table_column_statistics() {
+        super("update_table_column_statistics");
       }
 
-      public update_partition_column_statistics_args getEmptyArgsInstance() {
-        return new update_partition_column_statistics_args();
+      public update_table_column_statistics_args getEmptyArgsInstance() {
+        return new update_table_column_statistics_args();
       }
 
       public AsyncMethodCallback<Boolean> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
         return new AsyncMethodCallback<Boolean>() { 
           public void onComplete(Boolean o) {
-            update_partition_column_statistics_result result = new update_partition_column_statistics_result();
+            update_table_column_statistics_result result = new update_table_column_statistics_result();
             result.success = o;
             result.setSuccessIsSet(true);
             try {
@@ -23757,7 +23994,7 @@ import org.slf4j.LoggerFactory;
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            update_partition_column_statistics_result result = new update_partition_column_statistics_result();
+            update_table_column_statistics_result result = new update_table_column_statistics_result();
             if (e instanceof NoSuchObjectException) {
                         result.o1 = (NoSuchObjectException) e;
                         result.setO1IsSet(true);
@@ -23798,26 +24035,27 @@ import org.slf4j.LoggerFactory;
         return false;
       }
 
-      public void start(I iface, update_partition_column_statistics_args args, org.apache.thrift.async.AsyncMethodCallback<Boolean> resultHandler) throws TException {
-        iface.update_partition_column_statistics(args.stats_obj,resultHandler);
+      public void start(I iface, update_table_column_statistics_args args, org.apache.thrift.async.AsyncMethodCallback<Boolean> resultHandler) throws TException {
+        iface.update_table_column_statistics(args.stats_obj,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_table_column_statistics<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_column_statistics_args, ColumnStatistics> {
-      public get_table_column_statistics() {
-        super("get_table_column_statistics");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class update_partition_column_statistics<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, update_partition_column_statistics_args, Boolean> {
+      public update_partition_column_statistics() {
+        super("update_partition_column_statistics");
       }
 
-      public get_table_column_statistics_args getEmptyArgsInstance() {
-        return new get_table_column_statistics_args();
+      public update_partition_column_statistics_args getEmptyArgsInstance() {
+        return new update_partition_column_statistics_args();
       }
 
-      public AsyncMethodCallback<ColumnStatistics> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<Boolean> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<ColumnStatistics>() { 
-          public void onComplete(ColumnStatistics o) {
-            get_table_column_statistics_result result = new get_table_column_statistics_result();
+        return new AsyncMethodCallback<Boolean>() { 
+          public void onComplete(Boolean o) {
+            update_partition_column_statistics_result result = new update_partition_column_statistics_result();
             result.success = o;
+            result.setSuccessIsSet(true);
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -23829,24 +24067,96 @@ import org.slf4j.LoggerFactory;
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            get_table_column_statistics_result result = new get_table_column_statistics_result();
+            update_partition_column_statistics_result result = new update_partition_column_statistics_result();
             if (e instanceof NoSuchObjectException) {
                         result.o1 = (NoSuchObjectException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof MetaException) {
-                        result.o2 = (MetaException) e;
+            else             if (e instanceof InvalidObjectException) {
+                        result.o2 = (InvalidObjectException) e;
                         result.setO2IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof InvalidInputException) {
-                        result.o3 = (InvalidInputException) e;
+            else             if (e instanceof MetaException) {
+                        result.o3 = (MetaException) e;
                         result.setO3IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof InvalidObjectException) {
-                        result.o4 = (InvalidObjectException) e;
+            else             if (e instanceof InvalidInputException) {
+                        result.o4 = (InvalidInputException) e;
+                        result.setO4IsSet(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, update_partition_column_statistics_args args, org.apache.thrift.async.AsyncMethodCallback<Boolean> resultHandler) throws TException {
+        iface.update_partition_column_statistics(args.stats_obj,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_table_column_statistics<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_table_column_statistics_args, ColumnStatistics> {
+      public get_table_column_statistics() {
+        super("get_table_column_statistics");
+      }
+
+      public get_table_column_statistics_args getEmptyArgsInstance() {
+        return new get_table_column_statistics_args();
+      }
+
+      public AsyncMethodCallback<ColumnStatistics> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<ColumnStatistics>() { 
+          public void onComplete(ColumnStatistics o) {
+            get_table_column_statistics_result result = new get_table_column_statistics_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_table_column_statistics_result result = new get_table_column_statistics_result();
+            if (e instanceof NoSuchObjectException) {
+                        result.o1 = (NoSuchObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o2 = (MetaException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof InvalidInputException) {
+                        result.o3 = (InvalidInputException) e;
+                        result.setO3IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof InvalidObjectException) {
+                        result.o4 = (InvalidObjectException) e;
                         result.setO4IsSet(true);
                         msg = result;
             }
@@ -34354,13 +34664,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list858 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list858.size);
-                  String _elem859;
-                  for (int _i860 = 0; _i860 < _list858.size; ++_i860)
+                  org.apache.thrift.protocol.TList _list874 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list874.size);
+                  String _elem875;
+                  for (int _i876 = 0; _i876 < _list874.size; ++_i876)
                   {
-                    _elem859 = iprot.readString();
-                    struct.success.add(_elem859);
+                    _elem875 = iprot.readString();
+                    struct.success.add(_elem875);
                   }
                   iprot.readListEnd();
                 }
@@ -34395,9 +34705,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter861 : struct.success)
+            for (String _iter877 : struct.success)
             {
-              oprot.writeString(_iter861);
+              oprot.writeString(_iter877);
             }
             oprot.writeListEnd();
           }
@@ -34436,9 +34746,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter862 : struct.success)
+            for (String _iter878 : struct.success)
             {
-              oprot.writeString(_iter862);
+              oprot.writeString(_iter878);
             }
           }
         }
@@ -34453,13 +34763,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list863 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list863.size);
-            String _elem864;
-            for (int _i865 = 0; _i865 < _list863.size; ++_i865)
+            org.apache.thrift.protocol.TList _list879 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list879.size);
+            String _elem880;
+            for (int _i881 = 0; _i881 < _list879.size; ++_i881)
             {
-              _elem864 = iprot.readString();
-              struct.success.add(_elem864);
+              _elem880 = iprot.readString();
+              struct.success.add(_elem880);
             }
           }
           struct.setSuccessIsSet(true);
@@ -35113,13 +35423,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list866 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list866.size);
-                  String _elem867;
-                  for (int _i868 = 0; _i868 < _list866.size; ++_i868)
+                  org.apache.thrift.protocol.TList _list882 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list882.size);
+                  String _elem883;
+                  for (int _i884 = 0; _i884 < _list882.size; ++_i884)
                   {
-                    _elem867 = iprot.readString();
-                    struct.success.add(_elem867);
+                    _elem883 = iprot.readString();
+                    struct.success.add(_elem883);
                   }
                   iprot.readListEnd();
                 }
@@ -35154,9 +35464,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter869 : struct.success)
+            for (String _iter885 : struct.success)
             {
-              oprot.writeString(_iter869);
+              oprot.writeString(_iter885);
             }
             oprot.writeListEnd();
           }
@@ -35195,9 +35505,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter870 : struct.success)
+            for (String _iter886 : struct.success)
             {
-              oprot.writeString(_iter870);
+              oprot.writeString(_iter886);
             }
           }
         }
@@ -35212,13 +35522,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list871 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list871.size);
-            String _elem872;
-            for (int _i873 = 0; _i873 < _list871.size; ++_i873)
+            org.apache.thrift.protocol.TList _list887 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list887.size);
+            String _elem888;
+            for (int _i889 = 0; _i889 < _list887.size; ++_i889)
             {
-              _elem872 = iprot.readString();
-              struct.success.add(_elem872);
+              _elem888 = iprot.readString();
+              struct.success.add(_elem888);
             }
           }
           struct.setSuccessIsSet(true);
@@ -39825,16 +40135,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map874 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map874.size);
-                  String _key875;
-                  Type _val876;
-                  for (int _i877 = 0; _i877 < _map874.size; ++_i877)
+                  org.apache.thrift.protocol.TMap _map890 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map890.size);
+                  String _key891;
+                  Type _val892;
+                  for (int _i893 = 0; _i893 < _map890.size; ++_i893)
                   {
-                    _key875 = iprot.readString();
-                    _val876 = new Type();
-                    _val876.read(iprot);
-                    struct.success.put(_key875, _val876);
+                    _key891 = iprot.readString();
+                    _val892 = new Type();
+                    _val892.read(iprot);
+                    struct.success.put(_key891, _val892);
                   }
                   iprot.readMapEnd();
                 }
@@ -39869,10 +40179,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Map.Entry<String, Type> _iter878 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter894 : struct.success.entrySet())
             {
-              oprot.writeString(_iter878.getKey());
-              _iter878.getValue().write(oprot);
+              oprot.writeString(_iter894.getKey());
+              _iter894.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -39911,10 +40221,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter879 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter895 : struct.success.entrySet())
             {
-              oprot.writeString(_iter879.getKey());
-              _iter879.getValue().write(oprot);
+              oprot.writeString(_iter895.getKey());
+              _iter895.getValue().write(oprot);
             }
           }
         }
@@ -39929,16 +40239,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map880 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new HashMap<String,Type>(2*_map880.size);
-            String _key881;
-            Type _val882;
-            for (int _i883 = 0; _i883 < _map880.size; ++_i883)
+            org.apache.thrift.protocol.TMap _map896 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new HashMap<String,Type>(2*_map896.size);
+            String _key897;
+            Type _val898;
+            for (int _i899 = 0; _i899 < _map896.size; ++_i899)
             {
-              _key881 = iprot.readString();
-              _val882 = new Type();
-              _val882.read(iprot);
-              struct.success.put(_key881, _val882);
+              _key897 = iprot.readString();
+              _val898 = new Type();
+              _val898.read(iprot);
+              struct.success.put(_key897, _val898);
             }
           }
           struct.setSuccessIsSet(true);
@@ -40973,14 +41283,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list884 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list884.size);
-                  FieldSchema _elem885;
-                  for (int _i886 = 0; _i886 < _list884.size; ++_i886)
+                  org.apache.thrift.protocol.TList _list900 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list900.size);
+                  FieldSchema _elem901;
+                  for (int _i902 = 0; _i902 < _list900.size; ++_i902)
                   {
-                    _elem885 = new FieldSchema();
-                    _elem885.read(iprot);
-                    struct.success.add(_elem885);
+                    _elem901 = new FieldSchema();
+                    _elem901.read(iprot);
+                    struct.success.add(_elem901);
                   }
                   iprot.readListEnd();
                 }
@@ -41033,9 +41343,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter887 : struct.success)
+            for (FieldSchema _iter903 : struct.success)
             {
-              _iter887.write(oprot);
+              _iter903.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -41090,9 +41400,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter888 : struct.success)
+            for (FieldSchema _iter904 : struct.success)
             {
-              _iter888.write(oprot);
+              _iter904.write(oprot);
             }
           }
         }
@@ -41113,14 +41423,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list889 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list889.size);
-            FieldSchema _elem890;
-            for (int _i891 = 0; _i891 < _list889.size; ++_i891)
+            org.apache.thrift.protocol.TList _list905 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list905.size);
+            FieldSchema _elem906;
+            for (int _i907 = 0; _i907 < _list905.size; ++_i907)
             {
-              _elem890 = new FieldSchema();
-              _elem890.read(iprot);
-              struct.success.add(_elem890);
+              _elem906 = new FieldSchema();
+              _elem906.read(iprot);
+              struct.success.add(_elem906);
             }
           }
           struct.setSuccessIsSet(true);
@@ -42274,14 +42584,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list892 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list892.size);
-                  FieldSchema _elem893;
-                  for (int _i894 = 0; _i894 < _list892.size; ++_i894)
+                  org.apache.thrift.protocol.TList _list908 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list908.size);
+                  FieldSchema _elem909;
+                  for (int _i910 = 0; _i910 < _list908.size; ++_i910)
                   {
-                    _elem893 = new FieldSchema();
-                    _elem893.read(iprot);
-                    struct.success.add(_elem893);
+                    _elem909 = new FieldSchema();
+                    _elem909.read(iprot);
+                    struct.success.add(_elem909);
                   }
                   iprot.readListEnd();
                 }
@@ -42334,9 +42644,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter895 : struct.success)
+            for (FieldSchema _iter911 : struct.success)
             {
-              _iter895.write(oprot);
+              _iter911.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -42391,9 +42701,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter896 : struct.success)
+            for (FieldSchema _iter912 : struct.success)
             {
-              _iter896.write(oprot);
+              _iter912.write(oprot);
             }
           }
         }
@@ -42414,14 +42724,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list897 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list897.size);
-            FieldSchema _elem898;
-            for (int _i899 = 0; _i899 < _list897.size; ++_i899)
+            org.apache.thrift.protocol.TList _list913 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list913.size);
+            FieldSchema _elem914;
+            for (int _i915 = 0; _i915 < _list913.size; ++_i915)
             {
-              _elem898 = new FieldSchema();
-              _elem898.read(iprot);
-              struct.success.add(_elem898);
+              _elem914 = new FieldSchema();
+              _elem914.read(iprot);
+              struct.success.add(_elem914);
             }
           }
           struct.setSuccessIsSet(true);
@@ -43466,14 +43776,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list900 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list900.size);
-                  FieldSchema _elem901;
-                  for (int _i902 = 0; _i902 < _list900.size; ++_i902)
+                  org.apache.thrift.protocol.TList _list916 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list916.size);
+                  FieldSchema _elem917;
+                  for (int _i918 = 0; _i918 < _list916.size; ++_i918)
                   {
-                    _elem901 = new FieldSchema();
-                    _elem901.read(iprot);
-                    struct.success.add(_elem901);
+                    _elem917 = new FieldSchema();
+                    _elem917.read(iprot);
+                    struct.success.add(_elem917);
                   }
                   iprot.readListEnd();
                 }
@@ -43526,9 +43836,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter903 : struct.success)
+            for (FieldSchema _iter919 : struct.success)
             {
-              _iter903.write(oprot);
+              _iter919.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -43583,9 +43893,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter904 : struct.success)
+            for (FieldSchema _iter920 : struct.success)
             {
-              _iter904.write(oprot);
+              _iter920.write(oprot);
             }
           }
         }
@@ -43606,14 +43916,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list905 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list905.size);
-            FieldSchema _elem906;
-            for (int _i907 = 0; _i907 < _list905.size; ++_i907)
+            org.apache.thrift.protocol.TList _list921 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list921.size);
+            FieldSchema _elem922;
+            for (int _i923 = 0; _i923 < _list921.size; ++_i923)
             {
-              _elem906 = new FieldSchema();
-              _elem906.read(iprot);
-              struct.success.add(_elem906);
+              _elem922 = new FieldSchema();
+              _elem922.read(iprot);
+              struct.success.add(_elem922);
             }
           }
           struct.setSuccessIsSet(true);
@@ -44767,14 +45077,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list908 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list908.size);
-                  FieldSchema _elem909;
-                  for (int _i910 = 0; _i910 < _list908.size; ++_i910)
+                  org.apache.thrift.protocol.TList _list924 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list924.size);
+                  FieldSchema _elem925;
+                  for (int _i926 = 0; _i926 < _list924.size; ++_i926)
                   {
-                    _elem909 = new FieldSchema();
-                    _elem909.read(iprot);
-                    struct.success.add(_elem909);
+                    _elem925 = new FieldSchema();
+                    _elem925.read(iprot);
+                    struct.success.add(_elem925);
                   }
                   iprot.readListEnd();
                 }
@@ -44827,9 +45137,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter911 : struct.success)
+            for (FieldSchema _iter927 : struct.success)
             {
-              _iter911.write(oprot);
+              _iter927.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -44884,9 +45194,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter912 : struct.success)
+            for (FieldSchema _iter928 : struct.success)
             {
-              _iter912.write(oprot);
+              _iter928.write(oprot);
             }
           }
         }
@@ -44907,14 +45217,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list913 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list913.size);
-            FieldSchema _elem914;
-            for (int _i915 = 0; _i915 < _list913.size; ++_i915)
+            org.apache.thrift.protocol.TList _list929 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list929.size);
+            FieldSchema _elem930;
+            for (int _i931 = 0; _i931 < _list929.size; ++_i931)
             {
-              _elem914 = new FieldSchema();
-              _elem914.read(iprot);
-              struct.success.add(_elem914);
+              _elem930 = new FieldSchema();
+              _elem930.read(iprot);
+              struct.success.add(_elem930);
             }
           }
           struct.setSuccessIsSet(true);
@@ -47138,6 +47448,7 @@ import org.slf4j.LoggerFactory;
     private static final org.apache.thrift.protocol.TField FOREIGN_KEYS_FIELD_DESC = new org.apache.thrift.protocol.TField("foreignKeys", org.apache.thrift.protocol.TType.LIST, (short)3);
     private static final org.apache.thrift.protocol.TField UNIQUE_CONSTRAINTS_FIELD_DESC = new org.apache.thrift.protocol.TField("uniqueConstraints", org.apache.thrift.protocol.TType.LIST, (short)4);
     private static final org.apache.thrift.protocol.TField NOT_NULL_CONSTRAINTS_FIELD_DESC = new org.apache.thrift.protocol.TField("notNullConstraints", org.apache.thrift.protocol.TType.LIST, (short)5);
+    private static final org.apache.thrift.protocol.TField DEFAULT_CONSTRAINTS_FIELD_DESC = new org.apache.thrift.protocol.TField("defaultConstraints", org.apache.thrift.protocol.TType.LIST, (short)6);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
@@ -47150,6 +47461,7 @@ import org.slf4j.LoggerFactory;
     private List<SQLForeignKey> foreignKeys; // required
     private List<SQLUniqueConstraint> uniqueConstraints; // required
     private List<SQLNotNullConstraint> notNullConstraints; // required
+    private List<SQLDefaultConstraint> defaultConstraints; // 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 {
@@ -47157,7 +47469,8 @@ import org.slf4j.LoggerFactory;
       PRIMARY_KEYS((short)2, "primaryKeys"),
       FOREIGN_KEYS((short)3, "foreignKeys"),
       UNIQUE_CONSTRAINTS((short)4, "uniqueConstraints"),
-      NOT_NULL_CONSTRAINTS((short)5, "notNullConstraints");
+      NOT_NULL_CONSTRAINTS((short)5, "notNullConstraints"),
+      DEFAULT_CONSTRAINTS((short)6, "defaultConstraints");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -47182,6 +47495,8 @@ import org.slf4j.LoggerFactory;
             return UNIQUE_CONSTRAINTS;
           case 5: // NOT_NULL_CONSTRAINTS
             return NOT_NULL_CONSTRAINTS;
+          case 6: // DEFAULT_CONSTRAINTS
+            return DEFAULT_CONSTRAINTS;
           default:
             return null;
         }
@@ -47239,6 +47554,9 @@ import org.slf4j.LoggerFactory;
       tmpMap.put(_Fields.NOT_NULL_CONSTRAINTS, new org.apache.thrift.meta_data.FieldMetaData("notNullConstraints", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SQLNotNullConstraint.class))));
+      tmpMap.put(_Fields.DEFAULT_CONSTRAINTS, new org.apache.thrift.meta_data.FieldMetaData("defaultConstraints", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SQLDefaultConstraint.class))));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(create_table_with_constraints_args.class, metaDataMap);
     }
@@ -47251,7 +47569,8 @@ import org.slf4j.LoggerFactory;
       List<SQLPrimaryKey> primaryKeys,
       List<SQLForeignKey> foreignKeys,
       List<SQLUniqueConstraint> uniqueConstraints,
-      List<SQLNotNullConstraint> notNullConstraints)
+      List<SQLNotNullConstraint> notNullConstraints,
+      List<SQLDefaultConstraint> defaultConstraints)
     {
       this();
       this.tbl = tbl;
@@ -47259,6 +47578,7 @@ import org.slf4j.LoggerFactory;
       this.foreignKeys = foreignKeys;
       this.uniqueConstraints = uniqueConstraints;
       this.notNullConstraints = notNullConstraints;
+      this.defaultConstraints = defaultConstraints;
     }
 
     /**
@@ -47296,6 +47616,13 @@ import org.slf4j.LoggerFactory;
         }
         this.notNullConstraints = __this__notNullConstraints;
       }
+      if (other.isSetDefaultConstraints()) {
+        List<SQLDefaultConstraint> __this__defaultConstraints = new ArrayList<SQLDefaultConstraint>(other.defaultConstraints.size());
+        for (SQLDefaultConstraint other_element : other.defaultConstraints) {
+          __this__defaultConstraints.add(new SQLDefaultConstraint(other_element));
+        }
+        this.defaultConstraints = __this__defaultConstraints;
+      }
     }
 
     public create_table_with_constraints_args deepCopy() {
@@ -47309,6 +47636,7 @@ import org.slf4j.LoggerFactory;
       this.foreignKeys = null;
       this.uniqueConstraints = null;
       this.notNullConstraints = null;
+      this.defaultConstraints = null;
     }
 
     public Table getTbl() {
@@ -47486,6 +47814,44 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public int getDefaultConstraintsSize() {
+      return (this.defaultConstraints == null) ? 0 : this.defaultConstraints.size();
+    }
+
+    public java.util.Iterator<SQLDefaultConstraint> getDefaultConstraintsIterator() {
+      return (this.defaultConstraints == null) ? null : this.defaultConstraints.iterator();
+    }
+
+    public void addToDefaultConstraints(SQLDefaultConstraint elem) {
+      if (this.defaultConstraints == null) {
+        this.defaultConstraints = new ArrayList<SQLDefaultConstraint>();
+      }
+      this.defaultConstraints.add(elem);
+    }
+
+    public List<SQLDefaultConstraint> getDefaultConstraints() {
+      return this.defaultConstraints;
+    }
+
+    public void setDefaultConstraints(List<SQLDefaultConstraint> defaultConstraints) {
+      this.defaultConstraints = defaultConstraints;
+    }
+
+    public void unsetDefaultConstraints() {
+      this.defaultConstraints = null;
+    }
+
+    /** Returns true if field defaultConstraints is set (has been assigned a value) and false otherwise */
+    public boolean isSetDefaultConstraints() {
+      return this.defaultConstraints != null;
+    }
+
+    public void setDefaultConstraintsIsSet(boolean value) {
+      if (!value) {
+        this.defaultConstraints = null;
+      }
+    }
+
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
       case TBL:
@@ -47528,6 +47894,14 @@ import org.slf4j.LoggerFactory;
         }
         break;
 
+      case DEFAULT_CONSTRAINTS:
+        if (value == null) {
+          unsetDefaultConstraints();
+        } else {
+          setDefaultConstraints((List<SQLDefaultConstraint>)value);
+        }
+        break;
+
       }
     }
 
@@ -47548,6 +47922,9 @@ import org.slf4j.LoggerFactory;
       case NOT_NULL_CONSTRAINTS:
         return getNotNullConstraints();
 
+      case DEFAULT_CONSTRAINTS:
+        return getDefaultConstraints();
+
       }
       throw new IllegalStateException();
     }
@@ -47569,6 +47946,8 @@ import org.slf4j.LoggerFactory;
         return isSetUniqueConstraints();
       case NOT_NULL_CONSTRAINTS:
         return isSetNotNullConstraints();
+      case DEFAULT_CONSTRAINTS:
+        return isSetDefaultConstraints();
       }
       throw new IllegalStateException();
     }
@@ -47631,6 +48010,15 @@ import org.slf4j.LoggerFactory;
           return false;
       }
 
+      boolean this_present_defaultConstraints = true && this.isSetDefaultConstraints();
+      boolean that_present_defaultConstraints = true && that.isSetDefaultConstraints();
+      if (this_present_defaultConstraints || that_present_defaultConstraints) {
+        if (!(this_present_defaultConstraints && that_present_defaultConstraints))
+          return false;
+        if (!this.defaultConstraints.equals(that.defaultConstraints))
+          return false;
+      }
+
       return true;
     }
 
@@ -47663,6 +48051,11 @@ import org.slf4j.LoggerFactory;
       if (present_notNullConstraints)
         list.add(notNullConstraints);
 
+      boolean present_defaultConstraints = true && (isSetDefaultConstraints());
+      list.add(present_defaultConstraints);
+      if (present_defaultConstraints)
+        list.add(defaultConstraints);
+
       return list.hashCode();
     }
 
@@ -47724,6 +48117,16 @@ import org.slf4j.LoggerFactory;
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(isSetDefaultConstraints()).compareTo(other.isSetDefaultConstraints());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetDefaultConstraints()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.defaultConstraints, other.defaultConstraints);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -47783,6 +48186,14 @@ import org.slf4j.LoggerFactory;
         sb.append(this.notNullConstraints);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("defaultConstraints:");
+      if (this.defaultConstraints == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.defaultConstraints);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -47841,14 +48252,14 @@ import org.slf4j.LoggerFactory;
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list916 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list916.size);
-                  SQLPrimaryKey _elem917;
-                  for (int _i918 = 0; _i918 < _list916.size; ++_i918)
+                  org.apache.thrift.protocol.TList _list932 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list932.size);
+                  SQLPrimaryKey _elem933;
+                  for (int _i934 = 0; _i934 < _list932.size; ++_i934)
                   {
-                    _elem917 = new SQLPrimaryKey();
-                    _elem917.read(iprot);
-                    struct.primaryKeys.add(_elem917);
+                    _elem933 = new SQLPrimaryKey();
+                    _elem933.read(iprot);
+                    struct.primaryKeys.add(_elem933);
                   }
                   iprot.readListEnd();
                 }
@@ -47860,14 +48271,14 @@ import org.slf4j.LoggerFactory;
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list919 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list919.size);
-                  SQLForeignKey _elem920;
-                  for (int _i921 = 0; _i921 < _list919.size; ++_i921)
+                  org.apache.thrift.protocol.TList _list935 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list935.size);
+                  SQLForeignKey _elem936;
+                  for (int _i937 = 0; _i937 < _list935.size; ++_i937)
                   {
-                    _elem920 = new SQLForeignKey();
-                    _elem920.read(iprot);
-                    struct.foreignKeys.add(_elem920);
+                    _elem936 = new SQLForeignKey();
+                    _elem936.read(iprot);
+                    struct.foreignKeys.add(_elem936);
                   }
                   iprot.readListEnd();
                 }
@@ -47879,14 +48290,14 @@ import org.slf4j.LoggerFactory;
             case 4: // UNIQUE_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list922 = iprot.readListBegin();
-                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list922.size);
-                  SQLUniqueConstraint _elem923;
-                  for (int _i924 = 0; _i924 < _list922.size; ++_i924)
+                  org.apache.thrift.protocol.TList _list938 = iprot.readListBegin();
+                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list938.size);
+                  SQLUniqueConstraint _elem939;
+                  for (int _i940 = 0; _i940 < _list938.size; ++_i940)
                   {
-                    _elem923 = new SQLUniqueConstraint();
-                    _elem923.read(iprot);
-                    struct.uniqueConstraints.add(_elem923);
+                    _elem939 = new SQLUniqueConstraint();
+                    _elem939.read(iprot);
+                    struct.uniqueConstraints.add(_elem939);
                   }
                   iprot.readListEnd();
                 }
@@ -47898,14 +48309,14 @@ import org.slf4j.LoggerFactory;
             case 5: // NOT_NULL_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list925 = iprot.readListBegin();
-                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list925.size);
-                  SQLNotNullConstraint _elem926;
-                  for (int _i927 = 0; _i927 < _list925.size; ++_i927)
+                  org.apache.thrift.protocol.TList _list941 = iprot.readListBegin();
+                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list941.size);
+                  SQLNotNullConstraint _elem942;
+                  for (int _i943 = 0; _i943 < _list941.size; ++_i943)
                   {
-                    _elem926 = new SQLNotNullConstraint();
-                    _elem926.read(iprot);
-                    struct.notNullConstraints.add(_elem926);
+                    _elem942 = new SQLNotNullConstraint();
+                    _elem942.read(iprot);
+                    struct.notNullConstraints.add(_elem942);
                   }
                   iprot.readListEnd();
                 }
@@ -47914,6 +48325,25 @@ import org.slf4j.LoggerFactory;
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
+            case 6: // DEFAULT_CONSTRAINTS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list944 = iprot.readListBegin();
+                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list944.size);
+                  SQLDefaultConstraint _elem945;
+                  for (int _i946 = 0; _i946 < _list944.size; ++_i946)
+                  {
+                    _elem945 = new SQLDefaultConstraint();
+                    _elem945.read(iprot);
+                    struct.defaultConstraints.add(_elem945);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setDefaultConstraintsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -47936,9 +48366,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size()));
-            for (SQLPrimaryKey _iter928 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter947 : struct.primaryKeys)
             {
-              _iter928.write(oprot);
+              _iter947.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -47948,9 +48378,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size()));
-            for (SQLForeignKey _iter929 : struct.foreignKeys)
+            for (SQLForeignKey _iter948 : struct.foreignKeys)
             {
-              _iter929.write(oprot);
+              _iter948.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -47960,9 +48390,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(UNIQUE_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraints.size()));
-            for (SQLUniqueConstraint _iter930 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter949 : struct.uniqueConstraints)
             {
-              _iter930.write(oprot);
+              _iter949.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -47972,9 +48402,21 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraints.size()));
-            for (SQLNotNullConstraint _iter931 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter950 : struct.notNullConstraints)
+            {
+              _iter950.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.defaultConstraints != null) {
+          oprot.writeFieldBegin(DEFAULT_CONSTRAINTS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraints.size()));
+            for (SQLDefaultConstraint _iter951 : struct.defaultConstraints)
             {
-              _iter931.write(oprot);
+              _iter951.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -48013,43 +48455,55 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNotNullConstraints()) {
           optionals.set(4);
         }
-        oprot.writeBitSet(optionals, 5);
+        if (struct.isSetDefaultConstraints()) {
+          optionals.set(5);
+        }
+        oprot.writeBitSet(optionals, 6);
         if (struct.isSetTbl()) {
           struct.tbl.write(oprot);
         }
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter932 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter952 : struct.primaryKeys)
             {
-              _iter932.write(oprot);
+              _iter952.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter933 : struct.foreignKeys)
+            for (SQLForeignKey _iter953 : struct.foreignKeys)
             {
-              _iter933.write(oprot);
+              _iter953.write(oprot);
             }
           }
         }
         if (struct.isSetUniqueConstraints()) {
           {
             oprot.writeI32(struct.uniqueConstraints.size());
-            for (SQLUniqueConstraint _iter934 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter954 : struct.uniqueConstraints)
             {
-              _iter934.write(oprot);
+              _iter954.write(oprot);
             }
           }
         }
         if (struct.isSetNotNullConstraints()) {
           {
             oprot.writeI32(struct.notNullConstraints.size());
-            for (SQLNotNullConstraint _iter935 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter955 : struct.notNullConstraints)
+            {
+              _iter955.write(oprot);
+            }
+          }
+        }
+        if (struct.isSetDefaultConstraints()) {
+          {
+            oprot.writeI32(struct.defaultConstraints.size());
+            for (SQLDefaultConstraint _iter956 : struct.defaultConstraints)
             {
-              _iter935.write(oprot);
+              _iter956.write(oprot);
             }
           }
         }
@@ -48058,7 +48512,7 @@ import org.slf4j.LoggerFactory;
       @Override
       public void read(org.apache.thrift.protocol.TProtocol prot, create_table_with_constraints_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(5);
+        BitSet incoming = iprot.readBitSet(6);
         if (incoming.get(0)) {
           struct.tbl = new Table();
           struct.tbl.read(iprot);
@@ -48066,60 +48520,74 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list936 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list936.size);
-            SQLPrimaryKey _elem937;
-            for (int _i938 = 0; _i938 < _list936.size; ++_i938)
+            org.apache.thrift.protocol.TList _list957 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list957.size);
+            SQLPrimaryKey _elem958;
+            for (int _i959 = 0; _i959 < _list957.size; ++_i959)
             {
-              _elem937 = new SQLPrimaryKey();
-              _elem937.read(iprot);
-              struct.primaryKeys.add(_elem937);
+              _elem958 = new SQLPrimaryKey();
+              _elem958.read(iprot);
+              struct.primaryKeys.add(_elem958);
             }
           }
           struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list939 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list939.size);
-            SQLForeignKey _elem940;
-            for (int _i941 = 0; _i941 < _list939.size; ++_i941)
+            org.apache.thrift.protocol.TList _list960 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+      

<TRUNCATED>

[07/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
index 9c05a18..dedbcce 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
@@ -755,14 +755,14 @@ import org.slf4j.LoggerFactory;
           case 2: // POOLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list794 = iprot.readListBegin();
-                struct.pools = new ArrayList<WMPool>(_list794.size);
-                WMPool _elem795;
-                for (int _i796 = 0; _i796 < _list794.size; ++_i796)
+                org.apache.thrift.protocol.TList _list810 = iprot.readListBegin();
+                struct.pools = new ArrayList<WMPool>(_list810.size);
+                WMPool _elem811;
+                for (int _i812 = 0; _i812 < _list810.size; ++_i812)
                 {
-                  _elem795 = new WMPool();
-                  _elem795.read(iprot);
-                  struct.pools.add(_elem795);
+                  _elem811 = new WMPool();
+                  _elem811.read(iprot);
+                  struct.pools.add(_elem811);
                 }
                 iprot.readListEnd();
               }
@@ -774,14 +774,14 @@ import org.slf4j.LoggerFactory;
           case 3: // MAPPINGS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list797 = iprot.readListBegin();
-                struct.mappings = new ArrayList<WMMapping>(_list797.size);
-                WMMapping _elem798;
-                for (int _i799 = 0; _i799 < _list797.size; ++_i799)
+                org.apache.thrift.protocol.TList _list813 = iprot.readListBegin();
+                struct.mappings = new ArrayList<WMMapping>(_list813.size);
+                WMMapping _elem814;
+                for (int _i815 = 0; _i815 < _list813.size; ++_i815)
                 {
-                  _elem798 = new WMMapping();
-                  _elem798.read(iprot);
-                  struct.mappings.add(_elem798);
+                  _elem814 = new WMMapping();
+                  _elem814.read(iprot);
+                  struct.mappings.add(_elem814);
                 }
                 iprot.readListEnd();
               }
@@ -793,14 +793,14 @@ import org.slf4j.LoggerFactory;
           case 4: // TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list800 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list800.size);
-                WMTrigger _elem801;
-                for (int _i802 = 0; _i802 < _list800.size; ++_i802)
+                org.apache.thrift.protocol.TList _list816 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list816.size);
+                WMTrigger _elem817;
+                for (int _i818 = 0; _i818 < _list816.size; ++_i818)
                 {
-                  _elem801 = new WMTrigger();
-                  _elem801.read(iprot);
-                  struct.triggers.add(_elem801);
+                  _elem817 = new WMTrigger();
+                  _elem817.read(iprot);
+                  struct.triggers.add(_elem817);
                 }
                 iprot.readListEnd();
               }
@@ -812,14 +812,14 @@ import org.slf4j.LoggerFactory;
           case 5: // POOL_TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list803 = iprot.readListBegin();
-                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list803.size);
-                WMPoolTrigger _elem804;
-                for (int _i805 = 0; _i805 < _list803.size; ++_i805)
+                org.apache.thrift.protocol.TList _list819 = iprot.readListBegin();
+                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list819.size);
+                WMPoolTrigger _elem820;
+                for (int _i821 = 0; _i821 < _list819.size; ++_i821)
                 {
-                  _elem804 = new WMPoolTrigger();
-                  _elem804.read(iprot);
-                  struct.poolTriggers.add(_elem804);
+                  _elem820 = new WMPoolTrigger();
+                  _elem820.read(iprot);
+                  struct.poolTriggers.add(_elem820);
                 }
                 iprot.readListEnd();
               }
@@ -850,9 +850,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(POOLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.pools.size()));
-          for (WMPool _iter806 : struct.pools)
+          for (WMPool _iter822 : struct.pools)
           {
-            _iter806.write(oprot);
+            _iter822.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -863,9 +863,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(MAPPINGS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mappings.size()));
-            for (WMMapping _iter807 : struct.mappings)
+            for (WMMapping _iter823 : struct.mappings)
             {
-              _iter807.write(oprot);
+              _iter823.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -877,9 +877,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size()));
-            for (WMTrigger _iter808 : struct.triggers)
+            for (WMTrigger _iter824 : struct.triggers)
             {
-              _iter808.write(oprot);
+              _iter824.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -891,9 +891,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(POOL_TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.poolTriggers.size()));
-            for (WMPoolTrigger _iter809 : struct.poolTriggers)
+            for (WMPoolTrigger _iter825 : struct.poolTriggers)
             {
-              _iter809.write(oprot);
+              _iter825.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -920,9 +920,9 @@ import org.slf4j.LoggerFactory;
       struct.plan.write(oprot);
       {
         oprot.writeI32(struct.pools.size());
-        for (WMPool _iter810 : struct.pools)
+        for (WMPool _iter826 : struct.pools)
         {
-          _iter810.write(oprot);
+          _iter826.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -939,27 +939,27 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetMappings()) {
         {
           oprot.writeI32(struct.mappings.size());
-          for (WMMapping _iter811 : struct.mappings)
+          for (WMMapping _iter827 : struct.mappings)
           {
-            _iter811.write(oprot);
+            _iter827.write(oprot);
           }
         }
       }
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter812 : struct.triggers)
+          for (WMTrigger _iter828 : struct.triggers)
           {
-            _iter812.write(oprot);
+            _iter828.write(oprot);
           }
         }
       }
       if (struct.isSetPoolTriggers()) {
         {
           oprot.writeI32(struct.poolTriggers.size());
-          for (WMPoolTrigger _iter813 : struct.poolTriggers)
+          for (WMPoolTrigger _iter829 : struct.poolTriggers)
           {
-            _iter813.write(oprot);
+            _iter829.write(oprot);
           }
         }
       }
@@ -972,56 +972,56 @@ import org.slf4j.LoggerFactory;
       struct.plan.read(iprot);
       struct.setPlanIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list814 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.pools = new ArrayList<WMPool>(_list814.size);
-        WMPool _elem815;
-        for (int _i816 = 0; _i816 < _list814.size; ++_i816)
+        org.apache.thrift.protocol.TList _list830 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.pools = new ArrayList<WMPool>(_list830.size);
+        WMPool _elem831;
+        for (int _i832 = 0; _i832 < _list830.size; ++_i832)
         {
-          _elem815 = new WMPool();
-          _elem815.read(iprot);
-          struct.pools.add(_elem815);
+          _elem831 = new WMPool();
+          _elem831.read(iprot);
+          struct.pools.add(_elem831);
         }
       }
       struct.setPoolsIsSet(true);
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list817 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.mappings = new ArrayList<WMMapping>(_list817.size);
-          WMMapping _elem818;
-          for (int _i819 = 0; _i819 < _list817.size; ++_i819)
+          org.apache.thrift.protocol.TList _list833 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.mappings = new ArrayList<WMMapping>(_list833.size);
+          WMMapping _elem834;
+          for (int _i835 = 0; _i835 < _list833.size; ++_i835)
           {
-            _elem818 = new WMMapping();
-            _elem818.read(iprot);
-            struct.mappings.add(_elem818);
+            _elem834 = new WMMapping();
+            _elem834.read(iprot);
+            struct.mappings.add(_elem834);
           }
         }
         struct.setMappingsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list820 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list820.size);
-          WMTrigger _elem821;
-          for (int _i822 = 0; _i822 < _list820.size; ++_i822)
+          org.apache.thrift.protocol.TList _list836 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list836.size);
+          WMTrigger _elem837;
+          for (int _i838 = 0; _i838 < _list836.size; ++_i838)
           {
-            _elem821 = new WMTrigger();
-            _elem821.read(iprot);
-            struct.triggers.add(_elem821);
+            _elem837 = new WMTrigger();
+            _elem837.read(iprot);
+            struct.triggers.add(_elem837);
           }
         }
         struct.setTriggersIsSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list823 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list823.size);
-          WMPoolTrigger _elem824;
-          for (int _i825 = 0; _i825 < _list823.size; ++_i825)
+          org.apache.thrift.protocol.TList _list839 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list839.size);
+          WMPoolTrigger _elem840;
+          for (int _i841 = 0; _i841 < _list839.size; ++_i841)
           {
-            _elem824 = new WMPoolTrigger();
-            _elem824.read(iprot);
-            struct.poolTriggers.add(_elem824);
+            _elem840 = new WMPoolTrigger();
+            _elem840.read(iprot);
+            struct.poolTriggers.add(_elem840);
           }
         }
         struct.setPoolTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
index ba44e3a..59bf429 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // RESOURCE_PLANS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list826 = iprot.readListBegin();
-                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list826.size);
-                WMResourcePlan _elem827;
-                for (int _i828 = 0; _i828 < _list826.size; ++_i828)
+                org.apache.thrift.protocol.TList _list842 = iprot.readListBegin();
+                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list842.size);
+                WMResourcePlan _elem843;
+                for (int _i844 = 0; _i844 < _list842.size; ++_i844)
                 {
-                  _elem827 = new WMResourcePlan();
-                  _elem827.read(iprot);
-                  struct.resourcePlans.add(_elem827);
+                  _elem843 = new WMResourcePlan();
+                  _elem843.read(iprot);
+                  struct.resourcePlans.add(_elem843);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(RESOURCE_PLANS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.resourcePlans.size()));
-            for (WMResourcePlan _iter829 : struct.resourcePlans)
+            for (WMResourcePlan _iter845 : struct.resourcePlans)
             {
-              _iter829.write(oprot);
+              _iter845.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetResourcePlans()) {
         {
           oprot.writeI32(struct.resourcePlans.size());
-          for (WMResourcePlan _iter830 : struct.resourcePlans)
+          for (WMResourcePlan _iter846 : struct.resourcePlans)
           {
-            _iter830.write(oprot);
+            _iter846.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list831 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list831.size);
-          WMResourcePlan _elem832;
-          for (int _i833 = 0; _i833 < _list831.size; ++_i833)
+          org.apache.thrift.protocol.TList _list847 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list847.size);
+          WMResourcePlan _elem848;
+          for (int _i849 = 0; _i849 < _list847.size; ++_i849)
           {
-            _elem832 = new WMResourcePlan();
-            _elem832.read(iprot);
-            struct.resourcePlans.add(_elem832);
+            _elem848 = new WMResourcePlan();
+            _elem848.read(iprot);
+            struct.resourcePlans.add(_elem848);
           }
         }
         struct.setResourcePlansIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
index edec382..5c6d680 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list850 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list850.size);
-                WMTrigger _elem851;
-                for (int _i852 = 0; _i852 < _list850.size; ++_i852)
+                org.apache.thrift.protocol.TList _list866 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list866.size);
+                WMTrigger _elem867;
+                for (int _i868 = 0; _i868 < _list866.size; ++_i868)
                 {
-                  _elem851 = new WMTrigger();
-                  _elem851.read(iprot);
-                  struct.triggers.add(_elem851);
+                  _elem867 = new WMTrigger();
+                  _elem867.read(iprot);
+                  struct.triggers.add(_elem867);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size()));
-            for (WMTrigger _iter853 : struct.triggers)
+            for (WMTrigger _iter869 : struct.triggers)
             {
-              _iter853.write(oprot);
+              _iter869.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter854 : struct.triggers)
+          for (WMTrigger _iter870 : struct.triggers)
           {
-            _iter854.write(oprot);
+            _iter870.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list855 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list855.size);
-          WMTrigger _elem856;
-          for (int _i857 = 0; _i857 < _list855.size; ++_i857)
+          org.apache.thrift.protocol.TList _list871 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list871.size);
+          WMTrigger _elem872;
+          for (int _i873 = 0; _i873 < _list871.size; ++_i873)
           {
-            _elem856 = new WMTrigger();
-            _elem856.read(iprot);
-            struct.triggers.add(_elem856);
+            _elem872 = new WMTrigger();
+            _elem872.read(iprot);
+            struct.triggers.add(_elem872);
           }
         }
         struct.setTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
index 228f37f..dba307a 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
@@ -441,13 +441,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ERRORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list834 = iprot.readListBegin();
-                struct.errors = new ArrayList<String>(_list834.size);
-                String _elem835;
-                for (int _i836 = 0; _i836 < _list834.size; ++_i836)
+                org.apache.thrift.protocol.TList _list850 = iprot.readListBegin();
+                struct.errors = new ArrayList<String>(_list850.size);
+                String _elem851;
+                for (int _i852 = 0; _i852 < _list850.size; ++_i852)
                 {
-                  _elem835 = iprot.readString();
-                  struct.errors.add(_elem835);
+                  _elem851 = iprot.readString();
+                  struct.errors.add(_elem851);
                 }
                 iprot.readListEnd();
               }
@@ -459,13 +459,13 @@ import org.slf4j.LoggerFactory;
           case 2: // WARNINGS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list837 = iprot.readListBegin();
-                struct.warnings = new ArrayList<String>(_list837.size);
-                String _elem838;
-                for (int _i839 = 0; _i839 < _list837.size; ++_i839)
+                org.apache.thrift.protocol.TList _list853 = iprot.readListBegin();
+                struct.warnings = new ArrayList<String>(_list853.size);
+                String _elem854;
+                for (int _i855 = 0; _i855 < _list853.size; ++_i855)
                 {
-                  _elem838 = iprot.readString();
-                  struct.warnings.add(_elem838);
+                  _elem854 = iprot.readString();
+                  struct.warnings.add(_elem854);
                 }
                 iprot.readListEnd();
               }
@@ -492,9 +492,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(ERRORS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.errors.size()));
-            for (String _iter840 : struct.errors)
+            for (String _iter856 : struct.errors)
             {
-              oprot.writeString(_iter840);
+              oprot.writeString(_iter856);
             }
             oprot.writeListEnd();
           }
@@ -506,9 +506,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(WARNINGS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.warnings.size()));
-            for (String _iter841 : struct.warnings)
+            for (String _iter857 : struct.warnings)
             {
-              oprot.writeString(_iter841);
+              oprot.writeString(_iter857);
             }
             oprot.writeListEnd();
           }
@@ -543,18 +543,18 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetErrors()) {
         {
           oprot.writeI32(struct.errors.size());
-          for (String _iter842 : struct.errors)
+          for (String _iter858 : struct.errors)
           {
-            oprot.writeString(_iter842);
+            oprot.writeString(_iter858);
           }
         }
       }
       if (struct.isSetWarnings()) {
         {
           oprot.writeI32(struct.warnings.size());
-          for (String _iter843 : struct.warnings)
+          for (String _iter859 : struct.warnings)
           {
-            oprot.writeString(_iter843);
+            oprot.writeString(_iter859);
           }
         }
       }
@@ -566,26 +566,26 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list844 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.errors = new ArrayList<String>(_list844.size);
-          String _elem845;
-          for (int _i846 = 0; _i846 < _list844.size; ++_i846)
+          org.apache.thrift.protocol.TList _list860 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.errors = new ArrayList<String>(_list860.size);
+          String _elem861;
+          for (int _i862 = 0; _i862 < _list860.size; ++_i862)
           {
-            _elem845 = iprot.readString();
-            struct.errors.add(_elem845);
+            _elem861 = iprot.readString();
+            struct.errors.add(_elem861);
           }
         }
         struct.setErrorsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list847 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.warnings = new ArrayList<String>(_list847.size);
-          String _elem848;
-          for (int _i849 = 0; _i849 < _list847.size; ++_i849)
+          org.apache.thrift.protocol.TList _list863 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.warnings = new ArrayList<String>(_list863.size);
+          String _elem864;
+          for (int _i865 = 0; _i865 < _list863.size; ++_i865)
           {
-            _elem848 = iprot.readString();
-            struct.warnings.add(_elem848);
+            _elem864 = iprot.readString();
+            struct.warnings.add(_elem864);
           }
         }
         struct.setWarningsIsSet(true);


[17/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientpositive/llap/default_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/default_constraint.q.out b/ql/src/test/results/clientpositive/llap/default_constraint.q.out
new file mode 100644
index 0000000..89b1224
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/default_constraint.q.out
@@ -0,0 +1,2793 @@
+PREHOOK: query: CREATE TABLE numericDataType(a TINYINT CONSTRAINT tinyint_constraint DEFAULT 127Y ENABLE, b SMALLINT DEFAULT 32767S, c INT DEFAULT 2147483647,
+    d BIGINT DEFAULT  9223372036854775807L, e DOUBLE DEFAULT 3.4E38, f DECIMAL(9,2) DEFAULT 1234567.89)
+    clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@numericDataType
+POSTHOOK: query: CREATE TABLE numericDataType(a TINYINT CONSTRAINT tinyint_constraint DEFAULT 127Y ENABLE, b SMALLINT DEFAULT 32767S, c INT DEFAULT 2147483647,
+    d BIGINT DEFAULT  9223372036854775807L, e DOUBLE DEFAULT 3.4E38, f DECIMAL(9,2) DEFAULT 1234567.89)
+    clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@numericDataType
+PREHOOK: query: DESC FORMATTED numericDataType
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@numericdatatype
+POSTHOOK: query: DESC FORMATTED numericDataType
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@numericdatatype
+# col_name            	data_type           	comment             
+a                   	tinyint             	                    
+b                   	smallint            	                    
+c                   	int                 	                    
+d                   	bigint              	                    
+e                   	double              	                    
+f                   	decimal(9,2)        	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	transactional       	true                
+	transactional_properties	default             
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.ql.io.orc.OrcSerde	 
+InputFormat:        	org.apache.hadoop.hive.ql.io.orc.OrcInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	2                   	 
+Bucket Columns:     	[a]                 	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+	 	 
+# Constraints	 	 
+	 	 
+# Default Constraints	 	 
+Table:              	default.numericdatatype	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:b       	Default Value:32767S	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:c       	Default Value:2147483647	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:d       	Default Value:9223372036854775807L	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:e       	Default Value:3.4E38D	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:f       	Default Value:1234567.89	 
+	 	 
+Constraint Name:    	tinyint_constraint  	 
+Column Name:a       	Default Value:127Y  	 
+	 	 
+PREHOOK: query: EXPLAIN INSERT INTO numericDataType(a) values(3Y)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN INSERT INTO numericDataType(a) values(3Y)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: _dummy_table
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: array(const struct(3)) (type: array<struct<col1:int>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    UDTF Operator
+                      Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      function name: inline
+                      Select Operator
+                        expressions: col1 (type: int), 32767S (type: smallint), 2147483647 (type: int), 9223372036854775807L (type: bigint), 3.4E38D (type: double), 1234567.89 (type: decimal(9,2))
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                        Statistics: Num rows: 1 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          sort order: 
+                          Map-reduce partition columns: UDFToByte(_col0) (type: tinyint)
+                          Statistics: Num rows: 1 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: int), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: double), _col5 (type: decimal(9,2))
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: UDFToByte(VALUE._col0) (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: int), VALUE._col3 (type: bigint), VALUE._col4 (type: double), VALUE._col5 (type: decimal(9,2))
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 1 Data size: 140 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 140 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+                      serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+                      name: default.numericdatatype
+                  Write Type: INSERT
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.numericdatatype
+          Write Type: INSERT
+
+PREHOOK: query: INSERT INTO numericDataType(a) values(3Y)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@numericdatatype
+POSTHOOK: query: INSERT INTO numericDataType(a) values(3Y)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@numericdatatype
+POSTHOOK: Lineage: numericdatatype.a SCRIPT []
+POSTHOOK: Lineage: numericdatatype.b SIMPLE []
+POSTHOOK: Lineage: numericdatatype.c SIMPLE []
+POSTHOOK: Lineage: numericdatatype.d SIMPLE []
+POSTHOOK: Lineage: numericdatatype.e SIMPLE []
+POSTHOOK: Lineage: numericdatatype.f SIMPLE []
+PREHOOK: query: SELECT * FROM numericDataType
+PREHOOK: type: QUERY
+PREHOOK: Input: default@numericdatatype
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM numericDataType
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@numericdatatype
+#### A masked pattern was here ####
+3	32767	2147483647	9223372036854775807	3.4E38	1234567.89
+PREHOOK: query: EXPLAIN INSERT INTO numericDataType(e,f) values(4.5, 678.4)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN INSERT INTO numericDataType(e,f) values(4.5, 678.4)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: _dummy_table
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: array(const struct(4.5,678.4)) (type: array<struct<col1:decimal(2,1),col2:decimal(4,1)>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    UDTF Operator
+                      Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      function name: inline
+                      Select Operator
+                        expressions: 127Y (type: tinyint), 32767S (type: smallint), 2147483647 (type: int), 9223372036854775807L (type: bigint), col1 (type: decimal(2,1)), col2 (type: decimal(4,1))
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                        Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          sort order: 
+                          Map-reduce partition columns: _col0 (type: tinyint)
+                          Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: decimal(2,1)), _col5 (type: decimal(4,1))
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: int), VALUE._col3 (type: bigint), UDFToDouble(VALUE._col4) (type: double), CAST( VALUE._col5 AS decimal(9,2)) (type: decimal(9,2))
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 1 Data size: 140 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 140 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+                      serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+                      name: default.numericdatatype
+                  Write Type: INSERT
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.numericdatatype
+          Write Type: INSERT
+
+PREHOOK: query: INSERT INTO numericDataType(e,f) values(4.5, 678.4)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@numericdatatype
+POSTHOOK: query: INSERT INTO numericDataType(e,f) values(4.5, 678.4)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@numericdatatype
+POSTHOOK: Lineage: numericdatatype.a SIMPLE []
+POSTHOOK: Lineage: numericdatatype.b SIMPLE []
+POSTHOOK: Lineage: numericdatatype.c SIMPLE []
+POSTHOOK: Lineage: numericdatatype.d SIMPLE []
+POSTHOOK: Lineage: numericdatatype.e SCRIPT []
+POSTHOOK: Lineage: numericdatatype.f SCRIPT []
+PREHOOK: query: SELECT * FROM numericDataType
+PREHOOK: type: QUERY
+PREHOOK: Input: default@numericdatatype
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM numericDataType
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@numericdatatype
+#### A masked pattern was here ####
+3	32767	2147483647	9223372036854775807	3.4E38	1234567.89
+127	32767	2147483647	9223372036854775807	4.5	678.40
+PREHOOK: query: DROP TABLE numericDataType
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@numericdatatype
+PREHOOK: Output: default@numericdatatype
+POSTHOOK: query: DROP TABLE numericDataType
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@numericdatatype
+POSTHOOK: Output: default@numericdatatype
+PREHOOK: query: -- Date/time
+CREATE TABLE table1(d DATE DEFAULT DATE'2018-02-14', t TIMESTAMP DEFAULT TIMESTAMP'2016-02-22 12:45:07.000000000',
+    tz timestamp with local time zone DEFAULT TIMESTAMPLOCALTZ'2016-01-03 12:26:34 America/Los_Angeles',
+    d1 DATE DEFAULT current_date() ENABLE, t1 TIMESTAMP DEFAULT current_timestamp() DISABLE)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table1
+POSTHOOK: query: -- Date/time
+CREATE TABLE table1(d DATE DEFAULT DATE'2018-02-14', t TIMESTAMP DEFAULT TIMESTAMP'2016-02-22 12:45:07.000000000',
+    tz timestamp with local time zone DEFAULT TIMESTAMPLOCALTZ'2016-01-03 12:26:34 America/Los_Angeles',
+    d1 DATE DEFAULT current_date() ENABLE, t1 TIMESTAMP DEFAULT current_timestamp() DISABLE)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table1
+PREHOOK: query: DESC FORMATTED table1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table1
+POSTHOOK: query: DESC FORMATTED table1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table1
+# col_name            	data_type           	comment             
+d                   	date                	                    
+t                   	timestamp           	                    
+tz                  	timestamp with local time zone('US/Pacific')	                    
+d1                  	date                	                    
+t1                  	timestamp           	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
+InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+	 	 
+# Constraints	 	 
+	 	 
+# Default Constraints	 	 
+Table:              	default.table1      	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:d1      	Default Value:CURRENT_DATE()	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:d       	Default Value:DATE'2018-02-14'	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:t       	Default Value:TIMESTAMP'2016-02-22 12:45:07.0'	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:t1      	Default Value:CURRENT_TIMESTAMP()	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:tz      	Default Value:TIMESTAMPLOCALTZ'2016-01-03 12:26:34.0 America/Los_Angeles'	 
+	 	 
+PREHOOK: query: EXPLAIN INSERT INTO table1(t) values ("1985-12-31 12:45:07")
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN INSERT INTO table1(t) values ("1985-12-31 12:45:07")
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: _dummy_table
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: array(const struct('1985-12-31 12:45:07')) (type: array<struct<col1:string>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    UDTF Operator
+                      Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      function name: inline
+                      Select Operator
+                        expressions: DATE'2018-02-14' (type: date), CAST( col1 AS TIMESTAMP) (type: timestamp), TIMESTAMPLOCALTZ'2016-01-03 12:26:34.0 US/Pacific' (type: timestamp with local time zone), CURRENT_DATE() (type: date), null (type: timestamp)
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                        Statistics: Num rows: 1 Data size: 232 Basic stats: COMPLETE Column stats: COMPLETE
+                        File Output Operator
+                          compressed: false
+                          Statistics: Num rows: 1 Data size: 232 Basic stats: COMPLETE Column stats: COMPLETE
+                          table:
+                              input format: org.apache.hadoop.mapred.TextInputFormat
+                              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                              name: default.table1
+            Execution mode: llap
+            LLAP IO: no inputs
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.table1
+
+PREHOOK: query: INSERT INTO table1(t) values ("1985-12-31 12:45:07")
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@table1
+POSTHOOK: query: INSERT INTO table1(t) values ("1985-12-31 12:45:07")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@table1
+POSTHOOK: Lineage: table1.d SIMPLE []
+POSTHOOK: Lineage: table1.d1 EXPRESSION []
+POSTHOOK: Lineage: table1.t SCRIPT []
+POSTHOOK: Lineage: table1.t1 SIMPLE []
+POSTHOOK: Lineage: table1.tz SIMPLE []
+PREHOOK: query: SELECT d, t, tz,d1=current_date(), t1 from table1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@table1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT d, t, tz,d1=current_date(), t1 from table1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@table1
+#### A masked pattern was here ####
+2018-02-14	1985-12-31 12:45:07	2016-01-03 12:26:34.0 US/Pacific	true	NULL
+PREHOOK: query: EXPLAIN INSERT INTO table1(d, t1) values ("1985-12-31", '2018-02-27 17:32:14.259')
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN INSERT INTO table1(d, t1) values ("1985-12-31", '2018-02-27 17:32:14.259')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: _dummy_table
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: array(const struct('1985-12-31','2018-02-27 17:32:14.259')) (type: array<struct<col1:string,col2:string>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    UDTF Operator
+                      Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      function name: inline
+                      Select Operator
+                        expressions: CAST( col1 AS DATE) (type: date), TIMESTAMP'2016-02-22 12:45:07.0' (type: timestamp), TIMESTAMPLOCALTZ'2016-01-03 12:26:34.0 US/Pacific' (type: timestamp with local time zone), CURRENT_DATE() (type: date), CAST( col2 AS TIMESTAMP) (type: timestamp)
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                        Statistics: Num rows: 1 Data size: 232 Basic stats: COMPLETE Column stats: COMPLETE
+                        File Output Operator
+                          compressed: false
+                          Statistics: Num rows: 1 Data size: 232 Basic stats: COMPLETE Column stats: COMPLETE
+                          table:
+                              input format: org.apache.hadoop.mapred.TextInputFormat
+                              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                              name: default.table1
+            Execution mode: llap
+            LLAP IO: no inputs
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.table1
+
+PREHOOK: query: INSERT INTO table1(d, t1) values ("1985-12-31", '2018-02-27 17:32:14.259')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@table1
+POSTHOOK: query: INSERT INTO table1(d, t1) values ("1985-12-31", '2018-02-27 17:32:14.259')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@table1
+POSTHOOK: Lineage: table1.d SCRIPT []
+POSTHOOK: Lineage: table1.d1 EXPRESSION []
+POSTHOOK: Lineage: table1.t SIMPLE []
+POSTHOOK: Lineage: table1.t1 SCRIPT []
+POSTHOOK: Lineage: table1.tz SIMPLE []
+PREHOOK: query: SELECT d, t, tz,d1=current_date(), t1=current_timestamp() from table1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@table1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT d, t, tz,d1=current_date(), t1=current_timestamp() from table1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@table1
+#### A masked pattern was here ####
+2018-02-14	1985-12-31 12:45:07	2016-01-03 12:26:34.0 US/Pacific	true	NULL
+1985-12-31	2016-02-22 12:45:07	2016-01-03 12:26:34.0 US/Pacific	true	false
+PREHOOK: query: DROP TABLE table1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@table1
+PREHOOK: Output: default@table1
+POSTHOOK: query: DROP TABLE table1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@table1
+POSTHOOK: Output: default@table1
+PREHOOK: query: CREATE TABLE table2(i STRING DEFAULT 'current_database()', j STRING DEFAULT current_user(),
+    k STRING DEFAULT 'Current_User()', v varchar(350) DEFAULT cast('varchar_default_value' as varchar(350)),
+    c char(20) DEFAULT cast('char_value' as char(20)))
+    clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table2
+POSTHOOK: query: CREATE TABLE table2(i STRING DEFAULT 'current_database()', j STRING DEFAULT current_user(),
+    k STRING DEFAULT 'Current_User()', v varchar(350) DEFAULT cast('varchar_default_value' as varchar(350)),
+    c char(20) DEFAULT cast('char_value' as char(20)))
+    clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table2
+PREHOOK: query: DESC FORMATTED table2
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table2
+POSTHOOK: query: DESC FORMATTED table2
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table2
+# col_name            	data_type           	comment             
+i                   	string              	                    
+j                   	string              	                    
+k                   	string              	                    
+v                   	varchar(350)        	                    
+c                   	char(20)            	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	transactional       	true                
+	transactional_properties	default             
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.ql.io.orc.OrcSerde	 
+InputFormat:        	org.apache.hadoop.hive.ql.io.orc.OrcInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	2                   	 
+Bucket Columns:     	[i]                 	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+	 	 
+# Constraints	 	 
+	 	 
+# Default Constraints	 	 
+Table:              	default.table2      	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:j       	Default Value:CURRENT_USER()	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:k       	Default Value:'Current_User()'	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:v       	Default Value:CAST( 'varchar_default_value' AS varchar(350))	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:c       	Default Value:CAST( 'char_value' AS CHAR(20))	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:i       	Default Value:'current_database()'	 
+	 	 
+PREHOOK: query: EXPLAIN INSERT INTO table2(i) values('default')
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN INSERT INTO table2(i) values('default')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: _dummy_table
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: array(const struct('default')) (type: array<struct<col1:string>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    UDTF Operator
+                      Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      function name: inline
+                      Select Operator
+                        expressions: col1 (type: string), CURRENT_USER() (type: string), 'Current_User()' (type: string), CAST( 'varchar_default_value' AS varchar(350)) (type: varchar(350)), CAST( 'char_value' AS CHAR(20)) (type: char(20))
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                        Statistics: Num rows: 1 Data size: 405 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          sort order: 
+                          Map-reduce partition columns: _col0 (type: string)
+                          Statistics: Num rows: 1 Data size: 405 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: varchar(350)), _col4 (type: char(20))
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: varchar(350)), VALUE._col4 (type: char(20))
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                Statistics: Num rows: 1 Data size: 405 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 405 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+                      serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+                      name: default.table2
+                  Write Type: INSERT
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.table2
+          Write Type: INSERT
+
+PREHOOK: query: INSERT INTO table2(i) values('default')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@table2
+POSTHOOK: query: INSERT INTO table2(i) values('default')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@table2
+POSTHOOK: Lineage: table2.c EXPRESSION []
+POSTHOOK: Lineage: table2.i SCRIPT []
+POSTHOOK: Lineage: table2.j EXPRESSION []
+POSTHOOK: Lineage: table2.k SIMPLE []
+POSTHOOK: Lineage: table2.v EXPRESSION []
+PREHOOK: query: SELECT i,j=current_user(),k,v,c FROM table2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@table2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT i,j=current_user(),k,v,c FROM table2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@table2
+#### A masked pattern was here ####
+default	true	Current_User()	varchar_default_value	char_value          
+PREHOOK: query: EXPLAIN INSERT INTO table2(v, c) values('varchar_default2', 'char')
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN INSERT INTO table2(v, c) values('varchar_default2', 'char')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: _dummy_table
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: array(const struct('varchar_default2','char')) (type: array<struct<col1:string,col2:string>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    UDTF Operator
+                      Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      function name: inline
+                      Select Operator
+                        expressions: 'current_database()' (type: string), CURRENT_USER() (type: string), 'Current_User()' (type: string), col1 (type: string), col2 (type: string)
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                        Statistics: Num rows: 1 Data size: 298 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          sort order: 
+                          Map-reduce partition columns: _col0 (type: string)
+                          Statistics: Num rows: 1 Data size: 298 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string)
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), CAST( VALUE._col3 AS varchar(350)) (type: varchar(350)), CAST( VALUE._col4 AS CHAR(20)) (type: char(20))
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                Statistics: Num rows: 1 Data size: 836 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 836 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+                      serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+                      name: default.table2
+                  Write Type: INSERT
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.table2
+          Write Type: INSERT
+
+PREHOOK: query: INSERT INTO table2(v, c) values('varchar_default2', 'char')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@table2
+POSTHOOK: query: INSERT INTO table2(v, c) values('varchar_default2', 'char')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@table2
+POSTHOOK: Lineage: table2.c SCRIPT []
+POSTHOOK: Lineage: table2.i SIMPLE []
+POSTHOOK: Lineage: table2.j EXPRESSION []
+POSTHOOK: Lineage: table2.k SIMPLE []
+POSTHOOK: Lineage: table2.v SCRIPT []
+PREHOOK: query: SELECT i,j=current_user(),k,v,c FROM table2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@table2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT i,j=current_user(),k,v,c FROM table2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@table2
+#### A masked pattern was here ####
+default	true	Current_User()	varchar_default_value	char_value          
+current_database()	true	Current_User()	varchar_default2	char                
+PREHOOK: query: DROP TABLE table2
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@table2
+PREHOOK: Output: default@table2
+POSTHOOK: query: DROP TABLE table2
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@table2
+POSTHOOK: Output: default@table2
+PREHOOK: query: CREATE TABLE misc(b BOOLEAN DEFAULT true, b1 BINARY DEFAULT cast('bin' as binary))
+    clustered by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@misc
+POSTHOOK: query: CREATE TABLE misc(b BOOLEAN DEFAULT true, b1 BINARY DEFAULT cast('bin' as binary))
+    clustered by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@misc
+PREHOOK: query: DESC FORMATTED misc
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@misc
+POSTHOOK: query: DESC FORMATTED misc
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@misc
+# col_name            	data_type           	comment             
+b                   	boolean             	                    
+b1                  	binary              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	transactional       	true                
+	transactional_properties	default             
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.ql.io.orc.OrcSerde	 
+InputFormat:        	org.apache.hadoop.hive.ql.io.orc.OrcInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	2                   	 
+Bucket Columns:     	[b]                 	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+	 	 
+# Constraints	 	 
+	 	 
+# Default Constraints	 	 
+Table:              	default.misc        	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:b       	Default Value:true  	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:b1      	Default Value:CAST( 'bin' AS BINARY)	 
+	 	 
+PREHOOK: query: EXPLAIN INSERT INTO misc(b) values(false)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN INSERT INTO misc(b) values(false)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: _dummy_table
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: array(const struct(false)) (type: array<struct<col1:boolean>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    UDTF Operator
+                      Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      function name: inline
+                      Select Operator
+                        expressions: col1 (type: boolean), CAST( 'bin' AS BINARY) (type: binary)
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          sort order: 
+                          Map-reduce partition columns: _col0 (type: boolean)
+                          Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: boolean), _col1 (type: binary)
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: boolean), VALUE._col1 (type: binary)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+                      serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+                      name: default.misc
+                  Write Type: INSERT
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.misc
+          Write Type: INSERT
+
+PREHOOK: query: INSERT INTO misc(b) values(false)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@misc
+POSTHOOK: query: INSERT INTO misc(b) values(false)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@misc
+POSTHOOK: Lineage: misc.b SCRIPT []
+POSTHOOK: Lineage: misc.b1 EXPRESSION []
+PREHOOK: query: SELECT b, b1 from misc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@misc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT b, b1 from misc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@misc
+#### A masked pattern was here ####
+false	bin
+PREHOOK: query: EXPLAIN INSERT INTO misc(b1) values('011')
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN INSERT INTO misc(b1) values('011')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: _dummy_table
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: array(const struct('011')) (type: array<struct<col1:string>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    UDTF Operator
+                      Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      function name: inline
+                      Select Operator
+                        expressions: true (type: boolean), col1 (type: string)
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          sort order: 
+                          Map-reduce partition columns: _col0 (type: boolean)
+                          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: boolean), _col1 (type: string)
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: boolean), CAST( VALUE._col1 AS BINARY) (type: binary)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 148 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 148 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+                      serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+                      name: default.misc
+                  Write Type: INSERT
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.misc
+          Write Type: INSERT
+
+PREHOOK: query: INSERT INTO misc(b) values(false)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@misc
+POSTHOOK: query: INSERT INTO misc(b) values(false)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@misc
+POSTHOOK: Lineage: misc.b SCRIPT []
+POSTHOOK: Lineage: misc.b1 EXPRESSION []
+PREHOOK: query: SELECT b, b1 from misc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@misc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT b, b1 from misc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@misc
+#### A masked pattern was here ####
+false	bin
+false	bin
+PREHOOK: query: DROP TABLE misc
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@misc
+PREHOOK: Output: default@misc
+POSTHOOK: query: DROP TABLE misc
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@misc
+POSTHOOK: Output: default@misc
+PREHOOK: query: CREATE table t11(i int default cast(cast(4 as double) as int),
+    b1 boolean default cast ('true' as boolean), b2 int default cast (5.67 as int),
+    b3 tinyint default cast (45 as tinyint), b4 float default cast (45.4 as float),
+    b5 bigint default cast (567 as bigint), b6 smallint default cast (88 as smallint),
+    j varchar(50) default cast(current_timestamp() as varchar(50)),
+     k string default cast(cast(current_user() as varchar(50)) as string),
+     tz1 timestamp with local time zone DEFAULT cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with local time zone),
+     ts timestamp default cast('2016-01-01 12:01:01' as timestamp),
+     dc decimal(8,2) default cast(4.5 as decimal(8,2)),
+     c2 double default cast(5 as double), c4 char(2) default cast(cast(cast('ab' as string) as varchar(2)) as char(2)))
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t11
+POSTHOOK: query: CREATE table t11(i int default cast(cast(4 as double) as int),
+    b1 boolean default cast ('true' as boolean), b2 int default cast (5.67 as int),
+    b3 tinyint default cast (45 as tinyint), b4 float default cast (45.4 as float),
+    b5 bigint default cast (567 as bigint), b6 smallint default cast (88 as smallint),
+    j varchar(50) default cast(current_timestamp() as varchar(50)),
+     k string default cast(cast(current_user() as varchar(50)) as string),
+     tz1 timestamp with local time zone DEFAULT cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with local time zone),
+     ts timestamp default cast('2016-01-01 12:01:01' as timestamp),
+     dc decimal(8,2) default cast(4.5 as decimal(8,2)),
+     c2 double default cast(5 as double), c4 char(2) default cast(cast(cast('ab' as string) as varchar(2)) as char(2)))
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t11
+PREHOOK: query: DESC FORMATTED t11
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@t11
+POSTHOOK: query: DESC FORMATTED t11
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@t11
+# col_name            	data_type           	comment             
+i                   	int                 	                    
+b1                  	boolean             	                    
+b2                  	int                 	                    
+b3                  	tinyint             	                    
+b4                  	float               	                    
+b5                  	bigint              	                    
+b6                  	smallint            	                    
+j                   	varchar(50)         	                    
+k                   	string              	                    
+tz1                 	timestamp with local time zone('US/Pacific')	                    
+ts                  	timestamp           	                    
+dc                  	decimal(8,2)        	                    
+c2                  	double              	                    
+c4                  	char(2)             	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
+InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+	 	 
+# Constraints	 	 
+	 	 
+# Default Constraints	 	 
+Table:              	default.t11         	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:tz1     	Default Value:CAST( '2016-01-03 12:26:34 America/Los_Angeles' AS timestamp with local time zone)	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:b1      	Default Value:UDFToBoolean('true')	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:i       	Default Value:UDFToInteger(UDFToDouble(4))	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:b2      	Default Value:UDFToInteger(5.67)	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:j       	Default Value:CAST( CURRENT_TIMESTAMP() AS varchar(50))	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:b3      	Default Value:UDFToByte(45)	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:k       	Default Value:UDFToString(CAST( CURRENT_USER() AS varchar(50)))	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:b4      	Default Value:UDFToFloat(45.4)	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:b5      	Default Value:UDFToLong(567)	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:b6      	Default Value:UDFToShort(88)	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:c2      	Default Value:UDFToDouble(5)	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:c4      	Default Value:CAST( CAST( 'ab' AS varchar(2)) AS CHAR(2))	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:ts      	Default Value:CAST( '2016-01-01 12:01:01' AS TIMESTAMP)	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:dc      	Default Value:CAST( 4.5 AS decimal(8,2))	 
+	 	 
+PREHOOK: query: EXPLAIN INSERT INTO t11(c4) values('vi')
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN INSERT INTO t11(c4) values('vi')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: _dummy_table
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: array(const struct('vi')) (type: array<struct<col1:string>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    UDTF Operator
+                      Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      function name: inline
+                      Select Operator
+                        expressions: UDFToInteger(UDFToDouble(4)) (type: int), UDFToBoolean('true') (type: boolean), UDFToInteger(5.67) (type: int), UDFToByte(45) (type: tinyint), UDFToFloat(45.4) (type: float), UDFToLong(567) (type: bigint), UDFToShort(88) (type: smallint), CAST( CURRENT_TIMESTAMP() AS varchar(50)) (type: varchar(50)), UDFToString(CAST( CURRENT_USER() AS varchar(50))) (type: string), CAST( '2016-01-03 12:26:34 America/Los_Angeles' AS timestamp with local time zone) (type: timestamp with local time zone), CAST( '2016-01-01 12:01:01' AS TIMESTAMP) (type: timestamp), CAST( 4.5 AS decimal(8,2)) (type: decimal(8,2)), UDFToDouble(5) (type: double), CAST( col1 AS CHAR(2)) (type: char(2))
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
+                        Statistics: Num rows: 1 Data size: 523 Basic stats: COMPLETE Column stats: COMPLETE
+                        File Output Operator
+                          compressed: false
+                          Statistics: Num rows: 1 Data size: 523 Basic stats: COMPLETE Column stats: COMPLETE
+                          table:
+                              input format: org.apache.hadoop.mapred.TextInputFormat
+                              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                              name: default.t11
+            Execution mode: llap
+            LLAP IO: no inputs
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.t11
+
+PREHOOK: query: INSERT INTO t11(c4) values('vi')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@t11
+POSTHOOK: query: INSERT INTO t11(c4) values('vi')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@t11
+POSTHOOK: Lineage: t11.b1 EXPRESSION []
+POSTHOOK: Lineage: t11.b2 EXPRESSION []
+POSTHOOK: Lineage: t11.b3 EXPRESSION []
+POSTHOOK: Lineage: t11.b4 EXPRESSION []
+POSTHOOK: Lineage: t11.b5 EXPRESSION []
+POSTHOOK: Lineage: t11.b6 EXPRESSION []
+POSTHOOK: Lineage: t11.c2 EXPRESSION []
+POSTHOOK: Lineage: t11.c4 SCRIPT []
+POSTHOOK: Lineage: t11.dc EXPRESSION []
+POSTHOOK: Lineage: t11.i EXPRESSION []
+POSTHOOK: Lineage: t11.j EXPRESSION []
+POSTHOOK: Lineage: t11.k EXPRESSION []
+POSTHOOK: Lineage: t11.ts EXPRESSION []
+POSTHOOK: Lineage: t11.tz1 EXPRESSION []
+PREHOOK: query: SELECT ts, tz1, dc, b1,b2,b3,b4,b5,b6,j=cast(current_timestamp() as varchar(50)), k=cast(current_user() as string), c2, c4 from t11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t11
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT ts, tz1, dc, b1,b2,b3,b4,b5,b6,j=cast(current_timestamp() as varchar(50)), k=cast(current_user() as string), c2, c4 from t11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t11
+#### A masked pattern was here ####
+2016-01-01 12:01:01	2016-01-03 12:26:34.0 US/Pacific	4.50	true	5	45	45.4	567	88	false	true	5.0	vi
+PREHOOK: query: EXPLAIN INSERT INTO t11(b1,c4) values(true,'ga')
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN INSERT INTO t11(b1,c4) values(true,'ga')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: _dummy_table
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: array(const struct(true,'ga')) (type: array<struct<col1:boolean,col2:string>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    UDTF Operator
+                      Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      function name: inline
+                      Select Operator
+                        expressions: UDFToInteger(UDFToDouble(4)) (type: int), col1 (type: boolean), UDFToInteger(5.67) (type: int), UDFToByte(45) (type: tinyint), UDFToFloat(45.4) (type: float), UDFToLong(567) (type: bigint), UDFToShort(88) (type: smallint), CAST( CURRENT_TIMESTAMP() AS varchar(50)) (type: varchar(50)), UDFToString(CAST( CURRENT_USER() AS varchar(50))) (type: string), CAST( '2016-01-03 12:26:34 America/Los_Angeles' AS timestamp with local time zone) (type: timestamp with local time zone), CAST( '2016-01-01 12:01:01' AS TIMESTAMP) (type: timestamp), CAST( 4.5 AS decimal(8,2)) (type: decimal(8,2)), UDFToDouble(5) (type: double), CAST( col2 AS CHAR(2)) (type: char(2))
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
+                        Statistics: Num rows: 1 Data size: 519 Basic stats: COMPLETE Column stats: COMPLETE
+                        File Output Operator
+                          compressed: false
+                          Statistics: Num rows: 1 Data size: 519 Basic stats: COMPLETE Column stats: COMPLETE
+                          table:
+                              input format: org.apache.hadoop.mapred.TextInputFormat
+                              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                              name: default.t11
+            Execution mode: llap
+            LLAP IO: no inputs
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.t11
+
+PREHOOK: query: INSERT INTO t11(c4) values('vi')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@t11
+POSTHOOK: query: INSERT INTO t11(c4) values('vi')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@t11
+POSTHOOK: Lineage: t11.b1 EXPRESSION []
+POSTHOOK: Lineage: t11.b2 EXPRESSION []
+POSTHOOK: Lineage: t11.b3 EXPRESSION []
+POSTHOOK: Lineage: t11.b4 EXPRESSION []
+POSTHOOK: Lineage: t11.b5 EXPRESSION []
+POSTHOOK: Lineage: t11.b6 EXPRESSION []
+POSTHOOK: Lineage: t11.c2 EXPRESSION []
+POSTHOOK: Lineage: t11.c4 SCRIPT []
+POSTHOOK: Lineage: t11.dc EXPRESSION []
+POSTHOOK: Lineage: t11.i EXPRESSION []
+POSTHOOK: Lineage: t11.j EXPRESSION []
+POSTHOOK: Lineage: t11.k EXPRESSION []
+POSTHOOK: Lineage: t11.ts EXPRESSION []
+POSTHOOK: Lineage: t11.tz1 EXPRESSION []
+PREHOOK: query: SELECT ts, tz1, dc, b1,b2,b3,b4,b5,b6,j=cast(current_timestamp() as varchar(50)), k=cast(current_user() as string), c2, c4 from t11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t11
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT ts, tz1, dc, b1,b2,b3,b4,b5,b6,j=cast(current_timestamp() as varchar(50)), k=cast(current_user() as string), c2, c4 from t11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t11
+#### A masked pattern was here ####
+2016-01-01 12:01:01	2016-01-03 12:26:34.0 US/Pacific	4.50	true	5	45	45.4	567	88	false	true	5.0	vi
+2016-01-01 12:01:01	2016-01-03 12:26:34.0 US/Pacific	4.50	true	5	45	45.4	567	88	false	true	5.0	vi
+PREHOOK: query: DROP TABLE t11
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@t11
+PREHOOK: Output: default@t11
+POSTHOOK: query: DROP TABLE t11
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@t11
+POSTHOOK: Output: default@t11
+PREHOOK: query: CREATE TABLE numericDataType(a TINYINT CONSTRAINT tinyint_constraint DEFAULT 127Y ENABLE, b SMALLINT DEFAULT 32767S, c INT DEFAULT 2147483647,
+    d BIGINT DEFAULT  9223372036854775807L, e DOUBLE DEFAULT 3.4E38, f DECIMAL(9,2) DEFAULT 1234567.89)
+    clustered by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@numericDataType
+POSTHOOK: query: CREATE TABLE numericDataType(a TINYINT CONSTRAINT tinyint_constraint DEFAULT 127Y ENABLE, b SMALLINT DEFAULT 32767S, c INT DEFAULT 2147483647,
+    d BIGINT DEFAULT  9223372036854775807L, e DOUBLE DEFAULT 3.4E38, f DECIMAL(9,2) DEFAULT 1234567.89)
+    clustered by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@numericDataType
+PREHOOK: query: ALTER TABLE numericDataType DROP CONSTRAINT tinyint_constraint
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+POSTHOOK: query: ALTER TABLE numericDataType DROP CONSTRAINT tinyint_constraint
+POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
+PREHOOK: query: DESC FORMATTED numericDataType
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@numericdatatype
+POSTHOOK: query: DESC FORMATTED numericDataType
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@numericdatatype
+# col_name            	data_type           	comment             
+a                   	tinyint             	                    
+b                   	smallint            	                    
+c                   	int                 	                    
+d                   	bigint              	                    
+e                   	double              	                    
+f                   	decimal(9,2)        	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	transactional       	true                
+	transactional_properties	default             
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.ql.io.orc.OrcSerde	 
+InputFormat:        	org.apache.hadoop.hive.ql.io.orc.OrcInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	2                   	 
+Bucket Columns:     	[b]                 	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+	 	 
+# Constraints	 	 
+	 	 
+# Default Constraints	 	 
+Table:              	default.numericdatatype	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:b       	Default Value:32767S	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:c       	Default Value:2147483647	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:d       	Default Value:9223372036854775807L	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:e       	Default Value:3.4E38D	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:f       	Default Value:1234567.89	 
+	 	 
+PREHOOK: query: EXPLAIN INSERT INTO numericDataType(b) values(456)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN INSERT INTO numericDataType(b) values(456)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: _dummy_table
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: array(const struct(456)) (type: array<struct<col1:int>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    UDTF Operator
+                      Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      function name: inline
+                      Select Operator
+                        expressions: null (type: tinyint), col1 (type: int), 2147483647 (type: int), 9223372036854775807L (type: bigint), 3.4E38D (type: double), 1234567.89 (type: decimal(9,2))
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                        Statistics: Num rows: 1 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          sort order: 
+                          Map-reduce partition columns: UDFToShort(_col1) (type: smallint)
+                          Statistics: Num rows: 1 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: tinyint), _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: double), _col5 (type: decimal(9,2))
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: tinyint), UDFToShort(VALUE._col1) (type: smallint), VALUE._col2 (type: int), VALUE._col3 (type: bigint), VALUE._col4 (type: double), VALUE._col5 (type: decimal(9,2))
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 1 Data size: 140 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 140 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+                      serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+                      name: default.numericdatatype
+                  Write Type: INSERT
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.numericdatatype
+          Write Type: INSERT
+
+PREHOOK: query: INSERT INTO numericDataType(b) values(456)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@numericdatatype
+POSTHOOK: query: INSERT INTO numericDataType(b) values(456)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@numericdatatype
+POSTHOOK: Lineage: numericdatatype.a SIMPLE []
+POSTHOOK: Lineage: numericdatatype.b SCRIPT []
+POSTHOOK: Lineage: numericdatatype.c SIMPLE []
+POSTHOOK: Lineage: numericdatatype.d SIMPLE []
+POSTHOOK: Lineage: numericdatatype.e SIMPLE []
+POSTHOOK: Lineage: numericdatatype.f SIMPLE []
+PREHOOK: query: SELECT * from numericDataType
+PREHOOK: type: QUERY
+PREHOOK: Input: default@numericdatatype
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * from numericDataType
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@numericdatatype
+#### A masked pattern was here ####
+NULL	456	2147483647	9223372036854775807	3.4E38	1234567.89
+PREHOOK: query: ALTER TABLE numericDataType ADD CONSTRAINT uk1 UNIQUE(a,b) DISABLE NOVALIDATE
+PREHOOK: type: ALTERTABLE_ADDCONSTRAINT
+POSTHOOK: query: ALTER TABLE numericDataType ADD CONSTRAINT uk1 UNIQUE(a,b) DISABLE NOVALIDATE
+POSTHOOK: type: ALTERTABLE_ADDCONSTRAINT
+PREHOOK: query: DESC FORMATTED numericDataType
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@numericdatatype
+POSTHOOK: query: DESC FORMATTED numericDataType
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@numericdatatype
+# col_name            	data_type           	comment             
+a                   	tinyint             	                    
+b                   	smallint            	                    
+c                   	int                 	                    
+d                   	bigint              	                    
+e                   	double              	                    
+f                   	decimal(9,2)        	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	numFiles            	1                   
+	totalSize           	1063                
+	transactional       	true                
+	transactional_properties	default             
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.ql.io.orc.OrcSerde	 
+InputFormat:        	org.apache.hadoop.hive.ql.io.orc.OrcInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	2                   	 
+Bucket Columns:     	[b]                 	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+	 	 
+# Constraints	 	 
+	 	 
+# Unique Constraints	 	 
+Table:              	default.numericdatatype	 
+Constraint Name:    	uk1                 	 
+Column Name:a       	Key Sequence:1      	 
+Column Name:b       	Key Sequence:2      	 
+	 	 
+	 	 
+# Default Constraints	 	 
+Table:              	default.numericdatatype	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:b       	Default Value:32767S	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:c       	Default Value:2147483647	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:d       	Default Value:9223372036854775807L	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:e       	Default Value:3.4E38D	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:f       	Default Value:1234567.89	 
+	 	 
+PREHOOK: query: EXPLAIN INSERT INTO numericDataType(b) values(56)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN INSERT INTO numericDataType(b) values(56)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: _dummy_table
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: array(const struct(56)) (type: array<struct<col1:int>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    UDTF Operator
+                      Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      function name: inline
+                      Select Operator
+                        expressions: null (type: tinyint), col1 (type: int), 2147483647 (type: int), 9223372036854775807L (type: bigint), 3.4E38D (type: double), 1234567.89 (type: decimal(9,2))
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                        Statistics: Num rows: 1 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          sort order: 
+                          Map-reduce partition columns: UDFToShort(_col1) (type: smallint)
+                          Statistics: Num rows: 1 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: tinyint), _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: double), _col5 (type: decimal(9,2))
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: tinyint), UDFToShort(VALUE._col1) (type: smallint), VALUE._col2 (type: int), VALUE._col3 (type: bigint), VALUE._col4 (type: double), VALUE._col5 (type: decimal(9,2))
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 1 Data size: 140 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 140 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+                      serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+                      name: default.numericdatatype
+                  Write Type: INSERT
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.numericdatatype
+          Write Type: INSERT
+
+PREHOOK: query: INSERT INTO numericDataType(b) values(456)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@numericdatatype
+POSTHOOK: query: INSERT INTO numericDataType(b) values(456)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@numericdatatype
+POSTHOOK: Lineage: numericdatatype.a SIMPLE []
+POSTHOOK: Lineage: numericdatatype.b SCRIPT []
+POSTHOOK: Lineage: numericdatatype.c SIMPLE []
+POSTHOOK: Lineage: numericdatatype.d SIMPLE []
+POSTHOOK: Lineage: numericdatatype.e SIMPLE []
+POSTHOOK: Lineage: numericdatatype.f SIMPLE []
+PREHOOK: query: SELECT * from numericDataType
+PREHOOK: type: QUERY
+PREHOOK: Input: default@numericdatatype
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * from numericDataType
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@numericdatatype
+#### A masked pattern was here ####
+NULL	456	2147483647	9223372036854775807	3.4E38	1234567.89
+NULL	456	2147483647	9223372036854775807	3.4E38	1234567.89
+PREHOOK: query: ALTER TABLE numericDataType CHANGE a a TINYINT CONSTRAINT second_null_constraint NOT NULL ENABLE
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@numericdatatype
+PREHOOK: Output: default@numericdatatype
+POSTHOOK: query: ALTER TABLE numericDataType CHANGE a a TINYINT CONSTRAINT second_null_constraint NOT NULL ENABLE
+POSTHOOK: type: ALTERTABLE_RENAMECOL
+POSTHOOK: Input: default@numericdatatype
+POSTHOOK: Output: default@numericdatatype
+PREHOOK: query: DESC FORMATTED numericDataType
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@numericdatatype
+POSTHOOK: query: DESC FORMATTED numericDataType
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@numericdatatype
+# col_name            	data_type           	comment             
+a                   	tinyint             	                    
+b                   	smallint            	                    
+c                   	int                 	                    
+d                   	bigint              	                    
+e                   	double              	                    
+f                   	decimal(9,2)        	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+#### A masked pattern was here ####
+	numFiles            	2                   
+	totalSize           	2127                
+	transactional       	true                
+	transactional_properties	default             
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.ql.io.orc.OrcSerde	 
+InputFormat:        	org.apache.hadoop.hive.ql.io.orc.OrcInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	2                   	 
+Bucket Columns:     	[b]                 	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+	 	 
+# Constraints	 	 
+	 	 
+# Unique Constraints	 	 
+Table:              	default.numericdatatype	 
+Constraint Name:    	uk1                 	 
+Column Name:a       	Key Sequence:1      	 
+Column Name:b       	Key Sequence:2      	 
+	 	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	default.numericdatatype	 
+Constraint Name:    	second_null_constraint	 
+Column Name:        	a                   	 
+	 	 
+	 	 
+# Default Constraints	 	 
+Table:              	default.numericdatatype	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:b       	Default Value:32767S	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:c       	Default Value:2147483647	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:d       	Default Value:9223372036854775807L	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:e       	Default Value:3.4E38D	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:f       	Default Value:1234567.89	 
+	 	 
+PREHOOK: query: ALTER TABLE numericDataType CHANGE a a TINYINT CONSTRAINT default_constraint DEFAULT 127Y ENABLE
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@numericdatatype
+PREHOOK: Output: default@numericdatatype
+POSTHOOK: query: ALTER TABLE numericDataType CHANGE a a TINYINT CONSTRAINT default_constraint DEFAULT 127Y ENABLE
+POSTHOOK: type: ALTERTABLE_RENAMECOL
+POSTHOOK: Input: default@numericdatatype
+POSTHOOK: Output: default@numericdatatype
+PREHOOK: query: DESC FORMATTED numericDataType
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@numericdatatype
+POSTHOOK: query: DESC FORMATTED numericDataType
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@numericdatatype
+# col_name            	data_type           	comment             
+a                   	tinyint             	                    
+b                   	smallint            	                    
+c                   	int                 	                    
+d                   	bigint              	                    
+e                   	double              	                    
+f                   	decimal(9,2)        	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+#### A masked pattern was here ####
+	numFiles            	2                   
+	totalSize           	2127                
+	transactional       	true                
+	transactional_properties	default             
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.ql.io.orc.OrcSerde	 
+InputFormat:        	org.apache.hadoop.hive.ql.io.orc.OrcInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	2                   	 
+Bucket Columns:     	[b]                 	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+	 	 
+# Constraints	 	 
+	 	 
+# Unique Constraints	 	 
+Table:              	default.numericdatatype	 
+Constraint Name:    	uk1                 	 
+Column Name:a       	Key Sequence:1      	 
+Column Name:b       	Key Sequence:2      	 
+	 	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	default.numericdatatype	 
+Constraint Name:    	second_null_constraint	 
+Column Name:        	a                   	 
+	 	 
+	 	 
+# Default Constraints	 	 
+Table:              	default.numericdatatype	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:b       	Default Value:32767S	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:c       	Default Value:2147483647	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:d       	Default Value:9223372036854775807L	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:e       	Default Value:3.4E38D	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:f       	Default Value:1234567.89	 
+	 	 
+Constraint Name:    	default_constraint  	 
+Column Name:a       	Default Value:127Y  	 
+	 	 
+PREHOOK: query: EXPLAIN INSERT INTO numericDataType(f) values(847.45)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN INSERT INTO numericDataType(f) values(847.45)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: _dummy_table
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: array(const struct(847.45)) (type: array<struct<col1:decimal(5,2)>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    UDTF Operator
+                      Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      function name: inline
+                      Select Operator
+                        expressions: 127Y (type: tinyint), 32767S (type: smallint), 2147483647 (type: int), 9223372036854775807L (type: bigint), 3.4E38D (type: double), col1 (type: decimal(5,2))
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                        Statistics: Num rows: 1 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                        Filter Operator
+                          predicate: enforce_constraint(127Y is not null) (type: boolean)
+                          Statistics: Num rows: 1 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Map-reduce partition columns: _col1 (type: smallint)
+                            Statistics: Num rows: 1 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: double), _col5 (type: decimal(5,2))
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: int), VALUE._col3 (type: bigint), VALUE._col4 (type: double), CAST( VALUE._col5 AS decimal(9,2)) (type: decimal(9,2))
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 1 Data size: 140 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 140 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+                      serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+                      name: default.numericdatatype
+                  Write Type: INSERT
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.numericdatatype
+          Write Type: INSERT
+
+PREHOOK: query: --plan should have both DEFAULT and NOT NULL
+INSERT INTO numericDataType(f) values(847.45)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@numericdatatype
+POSTHOOK: query: --plan should have both DEFAULT and NOT NULL
+INSERT INTO numericDataType(f) values(847.45)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@numericdatatype
+POSTHOOK: Lineage: numericdata

<TRUNCATED>

[03/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index dfddd4a..392e8ca 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -156,7 +156,7 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
-  def create_table_with_constraints(self, tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints):
+  def create_table_with_constraints(self, tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints):
     """
     Parameters:
      - tbl
@@ -164,6 +164,7 @@ class Iface(fb303.FacebookService.Iface):
      - foreignKeys
      - uniqueConstraints
      - notNullConstraints
+     - defaultConstraints
     """
     pass
 
@@ -202,6 +203,13 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def add_default_constraint(self, req):
+    """
+    Parameters:
+     - req
+    """
+    pass
+
   def drop_table(self, dbname, name, deleteData):
     """
     Parameters:
@@ -832,6 +840,13 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def get_default_constraints(self, request):
+    """
+    Parameters:
+     - request
+    """
+    pass
+
   def update_table_column_statistics(self, stats_obj):
     """
     Parameters:
@@ -2105,7 +2120,7 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o4
     return
 
-  def create_table_with_constraints(self, tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints):
+  def create_table_with_constraints(self, tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints):
     """
     Parameters:
      - tbl
@@ -2113,11 +2128,12 @@ class Client(fb303.FacebookService.Client, Iface):
      - foreignKeys
      - uniqueConstraints
      - notNullConstraints
+     - defaultConstraints
     """
-    self.send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints)
+    self.send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints)
     self.recv_create_table_with_constraints()
 
-  def send_create_table_with_constraints(self, tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints):
+  def send_create_table_with_constraints(self, tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints):
     self._oprot.writeMessageBegin('create_table_with_constraints', TMessageType.CALL, self._seqid)
     args = create_table_with_constraints_args()
     args.tbl = tbl
@@ -2125,6 +2141,7 @@ class Client(fb303.FacebookService.Client, Iface):
     args.foreignKeys = foreignKeys
     args.uniqueConstraints = uniqueConstraints
     args.notNullConstraints = notNullConstraints
+    args.defaultConstraints = defaultConstraints
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
@@ -2315,6 +2332,39 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o2
     return
 
+  def add_default_constraint(self, req):
+    """
+    Parameters:
+     - req
+    """
+    self.send_add_default_constraint(req)
+    self.recv_add_default_constraint()
+
+  def send_add_default_constraint(self, req):
+    self._oprot.writeMessageBegin('add_default_constraint', TMessageType.CALL, self._seqid)
+    args = add_default_constraint_args()
+    args.req = req
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_add_default_constraint(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = add_default_constraint_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    return
+
   def drop_table(self, dbname, name, deleteData):
     """
     Parameters:
@@ -5064,6 +5114,41 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o2
     raise TApplicationException(TApplicationException.MISSING_RESULT, "get_not_null_constraints failed: unknown result")
 
+  def get_default_constraints(self, request):
+    """
+    Parameters:
+     - request
+    """
+    self.send_get_default_constraints(request)
+    return self.recv_get_default_constraints()
+
+  def send_get_default_constraints(self, request):
+    self._oprot.writeMessageBegin('get_default_constraints', TMessageType.CALL, self._seqid)
+    args = get_default_constraints_args()
+    args.request = request
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_default_constraints(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_default_constraints_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_default_constraints failed: unknown result")
+
   def update_table_column_statistics(self, stats_obj):
     """
     Parameters:
@@ -8120,6 +8205,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["add_foreign_key"] = Processor.process_add_foreign_key
     self._processMap["add_unique_constraint"] = Processor.process_add_unique_constraint
     self._processMap["add_not_null_constraint"] = Processor.process_add_not_null_constraint
+    self._processMap["add_default_constraint"] = Processor.process_add_default_constraint
     self._processMap["drop_table"] = Processor.process_drop_table
     self._processMap["drop_table_with_environment_context"] = Processor.process_drop_table_with_environment_context
     self._processMap["truncate_table"] = Processor.process_truncate_table
@@ -8191,6 +8277,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["get_foreign_keys"] = Processor.process_get_foreign_keys
     self._processMap["get_unique_constraints"] = Processor.process_get_unique_constraints
     self._processMap["get_not_null_constraints"] = Processor.process_get_not_null_constraints
+    self._processMap["get_default_constraints"] = Processor.process_get_default_constraints
     self._processMap["update_table_column_statistics"] = Processor.process_update_table_column_statistics
     self._processMap["update_partition_column_statistics"] = Processor.process_update_partition_column_statistics
     self._processMap["get_table_column_statistics"] = Processor.process_get_table_column_statistics
@@ -8770,7 +8857,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     iprot.readMessageEnd()
     result = create_table_with_constraints_result()
     try:
-      self._handler.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys, args.uniqueConstraints, args.notNullConstraints)
+      self._handler.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys, args.uniqueConstraints, args.notNullConstraints, args.defaultConstraints)
       msg_type = TMessageType.REPLY
     except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
       raise
@@ -8920,6 +9007,31 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_add_default_constraint(self, seqid, iprot, oprot):
+    args = add_default_constraint_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = add_default_constraint_result()
+    try:
+      self._handler.add_default_constraint(args.req)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchObjectException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except MetaException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("add_default_constraint", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_drop_table(self, seqid, iprot, oprot):
     args = drop_table_args()
     args.read(iprot)
@@ -10734,6 +10846,31 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_get_default_constraints(self, seqid, iprot, oprot):
+    args = get_default_constraints_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_default_constraints_result()
+    try:
+      result.success = self._handler.get_default_constraints(args.request)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except MetaException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except NoSuchObjectException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_default_constraints", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_update_table_column_statistics(self, seqid, iprot, oprot):
     args = update_table_column_statistics_args()
     args.read(iprot)
@@ -13728,10 +13865,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype758, _size755) = iprot.readListBegin()
-          for _i759 in xrange(_size755):
-            _elem760 = iprot.readString()
-            self.success.append(_elem760)
+          (_etype772, _size769) = iprot.readListBegin()
+          for _i773 in xrange(_size769):
+            _elem774 = iprot.readString()
+            self.success.append(_elem774)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13754,8 +13891,8 @@ class get_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter761 in self.success:
-        oprot.writeString(iter761)
+      for iter775 in self.success:
+        oprot.writeString(iter775)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13860,10 +13997,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype765, _size762) = iprot.readListBegin()
-          for _i766 in xrange(_size762):
-            _elem767 = iprot.readString()
-            self.success.append(_elem767)
+          (_etype779, _size776) = iprot.readListBegin()
+          for _i780 in xrange(_size776):
+            _elem781 = iprot.readString()
+            self.success.append(_elem781)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13886,8 +14023,8 @@ class get_all_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter768 in self.success:
-        oprot.writeString(iter768)
+      for iter782 in self.success:
+        oprot.writeString(iter782)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14657,12 +14794,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype770, _vtype771, _size769 ) = iprot.readMapBegin()
-          for _i773 in xrange(_size769):
-            _key774 = iprot.readString()
-            _val775 = Type()
-            _val775.read(iprot)
-            self.success[_key774] = _val775
+          (_ktype784, _vtype785, _size783 ) = iprot.readMapBegin()
+          for _i787 in xrange(_size783):
+            _key788 = iprot.readString()
+            _val789 = Type()
+            _val789.read(iprot)
+            self.success[_key788] = _val789
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -14685,9 +14822,9 @@ class get_type_all_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter776,viter777 in self.success.items():
-        oprot.writeString(kiter776)
-        viter777.write(oprot)
+      for kiter790,viter791 in self.success.items():
+        oprot.writeString(kiter790)
+        viter791.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -14830,11 +14967,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype781, _size778) = iprot.readListBegin()
-          for _i782 in xrange(_size778):
-            _elem783 = FieldSchema()
-            _elem783.read(iprot)
-            self.success.append(_elem783)
+          (_etype795, _size792) = iprot.readListBegin()
+          for _i796 in xrange(_size792):
+            _elem797 = FieldSchema()
+            _elem797.read(iprot)
+            self.success.append(_elem797)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14869,8 +15006,8 @@ class get_fields_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter784 in self.success:
-        iter784.write(oprot)
+      for iter798 in self.success:
+        iter798.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15037,11 +15174,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype788, _size785) = iprot.readListBegin()
-          for _i789 in xrange(_size785):
-            _elem790 = FieldSchema()
-            _elem790.read(iprot)
-            self.success.append(_elem790)
+          (_etype802, _size799) = iprot.readListBegin()
+          for _i803 in xrange(_size799):
+            _elem804 = FieldSchema()
+            _elem804.read(iprot)
+            self.success.append(_elem804)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15076,8 +15213,8 @@ class get_fields_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter791 in self.success:
-        iter791.write(oprot)
+      for iter805 in self.success:
+        iter805.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15230,11 +15367,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype795, _size792) = iprot.readListBegin()
-          for _i796 in xrange(_size792):
-            _elem797 = FieldSchema()
-            _elem797.read(iprot)
-            self.success.append(_elem797)
+          (_etype809, _size806) = iprot.readListBegin()
+          for _i810 in xrange(_size806):
+            _elem811 = FieldSchema()
+            _elem811.read(iprot)
+            self.success.append(_elem811)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15269,8 +15406,8 @@ class get_schema_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter798 in self.success:
-        iter798.write(oprot)
+      for iter812 in self.success:
+        iter812.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15437,11 +15574,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype802, _size799) = iprot.readListBegin()
-          for _i803 in xrange(_size799):
-            _elem804 = FieldSchema()
-            _elem804.read(iprot)
-            self.success.append(_elem804)
+          (_etype816, _size813) = iprot.readListBegin()
+          for _i817 in xrange(_size813):
+            _elem818 = FieldSchema()
+            _elem818.read(iprot)
+            self.success.append(_elem818)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15476,8 +15613,8 @@ class get_schema_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter805 in self.success:
-        iter805.write(oprot)
+      for iter819 in self.success:
+        iter819.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15888,6 +16025,7 @@ class create_table_with_constraints_args:
    - foreignKeys
    - uniqueConstraints
    - notNullConstraints
+   - defaultConstraints
   """
 
   thrift_spec = (
@@ -15897,14 +16035,16 @@ class create_table_with_constraints_args:
     (3, TType.LIST, 'foreignKeys', (TType.STRUCT,(SQLForeignKey, SQLForeignKey.thrift_spec)), None, ), # 3
     (4, TType.LIST, 'uniqueConstraints', (TType.STRUCT,(SQLUniqueConstraint, SQLUniqueConstraint.thrift_spec)), None, ), # 4
     (5, TType.LIST, 'notNullConstraints', (TType.STRUCT,(SQLNotNullConstraint, SQLNotNullConstraint.thrift_spec)), None, ), # 5
+    (6, TType.LIST, 'defaultConstraints', (TType.STRUCT,(SQLDefaultConstraint, SQLDefaultConstraint.thrift_spec)), None, ), # 6
   )
 
-  def __init__(self, tbl=None, primaryKeys=None, foreignKeys=None, uniqueConstraints=None, notNullConstraints=None,):
+  def __init__(self, tbl=None, primaryKeys=None, foreignKeys=None, uniqueConstraints=None, notNullConstraints=None, defaultConstraints=None,):
     self.tbl = tbl
     self.primaryKeys = primaryKeys
     self.foreignKeys = foreignKeys
     self.uniqueConstraints = uniqueConstraints
     self.notNullConstraints = notNullConstraints
+    self.defaultConstraints = defaultConstraints
 
   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:
@@ -15924,44 +16064,55 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype809, _size806) = iprot.readListBegin()
-          for _i810 in xrange(_size806):
-            _elem811 = SQLPrimaryKey()
-            _elem811.read(iprot)
-            self.primaryKeys.append(_elem811)
+          (_etype823, _size820) = iprot.readListBegin()
+          for _i824 in xrange(_size820):
+            _elem825 = SQLPrimaryKey()
+            _elem825.read(iprot)
+            self.primaryKeys.append(_elem825)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype815, _size812) = iprot.readListBegin()
-          for _i816 in xrange(_size812):
-            _elem817 = SQLForeignKey()
-            _elem817.read(iprot)
-            self.foreignKeys.append(_elem817)
+          (_etype829, _size826) = iprot.readListBegin()
+          for _i830 in xrange(_size826):
+            _elem831 = SQLForeignKey()
+            _elem831.read(iprot)
+            self.foreignKeys.append(_elem831)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.uniqueConstraints = []
-          (_etype821, _size818) = iprot.readListBegin()
-          for _i822 in xrange(_size818):
-            _elem823 = SQLUniqueConstraint()
-            _elem823.read(iprot)
-            self.uniqueConstraints.append(_elem823)
+          (_etype835, _size832) = iprot.readListBegin()
+          for _i836 in xrange(_size832):
+            _elem837 = SQLUniqueConstraint()
+            _elem837.read(iprot)
+            self.uniqueConstraints.append(_elem837)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.notNullConstraints = []
-          (_etype827, _size824) = iprot.readListBegin()
-          for _i828 in xrange(_size824):
-            _elem829 = SQLNotNullConstraint()
-            _elem829.read(iprot)
-            self.notNullConstraints.append(_elem829)
+          (_etype841, _size838) = iprot.readListBegin()
+          for _i842 in xrange(_size838):
+            _elem843 = SQLNotNullConstraint()
+            _elem843.read(iprot)
+            self.notNullConstraints.append(_elem843)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.LIST:
+          self.defaultConstraints = []
+          (_etype847, _size844) = iprot.readListBegin()
+          for _i848 in xrange(_size844):
+            _elem849 = SQLDefaultConstraint()
+            _elem849.read(iprot)
+            self.defaultConstraints.append(_elem849)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15982,29 +16133,36 @@ class create_table_with_constraints_args:
     if self.primaryKeys is not None:
       oprot.writeFieldBegin('primaryKeys', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
-      for iter830 in self.primaryKeys:
-        iter830.write(oprot)
+      for iter850 in self.primaryKeys:
+        iter850.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.foreignKeys is not None:
       oprot.writeFieldBegin('foreignKeys', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
-      for iter831 in self.foreignKeys:
-        iter831.write(oprot)
+      for iter851 in self.foreignKeys:
+        iter851.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.uniqueConstraints is not None:
       oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints))
-      for iter832 in self.uniqueConstraints:
-        iter832.write(oprot)
+      for iter852 in self.uniqueConstraints:
+        iter852.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.notNullConstraints is not None:
       oprot.writeFieldBegin('notNullConstraints', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints))
-      for iter833 in self.notNullConstraints:
-        iter833.write(oprot)
+      for iter853 in self.notNullConstraints:
+        iter853.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.defaultConstraints is not None:
+      oprot.writeFieldBegin('defaultConstraints', TType.LIST, 6)
+      oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraints))
+      for iter854 in self.defaultConstraints:
+        iter854.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16021,6 +16179,7 @@ class create_table_with_constraints_args:
     value = (value * 31) ^ hash(self.foreignKeys)
     value = (value * 31) ^ hash(self.uniqueConstraints)
     value = (value * 31) ^ hash(self.notNullConstraints)
+    value = (value * 31) ^ hash(self.defaultConstraints)
     return value
 
   def __repr__(self):
@@ -16872,6 +17031,152 @@ class add_not_null_constraint_result:
   def __ne__(self, other):
     return not (self == other)
 
+class add_default_constraint_args:
+  """
+  Attributes:
+   - req
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'req', (AddDefaultConstraintRequest, AddDefaultConstraintRequest.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, req=None,):
+    self.req = req
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.req = AddDefaultConstraintRequest()
+          self.req.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('add_default_constraint_args')
+    if self.req is not None:
+      oprot.writeFieldBegin('req', TType.STRUCT, 1)
+      self.req.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.req)
+    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 add_default_constraint_result:
+  """
+  Attributes:
+   - o1
+   - o2
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, o1=None, o2=None,):
+    self.o1 = o1
+    self.o2 = o2
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = NoSuchObjectException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.o2 = MetaException()
+          self.o2.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('add_default_constraint_result')
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o2 is not None:
+      oprot.writeFieldBegin('o2', TType.STRUCT, 2)
+      self.o2.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.o1)
+    value = (value * 31) ^ hash(self.o2)
+    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 drop_table_args:
   """
   Attributes:
@@ -17270,10 +17575,10 @@ class truncate_table_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype837, _size834) = iprot.readListBegin()
-          for _i838 in xrange(_size834):
-            _elem839 = iprot.readString()
-            self.partNames.append(_elem839)
+          (_etype858, _size855) = iprot.readListBegin()
+          for _i859 in xrange(_size855):
+            _elem860 = iprot.readString()
+            self.partNames.append(_elem860)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17298,8 +17603,8 @@ class truncate_table_args:
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter840 in self.partNames:
-        oprot.writeString(iter840)
+      for iter861 in self.partNames:
+        oprot.writeString(iter861)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17499,10 +17804,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype844, _size841) = iprot.readListBegin()
-          for _i845 in xrange(_size841):
-            _elem846 = iprot.readString()
-            self.success.append(_elem846)
+          (_etype865, _size862) = iprot.readListBegin()
+          for _i866 in xrange(_size862):
+            _elem867 = iprot.readString()
+            self.success.append(_elem867)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17525,8 +17830,8 @@ class get_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter847 in self.success:
-        oprot.writeString(iter847)
+      for iter868 in self.success:
+        oprot.writeString(iter868)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17676,10 +17981,10 @@ class get_tables_by_type_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype851, _size848) = iprot.readListBegin()
-          for _i852 in xrange(_size848):
-            _elem853 = iprot.readString()
-            self.success.append(_elem853)
+          (_etype872, _size869) = iprot.readListBegin()
+          for _i873 in xrange(_size869):
+            _elem874 = iprot.readString()
+            self.success.append(_elem874)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17702,8 +18007,8 @@ class get_tables_by_type_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter854 in self.success:
-        oprot.writeString(iter854)
+      for iter875 in self.success:
+        oprot.writeString(iter875)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17827,10 +18132,10 @@ class get_materialized_views_for_rewriting_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype858, _size855) = iprot.readListBegin()
-          for _i859 in xrange(_size855):
-            _elem860 = iprot.readString()
-            self.success.append(_elem860)
+          (_etype879, _size876) = iprot.readListBegin()
+          for _i880 in xrange(_size876):
+            _elem881 = iprot.readString()
+            self.success.append(_elem881)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17853,8 +18158,8 @@ class get_materialized_views_for_rewriting_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter861 in self.success:
-        oprot.writeString(iter861)
+      for iter882 in self.success:
+        oprot.writeString(iter882)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17927,10 +18232,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype865, _size862) = iprot.readListBegin()
-          for _i866 in xrange(_size862):
-            _elem867 = iprot.readString()
-            self.tbl_types.append(_elem867)
+          (_etype886, _size883) = iprot.readListBegin()
+          for _i887 in xrange(_size883):
+            _elem888 = iprot.readString()
+            self.tbl_types.append(_elem888)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17955,8 +18260,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 iter868 in self.tbl_types:
-        oprot.writeString(iter868)
+      for iter889 in self.tbl_types:
+        oprot.writeString(iter889)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18012,11 +18317,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype872, _size869) = iprot.readListBegin()
-          for _i873 in xrange(_size869):
-            _elem874 = TableMeta()
-            _elem874.read(iprot)
-            self.success.append(_elem874)
+          (_etype893, _size890) = iprot.readListBegin()
+          for _i894 in xrange(_size890):
+            _elem895 = TableMeta()
+            _elem895.read(iprot)
+            self.success.append(_elem895)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18039,8 +18344,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 iter875 in self.success:
-        iter875.write(oprot)
+      for iter896 in self.success:
+        iter896.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18164,10 +18469,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype879, _size876) = iprot.readListBegin()
-          for _i880 in xrange(_size876):
-            _elem881 = iprot.readString()
-            self.success.append(_elem881)
+          (_etype900, _size897) = iprot.readListBegin()
+          for _i901 in xrange(_size897):
+            _elem902 = iprot.readString()
+            self.success.append(_elem902)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18190,8 +18495,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 iter882 in self.success:
-        oprot.writeString(iter882)
+      for iter903 in self.success:
+        oprot.writeString(iter903)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18427,10 +18732,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype886, _size883) = iprot.readListBegin()
-          for _i887 in xrange(_size883):
-            _elem888 = iprot.readString()
-            self.tbl_names.append(_elem888)
+          (_etype907, _size904) = iprot.readListBegin()
+          for _i908 in xrange(_size904):
+            _elem909 = iprot.readString()
+            self.tbl_names.append(_elem909)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18451,8 +18756,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 iter889 in self.tbl_names:
-        oprot.writeString(iter889)
+      for iter910 in self.tbl_names:
+        oprot.writeString(iter910)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18504,11 +18809,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype893, _size890) = iprot.readListBegin()
-          for _i894 in xrange(_size890):
-            _elem895 = Table()
-            _elem895.read(iprot)
-            self.success.append(_elem895)
+          (_etype914, _size911) = iprot.readListBegin()
+          for _i915 in xrange(_size911):
+            _elem916 = Table()
+            _elem916.read(iprot)
+            self.success.append(_elem916)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18525,8 +18830,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 iter896 in self.success:
-        iter896.write(oprot)
+      for iter917 in self.success:
+        iter917.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18918,10 +19223,10 @@ class get_materialization_invalidation_info_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype900, _size897) = iprot.readListBegin()
-          for _i901 in xrange(_size897):
-            _elem902 = iprot.readString()
-            self.tbl_names.append(_elem902)
+          (_etype921, _size918) = iprot.readListBegin()
+          for _i922 in xrange(_size918):
+            _elem923 = iprot.readString()
+            self.tbl_names.append(_elem923)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18942,8 +19247,8 @@ class get_materialization_invalidation_info_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter903 in self.tbl_names:
-        oprot.writeString(iter903)
+      for iter924 in self.tbl_names:
+        oprot.writeString(iter924)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19004,12 +19309,12 @@ class get_materialization_invalidation_info_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype905, _vtype906, _size904 ) = iprot.readMapBegin()
-          for _i908 in xrange(_size904):
-            _key909 = iprot.readString()
-            _val910 = Materialization()
-            _val910.read(iprot)
-            self.success[_key909] = _val910
+          (_ktype926, _vtype927, _size925 ) = iprot.readMapBegin()
+          for _i929 in xrange(_size925):
+            _key930 = iprot.readString()
+            _val931 = Materialization()
+            _val931.read(iprot)
+            self.success[_key930] = _val931
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -19044,9 +19349,9 @@ class get_materialization_invalidation_info_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter911,viter912 in self.success.items():
-        oprot.writeString(kiter911)
-        viter912.write(oprot)
+      for kiter932,viter933 in self.success.items():
+        oprot.writeString(kiter932)
+        viter933.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19398,10 +19703,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype916, _size913) = iprot.readListBegin()
-          for _i917 in xrange(_size913):
-            _elem918 = iprot.readString()
-            self.success.append(_elem918)
+          (_etype937, _size934) = iprot.readListBegin()
+          for _i938 in xrange(_size934):
+            _elem939 = iprot.readString()
+            self.success.append(_elem939)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19436,8 +19741,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 iter919 in self.success:
-        oprot.writeString(iter919)
+      for iter940 in self.success:
+        oprot.writeString(iter940)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20407,11 +20712,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype923, _size920) = iprot.readListBegin()
-          for _i924 in xrange(_size920):
-            _elem925 = Partition()
-            _elem925.read(iprot)
-            self.new_parts.append(_elem925)
+          (_etype944, _size941) = iprot.readListBegin()
+          for _i945 in xrange(_size941):
+            _elem946 = Partition()
+            _elem946.read(iprot)
+            self.new_parts.append(_elem946)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20428,8 +20733,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 iter926 in self.new_parts:
-        iter926.write(oprot)
+      for iter947 in self.new_parts:
+        iter947.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20587,11 +20892,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype930, _size927) = iprot.readListBegin()
-          for _i931 in xrange(_size927):
-            _elem932 = PartitionSpec()
-            _elem932.read(iprot)
-            self.new_parts.append(_elem932)
+          (_etype951, _size948) = iprot.readListBegin()
+          for _i952 in xrange(_size948):
+            _elem953 = PartitionSpec()
+            _elem953.read(iprot)
+            self.new_parts.append(_elem953)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20608,8 +20913,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 iter933 in self.new_parts:
-        iter933.write(oprot)
+      for iter954 in self.new_parts:
+        iter954.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20783,10 +21088,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype937, _size934) = iprot.readListBegin()
-          for _i938 in xrange(_size934):
-            _elem939 = iprot.readString()
-            self.part_vals.append(_elem939)
+          (_etype958, _size955) = iprot.readListBegin()
+          for _i959 in xrange(_size955):
+            _elem960 = iprot.readString()
+            self.part_vals.append(_elem960)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20811,8 +21116,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 iter940 in self.part_vals:
-        oprot.writeString(iter940)
+      for iter961 in self.part_vals:
+        oprot.writeString(iter961)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21165,10 +21470,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype944, _size941) = iprot.readListBegin()
-          for _i945 in xrange(_size941):
-            _elem946 = iprot.readString()
-            self.part_vals.append(_elem946)
+          (_etype965, _size962) = iprot.readListBegin()
+          for _i966 in xrange(_size962):
+            _elem967 = iprot.readString()
+            self.part_vals.append(_elem967)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21199,8 +21504,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 iter947 in self.part_vals:
-        oprot.writeString(iter947)
+      for iter968 in self.part_vals:
+        oprot.writeString(iter968)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -21795,10 +22100,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype951, _size948) = iprot.readListBegin()
-          for _i952 in xrange(_size948):
-            _elem953 = iprot.readString()
-            self.part_vals.append(_elem953)
+          (_etype972, _size969) = iprot.readListBegin()
+          for _i973 in xrange(_size969):
+            _elem974 = iprot.readString()
+            self.part_vals.append(_elem974)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21828,8 +22133,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 iter954 in self.part_vals:
-        oprot.writeString(iter954)
+      for iter975 in self.part_vals:
+        oprot.writeString(iter975)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -22002,10 +22307,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype958, _size955) = iprot.readListBegin()
-          for _i959 in xrange(_size955):
-            _elem960 = iprot.readString()
-            self.part_vals.append(_elem960)
+          (_etype979, _size976) = iprot.readListBegin()
+          for _i980 in xrange(_size976):
+            _elem981 = iprot.readString()
+            self.part_vals.append(_elem981)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22041,8 +22346,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 iter961 in self.part_vals:
-        oprot.writeString(iter961)
+      for iter982 in self.part_vals:
+        oprot.writeString(iter982)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -22779,10 +23084,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype965, _size962) = iprot.readListBegin()
-          for _i966 in xrange(_size962):
-            _elem967 = iprot.readString()
-            self.part_vals.append(_elem967)
+          (_etype986, _size983) = iprot.readListBegin()
+          for _i987 in xrange(_size983):
+            _elem988 = iprot.readString()
+            self.part_vals.append(_elem988)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22807,8 +23112,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 iter968 in self.part_vals:
-        oprot.writeString(iter968)
+      for iter989 in self.part_vals:
+        oprot.writeString(iter989)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22967,11 +23272,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype970, _vtype971, _size969 ) = iprot.readMapBegin()
-          for _i973 in xrange(_size969):
-            _key974 = iprot.readString()
-            _val975 = iprot.readString()
-            self.partitionSpecs[_key974] = _val975
+          (_ktype991, _vtype992, _size990 ) = iprot.readMapBegin()
+          for _i994 in xrange(_size990):
+            _key995 = iprot.readString()
+            _val996 = iprot.readString()
+            self.partitionSpecs[_key995] = _val996
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -23008,9 +23313,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 kiter976,viter977 in self.partitionSpecs.items():
-        oprot.writeString(kiter976)
-        oprot.writeString(viter977)
+      for kiter997,viter998 in self.partitionSpecs.items():
+        oprot.writeString(kiter997)
+        oprot.writeString(viter998)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -23215,11 +23520,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype979, _vtype980, _size978 ) = iprot.readMapBegin()
-          for _i982 in xrange(_size978):
-            _key983 = iprot.readString()
-            _val984 = iprot.readString()
-            self.partitionSpecs[_key983] = _val984
+          (_ktype1000, _vtype1001, _size999 ) = iprot.readMapBegin()
+          for _i1003 in xrange(_size999):
+            _key1004 = iprot.readString()
+            _val1005 = iprot.readString()
+            self.partitionSpecs[_key1004] = _val1005
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -23256,9 +23561,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 kiter985,viter986 in self.partitionSpecs.items():
-        oprot.writeString(kiter985)
-        oprot.writeString(viter986)
+      for kiter1006,viter1007 in self.partitionSpecs.items():
+        oprot.writeString(kiter1006)
+        oprot.writeString(viter1007)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -23341,11 +23646,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype990, _size987) = iprot.readListBegin()
-          for _i991 in xrange(_size987):
-            _elem992 = Partition()
-            _elem992.read(iprot)
-            self.success.append(_elem992)
+          (_etype1011, _size1008) = iprot.readListBegin()
+          for _i1012 in xrange(_size1008):
+            _elem1013 = Partition()
+            _elem1013.read(iprot)
+            self.success.append(_elem1013)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23386,8 +23691,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 iter993 in self.success:
-        iter993.write(oprot)
+      for iter1014 in self.success:
+        iter1014.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23481,10 +23786,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype997, _size994) = iprot.readListBegin()
-          for _i998 in xrange(_size994):
-            _elem999 = iprot.readString()
-            self.part_vals.append(_elem999)
+          (_etype1018, _size1015) = iprot.readListBegin()
+          for _i1019 in xrange(_size1015):
+            _elem1020 = iprot.readString()
+            self.part_vals.append(_elem1020)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23496,10 +23801,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1003, _size1000) = iprot.readListBegin()
-          for _i1004 in xrange(_size1000):
-            _elem1005 = iprot.readString()
-            self.group_names.append(_elem1005)
+          (_etype1024, _size1021) = iprot.readListBegin()
+          for _i1025 in xrange(_size1021):
+            _elem1026 = iprot.readString()
+            self.group_names.append(_elem1026)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23524,8 +23829,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 iter1006 in self.part_vals:
-        oprot.writeString(iter1006)
+      for iter1027 in self.part_vals:
+        oprot.writeString(iter1027)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -23535,8 +23840,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 iter1007 in self.group_names:
-        oprot.writeString(iter1007)
+      for iter1028 in self.group_names:
+        oprot.writeString(iter1028)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23965,11 +24270,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1011, _size1008) = iprot.readListBegin()
-          for _i1012 in xrange(_size1008):
-            _elem1013 = Partition()
-            _elem1013.read(iprot)
-            self.success.append(_elem1013)
+          (_etype1032, _size1029) = iprot.readListBegin()
+          for _i1033 in xrange(_size1029):
+            _elem1034 = Partition()
+            _elem1034.read(iprot)
+            self.success.append(_elem1034)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23998,8 +24303,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 iter1014 in self.success:
-        iter1014.write(oprot)
+      for iter1035 in self.success:
+        iter1035.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24093,10 +24398,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1018, _size1015) = iprot.readListBegin()
-          for _i1019 in xrange(_size1015):
-            _elem1020 = iprot.readString()
-            self.group_names.append(_elem1020)
+          (_etype1039, _size1036) = iprot.readListBegin()
+          for _i1040 in xrange(_size1036):
+            _elem1041 = iprot.readString()
+            self.group_names.append(_elem1041)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24129,8 +24434,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 iter1021 in self.group_names:
-        oprot.writeString(iter1021)
+      for iter1042 in self.group_names:
+        oprot.writeString(iter1042)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -24191,11 +24496,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1025, _size1022) = iprot.readListBegin()
-          for _i1026 in xrange(_size1022):
-            _elem1027 = Partition()
-            _elem1027.read(iprot)
-            self.success.append(_elem1027)
+          (_etype1046, _size1043) = iprot.readListBegin()
+          for _i1047 in xrange(_size1043):
+            _elem1048 = Partition()
+            _elem1048.read(iprot)
+            self.success.append(_elem1048)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24224,8 +24529,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 iter1028 in self.success:
-        iter1028.write(oprot)
+      for iter1049 in self.success:
+        iter1049.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24383,11 +24688,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1032, _size1029) = iprot.readListBegin()
-          for _i1033 in xrange(_size1029):
-            _elem1034 = PartitionSpec()
-            _elem1034.read(iprot)
-            self.success.append(_elem1034)
+          (_etype1053, _size1050) = iprot.readListBegin()
+          for _i1054 in xrange(_size1050):
+            _elem1055 = PartitionSpec()
+            _elem1055.read(iprot)
+            self.success.append(_elem1055)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24416,8 +24721,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 iter1035 in self.success:
-        iter1035.write(oprot)
+      for iter1056 in self.success:
+        iter1056.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24575,10 +24880,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1039, _size1036) = iprot.readListBegin()
-          for _i1040 in xrange(_size1036):
-            _elem1041 = iprot.readString()
-            self.success.append(_elem1041)
+          (_etype1060, _size1057) = iprot.readListBegin()
+          for _i1061 in xrange(_size1057):
+            _elem1062 = iprot.readString()
+            self.success.append(_elem1062)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24607,8 +24912,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 iter1042 in self.success:
-        oprot.writeString(iter1042)
+      for iter1063 in self.success:
+        oprot.writeString(iter1063)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24848,10 +25153,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1046, _size1043) = iprot.readListBegin()
-          for _i1047 in xrange(_size1043):
-            _elem1048 = iprot.readString()
-            self.part_vals.append(_elem1048)
+          (_etype1067, _size1064) = iprot.readListBegin()
+          for _i1068 in xrange(_size1064):
+            _elem1069 = iprot.readString()
+            self.part_vals.append(_elem1069)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24881,8 +25186,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 iter1049 in self.part_vals:
-        oprot.writeString(iter1049)
+      for iter1070 in self.part_vals:
+        oprot.writeString(iter1070)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -24946,11 +25251,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1053, _size1050) = iprot.readListBegin()
-          for _i1054 in xrange(_size1050):
-            _elem1055 = Partition()
-            _elem1055.read(iprot)
-            self.success.append(_elem1055)
+          (_etype1074, _size1071) = iprot.readListBegin()
+          for _i1075 in xrange(_size1071):
+            _elem1076 = Partition()
+            _elem1076.read(iprot)
+            self.success.append(_elem1076)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24979,8 +25284,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 iter1056 in self.success:
-        iter1056.write(oprot)
+      for iter1077 in self.success:
+        iter1077.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25067,10 +25372,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1060, _size1057) = iprot.readListBegin()
-          for _i1061 in xrange(_size1057):
-            _elem1062 = iprot.readString()
-            self.part_vals.append(_elem1062)
+          (_etype1081, _size1078) = iprot.readListBegin()
+          for _i1082 in xrange(_size1078):
+            _elem1083 = iprot.readString()
+            self.part_vals.append(_elem1083)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25087,10 +25392,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1066, _size1063) = iprot.readListBegin()
-          for _i1067 in xrange(_size1063):
-            _elem1068 = iprot.readString()
-            self.group_names.append(_elem1068)
+          (_etype1087, _size1084) = iprot.readListBegin()
+          for _i1088 in xrange(_size1084):
+            _elem1089 = iprot.readString()
+            self.group_names.append(_elem1089)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25115,8 +25420,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 iter1069 in self.part_vals:
-        oprot.writeString(iter1069)
+      for iter1090 in self.part_vals:
+        oprot.writeString(iter1090)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -25130,8 +25435,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 iter1070 in self.group_names:
-        oprot.writeString(iter1070)
+      for iter1091 in self.group_names:
+        oprot.writeString(iter1091)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -25193,11 +25498,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1074, _size1071) = iprot.readListBegin()
-          for _i1075 in xrange(_size1071):
-            _elem1076 = Partition()
-            _elem1076.read(iprot)
-            self.success.append(_elem1076)
+          (_etype1095, _size1092) = iprot.readListBegin()
+          for _i1096 in xrange(_size1092):
+            _elem1097 = Partition()
+            _elem1097.read(iprot)
+            self.success.append(_elem1097)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25226,8 +25531,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 iter1077 in self.success:
-        iter1077.write(oprot)
+      for iter1098 in self.success:
+        iter1098.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25308,10 +25613,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1081, _size1078) = iprot.readListBegin()
-          for _i1082 in xrange(_size1078):
-            _elem1083 = iprot.readString()
-            self.part_vals.append(_elem1083)
+          (_etype1102, _size1099) = iprot.readListBegin()
+          for _i1103 in xrange(_size1099):
+            _elem1104 = iprot.readString()
+            self.part_vals.append(_elem1104)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25341,8 +25646,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 iter1084 in self.part_vals:
-        oprot.writeString(iter1084)
+      for iter1105 in self.part_vals:
+        oprot.writeString(iter1105)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -25406,10 +25711,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1088, _size1085) = iprot.readListBegin()
-          for _i1089 in xrange(_size1085):
-            _elem1090 = iprot.readString()
-            self.success.append(_elem1090)
+          (_etype1109, _size1106) = iprot.readListBegin()
+          for _i1110 in xrange(_size1106):
+            _elem1111 = iprot.readString()
+            self.success.append(_elem1111)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25438,8 +25743,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 iter1091 in self.success:
-        oprot.writeString(iter1091)
+      for iter1112 in self.success:
+        oprot.writeString(iter1112)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25610,11 +25915,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1095, _size1092) = iprot.readListBegin()
-          for _i1096 in xrange(_size1092):
-            _elem1097 = Partition()
-            _elem1097.read(iprot)
-            self.success.append(_elem1097)
+          (_etype1116, _size1113) = iprot.readListBegin()
+          for _i1117 in xrange(_size1113):
+            _elem1118 = Partition()
+            _elem1118.read(iprot)
+            self.success.append(_elem1118)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25643,8 +25948,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 iter1098 in self.success:
-        iter1098.write(oprot)
+      for iter1119 in self.success:
+        iter1119.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25815,11 +26120,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1102, _size1099) = iprot.readListBegin()
-          for _i1103 in xrange(_size1099):
-            _elem1104 = PartitionSpec()
-            _elem1104.read(iprot)
-            self.success.append(_elem1104)
+          (_etype1123, _size1120) = iprot.readListBegin()
+          for _i1124 in xrange(_size1120):
+            _elem1125 = PartitionSpec()
+            _elem1125.read(iprot)
+            self.success.append(_elem1125)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25848,8 +26153,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 iter1105 in self.success:
-        iter1105.write(oprot)
+      for iter1126 in self.success:
+        iter1126.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26269,10 +26574,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype1109, _size1106) = iprot.readListBegin()
-          for _i1110 in xrange(_size1106):
-            _elem1111 = iprot.readString()
-            self.names.append(_elem1111)
+          (_etype1130, _size1127) = iprot.readListBegin()
+          for _i1131 in xrange(_size1127):
+            _elem1132 = iprot.readString()
+            self.names.append(_elem1132)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26297,8 +26602,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 iter1112 in self.names:
-        oprot.writeString(iter1112)
+      for iter1133 in self.names:
+        oprot.writeString(iter1133)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26357,11 +26662,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1116, _size1113) = iprot.readListBegin()
-          for _i1117 in xrange(_size1113):
-            _elem1118 = Partition()
-            _elem1118.read(iprot)
-            self.success.append(_elem1118)
+          (_etype1137, _size1134) = iprot.readListBegin()
+          for _i1138 in xrange(_size1134):
+            _elem1139 = Partition()
+            _elem1139.read(iprot)
+            self.success.append(_elem1139)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26390,8 +26695,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 iter1119 in self.success:
-        iter1119.write(oprot)
+      for iter1140 in self.success:
+        iter1140.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26641,11 +26946,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1123, _size1120) = iprot.readListBegin()
-          for _i1124 in xrange(_size1120):
-            _elem1125 = Partition()
-            _elem1125.read(iprot)
-            self.new_parts.append(_elem1125)
+          (_etype1144, _size1141) = iprot.readListBegin()
+          for _i1145 in xrange(_size1141):
+            _elem1146 = Partition()
+            _elem1146.read(iprot)
+            self.new_parts.append(_elem1146)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26670,8 +26975,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 iter1126 in self.new_parts:
-        iter1126.write(oprot)
+      for iter1147 in self.new_parts:
+        iter1147.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26824,11 +27129,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1130, _size1127) = iprot.readListBegin()
-          for _i1131 in xrange(_size1127):
-            _elem1132 = Partition()
-            _elem1132.read(iprot)
-            self.new_parts.append(_elem1132)
+          (_etype1151, _size1148) = iprot.readListBegin()
+          for _i1152 in xrange(_size1148):
+            _elem1153 = Partition()
+            _elem1153.read(iprot)
+            self.new_parts.append(_elem1153)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26859,8 +27164,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 iter1133 in self.new_parts:
-        iter1133.write(oprot)
+      for iter1154 in self.new_parts:
+        iter1154.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -27204,10 +27509,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1137, _size1134) = iprot.readListBegin()
-          for _i1138 in xrange(_size1134):
-            _elem1139 = iprot.readString()
-            self.part_vals.append(_elem1139)
+          (_etype1158, _size1155) = iprot.readListBegin()
+          for _i1159 in xrange(_size1155):
+            _elem1160 = iprot.readString()
+            self.part_vals.append(_elem1160)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27238,8 +27543,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 iter1140 in self.part_vals:
-        oprot.writeString(iter1140)
+      for iter1161 in self.part_vals:
+        oprot.writeString(iter1161)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -27381,10 +27686,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1144, _size1141) = iprot.readListBegin()
-          for _i1145 in xrange(_size1141):
-            _elem1146 = iprot.readString()
-            self.part_vals.append(_elem1146)
+          (_etype1165, _size1162) = iprot.readListBegin()
+          for _i1166 in xrange(_size1162):
+            _elem1167 = iprot.readString()
+            self.part_vals.append(_elem1167)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27406,8 +27711,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 iter1147 in self.part_vals:
-        oprot.writeString(iter1147)
+      for iter1168 in self.part_vals:
+        oprot.writeString(iter1168)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -27765,10 +28070,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1151, _size1148) = iprot.readListBegin()
-          for _i1152 in xrange(_size1148):
-            _elem1153 = iprot.readString()
-            self.success.append(_elem1153)
+          (_etype1172, _size1169) = iprot.readListBegin()
+          for _i1173 in xrange(_size1169):
+            _elem1174 = iprot.readString()
+            self.success.append(_elem1174)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27791,8 +28096,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 iter1154 in self.success:
-        oprot.writeString(iter1154)
+      for iter1175 in self.success:
+        oprot.writeString(iter1175)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27916,11 +28221,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype1156, _vtype1157, _size1155 ) = iprot.readMapBegin()
-          for _i1159 in xrange(_size1155):
-            _key1160 = iprot.readString()
-            _val1161 = iprot.readString()
-            self.success[_key1160] = _val1161
+          (_ktype1177, _vtype1178, _size1176 ) = iprot.readMapBegin()
+          for _i1180 in xrange(_size1176):
+            _key1181 = iprot.readString()
+            _val1182 = iprot.readString()
+            self.success[_key1181] = _val1182
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -27943,9 +28248,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 kiter1162,viter1163 in self.success.items():
-        oprot.writeString(kiter1162)
-        oprot.writeString(viter1163)
+      for kiter1183,viter1184 in self.success.items():
+        oprot.writeString(kiter1183)
+        oprot.writeString(viter1184)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28021,11 +28326,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1165, _vtype1166, _size1164 ) = iprot.readMapBegin()
-          for _i1168 in xrange(_size1164):
-            _key1169 = iprot.readString()
-            _val1170 = iprot.readString()
-            self.part_vals[_key1169] = _val1170
+          (_ktype1186, _vtype1187, _size1185 ) = iprot.readMapBegin()
+          for _i1189 in xrange(_size1185):
+            _key1190 = iprot.readString()
+            _val1191 = iprot.readString()
+            self.part_vals[_key1190] = _val1191
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -28055,9 +28360,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 kiter1171,viter1172 in self.part_vals.items():
-        oprot.writeString(kiter1171)
-        oprot.writeString(viter1172)
+      for kiter1192,viter1193 in self.part_vals.items():
+        oprot.writeString(kiter1192)
+        oprot.writeString(viter1193)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -28271,11 +28576,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1174, _vtype1175, _size1173 ) = iprot.readMapBegin()
-          for _i1177 in xrange(_size1173):
-            _key1178 = iprot.readString()
-            _val1179 = iprot.readString()
-            self.part_vals[_key1178] = _val1179
+          (_ktype1195, _vtype1196, _size1194 ) = iprot.readMapBegin()
+          for _i1198 in xrange(_size1194):
+            _key1199 = iprot.readString()
+            _val1200 = iprot.readString()
+            self.part_vals[_key1199] = _val1200
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -28305,9 +28610,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 kiter1180,viter1181 in self.part_vals.items():
-        oprot.writeString(kiter1180)
-        oprot.writeString(viter1181)
+      for kiter1201,viter1202 in self.part_vals.items():
+        oprot.writeString(kiter1201)
+        oprot.writeString(viter1202)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -29362,11 +29667,11 @@ class get_indexes_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1185, _size1182) = iprot.readListBegin()
-          for _i1186 in xrange(_size1182):
-            _elem1187 = Index()
-            _elem1187.read(iprot)
-            self.success.append(_elem1187)
+          (_etype1206, _size1203) = iprot.readListBegin()
+          for _i1207 in xrange(_size1203):
+            _elem1208 = Index()
+            _elem1208.read(iprot)
+            self.success.append(_elem1208)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29395,8 +29700,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 iter1188 in self.success:
-        iter1188.write(oprot)
+      for iter1209 in self.success:
+        iter1209.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29551,10 +29856,10 @@ class get_index_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1192, _size1189) = iprot.readListBegin()
-          for _i1193 in xrange(_size1189):
-            _elem1194 = iprot.readString()
-            self.success.append(_elem1194)
+          (_etype1213, _size1210) = iprot.readListBegin()
+          for _i1214 in xrange(_size1210):
+            _elem1215 = iprot.readString()
+            self.success.append(_elem1215)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29577,8 +29882,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 iter1195 in self.success:
-        oprot.writeString(iter1195)
+      for iter1216 in self.success:
+        oprot.writeString(iter1216)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -30245,6 +30550,165 @@ class get_not_null_constraints_result:
   def __ne__(self, other):
     return not (self == other)
 
+class get_default_constraints_args:
+  """
+  Attributes:
+   - request
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'request', (DefaultConstraintsRequest, DefaultConstraintsRequest.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, request=None,):
+    self.request = request
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.request = DefaultConstraintsRequest()
+          self.request.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_default_constraints_args')
+    if self.request is not None:
+      oprot.writeFieldBegin('request', TType.STRUCT, 1)
+      self.request.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.request)
+    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_default_constraints_result:
+  """
+  Attributes:
+   - success
+   - o1
+   - o2
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (DefaultConstraintsResponse, DefaultConstraintsResponse.thrift_spec), None, ), # 0
+    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, success=None, o1=None, o2=None,):
+    self.success = success
+    self.o1 = o1
+    self.o2 = o2
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = DefaultConstraintsResponse()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = MetaException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.o2 = NoSuchObjectException()
+          self.o2.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_default_constraints_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o2 is not None:
+      oprot.writeFieldBegin('o2', TType.STRUCT, 2)
+      self.o2.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)
+    value = (value * 31) ^ hash(self.o2)
+    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)
+
 

<TRUNCATED>

[05/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
index a6047bf..fcdf235 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -1474,6 +1474,242 @@ class SQLNotNullConstraint {
 
 }
 
+class SQLDefaultConstraint {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $table_db = null;
+  /**
+   * @var string
+   */
+  public $table_name = null;
+  /**
+   * @var string
+   */
+  public $column_name = null;
+  /**
+   * @var string
+   */
+  public $default_value = null;
+  /**
+   * @var string
+   */
+  public $dc_name = null;
+  /**
+   * @var bool
+   */
+  public $enable_cstr = null;
+  /**
+   * @var bool
+   */
+  public $validate_cstr = null;
+  /**
+   * @var bool
+   */
+  public $rely_cstr = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'table_db',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'table_name',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'column_name',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'default_value',
+          'type' => TType::STRING,
+          ),
+        5 => array(
+          'var' => 'dc_name',
+          'type' => TType::STRING,
+          ),
+        6 => array(
+          'var' => 'enable_cstr',
+          'type' => TType::BOOL,
+          ),
+        7 => array(
+          'var' => 'validate_cstr',
+          'type' => TType::BOOL,
+          ),
+        8 => array(
+          'var' => 'rely_cstr',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['table_db'])) {
+        $this->table_db = $vals['table_db'];
+      }
+      if (isset($vals['table_name'])) {
+        $this->table_name = $vals['table_name'];
+      }
+      if (isset($vals['column_name'])) {
+        $this->column_name = $vals['column_name'];
+      }
+      if (isset($vals['default_value'])) {
+        $this->default_value = $vals['default_value'];
+      }
+      if (isset($vals['dc_name'])) {
+        $this->dc_name = $vals['dc_name'];
+      }
+      if (isset($vals['enable_cstr'])) {
+        $this->enable_cstr = $vals['enable_cstr'];
+      }
+      if (isset($vals['validate_cstr'])) {
+        $this->validate_cstr = $vals['validate_cstr'];
+      }
+      if (isset($vals['rely_cstr'])) {
+        $this->rely_cstr = $vals['rely_cstr'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'SQLDefaultConstraint';
+  }
+
+  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->table_db);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->table_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->column_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->default_value);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dc_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->enable_cstr);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->validate_cstr);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 8:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->rely_cstr);
+          } 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('SQLDefaultConstraint');
+    if ($this->table_db !== null) {
+      $xfer += $output->writeFieldBegin('table_db', TType::STRING, 1);
+      $xfer += $output->writeString($this->table_db);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->table_name !== null) {
+      $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2);
+      $xfer += $output->writeString($this->table_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->column_name !== null) {
+      $xfer += $output->writeFieldBegin('column_name', TType::STRING, 3);
+      $xfer += $output->writeString($this->column_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->default_value !== null) {
+      $xfer += $output->writeFieldBegin('default_value', TType::STRING, 4);
+      $xfer += $output->writeString($this->default_value);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->dc_name !== null) {
+      $xfer += $output->writeFieldBegin('dc_name', TType::STRING, 5);
+      $xfer += $output->writeString($this->dc_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->enable_cstr !== null) {
+      $xfer += $output->writeFieldBegin('enable_cstr', TType::BOOL, 6);
+      $xfer += $output->writeBool($this->enable_cstr);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->validate_cstr !== null) {
+      $xfer += $output->writeFieldBegin('validate_cstr', TType::BOOL, 7);
+      $xfer += $output->writeBool($this->validate_cstr);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->rely_cstr !== null) {
+      $xfer += $output->writeFieldBegin('rely_cstr', TType::BOOL, 8);
+      $xfer += $output->writeBool($this->rely_cstr);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class Type {
   static $_TSPEC;
 
@@ -10124,54 +10360,43 @@ class NotNullConstraintsResponse {
 
 }
 
-class DropConstraintRequest {
+class DefaultConstraintsRequest {
   static $_TSPEC;
 
   /**
    * @var string
    */
-  public $dbname = null;
-  /**
-   * @var string
-   */
-  public $tablename = null;
+  public $db_name = null;
   /**
    * @var string
    */
-  public $constraintname = null;
+  public $tbl_name = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'dbname',
+          'var' => 'db_name',
           'type' => TType::STRING,
           ),
         2 => array(
-          'var' => 'tablename',
-          'type' => TType::STRING,
-          ),
-        3 => array(
-          'var' => 'constraintname',
+          'var' => 'tbl_name',
           'type' => TType::STRING,
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['dbname'])) {
-        $this->dbname = $vals['dbname'];
-      }
-      if (isset($vals['tablename'])) {
-        $this->tablename = $vals['tablename'];
+      if (isset($vals['db_name'])) {
+        $this->db_name = $vals['db_name'];
       }
-      if (isset($vals['constraintname'])) {
-        $this->constraintname = $vals['constraintname'];
+      if (isset($vals['tbl_name'])) {
+        $this->tbl_name = $vals['tbl_name'];
       }
     }
   }
 
   public function getName() {
-    return 'DropConstraintRequest';
+    return 'DefaultConstraintsRequest';
   }
 
   public function read($input)
@@ -10191,24 +10416,236 @@ class DropConstraintRequest {
       {
         case 1:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->dbname);
+            $xfer += $input->readString($this->db_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->tablename);
+            $xfer += $input->readString($this->tbl_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 3:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->constraintname);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('DefaultConstraintsRequest');
+    if ($this->db_name !== null) {
+      $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1);
+      $xfer += $output->writeString($this->db_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tbl_name !== null) {
+      $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2);
+      $xfer += $output->writeString($this->tbl_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class DefaultConstraintsResponse {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\SQLDefaultConstraint[]
+   */
+  public $defaultConstraints = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'defaultConstraints',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\SQLDefaultConstraint',
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['defaultConstraints'])) {
+        $this->defaultConstraints = $vals['defaultConstraints'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'DefaultConstraintsResponse';
+  }
+
+  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::LST) {
+            $this->defaultConstraints = array();
+            $_size313 = 0;
+            $_etype316 = 0;
+            $xfer += $input->readListBegin($_etype316, $_size313);
+            for ($_i317 = 0; $_i317 < $_size313; ++$_i317)
+            {
+              $elem318 = null;
+              $elem318 = new \metastore\SQLDefaultConstraint();
+              $xfer += $elem318->read($input);
+              $this->defaultConstraints []= $elem318;
+            }
+            $xfer += $input->readListEnd();
+          } 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('DefaultConstraintsResponse');
+    if ($this->defaultConstraints !== null) {
+      if (!is_array($this->defaultConstraints)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('defaultConstraints', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints));
+        {
+          foreach ($this->defaultConstraints as $iter319)
+          {
+            $xfer += $iter319->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class DropConstraintRequest {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $dbname = null;
+  /**
+   * @var string
+   */
+  public $tablename = null;
+  /**
+   * @var string
+   */
+  public $constraintname = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'dbname',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'tablename',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'constraintname',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['dbname'])) {
+        $this->dbname = $vals['dbname'];
+      }
+      if (isset($vals['tablename'])) {
+        $this->tablename = $vals['tablename'];
+      }
+      if (isset($vals['constraintname'])) {
+        $this->constraintname = $vals['constraintname'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'DropConstraintRequest';
+  }
+
+  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->dbname);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tablename);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->constraintname);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
           break;
         default:
           $xfer += $input->skip($ftype);
@@ -10296,15 +10733,15 @@ class AddPrimaryKeyRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->primaryKeyCols = array();
-            $_size313 = 0;
-            $_etype316 = 0;
-            $xfer += $input->readListBegin($_etype316, $_size313);
-            for ($_i317 = 0; $_i317 < $_size313; ++$_i317)
+            $_size320 = 0;
+            $_etype323 = 0;
+            $xfer += $input->readListBegin($_etype323, $_size320);
+            for ($_i324 = 0; $_i324 < $_size320; ++$_i324)
             {
-              $elem318 = null;
-              $elem318 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem318->read($input);
-              $this->primaryKeyCols []= $elem318;
+              $elem325 = null;
+              $elem325 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem325->read($input);
+              $this->primaryKeyCols []= $elem325;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10332,9 +10769,9 @@ class AddPrimaryKeyRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeyCols));
         {
-          foreach ($this->primaryKeyCols as $iter319)
+          foreach ($this->primaryKeyCols as $iter326)
           {
-            $xfer += $iter319->write($output);
+            $xfer += $iter326->write($output);
           }
         }
         $output->writeListEnd();
@@ -10399,15 +10836,15 @@ class AddForeignKeyRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->foreignKeyCols = array();
-            $_size320 = 0;
-            $_etype323 = 0;
-            $xfer += $input->readListBegin($_etype323, $_size320);
-            for ($_i324 = 0; $_i324 < $_size320; ++$_i324)
+            $_size327 = 0;
+            $_etype330 = 0;
+            $xfer += $input->readListBegin($_etype330, $_size327);
+            for ($_i331 = 0; $_i331 < $_size327; ++$_i331)
             {
-              $elem325 = null;
-              $elem325 = new \metastore\SQLForeignKey();
-              $xfer += $elem325->read($input);
-              $this->foreignKeyCols []= $elem325;
+              $elem332 = null;
+              $elem332 = new \metastore\SQLForeignKey();
+              $xfer += $elem332->read($input);
+              $this->foreignKeyCols []= $elem332;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10435,9 +10872,9 @@ class AddForeignKeyRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeyCols));
         {
-          foreach ($this->foreignKeyCols as $iter326)
+          foreach ($this->foreignKeyCols as $iter333)
           {
-            $xfer += $iter326->write($output);
+            $xfer += $iter333->write($output);
           }
         }
         $output->writeListEnd();
@@ -10502,15 +10939,15 @@ class AddUniqueConstraintRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->uniqueConstraintCols = array();
-            $_size327 = 0;
-            $_etype330 = 0;
-            $xfer += $input->readListBegin($_etype330, $_size327);
-            for ($_i331 = 0; $_i331 < $_size327; ++$_i331)
+            $_size334 = 0;
+            $_etype337 = 0;
+            $xfer += $input->readListBegin($_etype337, $_size334);
+            for ($_i338 = 0; $_i338 < $_size334; ++$_i338)
             {
-              $elem332 = null;
-              $elem332 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem332->read($input);
-              $this->uniqueConstraintCols []= $elem332;
+              $elem339 = null;
+              $elem339 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem339->read($input);
+              $this->uniqueConstraintCols []= $elem339;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10538,9 +10975,9 @@ class AddUniqueConstraintRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraintCols));
         {
-          foreach ($this->uniqueConstraintCols as $iter333)
+          foreach ($this->uniqueConstraintCols as $iter340)
           {
-            $xfer += $iter333->write($output);
+            $xfer += $iter340->write($output);
           }
         }
         $output->writeListEnd();
@@ -10605,15 +11042,15 @@ class AddNotNullConstraintRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->notNullConstraintCols = array();
-            $_size334 = 0;
-            $_etype337 = 0;
-            $xfer += $input->readListBegin($_etype337, $_size334);
-            for ($_i338 = 0; $_i338 < $_size334; ++$_i338)
+            $_size341 = 0;
+            $_etype344 = 0;
+            $xfer += $input->readListBegin($_etype344, $_size341);
+            for ($_i345 = 0; $_i345 < $_size341; ++$_i345)
             {
-              $elem339 = null;
-              $elem339 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem339->read($input);
-              $this->notNullConstraintCols []= $elem339;
+              $elem346 = null;
+              $elem346 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem346->read($input);
+              $this->notNullConstraintCols []= $elem346;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10641,9 +11078,112 @@ class AddNotNullConstraintRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraintCols));
         {
-          foreach ($this->notNullConstraintCols as $iter340)
+          foreach ($this->notNullConstraintCols as $iter347)
           {
-            $xfer += $iter340->write($output);
+            $xfer += $iter347->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class AddDefaultConstraintRequest {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\SQLDefaultConstraint[]
+   */
+  public $defaultConstraintCols = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'defaultConstraintCols',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\SQLDefaultConstraint',
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['defaultConstraintCols'])) {
+        $this->defaultConstraintCols = $vals['defaultConstraintCols'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'AddDefaultConstraintRequest';
+  }
+
+  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::LST) {
+            $this->defaultConstraintCols = array();
+            $_size348 = 0;
+            $_etype351 = 0;
+            $xfer += $input->readListBegin($_etype351, $_size348);
+            for ($_i352 = 0; $_i352 < $_size348; ++$_i352)
+            {
+              $elem353 = null;
+              $elem353 = new \metastore\SQLDefaultConstraint();
+              $xfer += $elem353->read($input);
+              $this->defaultConstraintCols []= $elem353;
+            }
+            $xfer += $input->readListEnd();
+          } 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('AddDefaultConstraintRequest');
+    if ($this->defaultConstraintCols !== null) {
+      if (!is_array($this->defaultConstraintCols)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('defaultConstraintCols', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->defaultConstraintCols));
+        {
+          foreach ($this->defaultConstraintCols as $iter354)
+          {
+            $xfer += $iter354->write($output);
           }
         }
         $output->writeListEnd();
@@ -10719,15 +11259,15 @@ class PartitionsByExprResult {
         case 1:
           if ($ftype == TType::LST) {
             $this->partitions = array();
-            $_size341 = 0;
-            $_etype344 = 0;
-            $xfer += $input->readListBegin($_etype344, $_size341);
-            for ($_i345 = 0; $_i345 < $_size341; ++$_i345)
+            $_size355 = 0;
+            $_etype358 = 0;
+            $xfer += $input->readListBegin($_etype358, $_size355);
+            for ($_i359 = 0; $_i359 < $_size355; ++$_i359)
             {
-              $elem346 = null;
-              $elem346 = new \metastore\Partition();
-              $xfer += $elem346->read($input);
-              $this->partitions []= $elem346;
+              $elem360 = null;
+              $elem360 = new \metastore\Partition();
+              $xfer += $elem360->read($input);
+              $this->partitions []= $elem360;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10762,9 +11302,9 @@ class PartitionsByExprResult {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitions));
         {
-          foreach ($this->partitions as $iter347)
+          foreach ($this->partitions as $iter361)
           {
-            $xfer += $iter347->write($output);
+            $xfer += $iter361->write($output);
           }
         }
         $output->writeListEnd();
@@ -11001,15 +11541,15 @@ class TableStatsResult {
         case 1:
           if ($ftype == TType::LST) {
             $this->tableStats = array();
-            $_size348 = 0;
-            $_etype351 = 0;
-            $xfer += $input->readListBegin($_etype351, $_size348);
-            for ($_i352 = 0; $_i352 < $_size348; ++$_i352)
+            $_size362 = 0;
+            $_etype365 = 0;
+            $xfer += $input->readListBegin($_etype365, $_size362);
+            for ($_i366 = 0; $_i366 < $_size362; ++$_i366)
             {
-              $elem353 = null;
-              $elem353 = new \metastore\ColumnStatisticsObj();
-              $xfer += $elem353->read($input);
-              $this->tableStats []= $elem353;
+              $elem367 = null;
+              $elem367 = new \metastore\ColumnStatisticsObj();
+              $xfer += $elem367->read($input);
+              $this->tableStats []= $elem367;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11037,9 +11577,9 @@ class TableStatsResult {
       {
         $output->writeListBegin(TType::STRUCT, count($this->tableStats));
         {
-          foreach ($this->tableStats as $iter354)
+          foreach ($this->tableStats as $iter368)
           {
-            $xfer += $iter354->write($output);
+            $xfer += $iter368->write($output);
           }
         }
         $output->writeListEnd();
@@ -11112,28 +11652,28 @@ class PartitionsStatsResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partStats = array();
-            $_size355 = 0;
-            $_ktype356 = 0;
-            $_vtype357 = 0;
-            $xfer += $input->readMapBegin($_ktype356, $_vtype357, $_size355);
-            for ($_i359 = 0; $_i359 < $_size355; ++$_i359)
+            $_size369 = 0;
+            $_ktype370 = 0;
+            $_vtype371 = 0;
+            $xfer += $input->readMapBegin($_ktype370, $_vtype371, $_size369);
+            for ($_i373 = 0; $_i373 < $_size369; ++$_i373)
             {
-              $key360 = '';
-              $val361 = array();
-              $xfer += $input->readString($key360);
-              $val361 = array();
-              $_size362 = 0;
-              $_etype365 = 0;
-              $xfer += $input->readListBegin($_etype365, $_size362);
-              for ($_i366 = 0; $_i366 < $_size362; ++$_i366)
+              $key374 = '';
+              $val375 = array();
+              $xfer += $input->readString($key374);
+              $val375 = array();
+              $_size376 = 0;
+              $_etype379 = 0;
+              $xfer += $input->readListBegin($_etype379, $_size376);
+              for ($_i380 = 0; $_i380 < $_size376; ++$_i380)
               {
-                $elem367 = null;
-                $elem367 = new \metastore\ColumnStatisticsObj();
-                $xfer += $elem367->read($input);
-                $val361 []= $elem367;
+                $elem381 = null;
+                $elem381 = new \metastore\ColumnStatisticsObj();
+                $xfer += $elem381->read($input);
+                $val375 []= $elem381;
               }
               $xfer += $input->readListEnd();
-              $this->partStats[$key360] = $val361;
+              $this->partStats[$key374] = $val375;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -11161,15 +11701,15 @@ class PartitionsStatsResult {
       {
         $output->writeMapBegin(TType::STRING, TType::LST, count($this->partStats));
         {
-          foreach ($this->partStats as $kiter368 => $viter369)
+          foreach ($this->partStats as $kiter382 => $viter383)
           {
-            $xfer += $output->writeString($kiter368);
+            $xfer += $output->writeString($kiter382);
             {
-              $output->writeListBegin(TType::STRUCT, count($viter369));
+              $output->writeListBegin(TType::STRUCT, count($viter383));
               {
-                foreach ($viter369 as $iter370)
+                foreach ($viter383 as $iter384)
                 {
-                  $xfer += $iter370->write($output);
+                  $xfer += $iter384->write($output);
                 }
               }
               $output->writeListEnd();
@@ -11273,14 +11813,14 @@ class TableStatsRequest {
         case 3:
           if ($ftype == TType::LST) {
             $this->colNames = array();
-            $_size371 = 0;
-            $_etype374 = 0;
-            $xfer += $input->readListBegin($_etype374, $_size371);
-            for ($_i375 = 0; $_i375 < $_size371; ++$_i375)
+            $_size385 = 0;
+            $_etype388 = 0;
+            $xfer += $input->readListBegin($_etype388, $_size385);
+            for ($_i389 = 0; $_i389 < $_size385; ++$_i389)
             {
-              $elem376 = null;
-              $xfer += $input->readString($elem376);
-              $this->colNames []= $elem376;
+              $elem390 = null;
+              $xfer += $input->readString($elem390);
+              $this->colNames []= $elem390;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11318,9 +11858,9 @@ class TableStatsRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->colNames));
         {
-          foreach ($this->colNames as $iter377)
+          foreach ($this->colNames as $iter391)
           {
-            $xfer += $output->writeString($iter377);
+            $xfer += $output->writeString($iter391);
           }
         }
         $output->writeListEnd();
@@ -11435,14 +11975,14 @@ class PartitionsStatsRequest {
         case 3:
           if ($ftype == TType::LST) {
             $this->colNames = array();
-            $_size378 = 0;
-            $_etype381 = 0;
-            $xfer += $input->readListBegin($_etype381, $_size378);
-            for ($_i382 = 0; $_i382 < $_size378; ++$_i382)
+            $_size392 = 0;
+            $_etype395 = 0;
+            $xfer += $input->readListBegin($_etype395, $_size392);
+            for ($_i396 = 0; $_i396 < $_size392; ++$_i396)
             {
-              $elem383 = null;
-              $xfer += $input->readString($elem383);
-              $this->colNames []= $elem383;
+              $elem397 = null;
+              $xfer += $input->readString($elem397);
+              $this->colNames []= $elem397;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11452,14 +11992,14 @@ class PartitionsStatsRequest {
         case 4:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size384 = 0;
-            $_etype387 = 0;
-            $xfer += $input->readListBegin($_etype387, $_size384);
-            for ($_i388 = 0; $_i388 < $_size384; ++$_i388)
+            $_size398 = 0;
+            $_etype401 = 0;
+            $xfer += $input->readListBegin($_etype401, $_size398);
+            for ($_i402 = 0; $_i402 < $_size398; ++$_i402)
             {
-              $elem389 = null;
-              $xfer += $input->readString($elem389);
-              $this->partNames []= $elem389;
+              $elem403 = null;
+              $xfer += $input->readString($elem403);
+              $this->partNames []= $elem403;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11497,9 +12037,9 @@ class PartitionsStatsRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->colNames));
         {
-          foreach ($this->colNames as $iter390)
+          foreach ($this->colNames as $iter404)
           {
-            $xfer += $output->writeString($iter390);
+            $xfer += $output->writeString($iter404);
           }
         }
         $output->writeListEnd();
@@ -11514,9 +12054,9 @@ class PartitionsStatsRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter391)
+          foreach ($this->partNames as $iter405)
           {
-            $xfer += $output->writeString($iter391);
+            $xfer += $output->writeString($iter405);
           }
         }
         $output->writeListEnd();
@@ -11581,15 +12121,15 @@ class AddPartitionsResult {
         case 1:
           if ($ftype == TType::LST) {
             $this->partitions = array();
-            $_size392 = 0;
-            $_etype395 = 0;
-            $xfer += $input->readListBegin($_etype395, $_size392);
-            for ($_i396 = 0; $_i396 < $_size392; ++$_i396)
+            $_size406 = 0;
+            $_etype409 = 0;
+            $xfer += $input->readListBegin($_etype409, $_size406);
+            for ($_i410 = 0; $_i410 < $_size406; ++$_i410)
             {
-              $elem397 = null;
-              $elem397 = new \metastore\Partition();
-              $xfer += $elem397->read($input);
-              $this->partitions []= $elem397;
+              $elem411 = null;
+              $elem411 = new \metastore\Partition();
+              $xfer += $elem411->read($input);
+              $this->partitions []= $elem411;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11617,9 +12157,9 @@ class AddPartitionsResult {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitions));
         {
-          foreach ($this->partitions as $iter398)
+          foreach ($this->partitions as $iter412)
           {
-            $xfer += $iter398->write($output);
+            $xfer += $iter412->write($output);
           }
         }
         $output->writeListEnd();
@@ -11742,15 +12282,15 @@ class AddPartitionsRequest {
         case 3:
           if ($ftype == TType::LST) {
             $this->parts = array();
-            $_size399 = 0;
-            $_etype402 = 0;
-            $xfer += $input->readListBegin($_etype402, $_size399);
-            for ($_i403 = 0; $_i403 < $_size399; ++$_i403)
+            $_size413 = 0;
+            $_etype416 = 0;
+            $xfer += $input->readListBegin($_etype416, $_size413);
+            for ($_i417 = 0; $_i417 < $_size413; ++$_i417)
             {
-              $elem404 = null;
-              $elem404 = new \metastore\Partition();
-              $xfer += $elem404->read($input);
-              $this->parts []= $elem404;
+              $elem418 = null;
+              $elem418 = new \metastore\Partition();
+              $xfer += $elem418->read($input);
+              $this->parts []= $elem418;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11802,9 +12342,9 @@ class AddPartitionsRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->parts));
         {
-          foreach ($this->parts as $iter405)
+          foreach ($this->parts as $iter419)
           {
-            $xfer += $iter405->write($output);
+            $xfer += $iter419->write($output);
           }
         }
         $output->writeListEnd();
@@ -11879,15 +12419,15 @@ class DropPartitionsResult {
         case 1:
           if ($ftype == TType::LST) {
             $this->partitions = array();
-            $_size406 = 0;
-            $_etype409 = 0;
-            $xfer += $input->readListBegin($_etype409, $_size406);
-            for ($_i410 = 0; $_i410 < $_size406; ++$_i410)
+            $_size420 = 0;
+            $_etype423 = 0;
+            $xfer += $input->readListBegin($_etype423, $_size420);
+            for ($_i424 = 0; $_i424 < $_size420; ++$_i424)
             {
-              $elem411 = null;
-              $elem411 = new \metastore\Partition();
-              $xfer += $elem411->read($input);
-              $this->partitions []= $elem411;
+              $elem425 = null;
+              $elem425 = new \metastore\Partition();
+              $xfer += $elem425->read($input);
+              $this->partitions []= $elem425;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11915,9 +12455,9 @@ class DropPartitionsResult {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitions));
         {
-          foreach ($this->partitions as $iter412)
+          foreach ($this->partitions as $iter426)
           {
-            $xfer += $iter412->write($output);
+            $xfer += $iter426->write($output);
           }
         }
         $output->writeListEnd();
@@ -12095,14 +12635,14 @@ class RequestPartsSpec {
         case 1:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size413 = 0;
-            $_etype416 = 0;
-            $xfer += $input->readListBegin($_etype416, $_size413);
-            for ($_i417 = 0; $_i417 < $_size413; ++$_i417)
+            $_size427 = 0;
+            $_etype430 = 0;
+            $xfer += $input->readListBegin($_etype430, $_size427);
+            for ($_i431 = 0; $_i431 < $_size427; ++$_i431)
             {
-              $elem418 = null;
-              $xfer += $input->readString($elem418);
-              $this->names []= $elem418;
+              $elem432 = null;
+              $xfer += $input->readString($elem432);
+              $this->names []= $elem432;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12112,15 +12652,15 @@ class RequestPartsSpec {
         case 2:
           if ($ftype == TType::LST) {
             $this->exprs = array();
-            $_size419 = 0;
-            $_etype422 = 0;
-            $xfer += $input->readListBegin($_etype422, $_size419);
-            for ($_i423 = 0; $_i423 < $_size419; ++$_i423)
+            $_size433 = 0;
+            $_etype436 = 0;
+            $xfer += $input->readListBegin($_etype436, $_size433);
+            for ($_i437 = 0; $_i437 < $_size433; ++$_i437)
             {
-              $elem424 = null;
-              $elem424 = new \metastore\DropPartitionsExpr();
-              $xfer += $elem424->read($input);
-              $this->exprs []= $elem424;
+              $elem438 = null;
+              $elem438 = new \metastore\DropPartitionsExpr();
+              $xfer += $elem438->read($input);
+              $this->exprs []= $elem438;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12148,9 +12688,9 @@ class RequestPartsSpec {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter425)
+          foreach ($this->names as $iter439)
           {
-            $xfer += $output->writeString($iter425);
+            $xfer += $output->writeString($iter439);
           }
         }
         $output->writeListEnd();
@@ -12165,9 +12705,9 @@ class RequestPartsSpec {
       {
         $output->writeListBegin(TType::STRUCT, count($this->exprs));
         {
-          foreach ($this->exprs as $iter426)
+          foreach ($this->exprs as $iter440)
           {
-            $xfer += $iter426->write($output);
+            $xfer += $iter440->write($output);
           }
         }
         $output->writeListEnd();
@@ -12574,15 +13114,15 @@ class PartitionValuesRequest {
         case 3:
           if ($ftype == TType::LST) {
             $this->partitionKeys = array();
-            $_size427 = 0;
-            $_etype430 = 0;
-            $xfer += $input->readListBegin($_etype430, $_size427);
-            for ($_i431 = 0; $_i431 < $_size427; ++$_i431)
+            $_size441 = 0;
+            $_etype444 = 0;
+            $xfer += $input->readListBegin($_etype444, $_size441);
+            for ($_i445 = 0; $_i445 < $_size441; ++$_i445)
             {
-              $elem432 = null;
-              $elem432 = new \metastore\FieldSchema();
-              $xfer += $elem432->read($input);
-              $this->partitionKeys []= $elem432;
+              $elem446 = null;
+              $elem446 = new \metastore\FieldSchema();
+              $xfer += $elem446->read($input);
+              $this->partitionKeys []= $elem446;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12606,15 +13146,15 @@ class PartitionValuesRequest {
         case 6:
           if ($ftype == TType::LST) {
             $this->partitionOrder = array();
-            $_size433 = 0;
-            $_etype436 = 0;
-            $xfer += $input->readListBegin($_etype436, $_size433);
-            for ($_i437 = 0; $_i437 < $_size433; ++$_i437)
+            $_size447 = 0;
+            $_etype450 = 0;
+            $xfer += $input->readListBegin($_etype450, $_size447);
+            for ($_i451 = 0; $_i451 < $_size447; ++$_i451)
             {
-              $elem438 = null;
-              $elem438 = new \metastore\FieldSchema();
-              $xfer += $elem438->read($input);
-              $this->partitionOrder []= $elem438;
+              $elem452 = null;
+              $elem452 = new \metastore\FieldSchema();
+              $xfer += $elem452->read($input);
+              $this->partitionOrder []= $elem452;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12666,9 +13206,9 @@ class PartitionValuesRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitionKeys));
         {
-          foreach ($this->partitionKeys as $iter439)
+          foreach ($this->partitionKeys as $iter453)
           {
-            $xfer += $iter439->write($output);
+            $xfer += $iter453->write($output);
           }
         }
         $output->writeListEnd();
@@ -12693,9 +13233,9 @@ class PartitionValuesRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitionOrder));
         {
-          foreach ($this->partitionOrder as $iter440)
+          foreach ($this->partitionOrder as $iter454)
           {
-            $xfer += $iter440->write($output);
+            $xfer += $iter454->write($output);
           }
         }
         $output->writeListEnd();
@@ -12769,14 +13309,14 @@ class PartitionValuesRow {
         case 1:
           if ($ftype == TType::LST) {
             $this->row = array();
-            $_size441 = 0;
-            $_etype444 = 0;
-            $xfer += $input->readListBegin($_etype444, $_size441);
-            for ($_i445 = 0; $_i445 < $_size441; ++$_i445)
+            $_size455 = 0;
+            $_etype458 = 0;
+            $xfer += $input->readListBegin($_etype458, $_size455);
+            for ($_i459 = 0; $_i459 < $_size455; ++$_i459)
             {
-              $elem446 = null;
-              $xfer += $input->readString($elem446);
-              $this->row []= $elem446;
+              $elem460 = null;
+              $xfer += $input->readString($elem460);
+              $this->row []= $elem460;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12804,9 +13344,9 @@ class PartitionValuesRow {
       {
         $output->writeListBegin(TType::STRING, count($this->row));
         {
-          foreach ($this->row as $iter447)
+          foreach ($this->row as $iter461)
           {
-            $xfer += $output->writeString($iter447);
+            $xfer += $output->writeString($iter461);
           }
         }
         $output->writeListEnd();
@@ -12871,15 +13411,15 @@ class PartitionValuesResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->partitionValues = array();
-            $_size448 = 0;
-            $_etype451 = 0;
-            $xfer += $input->readListBegin($_etype451, $_size448);
-            for ($_i452 = 0; $_i452 < $_size448; ++$_i452)
+            $_size462 = 0;
+            $_etype465 = 0;
+            $xfer += $input->readListBegin($_etype465, $_size462);
+            for ($_i466 = 0; $_i466 < $_size462; ++$_i466)
             {
-              $elem453 = null;
-              $elem453 = new \metastore\PartitionValuesRow();
-              $xfer += $elem453->read($input);
-              $this->partitionValues []= $elem453;
+              $elem467 = null;
+              $elem467 = new \metastore\PartitionValuesRow();
+              $xfer += $elem467->read($input);
+              $this->partitionValues []= $elem467;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12907,9 +13447,9 @@ class PartitionValuesResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitionValues));
         {
-          foreach ($this->partitionValues as $iter454)
+          foreach ($this->partitionValues as $iter468)
           {
-            $xfer += $iter454->write($output);
+            $xfer += $iter468->write($output);
           }
         }
         $output->writeListEnd();
@@ -13198,15 +13738,15 @@ class Function {
         case 8:
           if ($ftype == TType::LST) {
             $this->resourceUris = array();
-            $_size455 = 0;
-            $_etype458 = 0;
-            $xfer += $input->readListBegin($_etype458, $_size455);
-            for ($_i459 = 0; $_i459 < $_size455; ++$_i459)
+            $_size469 = 0;
+            $_etype472 = 0;
+            $xfer += $input->readListBegin($_etype472, $_size469);
+            for ($_i473 = 0; $_i473 < $_size469; ++$_i473)
             {
-              $elem460 = null;
-              $elem460 = new \metastore\ResourceUri();
-              $xfer += $elem460->read($input);
-              $this->resourceUris []= $elem460;
+              $elem474 = null;
+              $elem474 = new \metastore\ResourceUri();
+              $xfer += $elem474->read($input);
+              $this->resourceUris []= $elem474;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13269,9 +13809,9 @@ class Function {
       {
         $output->writeListBegin(TType::STRUCT, count($this->resourceUris));
         {
-          foreach ($this->resourceUris as $iter461)
+          foreach ($this->resourceUris as $iter475)
           {
-            $xfer += $iter461->write($output);
+            $xfer += $iter475->write($output);
           }
         }
         $output->writeListEnd();
@@ -13613,15 +14153,15 @@ class GetOpenTxnsInfoResponse {
         case 2:
           if ($ftype == TType::LST) {
             $this->open_txns = array();
-            $_size462 = 0;
-            $_etype465 = 0;
-            $xfer += $input->readListBegin($_etype465, $_size462);
-            for ($_i466 = 0; $_i466 < $_size462; ++$_i466)
+            $_size476 = 0;
+            $_etype479 = 0;
+            $xfer += $input->readListBegin($_etype479, $_size476);
+            for ($_i480 = 0; $_i480 < $_size476; ++$_i480)
             {
-              $elem467 = null;
-              $elem467 = new \metastore\TxnInfo();
-              $xfer += $elem467->read($input);
-              $this->open_txns []= $elem467;
+              $elem481 = null;
+              $elem481 = new \metastore\TxnInfo();
+              $xfer += $elem481->read($input);
+              $this->open_txns []= $elem481;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13654,9 +14194,9 @@ class GetOpenTxnsInfoResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->open_txns));
         {
-          foreach ($this->open_txns as $iter468)
+          foreach ($this->open_txns as $iter482)
           {
-            $xfer += $iter468->write($output);
+            $xfer += $iter482->write($output);
           }
         }
         $output->writeListEnd();
@@ -13760,14 +14300,14 @@ class GetOpenTxnsResponse {
         case 2:
           if ($ftype == TType::LST) {
             $this->open_txns = array();
-            $_size469 = 0;
-            $_etype472 = 0;
-            $xfer += $input->readListBegin($_etype472, $_size469);
-            for ($_i473 = 0; $_i473 < $_size469; ++$_i473)
+            $_size483 = 0;
+            $_etype486 = 0;
+            $xfer += $input->readListBegin($_etype486, $_size483);
+            for ($_i487 = 0; $_i487 < $_size483; ++$_i487)
             {
-              $elem474 = null;
-              $xfer += $input->readI64($elem474);
-              $this->open_txns []= $elem474;
+              $elem488 = null;
+              $xfer += $input->readI64($elem488);
+              $this->open_txns []= $elem488;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13814,9 +14354,9 @@ class GetOpenTxnsResponse {
       {
         $output->writeListBegin(TType::I64, count($this->open_txns));
         {
-          foreach ($this->open_txns as $iter475)
+          foreach ($this->open_txns as $iter489)
           {
-            $xfer += $output->writeI64($iter475);
+            $xfer += $output->writeI64($iter489);
           }
         }
         $output->writeListEnd();
@@ -14034,14 +14574,14 @@ class OpenTxnsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->txn_ids = array();
-            $_size476 = 0;
-            $_etype479 = 0;
-            $xfer += $input->readListBegin($_etype479, $_size476);
-            for ($_i480 = 0; $_i480 < $_size476; ++$_i480)
+            $_size490 = 0;
+            $_etype493 = 0;
+            $xfer += $input->readListBegin($_etype493, $_size490);
+            for ($_i494 = 0; $_i494 < $_size490; ++$_i494)
             {
-              $elem481 = null;
-              $xfer += $input->readI64($elem481);
-              $this->txn_ids []= $elem481;
+              $elem495 = null;
+              $xfer += $input->readI64($elem495);
+              $this->txn_ids []= $elem495;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14069,9 +14609,9 @@ class OpenTxnsResponse {
       {
         $output->writeListBegin(TType::I64, count($this->txn_ids));
         {
-          foreach ($this->txn_ids as $iter482)
+          foreach ($this->txn_ids as $iter496)
           {
-            $xfer += $output->writeI64($iter482);
+            $xfer += $output->writeI64($iter496);
           }
         }
         $output->writeListEnd();
@@ -14210,14 +14750,14 @@ class AbortTxnsRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->txn_ids = array();
-            $_size483 = 0;
-            $_etype486 = 0;
-            $xfer += $input->readListBegin($_etype486, $_size483);
-            for ($_i487 = 0; $_i487 < $_size483; ++$_i487)
+            $_size497 = 0;
+            $_etype500 = 0;
+            $xfer += $input->readListBegin($_etype500, $_size497);
+            for ($_i501 = 0; $_i501 < $_size497; ++$_i501)
             {
-              $elem488 = null;
-              $xfer += $input->readI64($elem488);
-              $this->txn_ids []= $elem488;
+              $elem502 = null;
+              $xfer += $input->readI64($elem502);
+              $this->txn_ids []= $elem502;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14245,9 +14785,9 @@ class AbortTxnsRequest {
       {
         $output->writeListBegin(TType::I64, count($this->txn_ids));
         {
-          foreach ($this->txn_ids as $iter489)
+          foreach ($this->txn_ids as $iter503)
           {
-            $xfer += $output->writeI64($iter489);
+            $xfer += $output->writeI64($iter503);
           }
         }
         $output->writeListEnd();
@@ -14397,14 +14937,14 @@ class GetValidWriteIdsRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fullTableNames = array();
-            $_size490 = 0;
-            $_etype493 = 0;
-            $xfer += $input->readListBegin($_etype493, $_size490);
-            for ($_i494 = 0; $_i494 < $_size490; ++$_i494)
+            $_size504 = 0;
+            $_etype507 = 0;
+            $xfer += $input->readListBegin($_etype507, $_size504);
+            for ($_i508 = 0; $_i508 < $_size504; ++$_i508)
             {
-              $elem495 = null;
-              $xfer += $input->readString($elem495);
-              $this->fullTableNames []= $elem495;
+              $elem509 = null;
+              $xfer += $input->readString($elem509);
+              $this->fullTableNames []= $elem509;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14439,9 +14979,9 @@ class GetValidWriteIdsRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->fullTableNames));
         {
-          foreach ($this->fullTableNames as $iter496)
+          foreach ($this->fullTableNames as $iter510)
           {
-            $xfer += $output->writeString($iter496);
+            $xfer += $output->writeString($iter510);
           }
         }
         $output->writeListEnd();
@@ -14568,14 +15108,14 @@ class TableValidWriteIds {
         case 3:
           if ($ftype == TType::LST) {
             $this->invalidWriteIds = array();
-            $_size497 = 0;
-            $_etype500 = 0;
-            $xfer += $input->readListBegin($_etype500, $_size497);
-            for ($_i501 = 0; $_i501 < $_size497; ++$_i501)
+            $_size511 = 0;
+            $_etype514 = 0;
+            $xfer += $input->readListBegin($_etype514, $_size511);
+            for ($_i515 = 0; $_i515 < $_size511; ++$_i515)
             {
-              $elem502 = null;
-              $xfer += $input->readI64($elem502);
-              $this->invalidWriteIds []= $elem502;
+              $elem516 = null;
+              $xfer += $input->readI64($elem516);
+              $this->invalidWriteIds []= $elem516;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14627,9 +15167,9 @@ class TableValidWriteIds {
       {
         $output->writeListBegin(TType::I64, count($this->invalidWriteIds));
         {
-          foreach ($this->invalidWriteIds as $iter503)
+          foreach ($this->invalidWriteIds as $iter517)
           {
-            $xfer += $output->writeI64($iter503);
+            $xfer += $output->writeI64($iter517);
           }
         }
         $output->writeListEnd();
@@ -14704,15 +15244,15 @@ class GetValidWriteIdsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->tblValidWriteIds = array();
-            $_size504 = 0;
-            $_etype507 = 0;
-            $xfer += $input->readListBegin($_etype507, $_size504);
-            for ($_i508 = 0; $_i508 < $_size504; ++$_i508)
+            $_size518 = 0;
+            $_etype521 = 0;
+            $xfer += $input->readListBegin($_etype521, $_size518);
+            for ($_i522 = 0; $_i522 < $_size518; ++$_i522)
             {
-              $elem509 = null;
-              $elem509 = new \metastore\TableValidWriteIds();
-              $xfer += $elem509->read($input);
-              $this->tblValidWriteIds []= $elem509;
+              $elem523 = null;
+              $elem523 = new \metastore\TableValidWriteIds();
+              $xfer += $elem523->read($input);
+              $this->tblValidWriteIds []= $elem523;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14740,9 +15280,9 @@ class GetValidWriteIdsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->tblValidWriteIds));
         {
-          foreach ($this->tblValidWriteIds as $iter510)
+          foreach ($this->tblValidWriteIds as $iter524)
           {
-            $xfer += $iter510->write($output);
+            $xfer += $iter524->write($output);
           }
         }
         $output->writeListEnd();
@@ -14828,14 +15368,14 @@ class AllocateTableWriteIdsRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->txnIds = array();
-            $_size511 = 0;
-            $_etype514 = 0;
-            $xfer += $input->readListBegin($_etype514, $_size511);
-            for ($_i515 = 0; $_i515 < $_size511; ++$_i515)
+            $_size525 = 0;
+            $_etype528 = 0;
+            $xfer += $input->readListBegin($_etype528, $_size525);
+            for ($_i529 = 0; $_i529 < $_size525; ++$_i529)
             {
-              $elem516 = null;
-              $xfer += $input->readI64($elem516);
-              $this->txnIds []= $elem516;
+              $elem530 = null;
+              $xfer += $input->readI64($elem530);
+              $this->txnIds []= $elem530;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14877,9 +15417,9 @@ class AllocateTableWriteIdsRequest {
       {
         $output->writeListBegin(TType::I64, count($this->txnIds));
         {
-          foreach ($this->txnIds as $iter517)
+          foreach ($this->txnIds as $iter531)
           {
-            $xfer += $output->writeI64($iter517);
+            $xfer += $output->writeI64($iter531);
           }
         }
         $output->writeListEnd();
@@ -15052,15 +15592,15 @@ class AllocateTableWriteIdsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->txnToWriteIds = array();
-            $_size518 = 0;
-            $_etype521 = 0;
-            $xfer += $input->readListBegin($_etype521, $_size518);
-            for ($_i522 = 0; $_i522 < $_size518; ++$_i522)
+            $_size532 = 0;
+            $_etype535 = 0;
+            $xfer += $input->readListBegin($_etype535, $_size532);
+            for ($_i536 = 0; $_i536 < $_size532; ++$_i536)
             {
-              $elem523 = null;
-              $elem523 = new \metastore\TxnToWriteId();
-              $xfer += $elem523->read($input);
-              $this->txnToWriteIds []= $elem523;
+              $elem537 = null;
+              $elem537 = new \metastore\TxnToWriteId();
+              $xfer += $elem537->read($input);
+              $this->txnToWriteIds []= $elem537;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15088,9 +15628,9 @@ class AllocateTableWriteIdsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->txnToWriteIds));
         {
-          foreach ($this->txnToWriteIds as $iter524)
+          foreach ($this->txnToWriteIds as $iter538)
           {
-            $xfer += $iter524->write($output);
+            $xfer += $iter538->write($output);
           }
         }
         $output->writeListEnd();
@@ -15435,15 +15975,15 @@ class LockRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->component = array();
-            $_size525 = 0;
-            $_etype528 = 0;
-            $xfer += $input->readListBegin($_etype528, $_size525);
-            for ($_i529 = 0; $_i529 < $_size525; ++$_i529)
+            $_size539 = 0;
+            $_etype542 = 0;
+            $xfer += $input->readListBegin($_etype542, $_size539);
+            for ($_i543 = 0; $_i543 < $_size539; ++$_i543)
             {
-              $elem530 = null;
-              $elem530 = new \metastore\LockComponent();
-              $xfer += $elem530->read($input);
-              $this->component []= $elem530;
+              $elem544 = null;
+              $elem544 = new \metastore\LockComponent();
+              $xfer += $elem544->read($input);
+              $this->component []= $elem544;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15499,9 +16039,9 @@ class LockRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->component));
         {
-          foreach ($this->component as $iter531)
+          foreach ($this->component as $iter545)
           {
-            $xfer += $iter531->write($output);
+            $xfer += $iter545->write($output);
           }
         }
         $output->writeListEnd();
@@ -16444,15 +16984,15 @@ class ShowLocksResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->locks = array();
-            $_size532 = 0;
-            $_etype535 = 0;
-            $xfer += $input->readListBegin($_etype535, $_size532);
-            for ($_i536 = 0; $_i536 < $_size532; ++$_i536)
+            $_size546 = 0;
+            $_etype549 = 0;
+            $xfer += $input->readListBegin($_etype549, $_size546);
+            for ($_i550 = 0; $_i550 < $_size546; ++$_i550)
             {
-              $elem537 = null;
-              $elem537 = new \metastore\ShowLocksResponseElement();
-              $xfer += $elem537->read($input);
-              $this->locks []= $elem537;
+              $elem551 = null;
+              $elem551 = new \metastore\ShowLocksResponseElement();
+              $xfer += $elem551->read($input);
+              $this->locks []= $elem551;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16480,9 +17020,9 @@ class ShowLocksResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->locks));
         {
-          foreach ($this->locks as $iter538)
+          foreach ($this->locks as $iter552)
           {
-            $xfer += $iter538->write($output);
+            $xfer += $iter552->write($output);
           }
         }
         $output->writeListEnd();
@@ -16757,17 +17297,17 @@ class HeartbeatTxnRangeResponse {
         case 1:
           if ($ftype == TType::SET) {
             $this->aborted = array();
-            $_size539 = 0;
-            $_etype542 = 0;
-            $xfer += $input->readSetBegin($_etype542, $_size539);
-            for ($_i543 = 0; $_i543 < $_size539; ++$_i543)
+            $_size553 = 0;
+            $_etype556 = 0;
+            $xfer += $input->readSetBegin($_etype556, $_size553);
+            for ($_i557 = 0; $_i557 < $_size553; ++$_i557)
             {
-              $elem544 = null;
-              $xfer += $input->readI64($elem544);
-              if (is_scalar($elem544)) {
-                $this->aborted[$elem544] = true;
+              $elem558 = null;
+              $xfer += $input->readI64($elem558);
+              if (is_scalar($elem558)) {
+                $this->aborted[$elem558] = true;
               } else {
-                $this->aborted []= $elem544;
+                $this->aborted []= $elem558;
               }
             }
             $xfer += $input->readSetEnd();
@@ -16778,17 +17318,17 @@ class HeartbeatTxnRangeResponse {
         case 2:
           if ($ftype == TType::SET) {
             $this->nosuch = array();
-            $_size545 = 0;
-            $_etype548 = 0;
-            $xfer += $input->readSetBegin($_etype548, $_size545);
-            for ($_i549 = 0; $_i549 < $_size545; ++$_i549)
+            $_size559 = 0;
+            $_etype562 = 0;
+            $xfer += $input->readSetBegin($_etype562, $_size559);
+            for ($_i563 = 0; $_i563 < $_size559; ++$_i563)
             {
-              $elem550 = null;
-              $xfer += $input->readI64($elem550);
-              if (is_scalar($elem550)) {
-                $this->nosuch[$elem550] = true;
+              $elem564 = null;
+              $xfer += $input->readI64($elem564);
+              if (is_scalar($elem564)) {
+                $this->nosuch[$elem564] = true;
               } else {
-                $this->nosuch []= $elem550;
+                $this->nosuch []= $elem564;
               }
             }
             $xfer += $input->readSetEnd();
@@ -16817,12 +17357,12 @@ class HeartbeatTxnRangeResponse {
       {
         $output->writeSetBegin(TType::I64, count($this->aborted));
         {
-          foreach ($this->aborted as $iter551 => $iter552)
+          foreach ($this->aborted as $iter565 => $iter566)
           {
-            if (is_scalar($iter552)) {
-            $xfer += $output->writeI64($iter551);
+            if (is_scalar($iter566)) {
+            $xfer += $output->writeI64($iter565);
             } else {
-            $xfer += $output->writeI64($iter552);
+            $xfer += $output->writeI64($iter566);
             }
           }
         }
@@ -16838,12 +17378,12 @@ class HeartbeatTxnRangeResponse {
       {
         $output->writeSetBegin(TType::I64, count($this->nosuch));
         {
-          foreach ($this->nosuch as $iter553 => $iter554)
+          foreach ($this->nosuch as $iter567 => $iter568)
           {
-            if (is_scalar($iter554)) {
-            $xfer += $output->writeI64($iter553);
+            if (is_scalar($iter568)) {
+            $xfer += $output->writeI64($iter567);
             } else {
-            $xfer += $output->writeI64($iter554);
+            $xfer += $output->writeI64($iter568);
             }
           }
         }
@@ -17002,17 +17542,17 @@ class CompactionRequest {
         case 6:
           if ($ftype == TType::MAP) {
             $this->properties = array();
-            $_size555 = 0;
-            $_ktype556 = 0;
-            $_vtype557 = 0;
-            $xfer += $input->readMapBegin($_ktype556, $_vtype557, $_size555);
-            for ($_i559 = 0; $_i559 < $_size555; ++$_i559)
+            $_size569 = 0;
+            $_ktype570 = 0;
+            $_vtype571 = 0;
+            $xfer += $input->readMapBegin($_ktype570, $_vtype571, $_size569);
+            for ($_i573 = 0; $_i573 < $_size569; ++$_i573)
             {
-              $key560 = '';
-              $val561 = '';
-              $xfer += $input->readString($key560);
-              $xfer += $input->readString($val561);
-              $this->properties[$key560] = $val561;
+              $key574 = '';
+              $val575 = '';
+              $xfer += $input->readString($key574);
+              $xfer += $input->readString($val575);
+              $this->properties[$key574] = $val575;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -17065,10 +17605,10 @@ class CompactionRequest {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->properties));
         {
-          foreach ($this->properties as $kiter562 => $viter563)
+          foreach ($this->properties as $kiter576 => $viter577)
           {
-            $xfer += $output->writeString($kiter562);
-            $xfer += $output->writeString($viter563);
+            $xfer += $output->writeString($kiter576);
+            $xfer += $output->writeString($viter577);
           }
         }
         $output->writeMapEnd();
@@ -17655,15 +18195,15 @@ class ShowCompactResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->compacts = array();
-            $_size564 = 0;
-            $_etype567 = 0;
-            $xfer += $input->readListBegin($_etype567, $_size564);
-            for ($_i568 = 0; $_i568 < $_size564; ++$_i568)
+            $_size578 = 0;
+            $_etype581 = 0;
+            $xfer += $input->readListBegin($_etype581, $_size578);
+            for ($_i582 = 0; $_i582 < $_size578; ++$_i582)
             {
-              $elem569 = null;
-              $elem569 = new \metastore\ShowCompactResponseElement();
-              $xfer += $elem569->read($input);
-              $this->compacts []= $elem569;
+              $elem583 = null;
+              $elem583 = new \metastore\ShowCompactResponseElement();
+              $xfer += $elem583->read($input);
+              $this->compacts []= $elem583;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17691,9 +18231,9 @@ class ShowCompactResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->compacts));
         {
-          foreach ($this->compacts as $iter570)
+          foreach ($this->compacts as $iter584)
           {
-            $xfer += $iter570->write($output);
+            $xfer += $iter584->write($output);
           }
         }
         $output->writeListEnd();
@@ -17840,14 +18380,14 @@ class AddDynamicPartitions {
         case 5:
           if ($ftype == TType::LST) {
             $this->partitionnames = array();
-            $_size571 = 0;
-            $_etype574 = 0;
-            $xfer += $input->readListBegin($_etype574, $_size571);
-            for ($_i575 = 0; $_i575 < $_size571; ++$_i575)
+            $_size585 = 0;
+            $_etype588 = 0;
+            $xfer += $input->readListBegin($_etype588, $_size585);
+            for ($_i589 = 0; $_i589 < $_size585; ++$_i589)
             {
-              $elem576 = null;
-              $xfer += $input->readString($elem576);
-              $this->partitionnames []= $elem576;
+              $elem590 = null;
+              $xfer += $input->readString($elem590);
+              $this->partitionnames []= $elem590;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17902,9 +18442,9 @@ class AddDynamicPartitions {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionnames));
         {
-          foreach ($this->partitionnames as $iter577)
+          foreach ($this->partitionnames as $iter591)
           {
-            $xfer += $output->writeString($iter577);
+            $xfer += $output->writeString($iter591);
           }
         }
         $output->writeListEnd();
@@ -18210,17 +18750,17 @@ class CreationMetadata {
         case 3:
           if ($ftype == TType::SET) {
             $this->tablesUsed = array();
-            $_size578 = 0;
-            $_etype581 = 0;
-            $xfer += $input->readSetBegin($_etype581, $_size578);
-            for ($_i582 = 0; $_i582 < $_size578; ++$_i582)
+            $_size592 = 0;
+            $_etype595 = 0;
+            $xfer += $input->readSetBegin($_etype595, $_size592);
+            for ($_i596 = 0; $_i596 < $_size592; ++$_i596)
             {
-              $elem583 = null;
-              $xfer += $input->readString($elem583);
-              if (is_scalar($elem583)) {
-                $this->tablesUsed[$elem583] = true;
+              $elem597 = null;
+              $xfer += $input->readString($elem597);
+              if (is_scalar($elem597)) {
+                $this->tablesUsed[$elem597] = true;
               } else {
-                $this->tablesUsed []= $elem583;
+                $this->tablesUsed []= $elem597;
               }
             }
             $xfer += $input->readSetEnd();
@@ -18266,12 +18806,12 @@ class CreationMetadata {
       {
         $output->writeSetBegin(TType::STRING, count($this->tablesUsed));
         {
-          foreach ($this->tablesUsed as $iter584 => $iter585)
+          foreach ($this->tablesUsed as $iter598 => $iter599)
           {
-            if (is_scalar($iter585)) {
-            $xfer += $output->writeString($iter584);
+            if (is_scalar($iter599)) {
+            $xfer += $output->writeString($iter598);
             } else {
-            $xfer += $output->writeString($iter585);
+            $xfer += $output->writeString($iter599);
             }
           }
         }
@@ -18653,15 +19193,15 @@ class NotificationEventResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->events = array();
-            $_size586 = 0;
-            $_etype589 = 0;
-            $xfer += $input->readListBegin($_etype589, $_size586);
-            for ($_i590 = 0; $_i590 < $_size586; ++$_i590)
+            $_size600 = 0;
+            $_etype603 = 0;
+            $xfer += $input->readListBegin($_etype603, $_size600);
+            for ($_i604 = 0; $_i604 < $_size600; ++$_i604)
             {
-              $elem591 = null;
-              $elem591 = new \metastore\NotificationEvent();
-              $xfer += $elem591->read($input);
-              $this->events []= $elem591;
+              $elem605 = null;
+              $elem605 = new \metastore\NotificationEvent();
+              $xfer += $elem605->read($input);
+              $this->events []= $elem605;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18689,9 +19229,9 @@ class NotificationEventResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->events));
         {
-          foreach ($this->events as $iter592)
+          foreach ($this->events as $iter606)
           {
-            $xfer += $iter592->write($output);
+            $xfer += $iter606->write($output);
           }
         }
         $output->writeListEnd();
@@ -19036,14 +19576,14 @@ class InsertEventRequestData {
         case 2:
           if ($ftype == TType::LST) {
             $this->filesAdded = array();
-            $_size593 = 0;
-            $_etype596 = 0;
-            $xfer += $input->readListBegin($_etype596, $_size593);
-            for ($_i597 = 0; $_i597 < $_size593; ++$_i597)
+            $_size607 = 0;
+            $_etype610 = 0;
+            $xfer += $input->readListBegin($_etype610, $_size607);
+            for ($_i611 = 0; $_i611 < $_size607; ++$_i611)
             {
-              $elem598 = null;
-              $xfer += $input->readString($elem598);
-              $this->filesAdded []= $elem598;
+              $elem612 = null;
+              $xfer += $input->readString($elem612);
+              $this->filesAdded []= $elem612;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19053,14 +19593,14 @@ class InsertEventRequestData {
         case 3:
           if ($ftype == TType::LST) {
             $this->filesAddedChecksum = array();
-            $_size599 = 0;
-            $_etype602 = 0;
-            $xfer += $input->readListBegin($_etype602, $_size599);
-            for ($_i603 = 0; $_i603 < $_size599; ++$_i603)
+            $_size613 = 0;
+            $_etype616 = 0;
+            $xfer += $input->readListBegin($_etype616, $_size613);
+            for ($_i617 = 0; $_i617 < $_size613; ++$_i617)
             {
-              $elem604 = null;
-              $xfer += $input->readString($elem604);
-              $this->filesAddedChecksum []= $elem604;
+              $elem618 = null;
+              $xfer += $input->readString($elem618);
+              $this->filesAddedChecksum []= $elem618;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19093,9 +19633,9 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAdded));
         {
-          foreach ($this->filesAdded as $iter605)
+          foreach ($this->filesAdded as $iter619)
           {
-            $xfer += $output->writeString($iter605);
+            $xfer += $output->writeString($iter619);
           }
         }
         $output->writeListEnd();
@@ -19110,9 +19650,9 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAddedChecksum));
         {
-          foreach ($this->filesAddedChecksum as $iter606)
+          foreach ($this->filesAddedChecksum as $iter620)
           {
-            $xfer += $output->writeString($iter606);
+            $xfer += $output->writeString($iter620);
           }
         }
         $output->writeListEnd();
@@ -19330,14 +19870,14 @@ class FireEventRequest {
         case 5:
           if ($ftype == TType::LST) {
             $this->partitionVals = array();
-            $_size607 = 0;
-            $_etype610 = 0;
-            $xfer += $input->readListBegin($_etype610, $_size607);
-            for ($_i611 = 0; $_i611 < $_size607; ++$_i611)
+            $_size621 = 0;
+            $_etype624 = 0;
+            $xfer += $input->readListBegin($_etype624, $_size621);
+            for ($_i625 = 0; $_i625 < $_size621; ++$_i625)
             {
-              $elem612 = null;
-              $xfer += $input->readString($elem612);
-              $this->partitionVals []= $elem612;
+              $elem626 = null;
+              $xfer += $input->readString($elem626);
+              $this->partitionVals []= $elem626;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19388,9 +19928,9 @@ class FireEventRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionVals));
         {
-          foreach ($this->partitionVals as $iter613)
+          foreach ($this->partitionVals as $iter627)
           {
-            $xfer += $output->writeString($iter613);
+            $xfer += $output->writeString($iter627);
           }
         }
         $output->writeListEnd();
@@ -19618,18 +20158,18 @@ class GetFileMetadataByExprResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->metadata = array();
-            $_size614 = 0;
-            $_ktype615 = 0;
-            $_vtype616 = 0;
-            $xfer += $input->readMapBegin($_ktype615, $_vtype616, $_size614);
-            for ($_i618 = 0; $_i618 < $_size614; ++$_i618)
+            $_size628 = 0;
+            $_ktype629 = 0;
+            $_vtype630 = 0;
+            $xfer += $input->readMapBegin($_ktype629, $_vtype630, $_size628);
+            for ($_i632 = 0; $_i632 < $_size628; ++$_i632)
             {
-              $key619 = 0;
-              $val620 = new \metastore\MetadataPpdResult();
-              $xfer += $input->readI64($key619);
-              $val620 = new \metastore\MetadataPpdResult();
-              $xfer += $val620->read($input);
-              $this->metadata[$key619] = $val620;
+              $key633 = 0;
+              $val634 = new \metastore\MetadataPpdResult();
+              $xfer += $input->readI64($key633);
+              $val634 = new \metastore\MetadataPpdResult();
+              $xfer += $val634->read($input);
+              $this->metadata[$key633] = $val634;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19664,10 +20204,10 @@ class GetFileMetadataByExprResult {
       {
         $output->writeMapBegin(TType::I64, TType::STRUCT, count($this->metadata));
         {
-          foreach ($this->metadata as $kiter621 => $viter622)
+          foreach ($this->metadata as $kiter635 => $viter636)
           {
-            $xfer += $output->writeI64($kiter621);
-            $xfer += $viter622->write($output);
+            $xfer += $output->writeI64($kiter635);
+            $xfer += $viter636->write($output);
           }
         }
         $output->writeMapEnd();
@@ -19769,14 +20309,14 @@ class GetFileMetadataByExprRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size623 = 0;
-            $_etype626 = 0;
-            $xfer += $input->readListBegin($_etype626, $_size623);
-            for ($_i627 = 0; $_i627 < $_size623; ++$_i627)
+            $_size637 = 0;
+            $_etype640 = 0;
+            $xfer += $input->readListBegin($_etype640, $_size637);
+            for ($_i641 = 0; $_i641 < $_size637; ++$_i641)
             {
-              $elem628 = null;
-              $xfer += $input->readI64($elem628);
-              $this->fileIds []= $elem628;
+              $elem642 = null;
+              $xfer += $input->readI64($elem642);
+              $this->fileIds []= $elem642;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19825,9 +20365,9 @@ class GetFileMetadataByExprRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter629)
+          foreach ($this->fileIds as $iter643)
           {
-            $xfer += $output->writeI64($iter629);
+            $xfer += $output->writeI64($iter643);
           }
         }
         $output->writeListEnd();
@@ -19921,17 +20461,17 @@ class GetFileMetadataResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->metadata = array();
-            $_size630 = 0;
-            $_ktype631 = 0;
-            $_vtype632 = 0;
-            $xfer += $input->readMapBegin($_ktype631, $_vtype632, $_size630);
-            for ($_i634 = 0; $_i634 < $_size630; ++$_i634)
+            $_size644 = 0;
+            $_ktype645 = 0;
+            $_vtype646 = 0;
+            $xfer += $input->readMapBegin($_ktype645, $_vtype646, $_size644);
+            for ($_i648 = 0; $_i648 < $_size644; ++$_i648)
             {
-              $key635 = 0;
-              $val636 = '';
-              $xfer += $input->readI64($key635);
-              $xfer += $input->readString($val636);
-              $this->metadata[$key635] = $val636;
+              $key649 = 0;
+              $val650 = '';
+              $xfer += $input->readI64($key649);
+              $xfer += $input->readString($val650);
+              $this->metadata[$key649] = $val650;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19966,10 +20506,10 @@ class GetFileMetadataResult {
       {
         $output->writeMapBegin(TType::I64, TType::STRING, count($this->metadata));
         {
-          foreach ($this->metadata as $kiter637 => $viter638)
+          foreach ($this->metadata as $kiter651 => $viter652)
           {
-            $xfer += $output->writeI64($kiter637);
-            $xfer += $output->writeString($viter638);
+            $xfer += $output->writeI64($kiter651);
+            $xfer += $output->writeString($viter652);
           }
         }
         $output->writeMapEnd();
@@ -20038,14 +20578,14 @@ class GetFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size639 = 0;
-            $_etype642 = 0;
-            $xfer += $input->readListBegin($_etype642, $_size639);
-            for ($_i643 = 0; $_i643 < $_size639; ++$_i643)
+            $_size653 = 0;
+            $_etype656 = 0;
+            $xfer += $input->readListBegin($_etype656, $_size653);
+            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
             {
-              $elem644 = null;
-              $xfer += $input->readI64($elem644);
-              $this->fileIds []= $elem644;
+              $elem658 = null;
+              $xfer += $input->readI64($elem658);
+              $this->fileIds []= $elem658;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20073,9 +20613,9 @@ class GetFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter645)
+          foreach ($this->fileIds as $iter659)
           {
-            $xfer += $output->writeI64($iter645);
+            $xfer += $output->writeI64($iter659);
           }
         }
         $output->writeListEnd();
@@ -20215,14 +20755,14 @@ class PutFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size646 = 0;
-            $_etype649 = 0;
-            $xfer += $input->readListBegin($_etype649, $_size646);
-            for ($_i650 = 0; $_i650 < $_size646; ++$_i650)
+            $_size660 = 0;
+            $_etype663 = 0;
+            $xfer += $input->readListBegin($_etype663, $_size660);
+            for ($_i664 = 0; $_i664 < $_size660; ++$_i664)
             {
-              $elem651 = null;
-              $xfer += $input->readI64($elem651);
-              $this->fileIds []= $elem651;
+              $elem665 = null;
+              $xfer += $input->readI64($elem665);
+              $this->fileIds []= $elem665;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20232,14 +20772,14 @@ class PutFileMetadataRequest {
         case 2:
           if ($ftype == TType::LST) {
             $this->metadata = array();
-            $_size652 = 0;
-            $_etype655 = 0;
-            $xfer += $input->readListBegin($_etype655, $_size652);
-            for ($_i656 = 0; $_i656 < $_size652; ++$_i656)
+            $_size666 = 0;
+            $_etype669 = 0;
+            $xfer += $input->readListBegin($_etype669, $_size666);
+            for ($_i670 = 0; $_i670 < $_size666; ++$_i670)
             {
-              $elem657 = null;
-              $xfer += $input->readString($elem657);
-              $this->metadata []= $elem657;
+              $elem671 = null;
+              $xfer += $input->readString($elem671);
+              $this->metadata []= $elem671;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20274,9 +20814,9 @@ class PutFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter658)
+          foreach ($this->fileIds as $iter672)
           {
-            $xfer += $output->writeI64($iter658);
+            $xfer += $output->writeI64($iter672);
           }
         }
         $output->writeListEnd();
@@ -20291,9 +20831,9 @@ class PutFileMetadataRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->metadata));
         {
-          foreach ($this->metadata as $iter659)
+          foreach ($this->metadata as $iter673)
           {
-            $xfer += $output->writeString($iter659);
+            $xfer += $output->writeString($iter673);
           }
         }
         $output->writeListEnd();
@@ -20412,14 +20952,14 @@ class ClearFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size660 = 0;
-            $_etype663 = 0;
-            $xfer += $input->readListBegin($_etype663, $_size660);
-            for ($_i664 = 0; $_i664 < $_size660; ++$_i664)
+            $_size674 = 0;
+            $_etype677 = 0;
+            $xfer += $input->readListBegin($_etype677, $_size674);
+            for ($_i678 = 0; $_i678 < $_size674; ++$_i678)
             {
-              $elem665 = null;
-              $xfer += $input->readI64($elem665);
-              $this->fileIds []= $elem665;
+              $elem679 = null;
+              $xfer += $input->readI64($elem679);
+              $this->fileIds []= $elem679;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20447,9 +20987,9 @@ class ClearFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter666)
+          foreach ($this->fileIds as $iter680)
           {
-            $xfer += $output->writeI64($iter666);
+            $xfer += $output->writeI64($iter680);
           }
         }
         $output->writeListEnd();
@@ -20733,15 +21273,15 @@ class GetAllFunctionsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->functions = array();
-            $_size667 = 0;
-            $_etype670 = 0;
-            $xfer += $input->readListBegin($_etype670, $_size667);
-            for ($_i671 = 0; $_i671 < $_size667; ++$_i671)
+            $_size681 = 0;
+            $_etype684 = 0;
+            $xfer += $input->readListBegin($_etype684, $_size681);
+            for ($_i685 = 0; $_i685 < $_size681; ++$_i685)
             {
-              $elem672 = null;
-              $elem672 = new \metastore\Function();
-              $xfer += $elem672->read($input);
-              $this->functions []= $elem672;
+              $elem686 = null;
+              $elem686 = new \metastore\Function();
+              $xfer += $elem686->read($input);
+              $this->functions []= $elem686;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20769,9 +21309,9 @@ class GetAllFunctionsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->functions));
         {
-          foreach ($this->functions as $iter673)
+          foreach ($this->functions as $iter687)
           {
-            $xfer += $iter673->write($output);
+            $xfer += $iter687->write($output);
           }
         }
         $output->writeListEnd();
@@ -20835,14 +21375,14 @@ class ClientCapabilities {
         case 1:
           if ($ftype == TType::LST) {
             $this->values = array();
-            $_size674 = 0;
-            $_etype677 = 0;
-            $xfer += $input->readListBegin($_etype677, $_size674);
-            for ($_i678 = 0; $_i678 < $_size674; ++$_i678)
+            $_size688 = 0;
+            $_etype691 = 0;
+            $xfer += $input->readListBegin($_etype691, $_size688);
+            for ($_i692 = 0; $_i692 < $_size688; ++$_i692)
             {
-              $elem679 = null;
-              $xfer += $input->readI32($elem679);
-              $this->values []= $elem679;
+              $elem6

<TRUNCATED>

[18/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
index 00c0381..bf86cec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.plan;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -36,7 +37,6 @@ import com.google.common.collect.ImmutableList;
 
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -132,6 +132,7 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
   List<SQLForeignKey> foreignKeyCols;
   List<SQLUniqueConstraint> uniqueConstraintCols;
   List<SQLNotNullConstraint> notNullConstraintCols;
+  List<SQLDefaultConstraint> defaultConstraintsCols;
   ReplicationSpec replicationSpec;
 
   public AlterTableDesc() {
@@ -167,7 +168,7 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
       String oldColName, String newColName, String newType, String newComment,
       boolean first, String afterCol, boolean isCascade, List<SQLPrimaryKey> primaryKeyCols,
       List<SQLForeignKey> foreignKeyCols, List<SQLUniqueConstraint> uniqueConstraintCols,
-      List<SQLNotNullConstraint> notNullConstraintCols) {
+      List<SQLNotNullConstraint> notNullConstraintCols, List<SQLDefaultConstraint> defaultConstraints) {
     super();
     oldName = tblName;
     this.partSpec = partSpec;
@@ -183,6 +184,7 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
     this.foreignKeyCols = foreignKeyCols;
     this.uniqueConstraintCols = uniqueConstraintCols;
     this.notNullConstraintCols = notNullConstraintCols;
+    this.defaultConstraintsCols = defaultConstraints;
   }
 
   /**
@@ -342,12 +344,14 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
 
   public AlterTableDesc(String tableName, List<SQLPrimaryKey> primaryKeyCols,
       List<SQLForeignKey> foreignKeyCols, List<SQLUniqueConstraint> uniqueConstraintCols,
-      List<SQLNotNullConstraint> notNullConstraintCols, ReplicationSpec replicationSpec) {
+      List<SQLNotNullConstraint> notNullConstraintCols, List<SQLDefaultConstraint> defaultConstraints,
+                        ReplicationSpec replicationSpec) {
     this.oldName = tableName;
     this.primaryKeyCols = primaryKeyCols;
     this.foreignKeyCols = foreignKeyCols;
     this.uniqueConstraintCols = uniqueConstraintCols;
     this.notNullConstraintCols = notNullConstraintCols;
+    this.defaultConstraintsCols = defaultConstraints;
     this.replicationSpec = replicationSpec;
     op = AlterTableTypes.ADDCONSTRAINT;
   }
@@ -541,6 +545,13 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
   }
 
   /**
+   * @return the default constraint cols
+   */
+  public List<SQLDefaultConstraint> getDefaultConstraintCols() {
+    return defaultConstraintsCols;
+  }
+
+  /**
    * @return the drop constraint name of the table
    */
   @Explain(displayName = "drop constraint name", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
index 6228d4c..ff9df3d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
@@ -100,6 +101,7 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
   List<SQLForeignKey> foreignKeys;
   List<SQLUniqueConstraint> uniqueConstraints;
   List<SQLNotNullConstraint> notNullConstraints;
+  List<SQLDefaultConstraint> defaultConstraints;
   private Long initialMmWriteId; // Initial MM write ID for CTAS and import.
   // The FSOP configuration for the FSOP that is going to write initial data during ctas.
   // This is not needed beyond compilation, so it is transient.
@@ -119,14 +121,15 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
       Map<String, String> tblProps,
       boolean ifNotExists, List<String> skewedColNames, List<List<String>> skewedColValues,
       List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
-      List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints) {
+      List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints,
+      List<SQLDefaultConstraint> defaultConstraints) {
 
     this(tableName, isExternal, isTemporary, cols, partCols,
         bucketCols, sortCols, numBuckets, fieldDelim, fieldEscape,
         collItemDelim, mapKeyDelim, lineDelim, comment, inputFormat,
         outputFormat, location, serName, storageHandler, serdeProps,
         tblProps, ifNotExists, skewedColNames, skewedColValues,
-        primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints);
+        primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints);
 
     this.databaseName = databaseName;
   }
@@ -142,13 +145,14 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
                          Map<String, String> tblProps,
                          boolean ifNotExists, List<String> skewedColNames, List<List<String>> skewedColValues,
                          boolean isCTAS, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
-                         List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints) {
+                         List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints,
+                         List<SQLDefaultConstraint> defaultConstraints) {
     this(databaseName, tableName, isExternal, isTemporary, cols, partCols,
             bucketCols, sortCols, numBuckets, fieldDelim, fieldEscape,
             collItemDelim, mapKeyDelim, lineDelim, comment, inputFormat,
             outputFormat, location, serName, storageHandler, serdeProps,
             tblProps, ifNotExists, skewedColNames, skewedColValues,
-            primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints);
+            primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints);
     this.isCTAS = isCTAS;
 
   }
@@ -165,7 +169,8 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
       Map<String, String> tblProps,
       boolean ifNotExists, List<String> skewedColNames, List<List<String>> skewedColValues,
       List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
-      List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints) {
+      List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints,
+      List<SQLDefaultConstraint> defaultConstraints) {
     this.tableName = tableName;
     this.isExternal = isExternal;
     this.isTemporary = isTemporary;
@@ -194,6 +199,7 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
     this.foreignKeys = copyList(foreignKeys);
     this.uniqueConstraints = copyList(uniqueConstraints);
     this.notNullConstraints = copyList(notNullConstraints);
+    this.defaultConstraints = copyList(defaultConstraints);
   }
 
   private static <T> List<T> copyList(List<T> copy) {
@@ -272,6 +278,10 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
     return notNullConstraints;
   }
 
+  public List<SQLDefaultConstraint> getDefaultConstraints() {
+    return defaultConstraints;
+  }
+
   @Explain(displayName = "bucket columns")
   public List<String> getBucketCols() {
     return bucketCols;

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
index fcbac7d..a4e21c1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
@@ -82,7 +82,8 @@ public class ImportTableDesc {
                 null,
                 null,
                 null,
-                null);
+                null,
+            null);
         this.createTblDesc.setStoredAsSubDirectories(table.getSd().isStoredAsSubDirectories());
         break;
       case VIEW:

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToChar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToChar.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToChar.java
index b98ec68..899abf7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToChar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToChar.java
@@ -93,6 +93,7 @@ public class GenericUDFToChar extends GenericUDF implements SettableUDF, Seriali
     sb.append(" AS CHAR(");
     sb.append("" + typeInfo.getLength());
     sb.append(")");
+    sb.append(")");
     return sb.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/alter_external_with_constraint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_external_with_constraint.q b/ql/src/test/queries/clientnegative/alter_external_with_constraint.q
deleted file mode 100644
index 041585e..0000000
--- a/ql/src/test/queries/clientnegative/alter_external_with_constraint.q
+++ /dev/null
@@ -1,3 +0,0 @@
-CREATE external TABLE table1 (a STRING, b STRING); 
-Alter table table1 change b b STRING NOT NULL enable;
-

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/alter_external_with_default_constraint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_external_with_default_constraint.q b/ql/src/test/queries/clientnegative/alter_external_with_default_constraint.q
new file mode 100644
index 0000000..5169f95
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/alter_external_with_default_constraint.q
@@ -0,0 +1,3 @@
+CREATE external TABLE table1 (a STRING, b STRING); 
+Alter table table1 change b b STRING DEFAULT 'hive' enable;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/alter_external_with_notnull_constraint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_external_with_notnull_constraint.q b/ql/src/test/queries/clientnegative/alter_external_with_notnull_constraint.q
new file mode 100644
index 0000000..041585e
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/alter_external_with_notnull_constraint.q
@@ -0,0 +1,3 @@
+CREATE external TABLE table1 (a STRING, b STRING); 
+Alter table table1 change b b STRING NOT NULL enable;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/alter_tableprops_external_with_constraint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_tableprops_external_with_constraint.q b/ql/src/test/queries/clientnegative/alter_tableprops_external_with_constraint.q
deleted file mode 100644
index 9deefb1..0000000
--- a/ql/src/test/queries/clientnegative/alter_tableprops_external_with_constraint.q
+++ /dev/null
@@ -1,3 +0,0 @@
-CREATE TABLE table1 (a STRING NOT NULL enforced, b STRING); 
-Alter table table1 set TBLPROPERTIES('external'='true');
-

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/alter_tableprops_external_with_default_constraint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_tableprops_external_with_default_constraint.q b/ql/src/test/queries/clientnegative/alter_tableprops_external_with_default_constraint.q
new file mode 100644
index 0000000..c93ceba
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/alter_tableprops_external_with_default_constraint.q
@@ -0,0 +1,3 @@
+CREATE TABLE table1 (a STRING DEFAULT 'hive', b STRING);
+Alter table table1 set TBLPROPERTIES('external'='true');
+

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/alter_tableprops_external_with_notnull_constraint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_tableprops_external_with_notnull_constraint.q b/ql/src/test/queries/clientnegative/alter_tableprops_external_with_notnull_constraint.q
new file mode 100644
index 0000000..9deefb1
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/alter_tableprops_external_with_notnull_constraint.q
@@ -0,0 +1,3 @@
+CREATE TABLE table1 (a STRING NOT NULL enforced, b STRING); 
+Alter table table1 set TBLPROPERTIES('external'='true');
+

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/constraint_duplicate_name.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/constraint_duplicate_name.q b/ql/src/test/queries/clientnegative/constraint_duplicate_name.q
new file mode 100644
index 0000000..2b7429d
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/constraint_duplicate_name.q
@@ -0,0 +1,2 @@
+create table t(i int constraint c1 not null enable);
+create table t1(j int constraint c1 default 4);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/constraint_invalide_name.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/constraint_invalide_name.q b/ql/src/test/queries/clientnegative/constraint_invalide_name.q
new file mode 100644
index 0000000..a354d57
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/constraint_invalide_name.q
@@ -0,0 +1,3 @@
+-- max allowed length for constraint name is 255
+create table t (i int, j string constraint aaaaabcdatyaaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaa
+ default 'def_value');

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/constraint_partition_columns.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/constraint_partition_columns.q b/ql/src/test/queries/clientnegative/constraint_partition_columns.q
new file mode 100644
index 0000000..81e74ff
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/constraint_partition_columns.q
@@ -0,0 +1,2 @@
+-- partition columns aren't allowed to have not null or default constraints
+create table tpart(i int default 5, j int not null enable) partitioned by (s string not null);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/create_external_with_constraint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/create_external_with_constraint.q b/ql/src/test/queries/clientnegative/create_external_with_constraint.q
deleted file mode 100644
index 5b12662..0000000
--- a/ql/src/test/queries/clientnegative/create_external_with_constraint.q
+++ /dev/null
@@ -1 +0,0 @@
-CREATE external TABLE table1 (a STRING not null enforced, b STRING); 

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/create_external_with_default_constraint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/create_external_with_default_constraint.q b/ql/src/test/queries/clientnegative/create_external_with_default_constraint.q
new file mode 100644
index 0000000..4690c2c
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/create_external_with_default_constraint.q
@@ -0,0 +1 @@
+CREATE external TABLE table1 (a INT DEFAULT 56, b STRING);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/create_external_with_notnull_constraint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/create_external_with_notnull_constraint.q b/ql/src/test/queries/clientnegative/create_external_with_notnull_constraint.q
new file mode 100644
index 0000000..5b12662
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/create_external_with_notnull_constraint.q
@@ -0,0 +1 @@
+CREATE external TABLE table1 (a STRING not null enforced, b STRING); 

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/default_constraint_complex_default_value.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/default_constraint_complex_default_value.q b/ql/src/test/queries/clientnegative/default_constraint_complex_default_value.q
new file mode 100644
index 0000000..76c32aa
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/default_constraint_complex_default_value.q
@@ -0,0 +1,2 @@
+-- default for complex types are not allowed
+create table t (i int, j array<default> default array(1.3, 2.3));

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value.q b/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value.q
new file mode 100644
index 0000000..ff00ece
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value.q
@@ -0,0 +1,2 @@
+-- reference to other column in default value is not allowed
+create table t (i int, j double default cast(i as double));

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value2.q b/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value2.q
new file mode 100644
index 0000000..ec5b67a
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value2.q
@@ -0,0 +1,2 @@
+-- only certain UDFs are allowed as default
+create table t (i int, j string default repeat('s', 4));

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value_length.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value_length.q b/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value_length.q
new file mode 100644
index 0000000..a2a49ff
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value_length.q
@@ -0,0 +1,4 @@
+-- max allowed length for default value is 255 
+-- create default with length 256
+create table t (i int, j string default 
+	'12345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234');

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value_type.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value_type.q b/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value_type.q
new file mode 100644
index 0000000..1f1a9db
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value_type.q
@@ -0,0 +1,2 @@
+-- year() isn't valid
+create table t (i int, j string default cast(year("1970-01-01") as string));

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientnegative/default_constraint_invalid_type.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/default_constraint_invalid_type.q b/ql/src/test/queries/clientnegative/default_constraint_invalid_type.q
new file mode 100644
index 0000000..d1afb68
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/default_constraint_invalid_type.q
@@ -0,0 +1 @@
+create table t (i int, j double default 1);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientpositive/create_with_constraints.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/create_with_constraints.q b/ql/src/test/queries/clientpositive/create_with_constraints.q
index 27b4883..10712ca 100644
--- a/ql/src/test/queries/clientpositive/create_with_constraints.q
+++ b/ql/src/test/queries/clientpositive/create_with_constraints.q
@@ -24,6 +24,7 @@ CREATE TABLE table20 (a STRING, b STRING, CONSTRAINT uk20_1 UNIQUE (b) DISABLE R
 CREATE TABLE table21 (a STRING, CONSTRAINT uk21_1 UNIQUE (a,b) DISABLE) PARTITIONED BY (b STRING);
 CREATE TABLE table22 (a STRING, b STRING, CONSTRAINT fk22_1 FOREIGN KEY (a,b) REFERENCES table21(a,b) DISABLE);
 
+
 DESCRIBE EXTENDED table1;
 DESCRIBE EXTENDED table2;
 DESCRIBE EXTENDED table3;

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientpositive/default_constraint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/default_constraint.q b/ql/src/test/queries/clientpositive/default_constraint.q
new file mode 100644
index 0000000..c16dda3
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/default_constraint.q
@@ -0,0 +1,180 @@
+-- create table
+ -- numeric type
+ set hive.stats.autogather=false;
+ set hive.support.concurrency=true;
+ set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+CREATE TABLE numericDataType(a TINYINT CONSTRAINT tinyint_constraint DEFAULT 127Y ENABLE, b SMALLINT DEFAULT 32767S, c INT DEFAULT 2147483647,
+    d BIGINT DEFAULT  9223372036854775807L, e DOUBLE DEFAULT 3.4E38, f DECIMAL(9,2) DEFAULT 1234567.89)
+    clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
+DESC FORMATTED numericDataType;
+
+EXPLAIN INSERT INTO numericDataType(a) values(3Y);
+INSERT INTO numericDataType(a) values(3Y);
+SELECT * FROM numericDataType;
+
+EXPLAIN INSERT INTO numericDataType(e,f) values(4.5, 678.4);
+INSERT INTO numericDataType(e,f) values(4.5, 678.4);
+SELECT * FROM numericDataType;
+
+DROP TABLE numericDataType;
+
+  -- Date/time
+CREATE TABLE table1(d DATE DEFAULT DATE'2018-02-14', t TIMESTAMP DEFAULT TIMESTAMP'2016-02-22 12:45:07.000000000',
+    tz timestamp with local time zone DEFAULT TIMESTAMPLOCALTZ'2016-01-03 12:26:34 America/Los_Angeles',
+    d1 DATE DEFAULT current_date() ENABLE, t1 TIMESTAMP DEFAULT current_timestamp() DISABLE);
+DESC FORMATTED table1;
+
+EXPLAIN INSERT INTO table1(t) values ("1985-12-31 12:45:07");
+INSERT INTO table1(t) values ("1985-12-31 12:45:07");
+SELECT d, t, tz,d1=current_date(), t1 from table1;
+
+EXPLAIN INSERT INTO table1(d, t1) values ("1985-12-31", '2018-02-27 17:32:14.259');
+INSERT INTO table1(d, t1) values ("1985-12-31", '2018-02-27 17:32:14.259');
+SELECT d, t, tz,d1=current_date(), t1=current_timestamp() from table1;
+
+DROP TABLE table1;
+
+-- string type
+CREATE TABLE table2(i STRING DEFAULT 'current_database()', j STRING DEFAULT current_user(),
+    k STRING DEFAULT 'Current_User()', v varchar(350) DEFAULT cast('varchar_default_value' as varchar(350)),
+    c char(20) DEFAULT cast('char_value' as char(20)))
+    clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
+DESC FORMATTED table2;
+EXPLAIN INSERT INTO table2(i) values('default');
+INSERT INTO table2(i) values('default');
+SELECT i,j=current_user(),k,v,c FROM table2;
+
+EXPLAIN INSERT INTO table2(v, c) values('varchar_default2', 'char');
+INSERT INTO table2(v, c) values('varchar_default2', 'char');
+SELECT i,j=current_user(),k,v,c FROM table2;
+DROP TABLE table2;
+
+
+-- misc type
+CREATE TABLE misc(b BOOLEAN DEFAULT true, b1 BINARY DEFAULT cast('bin' as binary))
+    clustered by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
+DESC FORMATTED misc;
+EXPLAIN INSERT INTO misc(b) values(false);
+INSERT INTO misc(b) values(false);
+SELECT b, b1 from misc;
+EXPLAIN INSERT INTO misc(b1) values('011');
+INSERT INTO misc(b) values(false);
+SELECT b, b1 from misc;
+DROP TABLE misc;
+
+-- CAST
+CREATE table t11(i int default cast(cast(4 as double) as int),
+    b1 boolean default cast ('true' as boolean), b2 int default cast (5.67 as int),
+    b3 tinyint default cast (45 as tinyint), b4 float default cast (45.4 as float),
+    b5 bigint default cast (567 as bigint), b6 smallint default cast (88 as smallint),
+    j varchar(50) default cast(current_timestamp() as varchar(50)),
+     k string default cast(cast(current_user() as varchar(50)) as string),
+     tz1 timestamp with local time zone DEFAULT cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with local time zone),
+     ts timestamp default cast('2016-01-01 12:01:01' as timestamp),
+     dc decimal(8,2) default cast(4.5 as decimal(8,2)),
+     c2 double default cast(5 as double), c4 char(2) default cast(cast(cast('ab' as string) as varchar(2)) as char(2)));
+DESC FORMATTED t11;
+EXPLAIN INSERT INTO t11(c4) values('vi');
+INSERT INTO t11(c4) values('vi');
+SELECT ts, tz1, dc, b1,b2,b3,b4,b5,b6,j=cast(current_timestamp() as varchar(50)), k=cast(current_user() as string), c2, c4 from t11;
+
+EXPLAIN INSERT INTO t11(b1,c4) values(true,'ga');
+INSERT INTO t11(c4) values('vi');
+SELECT ts, tz1, dc, b1,b2,b3,b4,b5,b6,j=cast(current_timestamp() as varchar(50)), k=cast(current_user() as string), c2, c4 from t11;
+
+DROP TABLE t11;
+
+-- alter table
+-- drop constraint
+CREATE TABLE numericDataType(a TINYINT CONSTRAINT tinyint_constraint DEFAULT 127Y ENABLE, b SMALLINT DEFAULT 32767S, c INT DEFAULT 2147483647,
+    d BIGINT DEFAULT  9223372036854775807L, e DOUBLE DEFAULT 3.4E38, f DECIMAL(9,2) DEFAULT 1234567.89)
+    clustered by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
+ALTER TABLE numericDataType DROP CONSTRAINT tinyint_constraint;
+DESC FORMATTED numericDataType;
+
+EXPLAIN INSERT INTO numericDataType(b) values(456);
+INSERT INTO numericDataType(b) values(456);
+SELECT * from numericDataType;
+
+-- add another constraint on same column
+ALTER TABLE numericDataType ADD CONSTRAINT uk1 UNIQUE(a,b) DISABLE NOVALIDATE;
+DESC FORMATTED numericDataType;
+EXPLAIN INSERT INTO numericDataType(b) values(56);
+INSERT INTO numericDataType(b) values(456);
+SELECT * from numericDataType;
+
+-- alter table change column with constraint to add NOT NULL and then DEFAULT
+ALTER TABLE numericDataType CHANGE a a TINYINT CONSTRAINT second_null_constraint NOT NULL ENABLE;
+DESC FORMATTED numericDataType;
+ALTER TABLE numericDataType CHANGE a a TINYINT CONSTRAINT default_constraint DEFAULT 127Y ENABLE;
+DESC FORMATTED numericDataType;
+EXPLAIN INSERT INTO numericDataType(f) values(847.45); --plan should have both DEFAULT and NOT NULL
+INSERT INTO numericDataType(f) values(847.45);
+Select * from numericDataType;
+DESC FORMATTED numericDataType;
+
+-- drop constraint and add with same name again
+ALTER TABLE numericDataType DROP CONSTRAINT default_constraint;
+DESC FORMATTED numericDataType;
+ALTER TABLE numericDataType CHANGE a a TINYINT CONSTRAINT default_constraint DEFAULT 108Y ENABLE;
+DESC FORMATTED numericDataType;
+EXPLAIN INSERT INTO numericDataType(f) values(847.45);
+INSERT INTO numericDataType(f) values(847.45);
+Select * from numericDataType;
+DROP TABLE numericDataType;
+
+-- create default with maximum length allowed for default val (255)
+create table t (i int, j string default
+	'1234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123');
+desc formatted t;
+explain insert into t(i) values(3);
+insert into t(i) values(3);
+select * from t;
+drop table t;
+
+-- partitioned table
+set hive.exec.dynamic.partition.mode=nonstrict;
+-- Table with partition
+CREATE TABLE tablePartitioned (a STRING NOT NULL ENFORCED, url STRING constraint bdc1 default 'http://localhost',
+    c STRING NOT NULL ENFORCED)
+    PARTITIONED BY (p1 STRING, p2 INT);
+
+-- Insert into
+explain INSERT INTO tablePartitioned partition(p1='today', p2=10) values('not', 'null', 'constraint');
+INSERT INTO tablePartitioned partition(p1='today', p2=10) values('not', 'null', 'constraint');
+DROP TABLE tablePartitioned;
+
+-- try constraint with direct sql as false
+set hive.metastore.try.direct.sql=false;
+CREATE TABLE numericDataType(a TINYINT CONSTRAINT tinyint_constraint DEFAULT 127Y ENABLE, b SMALLINT DEFAULT 32767S, c INT DEFAULT 2147483647,
+    d BIGINT DEFAULT  9223372036854775807L, e DOUBLE DEFAULT 3.4E38, f DECIMAL(9,2) DEFAULT 1234567.89)
+    clustered by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
+ALTER TABLE numericDataType DROP CONSTRAINT tinyint_constraint;
+DESC FORMATTED numericDataType;
+
+EXPLAIN INSERT INTO numericDataType(b) values(456);
+INSERT INTO numericDataType(b) values(456);
+SELECT * from numericDataType;
+
+-- add another constraint on same column
+ALTER TABLE numericDataType ADD CONSTRAINT uk1 UNIQUE(a,b) DISABLE NOVALIDATE;
+DESC FORMATTED numericDataType;
+EXPLAIN INSERT INTO numericDataType(b) values(56);
+INSERT INTO numericDataType(b) values(456);
+SELECT * from numericDataType;
+DROP TABLE numericDataType;
+
+-- Following all are existing BUGS
+-- BUG1: alter table change constraint doesn't work, so following not working
+-- ALTER TABLE numericDataType change a a TINYINT CONSTRAINT default_constraint DEFAULT 1Y ENABLE; -- change default val
+-- ALTER TABLE numericDataType change a a TINYINT CONSTRAINT default_constraint_second DEFAULT 1Y ENABLE; -- change constraint name
+-- ALTER TABLE numericDataType change a a TINYINT CONSTRAINT default_constraint_second DEFAULT 1Y DISABLE; -- DISABLE constraint
+-- BUG2: ADD column not working
+-- ALTER TABLE numericDataType add columns (dd double);
+--BUG3: Following add multiple constraints
+--ALTER TABLE numericDataType CHANGE c c INT DEFAULT cast(4.5 as INT);
+-- BUG4 Replace column doesn't work, so following not workiing
+-- alter table numericDataType replace columns (a TINYINT);
+-- BUG5: select current_database() as default doesn't work
+

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/queries/clientpositive/enforce_constraint_notnull.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/enforce_constraint_notnull.q b/ql/src/test/queries/clientpositive/enforce_constraint_notnull.q
index b89bcb7..2aa4665 100644
--- a/ql/src/test/queries/clientpositive/enforce_constraint_notnull.q
+++ b/ql/src/test/queries/clientpositive/enforce_constraint_notnull.q
@@ -107,7 +107,7 @@ insert overwrite table src_multi2 select * where key > 10 and key < 20;
 set hive.exec.dynamic.partition.mode=nonstrict;
 -- Table with partition
 CREATE TABLE tablePartitioned (a STRING NOT NULL ENFORCED, b STRING, c STRING NOT NULL ENFORCED)
-    PARTITIONED BY (p1 STRING, p2 INT NOT NULL ENABLE);
+    PARTITIONED BY (p1 STRING, p2 INT NOT NULL DISABLE);
 
 -- Insert into
 explain INSERT INTO tablePartitioned partition(p1='today', p2=10) values('not', 'null', 'constraint');

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/alter_external_with_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_external_with_constraint.q.out b/ql/src/test/results/clientnegative/alter_external_with_constraint.q.out
deleted file mode 100644
index a6001a4..0000000
--- a/ql/src/test/results/clientnegative/alter_external_with_constraint.q.out
+++ /dev/null
@@ -1,9 +0,0 @@
-PREHOOK: query: CREATE external TABLE table1 (a STRING, b STRING)
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@table1
-POSTHOOK: query: CREATE external TABLE table1 (a STRING, b STRING)
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@table1
-FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Constraints are disallowed with External tables. Only RELY is allowed.

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/alter_external_with_default_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_external_with_default_constraint.q.out b/ql/src/test/results/clientnegative/alter_external_with_default_constraint.q.out
new file mode 100644
index 0000000..a6001a4
--- /dev/null
+++ b/ql/src/test/results/clientnegative/alter_external_with_default_constraint.q.out
@@ -0,0 +1,9 @@
+PREHOOK: query: CREATE external TABLE table1 (a STRING, b STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table1
+POSTHOOK: query: CREATE external TABLE table1 (a STRING, b STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table1
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Constraints are disallowed with External tables. Only RELY is allowed.

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/alter_external_with_notnull_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_external_with_notnull_constraint.q.out b/ql/src/test/results/clientnegative/alter_external_with_notnull_constraint.q.out
new file mode 100644
index 0000000..a6001a4
--- /dev/null
+++ b/ql/src/test/results/clientnegative/alter_external_with_notnull_constraint.q.out
@@ -0,0 +1,9 @@
+PREHOOK: query: CREATE external TABLE table1 (a STRING, b STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table1
+POSTHOOK: query: CREATE external TABLE table1 (a STRING, b STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table1
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Constraints are disallowed with External tables. Only RELY is allowed.

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/alter_tableprops_external_with_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_tableprops_external_with_constraint.q.out b/ql/src/test/results/clientnegative/alter_tableprops_external_with_constraint.q.out
deleted file mode 100644
index 6cfc932..0000000
--- a/ql/src/test/results/clientnegative/alter_tableprops_external_with_constraint.q.out
+++ /dev/null
@@ -1,9 +0,0 @@
-PREHOOK: query: CREATE TABLE table1 (a STRING NOT NULL enforced, b STRING)
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@table1
-POSTHOOK: query: CREATE TABLE table1 (a STRING NOT NULL enforced, b STRING)
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@table1
-FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Table: default.table1 has constraints enabled.Please remove those constraints to change this property.

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/alter_tableprops_external_with_default_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_tableprops_external_with_default_constraint.q.out b/ql/src/test/results/clientnegative/alter_tableprops_external_with_default_constraint.q.out
new file mode 100644
index 0000000..de5e276
--- /dev/null
+++ b/ql/src/test/results/clientnegative/alter_tableprops_external_with_default_constraint.q.out
@@ -0,0 +1,9 @@
+PREHOOK: query: CREATE TABLE table1 (a STRING DEFAULT 'hive', b STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table1
+POSTHOOK: query: CREATE TABLE table1 (a STRING DEFAULT 'hive', b STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table1
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Table: default.table1 has constraints enabled.Please remove those constraints to change this property.

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/alter_tableprops_external_with_notnull_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_tableprops_external_with_notnull_constraint.q.out b/ql/src/test/results/clientnegative/alter_tableprops_external_with_notnull_constraint.q.out
new file mode 100644
index 0000000..6cfc932
--- /dev/null
+++ b/ql/src/test/results/clientnegative/alter_tableprops_external_with_notnull_constraint.q.out
@@ -0,0 +1,9 @@
+PREHOOK: query: CREATE TABLE table1 (a STRING NOT NULL enforced, b STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table1
+POSTHOOK: query: CREATE TABLE table1 (a STRING NOT NULL enforced, b STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table1
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Table: default.table1 has constraints enabled.Please remove those constraints to change this property.

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/constraint_duplicate_name.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/constraint_duplicate_name.q.out b/ql/src/test/results/clientnegative/constraint_duplicate_name.q.out
new file mode 100644
index 0000000..8a154f6
--- /dev/null
+++ b/ql/src/test/results/clientnegative/constraint_duplicate_name.q.out
@@ -0,0 +1,13 @@
+PREHOOK: query: create table t(i int constraint c1 not null enable)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t
+POSTHOOK: query: create table t(i int constraint c1 not null enable)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t
+PREHOOK: query: create table t1(j int constraint c1 default 4)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidObjectException(message:Constraint name already exists: c1)

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/constraint_invalide_name.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/constraint_invalide_name.q.out b/ql/src/test/results/clientnegative/constraint_invalide_name.q.out
new file mode 100644
index 0000000..de749b4
--- /dev/null
+++ b/ql/src/test/results/clientnegative/constraint_invalide_name.q.out
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Constraint name: aaaaabcdatyaaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaabcdatyaaaa exceeded maximum allowed length: 255

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/constraint_partition_columns.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/constraint_partition_columns.q.out b/ql/src/test/results/clientnegative/constraint_partition_columns.q.out
new file mode 100644
index 0000000..d629ed0
--- /dev/null
+++ b/ql/src/test/results/clientnegative/constraint_partition_columns.q.out
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax NOT NULL and Default Constraints are not allowed with partition columns. 

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/create_external_with_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_external_with_constraint.q.out b/ql/src/test/results/clientnegative/create_external_with_constraint.q.out
deleted file mode 100644
index e69566d..0000000
--- a/ql/src/test/results/clientnegative/create_external_with_constraint.q.out
+++ /dev/null
@@ -1 +0,0 @@
-FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Constraints are disallowed with External tables. Only RELY is allowed.

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/create_external_with_default_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_external_with_default_constraint.q.out b/ql/src/test/results/clientnegative/create_external_with_default_constraint.q.out
new file mode 100644
index 0000000..e69566d
--- /dev/null
+++ b/ql/src/test/results/clientnegative/create_external_with_default_constraint.q.out
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Constraints are disallowed with External tables. Only RELY is allowed.

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/create_external_with_notnull_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_external_with_notnull_constraint.q.out b/ql/src/test/results/clientnegative/create_external_with_notnull_constraint.q.out
new file mode 100644
index 0000000..e69566d
--- /dev/null
+++ b/ql/src/test/results/clientnegative/create_external_with_notnull_constraint.q.out
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Constraints are disallowed with External tables. Only RELY is allowed.

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/default_constraint_complex_default_value.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/default_constraint_complex_default_value.q.out b/ql/src/test/results/clientnegative/default_constraint_complex_default_value.q.out
new file mode 100644
index 0000000..e178934
--- /dev/null
+++ b/ql/src/test/results/clientnegative/default_constraint_complex_default_value.q.out
@@ -0,0 +1 @@
+FAILED: ParseException line 2:31 cannot recognize input near 'default' '>' 'default' in list type

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/default_constraint_invalid_default_value.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/default_constraint_invalid_default_value.q.out b/ql/src/test/results/clientnegative/default_constraint_invalid_default_value.q.out
new file mode 100644
index 0000000..af727dc
--- /dev/null
+++ b/ql/src/test/results/clientnegative/default_constraint_invalid_default_value.q.out
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Invalid Default value!

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/default_constraint_invalid_default_value2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/default_constraint_invalid_default_value2.q.out b/ql/src/test/results/clientnegative/default_constraint_invalid_default_value2.q.out
new file mode 100644
index 0000000..76e5aeb
--- /dev/null
+++ b/ql/src/test/results/clientnegative/default_constraint_invalid_default_value2.q.out
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Invalid Default value: repeat('s', 4). DEFAULT only allows constant or function expressions

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/default_constraint_invalid_default_value_length.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/default_constraint_invalid_default_value_length.q.out b/ql/src/test/results/clientnegative/default_constraint_invalid_default_value_length.q.out
new file mode 100644
index 0000000..f08a6ac
--- /dev/null
+++ b/ql/src/test/results/clientnegative/default_constraint_invalid_default_value_length.q.out
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Invalid Default value:  '12345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234'Maximum character length allowed is 255 .

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/default_constraint_invalid_default_value_type.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/default_constraint_invalid_default_value_type.q.out b/ql/src/test/results/clientnegative/default_constraint_invalid_default_value_type.q.out
new file mode 100644
index 0000000..c60bc02
--- /dev/null
+++ b/ql/src/test/results/clientnegative/default_constraint_invalid_default_value_type.q.out
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Invalid Default value: UDFToString(year('1970-01-01')). DEFAULT only allows constant or function expressions

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientnegative/default_constraint_invalid_type.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/default_constraint_invalid_type.q.out b/ql/src/test/results/clientnegative/default_constraint_invalid_type.q.out
new file mode 100644
index 0000000..4e4a7b0
--- /dev/null
+++ b/ql/src/test/results/clientnegative/default_constraint_invalid_type.q.out
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Invalid type: int for default value: 1. Please make sure that the type is compatible with column type: double


[16/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientpositive/llap/enforce_constraint_notnull.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/enforce_constraint_notnull.q.out b/ql/src/test/results/clientpositive/llap/enforce_constraint_notnull.q.out
index 57f0cf6..cc12709 100644
--- a/ql/src/test/results/clientpositive/llap/enforce_constraint_notnull.q.out
+++ b/ql/src/test/results/clientpositive/llap/enforce_constraint_notnull.q.out
@@ -3564,12 +3564,12 @@ STAGE PLANS:
           Table: default.src_multi2
 
 PREHOOK: query: CREATE TABLE tablePartitioned (a STRING NOT NULL ENFORCED, b STRING, c STRING NOT NULL ENFORCED)
-    PARTITIONED BY (p1 STRING, p2 INT NOT NULL ENABLE)
+    PARTITIONED BY (p1 STRING, p2 INT NOT NULL DISABLE)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@tablePartitioned
 POSTHOOK: query: CREATE TABLE tablePartitioned (a STRING NOT NULL ENFORCED, b STRING, c STRING NOT NULL ENFORCED)
-    PARTITIONED BY (p1 STRING, p2 INT NOT NULL ENABLE)
+    PARTITIONED BY (p1 STRING, p2 INT NOT NULL DISABLE)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@tablePartitioned

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientpositive/llap/lineage2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/lineage2.q.out b/ql/src/test/results/clientpositive/llap/lineage2.q.out
index 4fb586d..bac5863 100644
--- a/ql/src/test/results/clientpositive/llap/lineage2.q.out
+++ b/ql/src/test/results/clientpositive/llap/lineage2.q.out
@@ -704,4 +704,4 @@ from relations lateral view explode(ep1_ids) rel1 as ep1_id
 PREHOOK: type: QUERY
 PREHOOK: Input: default@relations
 PREHOOK: Output: default@rels_exploded
-{"version":"1.0","engine":"tez","database":"default","hash":"56b2b197f394a30537ce1acf835ff8e1","queryText":"insert into rels_exploded select identity, type,\n  ep1_src_type, ep1_type, ep2_src_type, ep2_type, ep1_id, ep2_id\nfrom relations lateral view explode(ep1_ids) rel1 as ep1_id\n  lateral view explode (ep2_ids) rel2 as ep2_id","edges":[{"sources":[8],"targets":[0],"edgeType":"PROJECTION"},{"sources":[9],"targets":[1],"edgeType":"PROJECTION"},{"sources":[10],"targets":[2],"edgeType":"PROJECTION"},{"sources":[11],"targets":[3],"edgeType":"PROJECTION"},{"sources":[12],"targets":[4],"edgeType":"PROJECTION"},{"sources":[13],"targets":[5],"edgeType":"PROJECTION"},{"sources":[14],"targets":[6],"expression":"CAST( rel1._col11 AS CHAR(32)","edgeType":"PROJECTION"},{"sources":[15],"targets":[7],"expression":"CAST( rel2._col12 AS CHAR(32)","edgeType":"PROJECTION"},{"sources":[8],"targets":[0],"expression":"compute_stats(default.relations.identity, 'hll')","edgeType":"PROJECTION"},{"source
 s":[9],"targets":[1],"expression":"compute_stats(default.relations.type, 'hll')","edgeType":"PROJECTION"},{"sources":[10],"targets":[2],"expression":"compute_stats(default.relations.ep1_src_type, 'hll')","edgeType":"PROJECTION"},{"sources":[11],"targets":[3],"expression":"compute_stats(default.relations.ep1_type, 'hll')","edgeType":"PROJECTION"},{"sources":[12],"targets":[4],"expression":"compute_stats(default.relations.ep2_src_type, 'hll')","edgeType":"PROJECTION"},{"sources":[13],"targets":[5],"expression":"compute_stats(default.relations.ep2_type, 'hll')","edgeType":"PROJECTION"},{"sources":[14],"targets":[6],"expression":"compute_stats(CAST( rel1._col11 AS CHAR(32), 'hll')","edgeType":"PROJECTION"},{"sources":[15],"targets":[7],"expression":"compute_stats(CAST( rel2._col12 AS CHAR(32), 'hll')","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.rels_exploded.identity"},{"id":1,"vertexType":"COLUMN","vertexId":"default.rels_exploded.type"},{"id
 ":2,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep1_src_type"},{"id":3,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep1_type"},{"id":4,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep2_src_type"},{"id":5,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep2_type"},{"id":6,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep1_id"},{"id":7,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep2_id"},{"id":8,"vertexType":"COLUMN","vertexId":"default.relations.identity"},{"id":9,"vertexType":"COLUMN","vertexId":"default.relations.type"},{"id":10,"vertexType":"COLUMN","vertexId":"default.relations.ep1_src_type"},{"id":11,"vertexType":"COLUMN","vertexId":"default.relations.ep1_type"},{"id":12,"vertexType":"COLUMN","vertexId":"default.relations.ep2_src_type"},{"id":13,"vertexType":"COLUMN","vertexId":"default.relations.ep2_type"},{"id":14,"vertexType":"COLUMN","vertexId":"default.relations.ep1_ids"},{"id":15,"vertexType":"COLUMN","vertexId":
 "default.relations.ep2_ids"}]}
+{"version":"1.0","engine":"tez","database":"default","hash":"56b2b197f394a30537ce1acf835ff8e1","queryText":"insert into rels_exploded select identity, type,\n  ep1_src_type, ep1_type, ep2_src_type, ep2_type, ep1_id, ep2_id\nfrom relations lateral view explode(ep1_ids) rel1 as ep1_id\n  lateral view explode (ep2_ids) rel2 as ep2_id","edges":[{"sources":[8],"targets":[0],"edgeType":"PROJECTION"},{"sources":[9],"targets":[1],"edgeType":"PROJECTION"},{"sources":[10],"targets":[2],"edgeType":"PROJECTION"},{"sources":[11],"targets":[3],"edgeType":"PROJECTION"},{"sources":[12],"targets":[4],"edgeType":"PROJECTION"},{"sources":[13],"targets":[5],"edgeType":"PROJECTION"},{"sources":[14],"targets":[6],"expression":"CAST( rel1._col11 AS CHAR(32))","edgeType":"PROJECTION"},{"sources":[15],"targets":[7],"expression":"CAST( rel2._col12 AS CHAR(32))","edgeType":"PROJECTION"},{"sources":[8],"targets":[0],"expression":"compute_stats(default.relations.identity, 'hll')","edgeType":"PROJECTION"},{"sour
 ces":[9],"targets":[1],"expression":"compute_stats(default.relations.type, 'hll')","edgeType":"PROJECTION"},{"sources":[10],"targets":[2],"expression":"compute_stats(default.relations.ep1_src_type, 'hll')","edgeType":"PROJECTION"},{"sources":[11],"targets":[3],"expression":"compute_stats(default.relations.ep1_type, 'hll')","edgeType":"PROJECTION"},{"sources":[12],"targets":[4],"expression":"compute_stats(default.relations.ep2_src_type, 'hll')","edgeType":"PROJECTION"},{"sources":[13],"targets":[5],"expression":"compute_stats(default.relations.ep2_type, 'hll')","edgeType":"PROJECTION"},{"sources":[14],"targets":[6],"expression":"compute_stats(CAST( rel1._col11 AS CHAR(32)), 'hll')","edgeType":"PROJECTION"},{"sources":[15],"targets":[7],"expression":"compute_stats(CAST( rel2._col12 AS CHAR(32)), 'hll')","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.rels_exploded.identity"},{"id":1,"vertexType":"COLUMN","vertexId":"default.rels_exploded.type"},
 {"id":2,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep1_src_type"},{"id":3,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep1_type"},{"id":4,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep2_src_type"},{"id":5,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep2_type"},{"id":6,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep1_id"},{"id":7,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep2_id"},{"id":8,"vertexType":"COLUMN","vertexId":"default.relations.identity"},{"id":9,"vertexType":"COLUMN","vertexId":"default.relations.type"},{"id":10,"vertexType":"COLUMN","vertexId":"default.relations.ep1_src_type"},{"id":11,"vertexType":"COLUMN","vertexId":"default.relations.ep1_type"},{"id":12,"vertexType":"COLUMN","vertexId":"default.relations.ep2_src_type"},{"id":13,"vertexType":"COLUMN","vertexId":"default.relations.ep2_type"},{"id":14,"vertexType":"COLUMN","vertexId":"default.relations.ep1_ids"},{"id":15,"vertexType":"COLUMN","vertex
 Id":"default.relations.ep2_ids"}]}

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientpositive/llap/resourceplan.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/resourceplan.q.out b/ql/src/test/results/clientpositive/llap/resourceplan.q.out
index d790b44..6c4627b 100644
--- a/ql/src/test/results/clientpositive/llap/resourceplan.q.out
+++ b/ql/src/test/results/clientpositive/llap/resourceplan.q.out
@@ -2065,6 +2065,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS`
   `UPDATE_RULE` string,
   `DELETE_RULE` string,
   `ENABLE_VALIDATE_RELY` int,
+  `DEFAULT_VALUE` string,
   CONSTRAINT `SYS_PK_KEY_CONSTRAINTS` PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
@@ -2083,7 +2084,8 @@ TBLPROPERTIES (
   \"CONSTRAINT_TYPE\",
   \"UPDATE_RULE\",
   \"DELETE_RULE\",
-  \"ENABLE_VALIDATE_RELY\"
+  \"ENABLE_VALIDATE_RELY\",
+  \"DEFAULT_VALUE\"
 FROM
   \"KEY_CONSTRAINTS\""
 )
@@ -2104,6 +2106,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS`
   `UPDATE_RULE` string,
   `DELETE_RULE` string,
   `ENABLE_VALIDATE_RELY` int,
+  `DEFAULT_VALUE` string,
   CONSTRAINT `SYS_PK_KEY_CONSTRAINTS` PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
@@ -2122,7 +2125,8 @@ TBLPROPERTIES (
   \"CONSTRAINT_TYPE\",
   \"UPDATE_RULE\",
   \"DELETE_RULE\",
-  \"ENABLE_VALIDATE_RELY\"
+  \"ENABLE_VALIDATE_RELY\",
+  \"DEFAULT_VALUE\"
 FROM
   \"KEY_CONSTRAINTS\""
 )

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientpositive/llap/vector_char_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_char_4.q.out b/ql/src/test/results/clientpositive/llap/vector_char_4.q.out
index ba704350..ca4acf1 100644
--- a/ql/src/test/results/clientpositive/llap/vector_char_4.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_char_4.q.out
@@ -150,7 +150,7 @@ STAGE PLANS:
                   TableScan Vectorization:
                       native: true
                   Select Operator
-                    expressions: CAST( t AS CHAR(10) (type: char(10)), CAST( si AS CHAR(10) (type: char(10)), CAST( i AS CHAR(20) (type: char(20)), CAST( b AS CHAR(30) (type: char(30)), CAST( f AS CHAR(20) (type: char(20)), CAST( d AS CHAR(20) (type: char(20)), CAST( s AS CHAR(50) (type: char(50))
+                    expressions: CAST( t AS CHAR(10)) (type: char(10)), CAST( si AS CHAR(10)) (type: char(10)), CAST( i AS CHAR(20)) (type: char(20)), CAST( b AS CHAR(30)) (type: char(30)), CAST( f AS CHAR(20)) (type: char(20)), CAST( d AS CHAR(20)) (type: char(20)), CAST( s AS CHAR(50)) (type: char(50))
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                     Select Vectorization:
                         className: VectorSelectOperator

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientpositive/llap/vector_windowing_range_multiorder.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_windowing_range_multiorder.q.out b/ql/src/test/results/clientpositive/llap/vector_windowing_range_multiorder.q.out
index bf2591c..1817f64 100644
--- a/ql/src/test/results/clientpositive/llap/vector_windowing_range_multiorder.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_windowing_range_multiorder.q.out
@@ -12187,7 +12187,7 @@ STAGE PLANS:
                       native: true
                       vectorizationSchemaColumns: [0:t:tinyint, 1:si:smallint, 2:i:int, 3:b:bigint, 4:f:float, 5:d:double, 6:bo:boolean, 7:s:string, 8:ts:timestamp, 9:dec:decimal(4,2), 10:bin:binary, 11:ROW__ID:struct<transactionid:bigint,bucketid:int,rowid:bigint>]
                   Reduce Output Operator
-                    key expressions: i (type: int), CAST( s AS CHAR(12) (type: char(12))
+                    key expressions: i (type: int), CAST( s AS CHAR(12)) (type: char(12))
                     sort order: ++
                     Map-reduce partition columns: i (type: int)
                     Reduce Sink Vectorization:
@@ -12239,20 +12239,20 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: CAST( _col7 AS CHAR(12) ASC NULLS LAST
+                        order by: CAST( _col7 AS CHAR(12)) ASC NULLS LAST
                         partition by: _col2
                         raw input shape:
                         window functions:
                             window function definition
                               alias: rank_window_0
-                              arguments: CAST( _col7 AS CHAR(12)
+                              arguments: CAST( _col7 AS CHAR(12))
                               name: rank
                               window function: GenericUDAFRankEvaluator
                               window frame: RANGE PRECEDING(MAX)~FOLLOWING(MAX)
                               isPivotResult: true
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: CAST( _col7 AS CHAR(12) (type: char(12)), rank_window_0 (type: int)
+                    expressions: CAST( _col7 AS CHAR(12)) (type: char(12)), rank_window_0 (type: int)
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
                     Limit

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out b/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
index cc4e090..b6f9f4a 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
@@ -177,7 +177,7 @@ STAGE PLANS:
                     predicate: ((cbigint % 250) = 0) (type: boolean)
                     Statistics: Num rows: 6144 Data size: 842180 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: UDFToBoolean(ctinyint) (type: boolean), UDFToBoolean(csmallint) (type: boolean), UDFToBoolean(cint) (type: boolean), UDFToBoolean(cbigint) (type: boolean), UDFToBoolean(cfloat) (type: boolean), UDFToBoolean(cdouble) (type: boolean), cboolean1 (type: boolean), UDFToBoolean((cbigint * 0L)) (type: boolean), UDFToBoolean(ctimestamp1) (type: boolean), UDFToBoolean(cstring1) (type: boolean), UDFToInteger(ctinyint) (type: int), UDFToInteger(csmallint) (type: int), cint (type: int), UDFToInteger(cbigint) (type: int), UDFToInteger(cfloat) (type: int), UDFToInteger(cdouble) (type: int), UDFToInteger(cboolean1) (type: int), UDFToInteger(ctimestamp1) (type: int), UDFToInteger(cstring1) (type: int), UDFToInteger(substr(cstring1, 1, 1)) (type: int), UDFToByte(cfloat) (type: tinyint), UDFToShort(cfloat) (type: smallint), UDFToLong(cfloat) (type: bigint), UDFToDouble(ctinyint) (type: double), UDFToDouble(csmallint) (type: double), UDFToDouble(cint) (type: double),
  UDFToDouble(cbigint) (type: double), UDFToDouble(cfloat) (type: double), cdouble (type: double), UDFToDouble(cboolean1) (type: double), UDFToDouble(ctimestamp1) (type: double), UDFToDouble(cstring1) (type: double), UDFToDouble(substr(cstring1, 1, 1)) (type: double), UDFToFloat(cint) (type: float), UDFToFloat(cdouble) (type: float), CAST( ctinyint AS TIMESTAMP) (type: timestamp), CAST( csmallint AS TIMESTAMP) (type: timestamp), CAST( cint AS TIMESTAMP) (type: timestamp), CAST( cbigint AS TIMESTAMP) (type: timestamp), CAST( cfloat AS TIMESTAMP) (type: timestamp), CAST( cdouble AS TIMESTAMP) (type: timestamp), CAST( cboolean1 AS TIMESTAMP) (type: timestamp), CAST( (cbigint * 0L) AS TIMESTAMP) (type: timestamp), CAST( CAST( ctimestamp1 AS DATE) AS TIMESTAMP) (type: timestamp), ctimestamp1 (type: timestamp), CAST( cstring1 AS TIMESTAMP) (type: timestamp), CAST( substr(cstring1, 1, 1) AS TIMESTAMP) (type: timestamp), UDFToString(ctinyint) (type: string), UDFToString(csmallint) (type: str
 ing), UDFToString(cint) (type: string), UDFToString(cbigint) (type: string), UDFToString(cfloat) (type: string), UDFToString(cdouble) (type: string), UDFToString(cboolean1) (type: string), UDFToString((cbigint * 0L)) (type: string), UDFToString(ctimestamp1) (type: string), cstring1 (type: string), UDFToString(CAST( cstring1 AS CHAR(10)) (type: string), UDFToString(CAST( cstring1 AS varchar(10))) (type: string), UDFToFloat(UDFToInteger(cfloat)) (type: float), UDFToDouble((cint * 2)) (type: double), UDFToString(sin(cfloat)) (type: string), (UDFToDouble(UDFToFloat(cint)) + UDFToDouble(cboolean1)) (type: double)
+                      expressions: UDFToBoolean(ctinyint) (type: boolean), UDFToBoolean(csmallint) (type: boolean), UDFToBoolean(cint) (type: boolean), UDFToBoolean(cbigint) (type: boolean), UDFToBoolean(cfloat) (type: boolean), UDFToBoolean(cdouble) (type: boolean), cboolean1 (type: boolean), UDFToBoolean((cbigint * 0L)) (type: boolean), UDFToBoolean(ctimestamp1) (type: boolean), UDFToBoolean(cstring1) (type: boolean), UDFToInteger(ctinyint) (type: int), UDFToInteger(csmallint) (type: int), cint (type: int), UDFToInteger(cbigint) (type: int), UDFToInteger(cfloat) (type: int), UDFToInteger(cdouble) (type: int), UDFToInteger(cboolean1) (type: int), UDFToInteger(ctimestamp1) (type: int), UDFToInteger(cstring1) (type: int), UDFToInteger(substr(cstring1, 1, 1)) (type: int), UDFToByte(cfloat) (type: tinyint), UDFToShort(cfloat) (type: smallint), UDFToLong(cfloat) (type: bigint), UDFToDouble(ctinyint) (type: double), UDFToDouble(csmallint) (type: double), UDFToDouble(cint) (type: double),
  UDFToDouble(cbigint) (type: double), UDFToDouble(cfloat) (type: double), cdouble (type: double), UDFToDouble(cboolean1) (type: double), UDFToDouble(ctimestamp1) (type: double), UDFToDouble(cstring1) (type: double), UDFToDouble(substr(cstring1, 1, 1)) (type: double), UDFToFloat(cint) (type: float), UDFToFloat(cdouble) (type: float), CAST( ctinyint AS TIMESTAMP) (type: timestamp), CAST( csmallint AS TIMESTAMP) (type: timestamp), CAST( cint AS TIMESTAMP) (type: timestamp), CAST( cbigint AS TIMESTAMP) (type: timestamp), CAST( cfloat AS TIMESTAMP) (type: timestamp), CAST( cdouble AS TIMESTAMP) (type: timestamp), CAST( cboolean1 AS TIMESTAMP) (type: timestamp), CAST( (cbigint * 0L) AS TIMESTAMP) (type: timestamp), CAST( CAST( ctimestamp1 AS DATE) AS TIMESTAMP) (type: timestamp), ctimestamp1 (type: timestamp), CAST( cstring1 AS TIMESTAMP) (type: timestamp), CAST( substr(cstring1, 1, 1) AS TIMESTAMP) (type: timestamp), UDFToString(ctinyint) (type: string), UDFToString(csmallint) (type: str
 ing), UDFToString(cint) (type: string), UDFToString(cbigint) (type: string), UDFToString(cfloat) (type: string), UDFToString(cdouble) (type: string), UDFToString(cboolean1) (type: string), UDFToString((cbigint * 0L)) (type: string), UDFToString(ctimestamp1) (type: string), cstring1 (type: string), UDFToString(CAST( cstring1 AS CHAR(10))) (type: string), UDFToString(CAST( cstring1 AS varchar(10))) (type: string), UDFToFloat(UDFToInteger(cfloat)) (type: float), UDFToDouble((cint * 2)) (type: double), UDFToString(sin(cfloat)) (type: string), (UDFToDouble(UDFToFloat(cint)) + UDFToDouble(cboolean1)) (type: double)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44, _col45, _col46, _col47, _col48, _col49, _col50, _col51, _col52, _col53, _col54, _col55, _col56, _col57, _col58, _col59, _col60, _col61, _col62
                       Select Vectorization:
                           className: VectorSelectOperator

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientpositive/nonmr_fetch_threshold.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/nonmr_fetch_threshold.q.out b/ql/src/test/results/clientpositive/nonmr_fetch_threshold.q.out
index 6c36a35..9f88f8e 100644
--- a/ql/src/test/results/clientpositive/nonmr_fetch_threshold.q.out
+++ b/ql/src/test/results/clientpositive/nonmr_fetch_threshold.q.out
@@ -296,7 +296,7 @@ STAGE PLANS:
           alias: src
           Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: CAST( key AS CHAR(20) (type: char(20)), 1 (type: int)
+            expressions: CAST( key AS CHAR(20)) (type: char(20)), 1 (type: int)
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             ListSink

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/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 4e8ceaf..82fb5f4 100644
--- a/ql/src/test/results/clientpositive/show_functions.q.out
+++ b/ql/src/test/results/clientpositive/show_functions.q.out
@@ -254,6 +254,14 @@ translate
 trim
 trunc
 ucase
+udftoboolean
+udftobyte
+udftodouble
+udftofloat
+udftointeger
+udftolong
+udftoshort
+udftostring
 unbase64
 unhex
 unix_timestamp
@@ -355,6 +363,8 @@ space
 to_date
 translate
 ucase
+udftobyte
+udftodouble
 variance
 xpath_double
 PREHOOK: query: SHOW FUNCTIONS 'log.*'

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientpositive/spark/union32.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union32.q.out b/ql/src/test/results/clientpositive/spark/union32.q.out
index 5c7759a..f1dc705 100644
--- a/ql/src/test/results/clientpositive/spark/union32.q.out
+++ b/ql/src/test/results/clientpositive/spark/union32.q.out
@@ -491,7 +491,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 11 Data size: 77 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: UDFToDouble(UDFToLong(_col0)) (type: double), UDFToString(CAST( _col1 AS CHAR(20)) (type: string)
+                  expressions: UDFToDouble(UDFToLong(_col0)) (type: double), UDFToString(CAST( _col1 AS CHAR(20))) (type: string)
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 11 Data size: 77 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
@@ -637,7 +637,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 11 Data size: 77 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: UDFToDouble(UDFToLong(_col0)) (type: double), UDFToString(CAST( _col1 AS CHAR(20)) (type: string)
+                  expressions: UDFToDouble(UDFToLong(_col0)) (type: double), UDFToString(CAST( _col1 AS CHAR(20))) (type: string)
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 11 Data size: 77 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientpositive/spark/vector_char_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vector_char_4.q.out b/ql/src/test/results/clientpositive/spark/vector_char_4.q.out
index c03ca21..96b829e 100644
--- a/ql/src/test/results/clientpositive/spark/vector_char_4.q.out
+++ b/ql/src/test/results/clientpositive/spark/vector_char_4.q.out
@@ -149,7 +149,7 @@ STAGE PLANS:
                   TableScan Vectorization:
                       native: true
                   Select Operator
-                    expressions: CAST( t AS CHAR(10) (type: char(10)), CAST( si AS CHAR(10) (type: char(10)), CAST( i AS CHAR(20) (type: char(20)), CAST( b AS CHAR(30) (type: char(30)), CAST( f AS CHAR(20) (type: char(20)), CAST( d AS CHAR(20) (type: char(20)), CAST( s AS CHAR(50) (type: char(50))
+                    expressions: CAST( t AS CHAR(10)) (type: char(10)), CAST( si AS CHAR(10)) (type: char(10)), CAST( i AS CHAR(20)) (type: char(20)), CAST( b AS CHAR(30)) (type: char(30)), CAST( f AS CHAR(20)) (type: char(20)), CAST( d AS CHAR(20)) (type: char(20)), CAST( s AS CHAR(50)) (type: char(50))
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                     Select Vectorization:
                         className: VectorSelectOperator

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientpositive/union32.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union32.q.out b/ql/src/test/results/clientpositive/union32.q.out
index f0b2087..186b8d5 100644
--- a/ql/src/test/results/clientpositive/union32.q.out
+++ b/ql/src/test/results/clientpositive/union32.q.out
@@ -488,7 +488,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 11 Data size: 77 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: UDFToDouble(UDFToLong(_col0)) (type: double), UDFToString(CAST( _col1 AS CHAR(20)) (type: string)
+            expressions: UDFToDouble(UDFToLong(_col0)) (type: double), UDFToString(CAST( _col1 AS CHAR(20))) (type: string)
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 11 Data size: 77 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
@@ -643,7 +643,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 11 Data size: 77 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: UDFToDouble(UDFToLong(_col0)) (type: double), UDFToString(CAST( _col1 AS CHAR(20)) (type: string)
+            expressions: UDFToDouble(UDFToLong(_col0)) (type: double), UDFToString(CAST( _col1 AS CHAR(20))) (type: string)
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 11 Data size: 77 Basic stats: COMPLETE Column stats: NONE
             File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientpositive/vector_char_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_char_4.q.out b/ql/src/test/results/clientpositive/vector_char_4.q.out
index b388550..5b9f272 100644
--- a/ql/src/test/results/clientpositive/vector_char_4.q.out
+++ b/ql/src/test/results/clientpositive/vector_char_4.q.out
@@ -151,7 +151,7 @@ STAGE PLANS:
             TableScan Vectorization:
                 native: true
             Select Operator
-              expressions: CAST( t AS CHAR(10) (type: char(10)), CAST( si AS CHAR(10) (type: char(10)), CAST( i AS CHAR(20) (type: char(20)), CAST( b AS CHAR(30) (type: char(30)), CAST( f AS CHAR(20) (type: char(20)), CAST( d AS CHAR(20) (type: char(20)), CAST( s AS CHAR(50) (type: char(50))
+              expressions: CAST( t AS CHAR(10)) (type: char(10)), CAST( si AS CHAR(10)) (type: char(10)), CAST( i AS CHAR(20)) (type: char(20)), CAST( b AS CHAR(30)) (type: char(30)), CAST( f AS CHAR(20)) (type: char(20)), CAST( d AS CHAR(20)) (type: char(20)), CAST( s AS CHAR(50)) (type: char(50))
               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
               Select Vectorization:
                   className: VectorSelectOperator

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/test/results/clientpositive/vectorized_casts.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_casts.q.out b/ql/src/test/results/clientpositive/vectorized_casts.q.out
index 9919cab..4da55fe 100644
--- a/ql/src/test/results/clientpositive/vectorized_casts.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_casts.q.out
@@ -174,7 +174,7 @@ STAGE PLANS:
               predicate: ((cbigint % 250) = 0) (type: boolean)
               Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: UDFToBoolean(ctinyint) (type: boolean), UDFToBoolean(csmallint) (type: boolean), UDFToBoolean(cint) (type: boolean), UDFToBoolean(cbigint) (type: boolean), UDFToBoolean(cfloat) (type: boolean), UDFToBoolean(cdouble) (type: boolean), cboolean1 (type: boolean), UDFToBoolean((cbigint * 0L)) (type: boolean), UDFToBoolean(ctimestamp1) (type: boolean), UDFToBoolean(cstring1) (type: boolean), UDFToInteger(ctinyint) (type: int), UDFToInteger(csmallint) (type: int), cint (type: int), UDFToInteger(cbigint) (type: int), UDFToInteger(cfloat) (type: int), UDFToInteger(cdouble) (type: int), UDFToInteger(cboolean1) (type: int), UDFToInteger(ctimestamp1) (type: int), UDFToInteger(cstring1) (type: int), UDFToInteger(substr(cstring1, 1, 1)) (type: int), UDFToByte(cfloat) (type: tinyint), UDFToShort(cfloat) (type: smallint), UDFToLong(cfloat) (type: bigint), UDFToDouble(ctinyint) (type: double), UDFToDouble(csmallint) (type: double), UDFToDouble(cint) (type: double), UDFTo
 Double(cbigint) (type: double), UDFToDouble(cfloat) (type: double), cdouble (type: double), UDFToDouble(cboolean1) (type: double), UDFToDouble(ctimestamp1) (type: double), UDFToDouble(cstring1) (type: double), UDFToDouble(substr(cstring1, 1, 1)) (type: double), UDFToFloat(cint) (type: float), UDFToFloat(cdouble) (type: float), CAST( ctinyint AS TIMESTAMP) (type: timestamp), CAST( csmallint AS TIMESTAMP) (type: timestamp), CAST( cint AS TIMESTAMP) (type: timestamp), CAST( cbigint AS TIMESTAMP) (type: timestamp), CAST( cfloat AS TIMESTAMP) (type: timestamp), CAST( cdouble AS TIMESTAMP) (type: timestamp), CAST( cboolean1 AS TIMESTAMP) (type: timestamp), CAST( (cbigint * 0L) AS TIMESTAMP) (type: timestamp), CAST( CAST( ctimestamp1 AS DATE) AS TIMESTAMP) (type: timestamp), ctimestamp1 (type: timestamp), CAST( cstring1 AS TIMESTAMP) (type: timestamp), CAST( substr(cstring1, 1, 1) AS TIMESTAMP) (type: timestamp), UDFToString(ctinyint) (type: string), UDFToString(csmallint) (type: string), 
 UDFToString(cint) (type: string), UDFToString(cbigint) (type: string), UDFToString(cfloat) (type: string), UDFToString(cdouble) (type: string), UDFToString(cboolean1) (type: string), UDFToString((cbigint * 0L)) (type: string), UDFToString(ctimestamp1) (type: string), cstring1 (type: string), UDFToString(CAST( cstring1 AS CHAR(10)) (type: string), UDFToString(CAST( cstring1 AS varchar(10))) (type: string), UDFToFloat(UDFToInteger(cfloat)) (type: float), UDFToDouble((cint * 2)) (type: double), UDFToString(sin(cfloat)) (type: string), (UDFToDouble(UDFToFloat(cint)) + UDFToDouble(cboolean1)) (type: double)
+                expressions: UDFToBoolean(ctinyint) (type: boolean), UDFToBoolean(csmallint) (type: boolean), UDFToBoolean(cint) (type: boolean), UDFToBoolean(cbigint) (type: boolean), UDFToBoolean(cfloat) (type: boolean), UDFToBoolean(cdouble) (type: boolean), cboolean1 (type: boolean), UDFToBoolean((cbigint * 0L)) (type: boolean), UDFToBoolean(ctimestamp1) (type: boolean), UDFToBoolean(cstring1) (type: boolean), UDFToInteger(ctinyint) (type: int), UDFToInteger(csmallint) (type: int), cint (type: int), UDFToInteger(cbigint) (type: int), UDFToInteger(cfloat) (type: int), UDFToInteger(cdouble) (type: int), UDFToInteger(cboolean1) (type: int), UDFToInteger(ctimestamp1) (type: int), UDFToInteger(cstring1) (type: int), UDFToInteger(substr(cstring1, 1, 1)) (type: int), UDFToByte(cfloat) (type: tinyint), UDFToShort(cfloat) (type: smallint), UDFToLong(cfloat) (type: bigint), UDFToDouble(ctinyint) (type: double), UDFToDouble(csmallint) (type: double), UDFToDouble(cint) (type: double), UDFTo
 Double(cbigint) (type: double), UDFToDouble(cfloat) (type: double), cdouble (type: double), UDFToDouble(cboolean1) (type: double), UDFToDouble(ctimestamp1) (type: double), UDFToDouble(cstring1) (type: double), UDFToDouble(substr(cstring1, 1, 1)) (type: double), UDFToFloat(cint) (type: float), UDFToFloat(cdouble) (type: float), CAST( ctinyint AS TIMESTAMP) (type: timestamp), CAST( csmallint AS TIMESTAMP) (type: timestamp), CAST( cint AS TIMESTAMP) (type: timestamp), CAST( cbigint AS TIMESTAMP) (type: timestamp), CAST( cfloat AS TIMESTAMP) (type: timestamp), CAST( cdouble AS TIMESTAMP) (type: timestamp), CAST( cboolean1 AS TIMESTAMP) (type: timestamp), CAST( (cbigint * 0L) AS TIMESTAMP) (type: timestamp), CAST( CAST( ctimestamp1 AS DATE) AS TIMESTAMP) (type: timestamp), ctimestamp1 (type: timestamp), CAST( cstring1 AS TIMESTAMP) (type: timestamp), CAST( substr(cstring1, 1, 1) AS TIMESTAMP) (type: timestamp), UDFToString(ctinyint) (type: string), UDFToString(csmallint) (type: string), 
 UDFToString(cint) (type: string), UDFToString(cbigint) (type: string), UDFToString(cfloat) (type: string), UDFToString(cdouble) (type: string), UDFToString(cboolean1) (type: string), UDFToString((cbigint * 0L)) (type: string), UDFToString(ctimestamp1) (type: string), cstring1 (type: string), UDFToString(CAST( cstring1 AS CHAR(10))) (type: string), UDFToString(CAST( cstring1 AS varchar(10))) (type: string), UDFToFloat(UDFToInteger(cfloat)) (type: float), UDFToDouble((cint * 2)) (type: double), UDFToString(sin(cfloat)) (type: string), (UDFToDouble(UDFToFloat(cint)) + UDFToDouble(cboolean1)) (type: double)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44, _col45, _col46, _col47, _col48, _col49, _col50, _col51, _col52, _col53, _col54, _col55, _col56, _col57, _col58, _col59, _col60, _col61, _col62
                 Select Vectorization:
                     className: VectorSelectOperator


[06/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index d063de8..57b6bfa 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -162,12 +162,13 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    * @param \metastore\SQLForeignKey[] $foreignKeys
    * @param \metastore\SQLUniqueConstraint[] $uniqueConstraints
    * @param \metastore\SQLNotNullConstraint[] $notNullConstraints
+   * @param \metastore\SQLDefaultConstraint[] $defaultConstraints
    * @throws \metastore\AlreadyExistsException
    * @throws \metastore\InvalidObjectException
    * @throws \metastore\MetaException
    * @throws \metastore\NoSuchObjectException
    */
-  public function create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys, array $uniqueConstraints, array $notNullConstraints);
+  public function create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys, array $uniqueConstraints, array $notNullConstraints, array $defaultConstraints);
   /**
    * @param \metastore\DropConstraintRequest $req
    * @throws \metastore\NoSuchObjectException
@@ -199,6 +200,12 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function add_not_null_constraint(\metastore\AddNotNullConstraintRequest $req);
   /**
+   * @param \metastore\AddDefaultConstraintRequest $req
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function add_default_constraint(\metastore\AddDefaultConstraintRequest $req);
+  /**
    * @param string $dbname
    * @param string $name
    * @param bool $deleteData
@@ -828,6 +835,13 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function get_not_null_constraints(\metastore\NotNullConstraintsRequest $request);
   /**
+   * @param \metastore\DefaultConstraintsRequest $request
+   * @return \metastore\DefaultConstraintsResponse
+   * @throws \metastore\MetaException
+   * @throws \metastore\NoSuchObjectException
+   */
+  public function get_default_constraints(\metastore\DefaultConstraintsRequest $request);
+  /**
    * @param \metastore\ColumnStatistics $stats_obj
    * @return bool
    * @throws \metastore\NoSuchObjectException
@@ -2463,13 +2477,13 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     return;
   }
 
-  public function create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys, array $uniqueConstraints, array $notNullConstraints)
+  public function create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys, array $uniqueConstraints, array $notNullConstraints, array $defaultConstraints)
   {
-    $this->send_create_table_with_constraints($tbl, $primaryKeys, $foreignKeys, $uniqueConstraints, $notNullConstraints);
+    $this->send_create_table_with_constraints($tbl, $primaryKeys, $foreignKeys, $uniqueConstraints, $notNullConstraints, $defaultConstraints);
     $this->recv_create_table_with_constraints();
   }
 
-  public function send_create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys, array $uniqueConstraints, array $notNullConstraints)
+  public function send_create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys, array $uniqueConstraints, array $notNullConstraints, array $defaultConstraints)
   {
     $args = new \metastore\ThriftHiveMetastore_create_table_with_constraints_args();
     $args->tbl = $tbl;
@@ -2477,6 +2491,7 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     $args->foreignKeys = $foreignKeys;
     $args->uniqueConstraints = $uniqueConstraints;
     $args->notNullConstraints = $notNullConstraints;
+    $args->defaultConstraints = $defaultConstraints;
     $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
     if ($bin_accel)
     {
@@ -2797,6 +2812,60 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     return;
   }
 
+  public function add_default_constraint(\metastore\AddDefaultConstraintRequest $req)
+  {
+    $this->send_add_default_constraint($req);
+    $this->recv_add_default_constraint();
+  }
+
+  public function send_add_default_constraint(\metastore\AddDefaultConstraintRequest $req)
+  {
+    $args = new \metastore\ThriftHiveMetastore_add_default_constraint_args();
+    $args->req = $req;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'add_default_constraint', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('add_default_constraint', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_add_default_constraint()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_add_default_constraint_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_add_default_constraint_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    return;
+  }
+
   public function drop_table($dbname, $name, $deleteData)
   {
     $this->send_drop_table($dbname, $name, $deleteData);
@@ -6974,6 +7043,63 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("get_not_null_constraints failed: unknown result");
   }
 
+  public function get_default_constraints(\metastore\DefaultConstraintsRequest $request)
+  {
+    $this->send_get_default_constraints($request);
+    return $this->recv_get_default_constraints();
+  }
+
+  public function send_get_default_constraints(\metastore\DefaultConstraintsRequest $request)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_default_constraints_args();
+    $args->request = $request;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_default_constraints', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_default_constraints', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_default_constraints()
+  {
+    $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_default_constraints_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_default_constraints_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    throw new \Exception("get_default_constraints failed: unknown result");
+  }
+
   public function update_table_column_statistics(\metastore\ColumnStatistics $stats_obj)
   {
     $this->send_update_table_column_statistics($stats_obj);
@@ -13064,14 +13190,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size759 = 0;
-            $_etype762 = 0;
-            $xfer += $input->readListBegin($_etype762, $_size759);
-            for ($_i763 = 0; $_i763 < $_size759; ++$_i763)
+            $_size773 = 0;
+            $_etype776 = 0;
+            $xfer += $input->readListBegin($_etype776, $_size773);
+            for ($_i777 = 0; $_i777 < $_size773; ++$_i777)
             {
-              $elem764 = null;
-              $xfer += $input->readString($elem764);
-              $this->success []= $elem764;
+              $elem778 = null;
+              $xfer += $input->readString($elem778);
+              $this->success []= $elem778;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13107,9 +13233,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter765)
+          foreach ($this->success as $iter779)
           {
-            $xfer += $output->writeString($iter765);
+            $xfer += $output->writeString($iter779);
           }
         }
         $output->writeListEnd();
@@ -13240,14 +13366,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size766 = 0;
-            $_etype769 = 0;
-            $xfer += $input->readListBegin($_etype769, $_size766);
-            for ($_i770 = 0; $_i770 < $_size766; ++$_i770)
+            $_size780 = 0;
+            $_etype783 = 0;
+            $xfer += $input->readListBegin($_etype783, $_size780);
+            for ($_i784 = 0; $_i784 < $_size780; ++$_i784)
             {
-              $elem771 = null;
-              $xfer += $input->readString($elem771);
-              $this->success []= $elem771;
+              $elem785 = null;
+              $xfer += $input->readString($elem785);
+              $this->success []= $elem785;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13283,9 +13409,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter772)
+          foreach ($this->success as $iter786)
           {
-            $xfer += $output->writeString($iter772);
+            $xfer += $output->writeString($iter786);
           }
         }
         $output->writeListEnd();
@@ -14286,18 +14412,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size773 = 0;
-            $_ktype774 = 0;
-            $_vtype775 = 0;
-            $xfer += $input->readMapBegin($_ktype774, $_vtype775, $_size773);
-            for ($_i777 = 0; $_i777 < $_size773; ++$_i777)
+            $_size787 = 0;
+            $_ktype788 = 0;
+            $_vtype789 = 0;
+            $xfer += $input->readMapBegin($_ktype788, $_vtype789, $_size787);
+            for ($_i791 = 0; $_i791 < $_size787; ++$_i791)
             {
-              $key778 = '';
-              $val779 = new \metastore\Type();
-              $xfer += $input->readString($key778);
-              $val779 = new \metastore\Type();
-              $xfer += $val779->read($input);
-              $this->success[$key778] = $val779;
+              $key792 = '';
+              $val793 = new \metastore\Type();
+              $xfer += $input->readString($key792);
+              $val793 = new \metastore\Type();
+              $xfer += $val793->read($input);
+              $this->success[$key792] = $val793;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -14333,10 +14459,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter780 => $viter781)
+          foreach ($this->success as $kiter794 => $viter795)
           {
-            $xfer += $output->writeString($kiter780);
-            $xfer += $viter781->write($output);
+            $xfer += $output->writeString($kiter794);
+            $xfer += $viter795->write($output);
           }
         }
         $output->writeMapEnd();
@@ -14540,15 +14666,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size782 = 0;
-            $_etype785 = 0;
-            $xfer += $input->readListBegin($_etype785, $_size782);
-            for ($_i786 = 0; $_i786 < $_size782; ++$_i786)
+            $_size796 = 0;
+            $_etype799 = 0;
+            $xfer += $input->readListBegin($_etype799, $_size796);
+            for ($_i800 = 0; $_i800 < $_size796; ++$_i800)
             {
-              $elem787 = null;
-              $elem787 = new \metastore\FieldSchema();
-              $xfer += $elem787->read($input);
-              $this->success []= $elem787;
+              $elem801 = null;
+              $elem801 = new \metastore\FieldSchema();
+              $xfer += $elem801->read($input);
+              $this->success []= $elem801;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14600,9 +14726,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter788)
+          foreach ($this->success as $iter802)
           {
-            $xfer += $iter788->write($output);
+            $xfer += $iter802->write($output);
           }
         }
         $output->writeListEnd();
@@ -14844,15 +14970,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size789 = 0;
-            $_etype792 = 0;
-            $xfer += $input->readListBegin($_etype792, $_size789);
-            for ($_i793 = 0; $_i793 < $_size789; ++$_i793)
+            $_size803 = 0;
+            $_etype806 = 0;
+            $xfer += $input->readListBegin($_etype806, $_size803);
+            for ($_i807 = 0; $_i807 < $_size803; ++$_i807)
             {
-              $elem794 = null;
-              $elem794 = new \metastore\FieldSchema();
-              $xfer += $elem794->read($input);
-              $this->success []= $elem794;
+              $elem808 = null;
+              $elem808 = new \metastore\FieldSchema();
+              $xfer += $elem808->read($input);
+              $this->success []= $elem808;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14904,9 +15030,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter795)
+          foreach ($this->success as $iter809)
           {
-            $xfer += $iter795->write($output);
+            $xfer += $iter809->write($output);
           }
         }
         $output->writeListEnd();
@@ -15120,15 +15246,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size796 = 0;
-            $_etype799 = 0;
-            $xfer += $input->readListBegin($_etype799, $_size796);
-            for ($_i800 = 0; $_i800 < $_size796; ++$_i800)
+            $_size810 = 0;
+            $_etype813 = 0;
+            $xfer += $input->readListBegin($_etype813, $_size810);
+            for ($_i814 = 0; $_i814 < $_size810; ++$_i814)
             {
-              $elem801 = null;
-              $elem801 = new \metastore\FieldSchema();
-              $xfer += $elem801->read($input);
-              $this->success []= $elem801;
+              $elem815 = null;
+              $elem815 = new \metastore\FieldSchema();
+              $xfer += $elem815->read($input);
+              $this->success []= $elem815;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15180,9 +15306,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter802)
+          foreach ($this->success as $iter816)
           {
-            $xfer += $iter802->write($output);
+            $xfer += $iter816->write($output);
           }
         }
         $output->writeListEnd();
@@ -15424,15 +15550,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size803 = 0;
-            $_etype806 = 0;
-            $xfer += $input->readListBegin($_etype806, $_size803);
-            for ($_i807 = 0; $_i807 < $_size803; ++$_i807)
+            $_size817 = 0;
+            $_etype820 = 0;
+            $xfer += $input->readListBegin($_etype820, $_size817);
+            for ($_i821 = 0; $_i821 < $_size817; ++$_i821)
             {
-              $elem808 = null;
-              $elem808 = new \metastore\FieldSchema();
-              $xfer += $elem808->read($input);
-              $this->success []= $elem808;
+              $elem822 = null;
+              $elem822 = new \metastore\FieldSchema();
+              $xfer += $elem822->read($input);
+              $this->success []= $elem822;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15484,9 +15610,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter809)
+          foreach ($this->success as $iter823)
           {
-            $xfer += $iter809->write($output);
+            $xfer += $iter823->write($output);
           }
         }
         $output->writeListEnd();
@@ -16030,6 +16156,10 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
    * @var \metastore\SQLNotNullConstraint[]
    */
   public $notNullConstraints = null;
+  /**
+   * @var \metastore\SQLDefaultConstraint[]
+   */
+  public $defaultConstraints = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -16075,6 +16205,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
             'class' => '\metastore\SQLNotNullConstraint',
             ),
           ),
+        6 => array(
+          'var' => 'defaultConstraints',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\SQLDefaultConstraint',
+            ),
+          ),
         );
     }
     if (is_array($vals)) {
@@ -16093,6 +16232,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       if (isset($vals['notNullConstraints'])) {
         $this->notNullConstraints = $vals['notNullConstraints'];
       }
+      if (isset($vals['defaultConstraints'])) {
+        $this->defaultConstraints = $vals['defaultConstraints'];
+      }
     }
   }
 
@@ -16126,15 +16268,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size810 = 0;
-            $_etype813 = 0;
-            $xfer += $input->readListBegin($_etype813, $_size810);
-            for ($_i814 = 0; $_i814 < $_size810; ++$_i814)
+            $_size824 = 0;
+            $_etype827 = 0;
+            $xfer += $input->readListBegin($_etype827, $_size824);
+            for ($_i828 = 0; $_i828 < $_size824; ++$_i828)
             {
-              $elem815 = null;
-              $elem815 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem815->read($input);
-              $this->primaryKeys []= $elem815;
+              $elem829 = null;
+              $elem829 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem829->read($input);
+              $this->primaryKeys []= $elem829;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16144,15 +16286,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size816 = 0;
-            $_etype819 = 0;
-            $xfer += $input->readListBegin($_etype819, $_size816);
-            for ($_i820 = 0; $_i820 < $_size816; ++$_i820)
+            $_size830 = 0;
+            $_etype833 = 0;
+            $xfer += $input->readListBegin($_etype833, $_size830);
+            for ($_i834 = 0; $_i834 < $_size830; ++$_i834)
             {
-              $elem821 = null;
-              $elem821 = new \metastore\SQLForeignKey();
-              $xfer += $elem821->read($input);
-              $this->foreignKeys []= $elem821;
+              $elem835 = null;
+              $elem835 = new \metastore\SQLForeignKey();
+              $xfer += $elem835->read($input);
+              $this->foreignKeys []= $elem835;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16162,15 +16304,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 4:
           if ($ftype == TType::LST) {
             $this->uniqueConstraints = array();
-            $_size822 = 0;
-            $_etype825 = 0;
-            $xfer += $input->readListBegin($_etype825, $_size822);
-            for ($_i826 = 0; $_i826 < $_size822; ++$_i826)
+            $_size836 = 0;
+            $_etype839 = 0;
+            $xfer += $input->readListBegin($_etype839, $_size836);
+            for ($_i840 = 0; $_i840 < $_size836; ++$_i840)
             {
-              $elem827 = null;
-              $elem827 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem827->read($input);
-              $this->uniqueConstraints []= $elem827;
+              $elem841 = null;
+              $elem841 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem841->read($input);
+              $this->uniqueConstraints []= $elem841;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16180,15 +16322,33 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->notNullConstraints = array();
-            $_size828 = 0;
-            $_etype831 = 0;
-            $xfer += $input->readListBegin($_etype831, $_size828);
-            for ($_i832 = 0; $_i832 < $_size828; ++$_i832)
+            $_size842 = 0;
+            $_etype845 = 0;
+            $xfer += $input->readListBegin($_etype845, $_size842);
+            for ($_i846 = 0; $_i846 < $_size842; ++$_i846)
             {
-              $elem833 = null;
-              $elem833 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem833->read($input);
-              $this->notNullConstraints []= $elem833;
+              $elem847 = null;
+              $elem847 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem847->read($input);
+              $this->notNullConstraints []= $elem847;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::LST) {
+            $this->defaultConstraints = array();
+            $_size848 = 0;
+            $_etype851 = 0;
+            $xfer += $input->readListBegin($_etype851, $_size848);
+            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
+            {
+              $elem853 = null;
+              $elem853 = new \metastore\SQLDefaultConstraint();
+              $xfer += $elem853->read($input);
+              $this->defaultConstraints []= $elem853;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16224,9 +16384,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter834)
+          foreach ($this->primaryKeys as $iter854)
           {
-            $xfer += $iter834->write($output);
+            $xfer += $iter854->write($output);
           }
         }
         $output->writeListEnd();
@@ -16241,9 +16401,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter835)
+          foreach ($this->foreignKeys as $iter855)
           {
-            $xfer += $iter835->write($output);
+            $xfer += $iter855->write($output);
           }
         }
         $output->writeListEnd();
@@ -16258,9 +16418,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
         {
-          foreach ($this->uniqueConstraints as $iter836)
+          foreach ($this->uniqueConstraints as $iter856)
           {
-            $xfer += $iter836->write($output);
+            $xfer += $iter856->write($output);
           }
         }
         $output->writeListEnd();
@@ -16275,9 +16435,26 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
         {
-          foreach ($this->notNullConstraints as $iter837)
+          foreach ($this->notNullConstraints as $iter857)
           {
-            $xfer += $iter837->write($output);
+            $xfer += $iter857->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->defaultConstraints !== null) {
+      if (!is_array($this->defaultConstraints)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('defaultConstraints', TType::LST, 6);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints));
+        {
+          foreach ($this->defaultConstraints as $iter858)
+          {
+            $xfer += $iter858->write($output);
           }
         }
         $output->writeListEnd();
@@ -17353,54 +17530,33 @@ class ThriftHiveMetastore_add_not_null_constraint_result {
 
 }
 
-class ThriftHiveMetastore_drop_table_args {
+class ThriftHiveMetastore_add_default_constraint_args {
   static $_TSPEC;
 
   /**
-   * @var string
-   */
-  public $dbname = null;
-  /**
-   * @var string
-   */
-  public $name = null;
-  /**
-   * @var bool
+   * @var \metastore\AddDefaultConstraintRequest
    */
-  public $deleteData = null;
+  public $req = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'dbname',
-          'type' => TType::STRING,
-          ),
-        2 => array(
-          'var' => 'name',
-          'type' => TType::STRING,
-          ),
-        3 => array(
-          'var' => 'deleteData',
-          'type' => TType::BOOL,
+          'var' => 'req',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\AddDefaultConstraintRequest',
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['dbname'])) {
-        $this->dbname = $vals['dbname'];
-      }
-      if (isset($vals['name'])) {
-        $this->name = $vals['name'];
-      }
-      if (isset($vals['deleteData'])) {
-        $this->deleteData = $vals['deleteData'];
+      if (isset($vals['req'])) {
+        $this->req = $vals['req'];
       }
     }
   }
 
   public function getName() {
-    return 'ThriftHiveMetastore_drop_table_args';
+    return 'ThriftHiveMetastore_add_default_constraint_args';
   }
 
   public function read($input)
@@ -17419,22 +17575,9 @@ class ThriftHiveMetastore_drop_table_args {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->dbname);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 2:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->name);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 3:
-          if ($ftype == TType::BOOL) {
-            $xfer += $input->readBool($this->deleteData);
+          if ($ftype == TType::STRUCT) {
+            $this->req = new \metastore\AddDefaultConstraintRequest();
+            $xfer += $this->req->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -17451,20 +17594,13 @@ class ThriftHiveMetastore_drop_table_args {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_args');
-    if ($this->dbname !== null) {
-      $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1);
-      $xfer += $output->writeString($this->dbname);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->name !== null) {
-      $xfer += $output->writeFieldBegin('name', TType::STRING, 2);
-      $xfer += $output->writeString($this->name);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->deleteData !== null) {
-      $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 3);
-      $xfer += $output->writeBool($this->deleteData);
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_default_constraint_args');
+    if ($this->req !== null) {
+      if (!is_object($this->req)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('req', TType::STRUCT, 1);
+      $xfer += $this->req->write($output);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -17474,7 +17610,7 @@ class ThriftHiveMetastore_drop_table_args {
 
 }
 
-class ThriftHiveMetastore_drop_table_result {
+class ThriftHiveMetastore_add_default_constraint_result {
   static $_TSPEC;
 
   /**
@@ -17484,7 +17620,7 @@ class ThriftHiveMetastore_drop_table_result {
   /**
    * @var \metastore\MetaException
    */
-  public $o3 = null;
+  public $o2 = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -17495,7 +17631,7 @@ class ThriftHiveMetastore_drop_table_result {
           'class' => '\metastore\NoSuchObjectException',
           ),
         2 => array(
-          'var' => 'o3',
+          'var' => 'o2',
           'type' => TType::STRUCT,
           'class' => '\metastore\MetaException',
           ),
@@ -17505,14 +17641,237 @@ class ThriftHiveMetastore_drop_table_result {
       if (isset($vals['o1'])) {
         $this->o1 = $vals['o1'];
       }
-      if (isset($vals['o3'])) {
-        $this->o3 = $vals['o3'];
+      if (isset($vals['o2'])) {
+        $this->o2 = $vals['o2'];
       }
     }
   }
 
   public function getName() {
-    return 'ThriftHiveMetastore_drop_table_result';
+    return 'ThriftHiveMetastore_add_default_constraint_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 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\NoSuchObjectException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->o2 = new \metastore\MetaException();
+            $xfer += $this->o2->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_add_default_constraint_result');
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o2 !== null) {
+      $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2);
+      $xfer += $this->o2->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_drop_table_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $dbname = null;
+  /**
+   * @var string
+   */
+  public $name = null;
+  /**
+   * @var bool
+   */
+  public $deleteData = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'dbname',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'name',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'deleteData',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['dbname'])) {
+        $this->dbname = $vals['dbname'];
+      }
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+      if (isset($vals['deleteData'])) {
+        $this->deleteData = $vals['deleteData'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_drop_table_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->dbname);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->deleteData);
+          } 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_drop_table_args');
+    if ($this->dbname !== null) {
+      $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1);
+      $xfer += $output->writeString($this->dbname);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 2);
+      $xfer += $output->writeString($this->name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->deleteData !== null) {
+      $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 3);
+      $xfer += $output->writeBool($this->deleteData);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_drop_table_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\NoSuchObjectException
+   */
+  public $o1 = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o3 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\NoSuchObjectException',
+          ),
+        2 => array(
+          'var' => 'o3',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+      if (isset($vals['o3'])) {
+        $this->o3 = $vals['o3'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_drop_table_result';
   }
 
   public function read($input)
@@ -17913,14 +18272,14 @@ class ThriftHiveMetastore_truncate_table_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size838 = 0;
-            $_etype841 = 0;
-            $xfer += $input->readListBegin($_etype841, $_size838);
-            for ($_i842 = 0; $_i842 < $_size838; ++$_i842)
+            $_size859 = 0;
+            $_etype862 = 0;
+            $xfer += $input->readListBegin($_etype862, $_size859);
+            for ($_i863 = 0; $_i863 < $_size859; ++$_i863)
             {
-              $elem843 = null;
-              $xfer += $input->readString($elem843);
-              $this->partNames []= $elem843;
+              $elem864 = null;
+              $xfer += $input->readString($elem864);
+              $this->partNames []= $elem864;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17958,9 +18317,9 @@ class ThriftHiveMetastore_truncate_table_args {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter844)
+          foreach ($this->partNames as $iter865)
           {
-            $xfer += $output->writeString($iter844);
+            $xfer += $output->writeString($iter865);
           }
         }
         $output->writeListEnd();
@@ -18211,14 +18570,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size845 = 0;
-            $_etype848 = 0;
-            $xfer += $input->readListBegin($_etype848, $_size845);
-            for ($_i849 = 0; $_i849 < $_size845; ++$_i849)
+            $_size866 = 0;
+            $_etype869 = 0;
+            $xfer += $input->readListBegin($_etype869, $_size866);
+            for ($_i870 = 0; $_i870 < $_size866; ++$_i870)
             {
-              $elem850 = null;
-              $xfer += $input->readString($elem850);
-              $this->success []= $elem850;
+              $elem871 = null;
+              $xfer += $input->readString($elem871);
+              $this->success []= $elem871;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18254,9 +18613,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter851)
+          foreach ($this->success as $iter872)
           {
-            $xfer += $output->writeString($iter851);
+            $xfer += $output->writeString($iter872);
           }
         }
         $output->writeListEnd();
@@ -18458,14 +18817,14 @@ class ThriftHiveMetastore_get_tables_by_type_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size852 = 0;
-            $_etype855 = 0;
-            $xfer += $input->readListBegin($_etype855, $_size852);
-            for ($_i856 = 0; $_i856 < $_size852; ++$_i856)
+            $_size873 = 0;
+            $_etype876 = 0;
+            $xfer += $input->readListBegin($_etype876, $_size873);
+            for ($_i877 = 0; $_i877 < $_size873; ++$_i877)
             {
-              $elem857 = null;
-              $xfer += $input->readString($elem857);
-              $this->success []= $elem857;
+              $elem878 = null;
+              $xfer += $input->readString($elem878);
+              $this->success []= $elem878;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18501,9 +18860,9 @@ class ThriftHiveMetastore_get_tables_by_type_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter858)
+          foreach ($this->success as $iter879)
           {
-            $xfer += $output->writeString($iter858);
+            $xfer += $output->writeString($iter879);
           }
         }
         $output->writeListEnd();
@@ -18659,14 +19018,14 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size859 = 0;
-            $_etype862 = 0;
-            $xfer += $input->readListBegin($_etype862, $_size859);
-            for ($_i863 = 0; $_i863 < $_size859; ++$_i863)
+            $_size880 = 0;
+            $_etype883 = 0;
+            $xfer += $input->readListBegin($_etype883, $_size880);
+            for ($_i884 = 0; $_i884 < $_size880; ++$_i884)
             {
-              $elem864 = null;
-              $xfer += $input->readString($elem864);
-              $this->success []= $elem864;
+              $elem885 = null;
+              $xfer += $input->readString($elem885);
+              $this->success []= $elem885;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18702,9 +19061,9 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter865)
+          foreach ($this->success as $iter886)
           {
-            $xfer += $output->writeString($iter865);
+            $xfer += $output->writeString($iter886);
           }
         }
         $output->writeListEnd();
@@ -18809,14 +19168,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size866 = 0;
-            $_etype869 = 0;
-            $xfer += $input->readListBegin($_etype869, $_size866);
-            for ($_i870 = 0; $_i870 < $_size866; ++$_i870)
+            $_size887 = 0;
+            $_etype890 = 0;
+            $xfer += $input->readListBegin($_etype890, $_size887);
+            for ($_i891 = 0; $_i891 < $_size887; ++$_i891)
             {
-              $elem871 = null;
-              $xfer += $input->readString($elem871);
-              $this->tbl_types []= $elem871;
+              $elem892 = null;
+              $xfer += $input->readString($elem892);
+              $this->tbl_types []= $elem892;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18854,9 +19213,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter872)
+          foreach ($this->tbl_types as $iter893)
           {
-            $xfer += $output->writeString($iter872);
+            $xfer += $output->writeString($iter893);
           }
         }
         $output->writeListEnd();
@@ -18933,15 +19292,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size873 = 0;
-            $_etype876 = 0;
-            $xfer += $input->readListBegin($_etype876, $_size873);
-            for ($_i877 = 0; $_i877 < $_size873; ++$_i877)
+            $_size894 = 0;
+            $_etype897 = 0;
+            $xfer += $input->readListBegin($_etype897, $_size894);
+            for ($_i898 = 0; $_i898 < $_size894; ++$_i898)
             {
-              $elem878 = null;
-              $elem878 = new \metastore\TableMeta();
-              $xfer += $elem878->read($input);
-              $this->success []= $elem878;
+              $elem899 = null;
+              $elem899 = new \metastore\TableMeta();
+              $xfer += $elem899->read($input);
+              $this->success []= $elem899;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18977,9 +19336,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter879)
+          foreach ($this->success as $iter900)
           {
-            $xfer += $iter879->write($output);
+            $xfer += $iter900->write($output);
           }
         }
         $output->writeListEnd();
@@ -19135,14 +19494,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size880 = 0;
-            $_etype883 = 0;
-            $xfer += $input->readListBegin($_etype883, $_size880);
-            for ($_i884 = 0; $_i884 < $_size880; ++$_i884)
+            $_size901 = 0;
+            $_etype904 = 0;
+            $xfer += $input->readListBegin($_etype904, $_size901);
+            for ($_i905 = 0; $_i905 < $_size901; ++$_i905)
             {
-              $elem885 = null;
-              $xfer += $input->readString($elem885);
-              $this->success []= $elem885;
+              $elem906 = null;
+              $xfer += $input->readString($elem906);
+              $this->success []= $elem906;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19178,9 +19537,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter886)
+          foreach ($this->success as $iter907)
           {
-            $xfer += $output->writeString($iter886);
+            $xfer += $output->writeString($iter907);
           }
         }
         $output->writeListEnd();
@@ -19495,14 +19854,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size887 = 0;
-            $_etype890 = 0;
-            $xfer += $input->readListBegin($_etype890, $_size887);
-            for ($_i891 = 0; $_i891 < $_size887; ++$_i891)
+            $_size908 = 0;
+            $_etype911 = 0;
+            $xfer += $input->readListBegin($_etype911, $_size908);
+            for ($_i912 = 0; $_i912 < $_size908; ++$_i912)
             {
-              $elem892 = null;
-              $xfer += $input->readString($elem892);
-              $this->tbl_names []= $elem892;
+              $elem913 = null;
+              $xfer += $input->readString($elem913);
+              $this->tbl_names []= $elem913;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19535,9 +19894,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter893)
+          foreach ($this->tbl_names as $iter914)
           {
-            $xfer += $output->writeString($iter893);
+            $xfer += $output->writeString($iter914);
           }
         }
         $output->writeListEnd();
@@ -19602,15 +19961,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size894 = 0;
-            $_etype897 = 0;
-            $xfer += $input->readListBegin($_etype897, $_size894);
-            for ($_i898 = 0; $_i898 < $_size894; ++$_i898)
+            $_size915 = 0;
+            $_etype918 = 0;
+            $xfer += $input->readListBegin($_etype918, $_size915);
+            for ($_i919 = 0; $_i919 < $_size915; ++$_i919)
             {
-              $elem899 = null;
-              $elem899 = new \metastore\Table();
-              $xfer += $elem899->read($input);
-              $this->success []= $elem899;
+              $elem920 = null;
+              $elem920 = new \metastore\Table();
+              $xfer += $elem920->read($input);
+              $this->success []= $elem920;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19638,9 +19997,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter900)
+          foreach ($this->success as $iter921)
           {
-            $xfer += $iter900->write($output);
+            $xfer += $iter921->write($output);
           }
         }
         $output->writeListEnd();
@@ -20167,14 +20526,14 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size901 = 0;
-            $_etype904 = 0;
-            $xfer += $input->readListBegin($_etype904, $_size901);
-            for ($_i905 = 0; $_i905 < $_size901; ++$_i905)
+            $_size922 = 0;
+            $_etype925 = 0;
+            $xfer += $input->readListBegin($_etype925, $_size922);
+            for ($_i926 = 0; $_i926 < $_size922; ++$_i926)
             {
-              $elem906 = null;
-              $xfer += $input->readString($elem906);
-              $this->tbl_names []= $elem906;
+              $elem927 = null;
+              $xfer += $input->readString($elem927);
+              $this->tbl_names []= $elem927;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20207,9 +20566,9 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter907)
+          foreach ($this->tbl_names as $iter928)
           {
-            $xfer += $output->writeString($iter907);
+            $xfer += $output->writeString($iter928);
           }
         }
         $output->writeListEnd();
@@ -20314,18 +20673,18 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size908 = 0;
-            $_ktype909 = 0;
-            $_vtype910 = 0;
-            $xfer += $input->readMapBegin($_ktype909, $_vtype910, $_size908);
-            for ($_i912 = 0; $_i912 < $_size908; ++$_i912)
+            $_size929 = 0;
+            $_ktype930 = 0;
+            $_vtype931 = 0;
+            $xfer += $input->readMapBegin($_ktype930, $_vtype931, $_size929);
+            for ($_i933 = 0; $_i933 < $_size929; ++$_i933)
             {
-              $key913 = '';
-              $val914 = new \metastore\Materialization();
-              $xfer += $input->readString($key913);
-              $val914 = new \metastore\Materialization();
-              $xfer += $val914->read($input);
-              $this->success[$key913] = $val914;
+              $key934 = '';
+              $val935 = new \metastore\Materialization();
+              $xfer += $input->readString($key934);
+              $val935 = new \metastore\Materialization();
+              $xfer += $val935->read($input);
+              $this->success[$key934] = $val935;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -20377,10 +20736,10 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter915 => $viter916)
+          foreach ($this->success as $kiter936 => $viter937)
           {
-            $xfer += $output->writeString($kiter915);
-            $xfer += $viter916->write($output);
+            $xfer += $output->writeString($kiter936);
+            $xfer += $viter937->write($output);
           }
         }
         $output->writeMapEnd();
@@ -20869,14 +21228,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size917 = 0;
-            $_etype920 = 0;
-            $xfer += $input->readListBegin($_etype920, $_size917);
-            for ($_i921 = 0; $_i921 < $_size917; ++$_i921)
+            $_size938 = 0;
+            $_etype941 = 0;
+            $xfer += $input->readListBegin($_etype941, $_size938);
+            for ($_i942 = 0; $_i942 < $_size938; ++$_i942)
             {
-              $elem922 = null;
-              $xfer += $input->readString($elem922);
-              $this->success []= $elem922;
+              $elem943 = null;
+              $xfer += $input->readString($elem943);
+              $this->success []= $elem943;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20928,9 +21287,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter923)
+          foreach ($this->success as $iter944)
           {
-            $xfer += $output->writeString($iter923);
+            $xfer += $output->writeString($iter944);
           }
         }
         $output->writeListEnd();
@@ -22243,15 +22602,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size924 = 0;
-            $_etype927 = 0;
-            $xfer += $input->readListBegin($_etype927, $_size924);
-            for ($_i928 = 0; $_i928 < $_size924; ++$_i928)
+            $_size945 = 0;
+            $_etype948 = 0;
+            $xfer += $input->readListBegin($_etype948, $_size945);
+            for ($_i949 = 0; $_i949 < $_size945; ++$_i949)
             {
-              $elem929 = null;
-              $elem929 = new \metastore\Partition();
-              $xfer += $elem929->read($input);
-              $this->new_parts []= $elem929;
+              $elem950 = null;
+              $elem950 = new \metastore\Partition();
+              $xfer += $elem950->read($input);
+              $this->new_parts []= $elem950;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22279,9 +22638,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter930)
+          foreach ($this->new_parts as $iter951)
           {
-            $xfer += $iter930->write($output);
+            $xfer += $iter951->write($output);
           }
         }
         $output->writeListEnd();
@@ -22496,15 +22855,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size931 = 0;
-            $_etype934 = 0;
-            $xfer += $input->readListBegin($_etype934, $_size931);
-            for ($_i935 = 0; $_i935 < $_size931; ++$_i935)
+            $_size952 = 0;
+            $_etype955 = 0;
+            $xfer += $input->readListBegin($_etype955, $_size952);
+            for ($_i956 = 0; $_i956 < $_size952; ++$_i956)
             {
-              $elem936 = null;
-              $elem936 = new \metastore\PartitionSpec();
-              $xfer += $elem936->read($input);
-              $this->new_parts []= $elem936;
+              $elem957 = null;
+              $elem957 = new \metastore\PartitionSpec();
+              $xfer += $elem957->read($input);
+              $this->new_parts []= $elem957;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22532,9 +22891,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter937)
+          foreach ($this->new_parts as $iter958)
           {
-            $xfer += $iter937->write($output);
+            $xfer += $iter958->write($output);
           }
         }
         $output->writeListEnd();
@@ -22784,14 +23143,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size938 = 0;
-            $_etype941 = 0;
-            $xfer += $input->readListBegin($_etype941, $_size938);
-            for ($_i942 = 0; $_i942 < $_size938; ++$_i942)
+            $_size959 = 0;
+            $_etype962 = 0;
+            $xfer += $input->readListBegin($_etype962, $_size959);
+            for ($_i963 = 0; $_i963 < $_size959; ++$_i963)
             {
-              $elem943 = null;
-              $xfer += $input->readString($elem943);
-              $this->part_vals []= $elem943;
+              $elem964 = null;
+              $xfer += $input->readString($elem964);
+              $this->part_vals []= $elem964;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22829,9 +23188,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter944)
+          foreach ($this->part_vals as $iter965)
           {
-            $xfer += $output->writeString($iter944);
+            $xfer += $output->writeString($iter965);
           }
         }
         $output->writeListEnd();
@@ -23333,14 +23692,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size945 = 0;
-            $_etype948 = 0;
-            $xfer += $input->readListBegin($_etype948, $_size945);
-            for ($_i949 = 0; $_i949 < $_size945; ++$_i949)
+            $_size966 = 0;
+            $_etype969 = 0;
+            $xfer += $input->readListBegin($_etype969, $_size966);
+            for ($_i970 = 0; $_i970 < $_size966; ++$_i970)
             {
-              $elem950 = null;
-              $xfer += $input->readString($elem950);
-              $this->part_vals []= $elem950;
+              $elem971 = null;
+              $xfer += $input->readString($elem971);
+              $this->part_vals []= $elem971;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23386,9 +23745,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter951)
+          foreach ($this->part_vals as $iter972)
           {
-            $xfer += $output->writeString($iter951);
+            $xfer += $output->writeString($iter972);
           }
         }
         $output->writeListEnd();
@@ -24242,14 +24601,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size952 = 0;
-            $_etype955 = 0;
-            $xfer += $input->readListBegin($_etype955, $_size952);
-            for ($_i956 = 0; $_i956 < $_size952; ++$_i956)
+            $_size973 = 0;
+            $_etype976 = 0;
+            $xfer += $input->readListBegin($_etype976, $_size973);
+            for ($_i977 = 0; $_i977 < $_size973; ++$_i977)
             {
-              $elem957 = null;
-              $xfer += $input->readString($elem957);
-              $this->part_vals []= $elem957;
+              $elem978 = null;
+              $xfer += $input->readString($elem978);
+              $this->part_vals []= $elem978;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24294,9 +24653,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter958)
+          foreach ($this->part_vals as $iter979)
           {
-            $xfer += $output->writeString($iter958);
+            $xfer += $output->writeString($iter979);
           }
         }
         $output->writeListEnd();
@@ -24549,14 +24908,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size959 = 0;
-            $_etype962 = 0;
-            $xfer += $input->readListBegin($_etype962, $_size959);
-            for ($_i963 = 0; $_i963 < $_size959; ++$_i963)
+            $_size980 = 0;
+            $_etype983 = 0;
+            $xfer += $input->readListBegin($_etype983, $_size980);
+            for ($_i984 = 0; $_i984 < $_size980; ++$_i984)
             {
-              $elem964 = null;
-              $xfer += $input->readString($elem964);
-              $this->part_vals []= $elem964;
+              $elem985 = null;
+              $xfer += $input->readString($elem985);
+              $this->part_vals []= $elem985;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24609,9 +24968,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter965)
+          foreach ($this->part_vals as $iter986)
           {
-            $xfer += $output->writeString($iter965);
+            $xfer += $output->writeString($iter986);
           }
         }
         $output->writeListEnd();
@@ -25625,14 +25984,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size966 = 0;
-            $_etype969 = 0;
-            $xfer += $input->readListBegin($_etype969, $_size966);
-            for ($_i970 = 0; $_i970 < $_size966; ++$_i970)
+            $_size987 = 0;
+            $_etype990 = 0;
+            $xfer += $input->readListBegin($_etype990, $_size987);
+            for ($_i991 = 0; $_i991 < $_size987; ++$_i991)
             {
-              $elem971 = null;
-              $xfer += $input->readString($elem971);
-              $this->part_vals []= $elem971;
+              $elem992 = null;
+              $xfer += $input->readString($elem992);
+              $this->part_vals []= $elem992;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25670,9 +26029,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter972)
+          foreach ($this->part_vals as $iter993)
           {
-            $xfer += $output->writeString($iter972);
+            $xfer += $output->writeString($iter993);
           }
         }
         $output->writeListEnd();
@@ -25914,17 +26273,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size973 = 0;
-            $_ktype974 = 0;
-            $_vtype975 = 0;
-            $xfer += $input->readMapBegin($_ktype974, $_vtype975, $_size973);
-            for ($_i977 = 0; $_i977 < $_size973; ++$_i977)
+            $_size994 = 0;
+            $_ktype995 = 0;
+            $_vtype996 = 0;
+            $xfer += $input->readMapBegin($_ktype995, $_vtype996, $_size994);
+            for ($_i998 = 0; $_i998 < $_size994; ++$_i998)
             {
-              $key978 = '';
-              $val979 = '';
-              $xfer += $input->readString($key978);
-              $xfer += $input->readString($val979);
-              $this->partitionSpecs[$key978] = $val979;
+              $key999 = '';
+              $val1000 = '';
+              $xfer += $input->readString($key999);
+              $xfer += $input->readString($val1000);
+              $this->partitionSpecs[$key999] = $val1000;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25980,10 +26339,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter980 => $viter981)
+          foreach ($this->partitionSpecs as $kiter1001 => $viter1002)
           {
-            $xfer += $output->writeString($kiter980);
-            $xfer += $output->writeString($viter981);
+            $xfer += $output->writeString($kiter1001);
+            $xfer += $output->writeString($viter1002);
           }
         }
         $output->writeMapEnd();
@@ -26295,17 +26654,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size982 = 0;
-            $_ktype983 = 0;
-            $_vtype984 = 0;
-            $xfer += $input->readMapBegin($_ktype983, $_vtype984, $_size982);
-            for ($_i986 = 0; $_i986 < $_size982; ++$_i986)
+            $_size1003 = 0;
+            $_ktype1004 = 0;
+            $_vtype1005 = 0;
+            $xfer += $input->readMapBegin($_ktype1004, $_vtype1005, $_size1003);
+            for ($_i1007 = 0; $_i1007 < $_size1003; ++$_i1007)
             {
-              $key987 = '';
-              $val988 = '';
-              $xfer += $input->readString($key987);
-              $xfer += $input->readString($val988);
-              $this->partitionSpecs[$key987] = $val988;
+              $key1008 = '';
+              $val1009 = '';
+              $xfer += $input->readString($key1008);
+              $xfer += $input->readString($val1009);
+              $this->partitionSpecs[$key1008] = $val1009;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -26361,10 +26720,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter989 => $viter990)
+          foreach ($this->partitionSpecs as $kiter1010 => $viter1011)
           {
-            $xfer += $output->writeString($kiter989);
-            $xfer += $output->writeString($viter990);
+            $xfer += $output->writeString($kiter1010);
+            $xfer += $output->writeString($viter1011);
           }
         }
         $output->writeMapEnd();
@@ -26497,15 +26856,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size991 = 0;
-            $_etype994 = 0;
-            $xfer += $input->readListBegin($_etype994, $_size991);
-            for ($_i995 = 0; $_i995 < $_size991; ++$_i995)
+            $_size1012 = 0;
+            $_etype1015 = 0;
+            $xfer += $input->readListBegin($_etype1015, $_size1012);
+            for ($_i1016 = 0; $_i1016 < $_size1012; ++$_i1016)
             {
-              $elem996 = null;
-              $elem996 = new \metastore\Partition();
-              $xfer += $elem996->read($input);
-              $this->success []= $elem996;
+              $elem1017 = null;
+              $elem1017 = new \metastore\Partition();
+              $xfer += $elem1017->read($input);
+              $this->success []= $elem1017;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26565,9 +26924,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter997)
+          foreach ($this->success as $iter1018)
           {
-            $xfer += $iter997->write($output);
+            $xfer += $iter1018->write($output);
           }
         }
         $output->writeListEnd();
@@ -26713,14 +27072,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size998 = 0;
-            $_etype1001 = 0;
-            $xfer += $input->readListBegin($_etype1001, $_size998);
-            for ($_i1002 = 0; $_i1002 < $_size998; ++$_i1002)
+            $_size1019 = 0;
+            $_etype1022 = 0;
+            $xfer += $input->readListBegin($_etype1022, $_size1019);
+            for ($_i1023 = 0; $_i1023 < $_size1019; ++$_i1023)
             {
-              $elem1003 = null;
-              $xfer += $input->readString($elem1003);
-              $this->part_vals []= $elem1003;
+              $elem1024 = null;
+              $xfer += $input->readString($elem1024);
+              $this->part_vals []= $elem1024;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26737,14 +27096,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1004 = 0;
-            $_etype1007 = 0;
-            $xfer += $input->readListBegin($_etype1007, $_size1004);
-            for ($_i1008 = 0; $_i1008 < $_size1004; ++$_i1008)
+            $_size1025 = 0;
+            $_etype1028 = 0;
+            $xfer += $input->readListBegin($_etype1028, $_size1025);
+            for ($_i1029 = 0; $_i1029 < $_size1025; ++$_i1029)
             {
-              $elem1009 = null;
-              $xfer += $input->readString($elem1009);
-              $this->group_names []= $elem1009;
+              $elem1030 = null;
+              $xfer += $input->readString($elem1030);
+              $this->group_names []= $elem1030;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26782,9 +27141,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1010)
+          foreach ($this->part_vals as $iter1031)
           {
-            $xfer += $output->writeString($iter1010);
+            $xfer += $output->writeString($iter1031);
           }
         }
         $output->writeListEnd();
@@ -26804,9 +27163,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1011)
+          foreach ($this->group_names as $iter1032)
           {
-            $xfer += $output->writeString($iter1011);
+            $xfer += $output->writeString($iter1032);
           }
         }
         $output->writeListEnd();
@@ -27397,15 +27756,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1012 = 0;
-            $_etype1015 = 0;
-            $xfer += $input->readListBegin($_etype1015, $_size1012);
-            for ($_i1016 = 0; $_i1016 < $_size1012; ++$_i1016)
+            $_size1033 = 0;
+            $_etype1036 = 0;
+            $xfer += $input->readListBegin($_etype1036, $_size1033);
+            for ($_i1037 = 0; $_i1037 < $_size1033; ++$_i1037)
             {
-              $elem1017 = null;
-              $elem1017 = new \metastore\Partition();
-              $xfer += $elem1017->read($input);
-              $this->success []= $elem1017;
+              $elem1038 = null;
+              $elem1038 = new \metastore\Partition();
+              $xfer += $elem1038->read($input);
+              $this->success []= $elem1038;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27449,9 +27808,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1018)
+          foreach ($this->success as $iter1039)
           {
-            $xfer += $iter1018->write($output);
+            $xfer += $iter1039->write($output);
           }
         }
         $output->writeListEnd();
@@ -27597,14 +27956,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1019 = 0;
-            $_etype1022 = 0;
-            $xfer += $input->readListBegin($_etype1022, $_size1019);
-            for ($_i1023 = 0; $_i1023 < $_size1019; ++$_i1023)
+            $_size1040 = 0;
+            $_etype1043 = 0;
+            $xfer += $input->readListBegin($_etype1043, $_size1040);
+            for ($_i1044 = 0; $_i1044 < $_size1040; ++$_i1044)
             {
-              $elem1024 = null;
-              $xfer += $input->readString($elem1024);
-              $this->group_names []= $elem1024;
+              $elem1045 = null;
+              $xfer += $input->readString($elem1045);
+              $this->group_names []= $elem1045;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27652,9 +28011,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1025)
+          foreach ($this->group_names as $iter1046)
           {
-            $xfer += $output->writeString($iter1025);
+            $xfer += $output->writeString($iter1046);
           }
         }
         $output->writeListEnd();
@@ -27743,15 +28102,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1026 = 0;
-            $_etype1029 = 0;
-            $xfer += $input->readListBegin($_etype1029, $_size1026);
-            for ($_i1030 = 0; $_i1030 < $_size1026; ++$_i1030)
+            $_size1047 = 0;
+            $_etype1050 = 0;
+            $xfer += $input->readListBegin($_etype1050, $_size1047);
+            for ($_i1051 = 0; $_i1051 < $_size1047; ++$_i1051)
             {
-              $elem1031 = null;
-              $elem1031 = new \metastore\Partition();
-              $xfer += $elem1031->read($input);
-              $this->success []= $elem1031;
+              $elem1052 = null;
+              $elem1052 = new \metastore\Partition();
+              $xfer += $elem1052->read($input);
+              $this->success []= $elem1052;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27795,9 +28154,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1032)
+          foreach ($this->success as $iter1053)
           {
-            $xfer += $iter1032->write($output);
+            $xfer += $iter1053->write($output);
           }
         }
         $output->writeListEnd();
@@ -28017,15 +28376,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1033 = 0;
-            $_etype1036 = 0;
-            $xfer += $input->readListBegin($_etype1036, $_size1033);
-            for ($_i1037 = 0; $_i1037 < $_size1033; ++$_i1037)
+            $_size1054 = 0;
+            $_etype1057 = 0;
+            $xfer += $input->readListBegin($_etype1057, $_size1054);
+            for ($_i1058 = 0; $_i1058 < $_size1054; ++$_i1058)
             {
-              $elem1038 = null;
-              $elem1038 = new \metastore\PartitionSpec();
-              $xfer += $elem1038->read($input);
-              $this->success []= $elem1038;
+              $elem1059 = null;
+              $elem1059 = new \metastore\PartitionSpec();
+              $xfer += $elem1059->read($input);
+              $this->success []= $elem1059;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28069,9 +28428,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1039)
+          foreach ($this->success as $iter1060)
           {
-            $xfer += $iter1039->write($output);
+            $xfer += $iter1060->write($output);
           }
         }
         $output->writeListEnd();
@@ -28290,14 +28649,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1040 = 0;
-            $_etype1043 = 0;
-            $xfer += $input->readListBegin($_etype1043, $_size1040);
-            for ($_i1044 = 0; $_i1044 < $_size1040; ++$_i1044)
+            $_size1061 = 0;
+            $_etype1064 = 0;
+            $xfer += $input->readListBegin($_etype1064, $_size1061);
+            for ($_i1065 = 0; $_i1065 < $_size1061; ++$_i1065)
             {
-              $elem1045 = null;
-              $xfer += $input->readString($elem1045);
-              $this->success []= $elem1045;
+              $elem1066 = null;
+              $xfer += $input->readString($elem1066);
+              $this->success []= $elem1066;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28341,9 +28700,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1046)
+          foreach ($this->success as $iter1067)
           {
-            $xfer += $output->writeString($iter1046);
+            $xfer += $output->writeString($iter1067);
           }
         }
         $output->writeListEnd();
@@ -28674,14 +29033,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1047 = 0;
-            $_etype1050 = 0;
-            $xfer += $input->readListBegin($_etype1050, $_size1047);
-            for ($_i1051 = 0; $_i1051 < $_size1047; ++$_i1051)
+            $_size1068 = 0;
+            $_etype1071 = 0;
+            $xfer += $input->readListBegin($_etype1071, $_size1068);
+            for ($_i1072 = 0; $_i1072 < $_size1068; ++$_i1072)
             {
-              $elem1052 = null;
-              $xfer += $input->readString($elem1052);
-              $this->part_vals []= $elem1052;
+              $elem1073 = null;
+              $xfer += $input->readString($elem1073);
+              $this->part_vals []= $elem1073;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28726,9 +29085,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1053)
+          foreach ($this->part_vals as $iter1074)
           {
-            $xfer += $output->writeString($iter1053);
+            $xfer += $output->writeString($iter1074);
           }
         }
         $output->writeListEnd();
@@ -28822,15 +29181,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1054 = 0;
-            $_etype1057 = 0;
-            $xfer += $input->readListBegin($_etype1057, $_size1054);
-            for ($_i1058 = 0; $_i1058 < $_size1054; ++$_i1058)
+            $_size1075 = 0;
+            $_etype1078 = 0;
+            $xfer += $input->readListBegin($_etype1078, $_size1075);
+            for ($_i1079 = 0; $_i1079 < $_size1075; ++$_i1079)
             {
-              $elem1059 = null;
-              $elem1059 = new \metastore\Partition();
-              $xfer += $elem1059->read($input);
-              $this->success []= $elem1059;
+              $elem1080 = null;
+              $elem1080 = new \metastore\Partition();
+              $xfer += $elem1080->read($input);
+              $this->success []= $elem1080;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28874,9 +29233,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1060)
+          foreach ($this->success as $iter1081)
           {
-            $xfer += $iter1060->write($output);
+            $xfer += $iter1081->write($output);
           }
         }
         $output->writeListEnd();
@@ -29023,14 +29382,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1061 = 0;
-            $_etype1064 = 0;
-            $xfer += $input->readListBegin($_etype1064, $_size1061);
-            for ($_i1065 = 0; $_i1065 < $_size1061; ++$_i1065)
+            $_size1082 = 0;
+            $_etype1085 = 0;
+            $xfer += $input->readListBegin($_etype1085, $_size1082);
+            for ($_i1086 = 0; $_i1086 < $_size1082; ++$_i1086)
             {
-              $elem1066 = null;
-              $xfer += $input->readString($elem1066);
-              $this->part_vals []= $elem1066;
+              $elem1087 = null;
+              $xfer += $input->readString($elem1087);
+              $this->part_vals []= $elem1087;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29054,14 +29413,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1067 = 0;
-            $_etype1070 = 0;
-            $xfer += $input->readListBegin($_etype1070, $_size1067);
-            for ($_i1071 = 0; $_i1071 < $_size1067; ++$_i1071)
+            $_size1088 = 0;
+            $_etype1091 = 0;
+            $xfer += $input->readListBegin($_etype1091, $_size1088);
+            for ($_i1092 = 0; $_i1092 < $_size1088; ++$_i1092)
             {
-              $elem1072 = null;
-              $xfer += $input->readString($elem1072);
-              $this->group_names []= $elem1072;
+              $elem1093 = null;
+              $xfer += $input->readString($elem1093);
+              $this->group_names []= $elem1093;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29099,9 +29458,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1073)
+          foreach ($this->part_vals as $iter1094)
           {
-            $xfer += $output->writeString($iter1073);
+            $xfer += $output->writeString($iter1094);
           }
         }
         $output->writeListEnd();
@@ -29126,9 +29485,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1074)
+          foreach ($this->group_names as $iter1095)
           {
-            $xfer += $output->writeString($iter1074);
+            $xfer += $output->writeString($iter1095);
           }
         }
         $output->writeListEnd();
@@ -29217,15 +29576,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1075 = 0;
-            $_etype1078 = 0;
-            $xfer += $input->readListBegin($_etype1078, $_size1075);
-            for ($_i1079 = 0; $_i1079 < $_size1075; ++$_i1079)
+            $_size1096 = 0;
+            $_etype1099 = 0;
+            $xfer += $input->readListBegin($_etype1099, $_size1096);
+            for ($_i1100 = 0; $_i1100 < $_size1096; ++$_i1100)
             {
-              $elem1080 = null;
-              $elem1080 = new \metastore\Partition();
-              $xfer += $elem1080->read($input);
-              $this->success []= $elem1080;
+              $elem1101 = null;
+              $elem1101 = new \metastore\Partition();
+              $xfer += $elem1101->read($input);
+              $this->success []= $elem1101;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29269,9 +29628,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1081)
+          foreach ($this->success as $iter1102)
           {
-            $xfer += $iter1081->write($output);
+            $xfer += $iter1102->write($output);
           }
         }
         $output->writeListEnd();
@@ -29392,14 +29751,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1082 = 0;
-            $_etype1085 = 0;
-            $xfer += $input->readListBegin($_etype1085, $_size1082);
-            for ($_i1086 = 0; $_i1086 < $_size1082; ++$_i1086)
+            $_size1103 = 0;
+            $_etype1106 = 0;
+            $xfer += $input->readListBegin($_etype1106, $_size1103);
+            for ($_i1107 = 0; $_i1107 < $_size1103; ++$_i1107)
             {
-              $elem1087 = null;
-              $xfer += $input->readString($elem1087);
-              $this->part_vals []= $elem1087;
+              $elem1108 = null;
+              $xfer += $input->readString($elem1108);
+              $this->part_vals []= $elem1108;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29444,9 +29803,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1088)
+          foreach ($this->part_vals as $iter1109)
           {
-            $xfer += $output->writeString($iter1088);
+            $xfer += $output->writeString($iter1109);
           }
         }
         $output->writeListEnd();
@@ -29539,14 +29898,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1089 = 0;
-            $_etype1092 = 0;
-            $xfer += $input->readListBegin($_etype1092, $_size1089);
-            for ($_i1093 = 0; $_i1093 < $_size1089; ++$_i1093)
+            $_size1110 = 0;
+            $_etype1113 = 0;
+            $xfer += $input->readListBegin($_etype1113, $_size1110);
+            for ($_i1114 = 0; $_i1114 < $_size1110; ++$_i1114)
             {
-              $elem1094 = null;
-              $xf

<TRUNCATED>

[10/19] hive git commit: HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
index 0bcd837..796bbfd 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
@@ -453,13 +453,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ABORTED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set612 = iprot.readSetBegin();
-                struct.aborted = new HashSet<Long>(2*_set612.size);
-                long _elem613;
-                for (int _i614 = 0; _i614 < _set612.size; ++_i614)
+                org.apache.thrift.protocol.TSet _set628 = iprot.readSetBegin();
+                struct.aborted = new HashSet<Long>(2*_set628.size);
+                long _elem629;
+                for (int _i630 = 0; _i630 < _set628.size; ++_i630)
                 {
-                  _elem613 = iprot.readI64();
-                  struct.aborted.add(_elem613);
+                  _elem629 = iprot.readI64();
+                  struct.aborted.add(_elem629);
                 }
                 iprot.readSetEnd();
               }
@@ -471,13 +471,13 @@ import org.slf4j.LoggerFactory;
           case 2: // NOSUCH
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set615 = iprot.readSetBegin();
-                struct.nosuch = new HashSet<Long>(2*_set615.size);
-                long _elem616;
-                for (int _i617 = 0; _i617 < _set615.size; ++_i617)
+                org.apache.thrift.protocol.TSet _set631 = iprot.readSetBegin();
+                struct.nosuch = new HashSet<Long>(2*_set631.size);
+                long _elem632;
+                for (int _i633 = 0; _i633 < _set631.size; ++_i633)
                 {
-                  _elem616 = iprot.readI64();
-                  struct.nosuch.add(_elem616);
+                  _elem632 = iprot.readI64();
+                  struct.nosuch.add(_elem632);
                 }
                 iprot.readSetEnd();
               }
@@ -503,9 +503,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(ABORTED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.aborted.size()));
-          for (long _iter618 : struct.aborted)
+          for (long _iter634 : struct.aborted)
           {
-            oprot.writeI64(_iter618);
+            oprot.writeI64(_iter634);
           }
           oprot.writeSetEnd();
         }
@@ -515,9 +515,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(NOSUCH_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.nosuch.size()));
-          for (long _iter619 : struct.nosuch)
+          for (long _iter635 : struct.nosuch)
           {
-            oprot.writeI64(_iter619);
+            oprot.writeI64(_iter635);
           }
           oprot.writeSetEnd();
         }
@@ -542,16 +542,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.aborted.size());
-        for (long _iter620 : struct.aborted)
+        for (long _iter636 : struct.aborted)
         {
-          oprot.writeI64(_iter620);
+          oprot.writeI64(_iter636);
         }
       }
       {
         oprot.writeI32(struct.nosuch.size());
-        for (long _iter621 : struct.nosuch)
+        for (long _iter637 : struct.nosuch)
         {
-          oprot.writeI64(_iter621);
+          oprot.writeI64(_iter637);
         }
       }
     }
@@ -560,24 +560,24 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, HeartbeatTxnRangeResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TSet _set622 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.aborted = new HashSet<Long>(2*_set622.size);
-        long _elem623;
-        for (int _i624 = 0; _i624 < _set622.size; ++_i624)
+        org.apache.thrift.protocol.TSet _set638 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.aborted = new HashSet<Long>(2*_set638.size);
+        long _elem639;
+        for (int _i640 = 0; _i640 < _set638.size; ++_i640)
         {
-          _elem623 = iprot.readI64();
-          struct.aborted.add(_elem623);
+          _elem639 = iprot.readI64();
+          struct.aborted.add(_elem639);
         }
       }
       struct.setAbortedIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set625 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.nosuch = new HashSet<Long>(2*_set625.size);
-        long _elem626;
-        for (int _i627 = 0; _i627 < _set625.size; ++_i627)
+        org.apache.thrift.protocol.TSet _set641 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.nosuch = new HashSet<Long>(2*_set641.size);
+        long _elem642;
+        for (int _i643 = 0; _i643 < _set641.size; ++_i643)
         {
-          _elem626 = iprot.readI64();
-          struct.nosuch.add(_elem626);
+          _elem642 = iprot.readI64();
+          struct.nosuch.add(_elem642);
         }
       }
       struct.setNosuchIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
index 85272dd..98647b0 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
@@ -538,13 +538,13 @@ import org.slf4j.LoggerFactory;
           case 2: // FILES_ADDED
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list670 = iprot.readListBegin();
-                struct.filesAdded = new ArrayList<String>(_list670.size);
-                String _elem671;
-                for (int _i672 = 0; _i672 < _list670.size; ++_i672)
+                org.apache.thrift.protocol.TList _list686 = iprot.readListBegin();
+                struct.filesAdded = new ArrayList<String>(_list686.size);
+                String _elem687;
+                for (int _i688 = 0; _i688 < _list686.size; ++_i688)
                 {
-                  _elem671 = iprot.readString();
-                  struct.filesAdded.add(_elem671);
+                  _elem687 = iprot.readString();
+                  struct.filesAdded.add(_elem687);
                 }
                 iprot.readListEnd();
               }
@@ -556,13 +556,13 @@ import org.slf4j.LoggerFactory;
           case 3: // FILES_ADDED_CHECKSUM
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list673 = iprot.readListBegin();
-                struct.filesAddedChecksum = new ArrayList<String>(_list673.size);
-                String _elem674;
-                for (int _i675 = 0; _i675 < _list673.size; ++_i675)
+                org.apache.thrift.protocol.TList _list689 = iprot.readListBegin();
+                struct.filesAddedChecksum = new ArrayList<String>(_list689.size);
+                String _elem690;
+                for (int _i691 = 0; _i691 < _list689.size; ++_i691)
                 {
-                  _elem674 = iprot.readString();
-                  struct.filesAddedChecksum.add(_elem674);
+                  _elem690 = iprot.readString();
+                  struct.filesAddedChecksum.add(_elem690);
                 }
                 iprot.readListEnd();
               }
@@ -593,9 +593,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILES_ADDED_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAdded.size()));
-          for (String _iter676 : struct.filesAdded)
+          for (String _iter692 : struct.filesAdded)
           {
-            oprot.writeString(_iter676);
+            oprot.writeString(_iter692);
           }
           oprot.writeListEnd();
         }
@@ -606,9 +606,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FILES_ADDED_CHECKSUM_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAddedChecksum.size()));
-            for (String _iter677 : struct.filesAddedChecksum)
+            for (String _iter693 : struct.filesAddedChecksum)
             {
-              oprot.writeString(_iter677);
+              oprot.writeString(_iter693);
             }
             oprot.writeListEnd();
           }
@@ -634,9 +634,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.filesAdded.size());
-        for (String _iter678 : struct.filesAdded)
+        for (String _iter694 : struct.filesAdded)
         {
-          oprot.writeString(_iter678);
+          oprot.writeString(_iter694);
         }
       }
       BitSet optionals = new BitSet();
@@ -653,9 +653,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFilesAddedChecksum()) {
         {
           oprot.writeI32(struct.filesAddedChecksum.size());
-          for (String _iter679 : struct.filesAddedChecksum)
+          for (String _iter695 : struct.filesAddedChecksum)
           {
-            oprot.writeString(_iter679);
+            oprot.writeString(_iter695);
           }
         }
       }
@@ -665,13 +665,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestData struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list680 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.filesAdded = new ArrayList<String>(_list680.size);
-        String _elem681;
-        for (int _i682 = 0; _i682 < _list680.size; ++_i682)
+        org.apache.thrift.protocol.TList _list696 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.filesAdded = new ArrayList<String>(_list696.size);
+        String _elem697;
+        for (int _i698 = 0; _i698 < _list696.size; ++_i698)
         {
-          _elem681 = iprot.readString();
-          struct.filesAdded.add(_elem681);
+          _elem697 = iprot.readString();
+          struct.filesAdded.add(_elem697);
         }
       }
       struct.setFilesAddedIsSet(true);
@@ -682,13 +682,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list683 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.filesAddedChecksum = new ArrayList<String>(_list683.size);
-          String _elem684;
-          for (int _i685 = 0; _i685 < _list683.size; ++_i685)
+          org.apache.thrift.protocol.TList _list699 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.filesAddedChecksum = new ArrayList<String>(_list699.size);
+          String _elem700;
+          for (int _i701 = 0; _i701 < _list699.size; ++_i701)
           {
-            _elem684 = iprot.readString();
-            struct.filesAddedChecksum.add(_elem684);
+            _elem700 = iprot.readString();
+            struct.filesAddedChecksum.add(_elem700);
           }
         }
         struct.setFilesAddedChecksumIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
index cfdd0bd..64bf58c 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
@@ -689,14 +689,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COMPONENT
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list596 = iprot.readListBegin();
-                struct.component = new ArrayList<LockComponent>(_list596.size);
-                LockComponent _elem597;
-                for (int _i598 = 0; _i598 < _list596.size; ++_i598)
+                org.apache.thrift.protocol.TList _list612 = iprot.readListBegin();
+                struct.component = new ArrayList<LockComponent>(_list612.size);
+                LockComponent _elem613;
+                for (int _i614 = 0; _i614 < _list612.size; ++_i614)
                 {
-                  _elem597 = new LockComponent();
-                  _elem597.read(iprot);
-                  struct.component.add(_elem597);
+                  _elem613 = new LockComponent();
+                  _elem613.read(iprot);
+                  struct.component.add(_elem613);
                 }
                 iprot.readListEnd();
               }
@@ -754,9 +754,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COMPONENT_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.component.size()));
-          for (LockComponent _iter599 : struct.component)
+          for (LockComponent _iter615 : struct.component)
           {
-            _iter599.write(oprot);
+            _iter615.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.component.size());
-        for (LockComponent _iter600 : struct.component)
+        for (LockComponent _iter616 : struct.component)
         {
-          _iter600.write(oprot);
+          _iter616.write(oprot);
         }
       }
       oprot.writeString(struct.user);
@@ -830,14 +830,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, LockRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list601 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.component = new ArrayList<LockComponent>(_list601.size);
-        LockComponent _elem602;
-        for (int _i603 = 0; _i603 < _list601.size; ++_i603)
+        org.apache.thrift.protocol.TList _list617 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.component = new ArrayList<LockComponent>(_list617.size);
+        LockComponent _elem618;
+        for (int _i619 = 0; _i619 < _list617.size; ++_i619)
         {
-          _elem602 = new LockComponent();
-          _elem602.read(iprot);
-          struct.component.add(_elem602);
+          _elem618 = new LockComponent();
+          _elem618.read(iprot);
+          struct.component.add(_elem618);
         }
       }
       struct.setComponentIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
index c91b9cf..d94a929 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
@@ -518,13 +518,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TABLES_USED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set786 = iprot.readSetBegin();
-                struct.tablesUsed = new HashSet<String>(2*_set786.size);
-                String _elem787;
-                for (int _i788 = 0; _i788 < _set786.size; ++_i788)
+                org.apache.thrift.protocol.TSet _set802 = iprot.readSetBegin();
+                struct.tablesUsed = new HashSet<String>(2*_set802.size);
+                String _elem803;
+                for (int _i804 = 0; _i804 < _set802.size; ++_i804)
                 {
-                  _elem787 = iprot.readString();
-                  struct.tablesUsed.add(_elem787);
+                  _elem803 = iprot.readString();
+                  struct.tablesUsed.add(_elem803);
                 }
                 iprot.readSetEnd();
               }
@@ -566,9 +566,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_USED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.tablesUsed.size()));
-          for (String _iter789 : struct.tablesUsed)
+          for (String _iter805 : struct.tablesUsed)
           {
-            oprot.writeString(_iter789);
+            oprot.writeString(_iter805);
           }
           oprot.writeSetEnd();
         }
@@ -603,9 +603,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tablesUsed.size());
-        for (String _iter790 : struct.tablesUsed)
+        for (String _iter806 : struct.tablesUsed)
         {
-          oprot.writeString(_iter790);
+          oprot.writeString(_iter806);
         }
       }
       oprot.writeI64(struct.invalidationTime);
@@ -623,13 +623,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, Materialization struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TSet _set791 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.tablesUsed = new HashSet<String>(2*_set791.size);
-        String _elem792;
-        for (int _i793 = 0; _i793 < _set791.size; ++_i793)
+        org.apache.thrift.protocol.TSet _set807 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.tablesUsed = new HashSet<String>(2*_set807.size);
+        String _elem808;
+        for (int _i809 = 0; _i809 < _set807.size; ++_i809)
         {
-          _elem792 = iprot.readString();
-          struct.tablesUsed.add(_elem792);
+          _elem808 = iprot.readString();
+          struct.tablesUsed.add(_elem808);
         }
       }
       struct.setTablesUsedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
index 549c14b..3405ecc 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // EVENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list662 = iprot.readListBegin();
-                struct.events = new ArrayList<NotificationEvent>(_list662.size);
-                NotificationEvent _elem663;
-                for (int _i664 = 0; _i664 < _list662.size; ++_i664)
+                org.apache.thrift.protocol.TList _list678 = iprot.readListBegin();
+                struct.events = new ArrayList<NotificationEvent>(_list678.size);
+                NotificationEvent _elem679;
+                for (int _i680 = 0; _i680 < _list678.size; ++_i680)
                 {
-                  _elem663 = new NotificationEvent();
-                  _elem663.read(iprot);
-                  struct.events.add(_elem663);
+                  _elem679 = new NotificationEvent();
+                  _elem679.read(iprot);
+                  struct.events.add(_elem679);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(EVENTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.events.size()));
-          for (NotificationEvent _iter665 : struct.events)
+          for (NotificationEvent _iter681 : struct.events)
           {
-            _iter665.write(oprot);
+            _iter681.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.events.size());
-        for (NotificationEvent _iter666 : struct.events)
+        for (NotificationEvent _iter682 : struct.events)
         {
-          _iter666.write(oprot);
+          _iter682.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, NotificationEventResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list667 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.events = new ArrayList<NotificationEvent>(_list667.size);
-        NotificationEvent _elem668;
-        for (int _i669 = 0; _i669 < _list667.size; ++_i669)
+        org.apache.thrift.protocol.TList _list683 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.events = new ArrayList<NotificationEvent>(_list683.size);
+        NotificationEvent _elem684;
+        for (int _i685 = 0; _i685 < _list683.size; ++_i685)
         {
-          _elem668 = new NotificationEvent();
-          _elem668.read(iprot);
-          struct.events.add(_elem668);
+          _elem684 = new NotificationEvent();
+          _elem684.read(iprot);
+          struct.events.add(_elem684);
         }
       }
       struct.setEventsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
index ee7ae39..672c688 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list540 = iprot.readListBegin();
-                struct.txn_ids = new ArrayList<Long>(_list540.size);
-                long _elem541;
-                for (int _i542 = 0; _i542 < _list540.size; ++_i542)
+                org.apache.thrift.protocol.TList _list556 = iprot.readListBegin();
+                struct.txn_ids = new ArrayList<Long>(_list556.size);
+                long _elem557;
+                for (int _i558 = 0; _i558 < _list556.size; ++_i558)
                 {
-                  _elem541 = iprot.readI64();
-                  struct.txn_ids.add(_elem541);
+                  _elem557 = iprot.readI64();
+                  struct.txn_ids.add(_elem557);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txn_ids.size()));
-          for (long _iter543 : struct.txn_ids)
+          for (long _iter559 : struct.txn_ids)
           {
-            oprot.writeI64(_iter543);
+            oprot.writeI64(_iter559);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txn_ids.size());
-        for (long _iter544 : struct.txn_ids)
+        for (long _iter560 : struct.txn_ids)
         {
-          oprot.writeI64(_iter544);
+          oprot.writeI64(_iter560);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, OpenTxnsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list545 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.txn_ids = new ArrayList<Long>(_list545.size);
-        long _elem546;
-        for (int _i547 = 0; _i547 < _list545.size; ++_i547)
+        org.apache.thrift.protocol.TList _list561 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.txn_ids = new ArrayList<Long>(_list561.size);
+        long _elem562;
+        for (int _i563 = 0; _i563 < _list561.size; ++_i563)
         {
-          _elem546 = iprot.readI64();
-          struct.txn_ids.add(_elem546);
+          _elem562 = iprot.readI64();
+          struct.txn_ids.add(_elem562);
         }
       }
       struct.setTxn_idsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java
index 2283c24..93a5034 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java
@@ -961,14 +961,14 @@ import org.slf4j.LoggerFactory;
           case 3: // PARTITION_KEYS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list484 = iprot.readListBegin();
-                struct.partitionKeys = new ArrayList<FieldSchema>(_list484.size);
-                FieldSchema _elem485;
-                for (int _i486 = 0; _i486 < _list484.size; ++_i486)
+                org.apache.thrift.protocol.TList _list500 = iprot.readListBegin();
+                struct.partitionKeys = new ArrayList<FieldSchema>(_list500.size);
+                FieldSchema _elem501;
+                for (int _i502 = 0; _i502 < _list500.size; ++_i502)
                 {
-                  _elem485 = new FieldSchema();
-                  _elem485.read(iprot);
-                  struct.partitionKeys.add(_elem485);
+                  _elem501 = new FieldSchema();
+                  _elem501.read(iprot);
+                  struct.partitionKeys.add(_elem501);
                 }
                 iprot.readListEnd();
               }
@@ -996,14 +996,14 @@ import org.slf4j.LoggerFactory;
           case 6: // PARTITION_ORDER
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list487 = iprot.readListBegin();
-                struct.partitionOrder = new ArrayList<FieldSchema>(_list487.size);
-                FieldSchema _elem488;
-                for (int _i489 = 0; _i489 < _list487.size; ++_i489)
+                org.apache.thrift.protocol.TList _list503 = iprot.readListBegin();
+                struct.partitionOrder = new ArrayList<FieldSchema>(_list503.size);
+                FieldSchema _elem504;
+                for (int _i505 = 0; _i505 < _list503.size; ++_i505)
                 {
-                  _elem488 = new FieldSchema();
-                  _elem488.read(iprot);
-                  struct.partitionOrder.add(_elem488);
+                  _elem504 = new FieldSchema();
+                  _elem504.read(iprot);
+                  struct.partitionOrder.add(_elem504);
                 }
                 iprot.readListEnd();
               }
@@ -1055,9 +1055,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITION_KEYS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionKeys.size()));
-          for (FieldSchema _iter490 : struct.partitionKeys)
+          for (FieldSchema _iter506 : struct.partitionKeys)
           {
-            _iter490.write(oprot);
+            _iter506.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1080,9 +1080,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITION_ORDER_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionOrder.size()));
-            for (FieldSchema _iter491 : struct.partitionOrder)
+            for (FieldSchema _iter507 : struct.partitionOrder)
             {
-              _iter491.write(oprot);
+              _iter507.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -1120,9 +1120,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.partitionKeys.size());
-        for (FieldSchema _iter492 : struct.partitionKeys)
+        for (FieldSchema _iter508 : struct.partitionKeys)
         {
-          _iter492.write(oprot);
+          _iter508.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -1151,9 +1151,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionOrder()) {
         {
           oprot.writeI32(struct.partitionOrder.size());
-          for (FieldSchema _iter493 : struct.partitionOrder)
+          for (FieldSchema _iter509 : struct.partitionOrder)
           {
-            _iter493.write(oprot);
+            _iter509.write(oprot);
           }
         }
       }
@@ -1173,14 +1173,14 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list494 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.partitionKeys = new ArrayList<FieldSchema>(_list494.size);
-        FieldSchema _elem495;
-        for (int _i496 = 0; _i496 < _list494.size; ++_i496)
+        org.apache.thrift.protocol.TList _list510 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.partitionKeys = new ArrayList<FieldSchema>(_list510.size);
+        FieldSchema _elem511;
+        for (int _i512 = 0; _i512 < _list510.size; ++_i512)
         {
-          _elem495 = new FieldSchema();
-          _elem495.read(iprot);
-          struct.partitionKeys.add(_elem495);
+          _elem511 = new FieldSchema();
+          _elem511.read(iprot);
+          struct.partitionKeys.add(_elem511);
         }
       }
       struct.setPartitionKeysIsSet(true);
@@ -1195,14 +1195,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list497 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitionOrder = new ArrayList<FieldSchema>(_list497.size);
-          FieldSchema _elem498;
-          for (int _i499 = 0; _i499 < _list497.size; ++_i499)
+          org.apache.thrift.protocol.TList _list513 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitionOrder = new ArrayList<FieldSchema>(_list513.size);
+          FieldSchema _elem514;
+          for (int _i515 = 0; _i515 < _list513.size; ++_i515)
           {
-            _elem498 = new FieldSchema();
-            _elem498.read(iprot);
-            struct.partitionOrder.add(_elem498);
+            _elem514 = new FieldSchema();
+            _elem514.read(iprot);
+            struct.partitionOrder.add(_elem514);
           }
         }
         struct.setPartitionOrderIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java
index f551156..e1e0de4 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITION_VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list508 = iprot.readListBegin();
-                struct.partitionValues = new ArrayList<PartitionValuesRow>(_list508.size);
-                PartitionValuesRow _elem509;
-                for (int _i510 = 0; _i510 < _list508.size; ++_i510)
+                org.apache.thrift.protocol.TList _list524 = iprot.readListBegin();
+                struct.partitionValues = new ArrayList<PartitionValuesRow>(_list524.size);
+                PartitionValuesRow _elem525;
+                for (int _i526 = 0; _i526 < _list524.size; ++_i526)
                 {
-                  _elem509 = new PartitionValuesRow();
-                  _elem509.read(iprot);
-                  struct.partitionValues.add(_elem509);
+                  _elem525 = new PartitionValuesRow();
+                  _elem525.read(iprot);
+                  struct.partitionValues.add(_elem525);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITION_VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionValues.size()));
-          for (PartitionValuesRow _iter511 : struct.partitionValues)
+          for (PartitionValuesRow _iter527 : struct.partitionValues)
           {
-            _iter511.write(oprot);
+            _iter527.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.partitionValues.size());
-        for (PartitionValuesRow _iter512 : struct.partitionValues)
+        for (PartitionValuesRow _iter528 : struct.partitionValues)
         {
-          _iter512.write(oprot);
+          _iter528.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionValuesResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list513 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.partitionValues = new ArrayList<PartitionValuesRow>(_list513.size);
-        PartitionValuesRow _elem514;
-        for (int _i515 = 0; _i515 < _list513.size; ++_i515)
+        org.apache.thrift.protocol.TList _list529 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.partitionValues = new ArrayList<PartitionValuesRow>(_list529.size);
+        PartitionValuesRow _elem530;
+        for (int _i531 = 0; _i531 < _list529.size; ++_i531)
         {
-          _elem514 = new PartitionValuesRow();
-          _elem514.read(iprot);
-          struct.partitionValues.add(_elem514);
+          _elem530 = new PartitionValuesRow();
+          _elem530.read(iprot);
+          struct.partitionValues.add(_elem530);
         }
       }
       struct.setPartitionValuesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java
index 3f3c3b9..e39063f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ROW
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list500 = iprot.readListBegin();
-                struct.row = new ArrayList<String>(_list500.size);
-                String _elem501;
-                for (int _i502 = 0; _i502 < _list500.size; ++_i502)
+                org.apache.thrift.protocol.TList _list516 = iprot.readListBegin();
+                struct.row = new ArrayList<String>(_list516.size);
+                String _elem517;
+                for (int _i518 = 0; _i518 < _list516.size; ++_i518)
                 {
-                  _elem501 = iprot.readString();
-                  struct.row.add(_elem501);
+                  _elem517 = iprot.readString();
+                  struct.row.add(_elem517);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(ROW_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.row.size()));
-          for (String _iter503 : struct.row)
+          for (String _iter519 : struct.row)
           {
-            oprot.writeString(_iter503);
+            oprot.writeString(_iter519);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.row.size());
-        for (String _iter504 : struct.row)
+        for (String _iter520 : struct.row)
         {
-          oprot.writeString(_iter504);
+          oprot.writeString(_iter520);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionValuesRow struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list505 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.row = new ArrayList<String>(_list505.size);
-        String _elem506;
-        for (int _i507 = 0; _i507 < _list505.size; ++_i507)
+        org.apache.thrift.protocol.TList _list521 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.row = new ArrayList<String>(_list521.size);
+        String _elem522;
+        for (int _i523 = 0; _i523 < _list521.size; ++_i523)
         {
-          _elem506 = iprot.readString();
-          struct.row.add(_elem506);
+          _elem522 = iprot.readString();
+          struct.row.add(_elem522);
         }
       }
       struct.setRowIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
index 3ccf5ee..1069acc 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
@@ -439,14 +439,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list386 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list386.size);
-                Partition _elem387;
-                for (int _i388 = 0; _i388 < _list386.size; ++_i388)
+                org.apache.thrift.protocol.TList _list402 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list402.size);
+                Partition _elem403;
+                for (int _i404 = 0; _i404 < _list402.size; ++_i404)
                 {
-                  _elem387 = new Partition();
-                  _elem387.read(iprot);
-                  struct.partitions.add(_elem387);
+                  _elem403 = new Partition();
+                  _elem403.read(iprot);
+                  struct.partitions.add(_elem403);
                 }
                 iprot.readListEnd();
               }
@@ -480,9 +480,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-          for (Partition _iter389 : struct.partitions)
+          for (Partition _iter405 : struct.partitions)
           {
-            _iter389.write(oprot);
+            _iter405.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -510,9 +510,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.partitions.size());
-        for (Partition _iter390 : struct.partitions)
+        for (Partition _iter406 : struct.partitions)
         {
-          _iter390.write(oprot);
+          _iter406.write(oprot);
         }
       }
       oprot.writeBool(struct.hasUnknownPartitions);
@@ -522,14 +522,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsByExprResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list391 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.partitions = new ArrayList<Partition>(_list391.size);
-        Partition _elem392;
-        for (int _i393 = 0; _i393 < _list391.size; ++_i393)
+        org.apache.thrift.protocol.TList _list407 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.partitions = new ArrayList<Partition>(_list407.size);
+        Partition _elem408;
+        for (int _i409 = 0; _i409 < _list407.size; ++_i409)
         {
-          _elem392 = new Partition();
-          _elem392.read(iprot);
-          struct.partitions.add(_elem392);
+          _elem408 = new Partition();
+          _elem408.read(iprot);
+          struct.partitions.add(_elem408);
         }
       }
       struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
index 9941fa5..1de1233 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
@@ -639,13 +639,13 @@ import org.slf4j.LoggerFactory;
           case 3: // COL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list428 = iprot.readListBegin();
-                struct.colNames = new ArrayList<String>(_list428.size);
-                String _elem429;
-                for (int _i430 = 0; _i430 < _list428.size; ++_i430)
+                org.apache.thrift.protocol.TList _list444 = iprot.readListBegin();
+                struct.colNames = new ArrayList<String>(_list444.size);
+                String _elem445;
+                for (int _i446 = 0; _i446 < _list444.size; ++_i446)
                 {
-                  _elem429 = iprot.readString();
-                  struct.colNames.add(_elem429);
+                  _elem445 = iprot.readString();
+                  struct.colNames.add(_elem445);
                 }
                 iprot.readListEnd();
               }
@@ -657,13 +657,13 @@ import org.slf4j.LoggerFactory;
           case 4: // PART_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list431 = iprot.readListBegin();
-                struct.partNames = new ArrayList<String>(_list431.size);
-                String _elem432;
-                for (int _i433 = 0; _i433 < _list431.size; ++_i433)
+                org.apache.thrift.protocol.TList _list447 = iprot.readListBegin();
+                struct.partNames = new ArrayList<String>(_list447.size);
+                String _elem448;
+                for (int _i449 = 0; _i449 < _list447.size; ++_i449)
                 {
-                  _elem432 = iprot.readString();
-                  struct.partNames.add(_elem432);
+                  _elem448 = iprot.readString();
+                  struct.partNames.add(_elem448);
                 }
                 iprot.readListEnd();
               }
@@ -699,9 +699,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COL_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.colNames.size()));
-          for (String _iter434 : struct.colNames)
+          for (String _iter450 : struct.colNames)
           {
-            oprot.writeString(_iter434);
+            oprot.writeString(_iter450);
           }
           oprot.writeListEnd();
         }
@@ -711,9 +711,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-          for (String _iter435 : struct.partNames)
+          for (String _iter451 : struct.partNames)
           {
-            oprot.writeString(_iter435);
+            oprot.writeString(_iter451);
           }
           oprot.writeListEnd();
         }
@@ -740,16 +740,16 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.colNames.size());
-        for (String _iter436 : struct.colNames)
+        for (String _iter452 : struct.colNames)
         {
-          oprot.writeString(_iter436);
+          oprot.writeString(_iter452);
         }
       }
       {
         oprot.writeI32(struct.partNames.size());
-        for (String _iter437 : struct.partNames)
+        for (String _iter453 : struct.partNames)
         {
-          oprot.writeString(_iter437);
+          oprot.writeString(_iter453);
         }
       }
     }
@@ -762,24 +762,24 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list438 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.colNames = new ArrayList<String>(_list438.size);
-        String _elem439;
-        for (int _i440 = 0; _i440 < _list438.size; ++_i440)
+        org.apache.thrift.protocol.TList _list454 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.colNames = new ArrayList<String>(_list454.size);
+        String _elem455;
+        for (int _i456 = 0; _i456 < _list454.size; ++_i456)
         {
-          _elem439 = iprot.readString();
-          struct.colNames.add(_elem439);
+          _elem455 = iprot.readString();
+          struct.colNames.add(_elem455);
         }
       }
       struct.setColNamesIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list441 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partNames = new ArrayList<String>(_list441.size);
-        String _elem442;
-        for (int _i443 = 0; _i443 < _list441.size; ++_i443)
+        org.apache.thrift.protocol.TList _list457 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partNames = new ArrayList<String>(_list457.size);
+        String _elem458;
+        for (int _i459 = 0; _i459 < _list457.size; ++_i459)
         {
-          _elem442 = iprot.readString();
-          struct.partNames.add(_elem442);
+          _elem458 = iprot.readString();
+          struct.partNames.add(_elem458);
         }
       }
       struct.setPartNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
index 8a0e5a5..c2183e5 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
@@ -363,26 +363,26 @@ import org.slf4j.LoggerFactory;
           case 1: // PART_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map402 = iprot.readMapBegin();
-                struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map402.size);
-                String _key403;
-                List<ColumnStatisticsObj> _val404;
-                for (int _i405 = 0; _i405 < _map402.size; ++_i405)
+                org.apache.thrift.protocol.TMap _map418 = iprot.readMapBegin();
+                struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map418.size);
+                String _key419;
+                List<ColumnStatisticsObj> _val420;
+                for (int _i421 = 0; _i421 < _map418.size; ++_i421)
                 {
-                  _key403 = iprot.readString();
+                  _key419 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TList _list406 = iprot.readListBegin();
-                    _val404 = new ArrayList<ColumnStatisticsObj>(_list406.size);
-                    ColumnStatisticsObj _elem407;
-                    for (int _i408 = 0; _i408 < _list406.size; ++_i408)
+                    org.apache.thrift.protocol.TList _list422 = iprot.readListBegin();
+                    _val420 = new ArrayList<ColumnStatisticsObj>(_list422.size);
+                    ColumnStatisticsObj _elem423;
+                    for (int _i424 = 0; _i424 < _list422.size; ++_i424)
                     {
-                      _elem407 = new ColumnStatisticsObj();
-                      _elem407.read(iprot);
-                      _val404.add(_elem407);
+                      _elem423 = new ColumnStatisticsObj();
+                      _elem423.read(iprot);
+                      _val420.add(_elem423);
                     }
                     iprot.readListEnd();
                   }
-                  struct.partStats.put(_key403, _val404);
+                  struct.partStats.put(_key419, _val420);
                 }
                 iprot.readMapEnd();
               }
@@ -408,14 +408,14 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PART_STATS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.partStats.size()));
-          for (Map.Entry<String, List<ColumnStatisticsObj>> _iter409 : struct.partStats.entrySet())
+          for (Map.Entry<String, List<ColumnStatisticsObj>> _iter425 : struct.partStats.entrySet())
           {
-            oprot.writeString(_iter409.getKey());
+            oprot.writeString(_iter425.getKey());
             {
-              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter409.getValue().size()));
-              for (ColumnStatisticsObj _iter410 : _iter409.getValue())
+              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter425.getValue().size()));
+              for (ColumnStatisticsObj _iter426 : _iter425.getValue())
               {
-                _iter410.write(oprot);
+                _iter426.write(oprot);
               }
               oprot.writeListEnd();
             }
@@ -443,14 +443,14 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.partStats.size());
-        for (Map.Entry<String, List<ColumnStatisticsObj>> _iter411 : struct.partStats.entrySet())
+        for (Map.Entry<String, List<ColumnStatisticsObj>> _iter427 : struct.partStats.entrySet())
         {
-          oprot.writeString(_iter411.getKey());
+          oprot.writeString(_iter427.getKey());
           {
-            oprot.writeI32(_iter411.getValue().size());
-            for (ColumnStatisticsObj _iter412 : _iter411.getValue())
+            oprot.writeI32(_iter427.getValue().size());
+            for (ColumnStatisticsObj _iter428 : _iter427.getValue())
             {
-              _iter412.write(oprot);
+              _iter428.write(oprot);
             }
           }
         }
@@ -461,25 +461,25 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsStatsResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map413 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-        struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map413.size);
-        String _key414;
-        List<ColumnStatisticsObj> _val415;
-        for (int _i416 = 0; _i416 < _map413.size; ++_i416)
+        org.apache.thrift.protocol.TMap _map429 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+        struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map429.size);
+        String _key430;
+        List<ColumnStatisticsObj> _val431;
+        for (int _i432 = 0; _i432 < _map429.size; ++_i432)
         {
-          _key414 = iprot.readString();
+          _key430 = iprot.readString();
           {
-            org.apache.thrift.protocol.TList _list417 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            _val415 = new ArrayList<ColumnStatisticsObj>(_list417.size);
-            ColumnStatisticsObj _elem418;
-            for (int _i419 = 0; _i419 < _list417.size; ++_i419)
+            org.apache.thrift.protocol.TList _list433 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            _val431 = new ArrayList<ColumnStatisticsObj>(_list433.size);
+            ColumnStatisticsObj _elem434;
+            for (int _i435 = 0; _i435 < _list433.size; ++_i435)
             {
-              _elem418 = new ColumnStatisticsObj();
-              _elem418.read(iprot);
-              _val415.add(_elem418);
+              _elem434 = new ColumnStatisticsObj();
+              _elem434.read(iprot);
+              _val431.add(_elem434);
             }
           }
-          struct.partStats.put(_key414, _val415);
+          struct.partStats.put(_key430, _val431);
         }
       }
       struct.setPartStatsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
index e4089c5..ba2b320 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
@@ -547,13 +547,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list730 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list730.size);
-                long _elem731;
-                for (int _i732 = 0; _i732 < _list730.size; ++_i732)
+                org.apache.thrift.protocol.TList _list746 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list746.size);
+                long _elem747;
+                for (int _i748 = 0; _i748 < _list746.size; ++_i748)
                 {
-                  _elem731 = iprot.readI64();
-                  struct.fileIds.add(_elem731);
+                  _elem747 = iprot.readI64();
+                  struct.fileIds.add(_elem747);
                 }
                 iprot.readListEnd();
               }
@@ -565,13 +565,13 @@ import org.slf4j.LoggerFactory;
           case 2: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list733 = iprot.readListBegin();
-                struct.metadata = new ArrayList<ByteBuffer>(_list733.size);
-                ByteBuffer _elem734;
-                for (int _i735 = 0; _i735 < _list733.size; ++_i735)
+                org.apache.thrift.protocol.TList _list749 = iprot.readListBegin();
+                struct.metadata = new ArrayList<ByteBuffer>(_list749.size);
+                ByteBuffer _elem750;
+                for (int _i751 = 0; _i751 < _list749.size; ++_i751)
                 {
-                  _elem734 = iprot.readBinary();
-                  struct.metadata.add(_elem734);
+                  _elem750 = iprot.readBinary();
+                  struct.metadata.add(_elem750);
                 }
                 iprot.readListEnd();
               }
@@ -605,9 +605,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter736 : struct.fileIds)
+          for (long _iter752 : struct.fileIds)
           {
-            oprot.writeI64(_iter736);
+            oprot.writeI64(_iter752);
           }
           oprot.writeListEnd();
         }
@@ -617,9 +617,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (ByteBuffer _iter737 : struct.metadata)
+          for (ByteBuffer _iter753 : struct.metadata)
           {
-            oprot.writeBinary(_iter737);
+            oprot.writeBinary(_iter753);
           }
           oprot.writeListEnd();
         }
@@ -651,16 +651,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter738 : struct.fileIds)
+        for (long _iter754 : struct.fileIds)
         {
-          oprot.writeI64(_iter738);
+          oprot.writeI64(_iter754);
         }
       }
       {
         oprot.writeI32(struct.metadata.size());
-        for (ByteBuffer _iter739 : struct.metadata)
+        for (ByteBuffer _iter755 : struct.metadata)
         {
-          oprot.writeBinary(_iter739);
+          oprot.writeBinary(_iter755);
         }
       }
       BitSet optionals = new BitSet();
@@ -677,24 +677,24 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PutFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list740 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list740.size);
-        long _elem741;
-        for (int _i742 = 0; _i742 < _list740.size; ++_i742)
+        org.apache.thrift.protocol.TList _list756 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list756.size);
+        long _elem757;
+        for (int _i758 = 0; _i758 < _list756.size; ++_i758)
         {
-          _elem741 = iprot.readI64();
-          struct.fileIds.add(_elem741);
+          _elem757 = iprot.readI64();
+          struct.fileIds.add(_elem757);
         }
       }
       struct.setFileIdsIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list743 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new ArrayList<ByteBuffer>(_list743.size);
-        ByteBuffer _elem744;
-        for (int _i745 = 0; _i745 < _list743.size; ++_i745)
+        org.apache.thrift.protocol.TList _list759 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new ArrayList<ByteBuffer>(_list759.size);
+        ByteBuffer _elem760;
+        for (int _i761 = 0; _i761 < _list759.size; ++_i761)
         {
-          _elem744 = iprot.readBinary();
-          struct.metadata.add(_elem744);
+          _elem760 = iprot.readBinary();
+          struct.metadata.add(_elem760);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
index d1b5247..96fd264 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
@@ -168,13 +168,13 @@ import org.slf4j.LoggerFactory;
           if (field.type == NAMES_FIELD_DESC.type) {
             List<String> names;
             {
-              org.apache.thrift.protocol.TList _list468 = iprot.readListBegin();
-              names = new ArrayList<String>(_list468.size);
-              String _elem469;
-              for (int _i470 = 0; _i470 < _list468.size; ++_i470)
+              org.apache.thrift.protocol.TList _list484 = iprot.readListBegin();
+              names = new ArrayList<String>(_list484.size);
+              String _elem485;
+              for (int _i486 = 0; _i486 < _list484.size; ++_i486)
               {
-                _elem469 = iprot.readString();
-                names.add(_elem469);
+                _elem485 = iprot.readString();
+                names.add(_elem485);
               }
               iprot.readListEnd();
             }
@@ -187,14 +187,14 @@ import org.slf4j.LoggerFactory;
           if (field.type == EXPRS_FIELD_DESC.type) {
             List<DropPartitionsExpr> exprs;
             {
-              org.apache.thrift.protocol.TList _list471 = iprot.readListBegin();
-              exprs = new ArrayList<DropPartitionsExpr>(_list471.size);
-              DropPartitionsExpr _elem472;
-              for (int _i473 = 0; _i473 < _list471.size; ++_i473)
+              org.apache.thrift.protocol.TList _list487 = iprot.readListBegin();
+              exprs = new ArrayList<DropPartitionsExpr>(_list487.size);
+              DropPartitionsExpr _elem488;
+              for (int _i489 = 0; _i489 < _list487.size; ++_i489)
               {
-                _elem472 = new DropPartitionsExpr();
-                _elem472.read(iprot);
-                exprs.add(_elem472);
+                _elem488 = new DropPartitionsExpr();
+                _elem488.read(iprot);
+                exprs.add(_elem488);
               }
               iprot.readListEnd();
             }
@@ -219,9 +219,9 @@ import org.slf4j.LoggerFactory;
         List<String> names = (List<String>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, names.size()));
-          for (String _iter474 : names)
+          for (String _iter490 : names)
           {
-            oprot.writeString(_iter474);
+            oprot.writeString(_iter490);
           }
           oprot.writeListEnd();
         }
@@ -230,9 +230,9 @@ import org.slf4j.LoggerFactory;
         List<DropPartitionsExpr> exprs = (List<DropPartitionsExpr>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, exprs.size()));
-          for (DropPartitionsExpr _iter475 : exprs)
+          for (DropPartitionsExpr _iter491 : exprs)
           {
-            _iter475.write(oprot);
+            _iter491.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -250,13 +250,13 @@ import org.slf4j.LoggerFactory;
         case NAMES:
           List<String> names;
           {
-            org.apache.thrift.protocol.TList _list476 = iprot.readListBegin();
-            names = new ArrayList<String>(_list476.size);
-            String _elem477;
-            for (int _i478 = 0; _i478 < _list476.size; ++_i478)
+            org.apache.thrift.protocol.TList _list492 = iprot.readListBegin();
+            names = new ArrayList<String>(_list492.size);
+            String _elem493;
+            for (int _i494 = 0; _i494 < _list492.size; ++_i494)
             {
-              _elem477 = iprot.readString();
-              names.add(_elem477);
+              _elem493 = iprot.readString();
+              names.add(_elem493);
             }
             iprot.readListEnd();
           }
@@ -264,14 +264,14 @@ import org.slf4j.LoggerFactory;
         case EXPRS:
           List<DropPartitionsExpr> exprs;
           {
-            org.apache.thrift.protocol.TList _list479 = iprot.readListBegin();
-            exprs = new ArrayList<DropPartitionsExpr>(_list479.size);
-            DropPartitionsExpr _elem480;
-            for (int _i481 = 0; _i481 < _list479.size; ++_i481)
+            org.apache.thrift.protocol.TList _list495 = iprot.readListBegin();
+            exprs = new ArrayList<DropPartitionsExpr>(_list495.size);
+            DropPartitionsExpr _elem496;
+            for (int _i497 = 0; _i497 < _list495.size; ++_i497)
             {
-              _elem480 = new DropPartitionsExpr();
-              _elem480.read(iprot);
-              exprs.add(_elem480);
+              _elem496 = new DropPartitionsExpr();
+              _elem496.read(iprot);
+              exprs.add(_elem496);
             }
             iprot.readListEnd();
           }
@@ -291,9 +291,9 @@ import org.slf4j.LoggerFactory;
         List<String> names = (List<String>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, names.size()));
-          for (String _iter482 : names)
+          for (String _iter498 : names)
           {
-            oprot.writeString(_iter482);
+            oprot.writeString(_iter498);
           }
           oprot.writeListEnd();
         }
@@ -302,9 +302,9 @@ import org.slf4j.LoggerFactory;
         List<DropPartitionsExpr> exprs = (List<DropPartitionsExpr>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, exprs.size()));
-          for (DropPartitionsExpr _iter483 : exprs)
+          for (DropPartitionsExpr _iter499 : exprs)
           {
-            _iter483.write(oprot);
+            _iter499.write(oprot);
           }
           oprot.writeListEnd();
         }